From a1ff0edf60d982ab14a693261d304b047115f4a5 Mon Sep 17 00:00:00 2001 From: Adam Storm <storm@cockroachlabs.com> Date: Wed, 14 Apr 2021 16:38:24 -0400 Subject: [PATCH 01/37] sql: Don't clear zone configs on indexes during a RBR transition Previously we were wiping out the zone configs for all indexes when transitioning from REGIONAL BY ROW to either GLOBAL or REGIONAL BY TABLE. This was problematic because there could have been zone configs which were setup by the user and didn't have any multi-region zone config fields set. This fix ensures that we don't wipe out the zone configs for indexes, provided that they contain at least one field set that isn't governed by the multi-region syntax. This commit also makes a minor change to ClearFieldsOfAllSubzones to use a NewZoneConfig instead of an empty ZoneConfig. Doing so ensures that ClearFieldsOfAllSubzones doesn't clear the inherited fields of the zone config. Release note (sql change): Fixes a bug where transitioning from locality REGIONAL BY ROW to GLOBAL or REGIONAL BY TABLE could mistakenly remove a zone configuration on an index which has no multi-region fields set. --- .../logic_test/multi_region_zone_configs | 30 +++++++++++++++++++ pkg/config/zonepb/zone.go | 17 +++++++++++ pkg/sql/region_util.go | 19 ++++-------- pkg/sql/schema_changer.go | 12 ++++---- 4 files changed, 59 insertions(+), 19 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs index 9d70e049b984..629da9c120f5 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs +++ b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs @@ -869,11 +869,41 @@ SET override_multi_region_zone_config = false statement error attempting to update zone config which contains an extra zone configuration for index tbl2@tbl2_i_idx with field num_replicas populated ALTER TABLE tbl2 SET LOCALITY GLOBAL +statement ok +ALTER INDEX tbl2@tbl2_i_idx CONFIGURE ZONE USING gc.ttlseconds=10 + +query TT +SHOW ZONE CONFIGURATION FOR INDEX tbl2@tbl2_i_idx +---- +INDEX tbl2@tbl2_i_idx ALTER INDEX tbl2@tbl2_i_idx CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 10, + num_replicas = 10, + num_voters = 3, + constraints = '{+region=us-east-1: 1}', + voter_constraints = '[+region=us-east-1]', + lease_preferences = '[[+region=us-east-1]]' + statement ok SET override_multi_region_zone_config = true; ALTER TABLE tbl2 SET LOCALITY GLOBAL; SET override_multi_region_zone_config = false +# Validate that we don't overwrite gc.ttlseconds +query TT +SHOW ZONE CONFIGURATION FOR INDEX tbl2@tbl2_i_idx +---- +INDEX tbl2@tbl2_i_idx ALTER INDEX tbl2@tbl2_i_idx CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 10, + num_replicas = 3, + num_voters = 3, + constraints = '{+region=us-east-1: 1}', + voter_constraints = '[+region=us-east-1]', + lease_preferences = '[[+region=us-east-1]]' + statement ok ALTER TABLE tbl2 SET LOCALITY REGIONAL BY ROW diff --git a/pkg/config/zonepb/zone.go b/pkg/config/zonepb/zone.go index 7e3a810ea9ab..31d6d8f7a9dc 100644 --- a/pkg/config/zonepb/zone.go +++ b/pkg/config/zonepb/zone.go @@ -849,6 +849,23 @@ func (z *ZoneConfig) DiffWithZone( return true, DiffWithZoneMismatch{}, nil } +// ClearFieldsOfAllSubzones uses the supplied fieldList and clears those fields +// from all of the zone config's subzones. +func (z *ZoneConfig) ClearFieldsOfAllSubzones(fieldList []tree.Name) { + newSubzones := z.Subzones[:0] + emptyZone := NewZoneConfig() + for _, sz := range z.Subzones { + // By copying from an empty zone, we'll end up clearing out all of the + // fields in the fieldList. + sz.Config.CopyFromZone(*emptyZone, fieldList) + // If we haven't emptied out the subzone, append it to the new slice. + if !sz.Config.Equal(emptyZone) { + newSubzones = append(newSubzones, sz) + } + } + z.Subzones = newSubzones +} + // StoreSatisfiesConstraint checks whether a store satisfies the given constraint. // If the constraint is of the PROHIBITED type, satisfying it means the store // not matching the constraint's spec. diff --git a/pkg/sql/region_util.go b/pkg/sql/region_util.go index d8d43ee56130..955edbcedc54 100644 --- a/pkg/sql/region_util.go +++ b/pkg/sql/region_util.go @@ -440,11 +440,10 @@ func dropZoneConfigsForMultiRegionIndexes( regionConfig multiregion.RegionConfig, table catalog.TableDescriptor, ) (hasNewSubzones bool, newZoneConfig zonepb.ZoneConfig, err error) { - for _, indexID := range indexIDs { - for _, region := range regionConfig.Regions() { - zoneConfig.DeleteSubzone(uint32(indexID), string(region)) - } - } + // Clear all multi-region fields of the subzones. If this leaves them + // empty, they will automatically be removed. + zoneConfig.ClearFieldsOfAllSubzones(zonepb.MultiRegionZoneConfigFields) + // Strip placeholder status and spans if there are no more subzones. if len(zoneConfig.Subzones) == 0 && zoneConfig.IsSubzonePlaceholder() { zoneConfig.NumReplicas = nil @@ -516,15 +515,7 @@ var ApplyZoneConfigForMultiRegionTableOptionTableAndIndexes = func( // have to override to perform the ALTER, we want to wipe out the index // zone config so that the user won't have to override again the next time // the want to ALTER the table locality. - newSubzones := zc.Subzones[:0] - for _, sz := range zc.Subzones { - sz.Config.CopyFromZone(zonepb.ZoneConfig{}, zonepb.MultiRegionZoneConfigFields) - // If we haven't emptied out the subzone, append it to the new slice. - if !sz.Config.Equal(zonepb.ZoneConfig{}) && !sz.Config.Equal(zonepb.NewZoneConfig()) { - newSubzones = append(newSubzones, sz) - } - } - zc.Subzones = newSubzones + zc.ClearFieldsOfAllSubzones(zonepb.MultiRegionZoneConfigFields) zc.CopyFromZone(*localityZoneConfig, zonepb.MultiRegionZoneConfigFields) diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 895c8f795b8e..abd8ff248dcb 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -2507,19 +2507,21 @@ func (sc *SchemaChanger) applyZoneConfigChangeForMutation( if err != nil { return err } - return ApplyZoneConfigForMultiRegionTable( + if err := ApplyZoneConfigForMultiRegionTable( ctx, txn, sc.execCfg, regionConfig, tableDesc, opts..., - ) + ); err != nil { + return err + } } - // For the plain ALTER PRIMARY KEY case, copy the zone configs over - // for any new indexes. - // Note this is done even for isDone = true, though not strictly necessary. + // In all cases, we now copy the zone configs over for any new indexes. + // Note this is done even for isDone = true, though not strictly + // necessary. return maybeUpdateZoneConfigsForPKChange( ctx, txn, sc.execCfg, tableDesc, pkSwap, ) From 2cafaf8d1ef7496561583a0da36ce6c0bf11fb53 Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Thu, 15 Apr 2021 15:06:43 -0400 Subject: [PATCH 02/37] kvserver: use replica clock for determining stale closed ts Replicas evaluate whether a closed timestamp they find out about is older than they wish. For this, they were using the system time. Most things in CRDB use an hlc.Clock - for example because in tests that one can be stopped. This patch switches the staleness determination to use the hlc. Release note: None --- pkg/kv/kvserver/replica_rangefeed.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index dccf2309e2ae..55bd4ea6fa75 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -30,7 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" ) @@ -611,14 +610,15 @@ func (r *Replica) handleClosedTimestampUpdateRaftMuLocked(ctx context.Context) { // If the closed timestamp is sufficiently stale, signal that we want an // update to the leaseholder so that it will eventually begin to progress // again. + behind := r.Clock().PhysicalTime().Sub(closedTS.GoTime()) slowClosedTSThresh := 5 * closedts.TargetDuration.Get(&r.store.cfg.Settings.SV) - if d := timeutil.Since(closedTS.GoTime()); d > slowClosedTSThresh { + if behind > slowClosedTSThresh { m := r.store.metrics.RangeFeedMetrics if m.RangeFeedSlowClosedTimestampLogN.ShouldLog() { if closedTS.IsEmpty() { log.Infof(ctx, "RangeFeed closed timestamp is empty") } else { - log.Infof(ctx, "RangeFeed closed timestamp %s is behind by %s", closedTS, d) + log.Infof(ctx, "RangeFeed closed timestamp %s is behind by %s", closedTS, behind) } } From 60dba102d2be8be37ecf4b8ce9461511e4bb8886 Mon Sep 17 00:00:00 2001 From: Bilal Akhtar <bilal@cockroachlabs.com> Date: Tue, 13 Apr 2021 17:14:22 -0400 Subject: [PATCH 03/37] server, security: Fix one-way connectivity with connect cmd Previously, non-trust-leader nodes couldn't connect back to the trust leader due to the presence of the wrong `ca-client.crt` on their disk; the main CA cert/key was being written in four places. This change fixes that bug, and also creates a new `client.node.crt` certificate to prevent other subsequent errors from being thrown. Fixes #61624. Release note: None. --- pkg/security/certificate_manager.go | 12 +++++ pkg/server/auto_tls_init.go | 76 ++++++++++------------------- pkg/server/auto_tls_init_test.go | 26 +++++----- 3 files changed, 54 insertions(+), 60 deletions(-) diff --git a/pkg/security/certificate_manager.go b/pkg/security/certificate_manager.go index d0d804d55389..b2752f5259b8 100644 --- a/pkg/security/certificate_manager.go +++ b/pkg/security/certificate_manager.go @@ -304,6 +304,18 @@ func (cl CertsLocator) ClientCAKeyPath() string { return filepath.Join(cl.certsDir, "ca-client"+keyExtension) } +// ClientNodeCertPath returns the expected file path for the certificate used +// by other nodes to verify outgoing RPCs from this node. +func (cl CertsLocator) ClientNodeCertPath() string { + return filepath.Join(cl.certsDir, "client.node"+certExtension) +} + +// ClientNodeKeyPath returns the expected file path for the key used +// to sign outgoing RPCs. +func (cl CertsLocator) ClientNodeKeyPath() string { + return filepath.Join(cl.certsDir, "client.node"+keyExtension) +} + // UICACertPath returns the expected file path for the CA certificate // used to verify Admin UI certificates. func (cl CertsLocator) UICACertPath() string { diff --git a/pkg/server/auto_tls_init.go b/pkg/server/auto_tls_init.go index 4c3a7d4ccad8..55838cb90833 100644 --- a/pkg/server/auto_tls_init.go +++ b/pkg/server/auto_tls_init.go @@ -92,49 +92,6 @@ func (sb *ServiceCertificateBundle) loadCACertAndKey(certPath string, keyPath st return } -// LoadUserAuthCACertAndKey loads host certificate and key from disk or fails with error. -func (sb *ServiceCertificateBundle) loadOrCreateUserAuthCACertAndKey( - ctx context.Context, - caCertPath string, - caKeyPath string, - initLifespan time.Duration, - serviceName string, -) (err error) { - log.Ops.Infof(ctx, "attempting to load CA cert: %s", caCertPath) - // Attempt to load cert into ServiceCertificateBundle. - sb.CACertificate, err = loadCertificateFile(caCertPath) - if err != nil { - if oserror.IsNotExist(err) { - log.Ops.Infof(ctx, "not found; auto-generating") - // Certificate not found, attempt to create both cert and key now. - err = sb.createServiceCA(ctx, caCertPath, caKeyPath, initLifespan, serviceName) - if err != nil { - return err - } - - // Both key and cert should now be populated. - return nil - } - - // Some error unrelated to file existence occurred. - return err - } - - log.Ops.Infof(ctx, "found; loading CA key: %s", caKeyPath) - // Load the key only if it exists. - sb.CAKey, err = loadKeyFile(caKeyPath) - if err != nil { - if !oserror.IsNotExist(err) { - // An error returned but it was not that the file didn't exist; - // this is an error. - return err - } - log.Ops.Infof(ctx, "CA key not found") - } - - return nil -} - // loadOrCreateServiceCertificates will attempt to load the service cert/key // into the service bundle. // * If they do not exist: @@ -373,12 +330,18 @@ func (b *CertificateBundle) InitializeFromConfig(ctx context.Context, c base.Con } // Initialize User auth certificates. - if err := b.UserAuth.loadOrCreateUserAuthCACertAndKey( + if err := b.UserAuth.loadOrCreateServiceCertificates( ctx, + cl.ClientNodeCertPath(), + cl.ClientNodeKeyPath(), cl.ClientCACertPath(), cl.ClientCAKeyPath(), + defaultCertLifetime, defaultCALifetime, + security.NodeUser, serviceNameUserAuth, + nil, + true, /* serviceCertIsAlsoValidAsClient */ ); err != nil { return errors.Wrap(err, "failed to load or create User auth certificate(s)") @@ -494,22 +457,22 @@ func (b *CertificateBundle) InitializeNodeFromBundle(ctx context.Context, c base } // Attempt to write ClientCA to disk. - if err := b.InterNode.writeCAOrFail(cl.ClientCACertPath(), cl.ClientCAKeyPath()); err != nil { + if err := b.UserAuth.writeCAOrFail(cl.ClientCACertPath(), cl.ClientCAKeyPath()); err != nil { return errors.Wrap(err, "failed to write ClientCA to disk") } // Attempt to write SQLServiceCA to disk. - if err := b.InterNode.writeCAOrFail(cl.SQLServiceCACertPath(), cl.SQLServiceCAKeyPath()); err != nil { + if err := b.SQLService.writeCAOrFail(cl.SQLServiceCACertPath(), cl.SQLServiceCAKeyPath()); err != nil { return errors.Wrap(err, "failed to write SQLServiceCA to disk") } // Attempt to write RPCServiceCA to disk. - if err := b.InterNode.writeCAOrFail(cl.RPCServiceCACertPath(), cl.RPCServiceCAKeyPath()); err != nil { + if err := b.RPCService.writeCAOrFail(cl.RPCServiceCACertPath(), cl.RPCServiceCAKeyPath()); err != nil { return errors.Wrap(err, "failed to write RPCServiceCA to disk") } // Attempt to write AdminUIServiceCA to disk. - if err := b.InterNode.writeCAOrFail(cl.UICACertPath(), cl.UICAKeyPath()); err != nil { + if err := b.AdminUIService.writeCAOrFail(cl.UICACertPath(), cl.UICAKeyPath()); err != nil { return errors.Wrap(err, "failed to write AdminUIServiceCA to disk") } @@ -644,7 +607,22 @@ func rotateGeneratedCerts(ctx context.Context, c base.Config) error { } } - // TODO(aaron-crl): Should we rotate UserAuth Certs. + // Rotate UserAuth certificate + if b.UserAuth.CACertificate != nil { + err = b.UserAuth.rotateServiceCert( + ctx, + cl.ClientNodeCertPath(), + cl.ClientNodeKeyPath(), + defaultCertLifetime, + security.NodeUser, + serviceNameUserAuth, + nil, + true, /* serviceCertIsAlsoValidAsClient */ + ) + if err != nil { + return errors.Wrap(err, "failed to rotate InterNode cert") + } + } // Rotate SQLService Certs. if b.SQLService.CACertificate != nil { diff --git a/pkg/server/auto_tls_init_test.go b/pkg/server/auto_tls_init_test.go index 1c96ec30a3be..3b944a92369e 100644 --- a/pkg/server/auto_tls_init_test.go +++ b/pkg/server/auto_tls_init_test.go @@ -77,10 +77,13 @@ func loadAllCertsFromDisk(ctx context.Context, cfg base.Config) (CertificateBund return bundleFromDisk, err } - // TODO(aaron-crl): Figure out how to handle client auth case. - //bundleFromDisk.UserAuth.loadOrCreateServiceCertificates( - // ctx, cl.ClientCertPath(), cl.ClientKeyPath(), "", "", 0, 0, security.NodeUser, "", []string{}, - //) + err = bundleFromDisk.UserAuth.loadOrCreateServiceCertificates( + ctx, cl.ClientNodeCertPath(), cl.ClientNodeKeyPath(), "", "", 0, 0, security.NodeUser, "", []string{}, + true, /* serviceCertIsAlsoValidAsClient */ + ) + if err != nil { + return bundleFromDisk, err + } err = bundleFromDisk.SQLService.loadOrCreateServiceCertificates( ctx, cl.SQLServiceCertPath(), cl.SQLServiceKeyPath(), "", "", 0, 0, security.NodeUser, "", []string{}, false, /* serviceCertIsAlsoValidAsClient */ @@ -181,6 +184,13 @@ func compareBundleServiceCerts( cb1.InterNode.HostKey, cb2.InterNode.HostKey, serviceNameInterNode+" Host key") + cmp( + cb1.UserAuth.HostCertificate, + cb2.UserAuth.HostCertificate, serviceNameUserAuth+" Host cert") + cmp( + cb1.UserAuth.HostKey, + cb2.UserAuth.HostKey, serviceNameUserAuth+" Host key") + cmp( cb1.SQLService.HostCertificate, cb2.SQLService.HostCertificate, serviceNameSQL+" Host cert") @@ -378,18 +388,12 @@ func TestRotationOnPartialIntializedNode(t *testing.T) { t.Fatalf("rotation failed; expected err=nil, got: %q", err) } - // Verify that client and UI service host certs are unchanged. + // Verify that the UI service host certs is unchanged. diskBundle, err := loadAllCertsFromDisk(ctx, cfg) if err != nil { t.Fatalf("failed loading certs from disk, got: %q", err) } cmp := certCompareHelper(t, true) - cmp( - certBundle.UserAuth.HostCertificate, - diskBundle.UserAuth.HostCertificate, "UserAuth host cert") - cmp( - certBundle.UserAuth.HostKey, - diskBundle.UserAuth.HostKey, "UserAuth host key") cmp( certBundle.AdminUIService.HostCertificate, diskBundle.AdminUIService.HostCertificate, "AdminUIService host cert") From 026e6301509481c11df7414ef02dd4edcf635823 Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Thu, 8 Apr 2021 14:05:35 -0400 Subject: [PATCH 04/37] kvserver: improve a TODO The TODO was proposing a new interface, whereas all that's require is a bit of code reordering I think. Release note: None --- pkg/kv/kvserver/replica_write.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index db34492ea527..a4b6e40cefac 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -94,11 +94,12 @@ func (r *Replica) executeWriteBatch( // We need to start tracking this request before we know the final write // timestamp at which this request will evaluate because we need to atomically // read the closed timestamp and start to be tracked. - // TODO(andrei): The timestamp cache might bump us above the timestamp at - // which we're registering with the proposalBuf. In that case, this request - // will be tracked at an unnecessarily low timestamp. We could invent an - // interface through which to communicate the updated timestamp to the - // proposalBuf. + // TODO(andrei): The timestamp cache (and also the "old closed timestamp + // mechanism" in the form of minTS) might bump us above the timestamp at which + // we're registering with the proposalBuf. In that case, this request will be + // tracked at an unnecessarily low timestamp which can block the closing of + // this low timestamp for no reason. We should refactor such that the request + // starts being tracked after we apply the timestamp cache. minTS2, tok := r.mu.proposalBuf.TrackEvaluatingRequest(ctx, ba.WriteTimestamp()) defer tok.DoneIfNotMoved(ctx) minTS.Forward(minTS2) From 2d5a43e050bf073cc56a9b06252353e5029a3026 Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Thu, 8 Apr 2021 14:09:56 -0400 Subject: [PATCH 05/37] kvserver: don't track non-MVCC requests Before this patch, all requests were being tracked by the proposal buffer during their evaluation, and thus all requests' timestamps were blocking the closed timestamp from progressing above them. This is only necessary for MVCC requests (ba.IsIntentWrite()) - these are the requests that are not allowed to write under the closed timestamp (or under any other read, as enforced by the timestamp cache). And indeed, these are the requests for which we assert that their write timestamp is above the closed timestamp below Raft. Tracking other requests was needless, confusing, and suboptimal since they were potentially holding back the closed timestamp progress for no reason. Release note: None --- pkg/kv/kvserver/replica_proposal_buf.go | 16 +++++++++++++--- pkg/kv/kvserver/replica_write.go | 23 +++++++++++++++-------- 2 files changed, 28 insertions(+), 11 deletions(-) diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 85deb72a6e99..015181f4fa4f 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -836,6 +836,7 @@ func (b *propBuf) EvaluatingRequestsCount() int { // fn(tok.Move()) type TrackedRequestToken struct { done bool + noop bool tok tracker.RemovalToken b *propBuf } @@ -849,7 +850,7 @@ type TrackedRequestToken struct { // tokens are expected to be destroyed at once by the propBuf (which calls // doneLocked). func (t *TrackedRequestToken) DoneIfNotMoved(ctx context.Context) { - if t.done { + if t.done || t.noop { return } t.b.p.locker().Lock() @@ -861,7 +862,7 @@ func (t *TrackedRequestToken) DoneIfNotMoved(ctx context.Context) { // this is used when wanting to untrack a proposal that might, in fact, be a // reproposal. func (t *TrackedRequestToken) doneIfNotMovedLocked(ctx context.Context) { - if t.done { + if t.done || t.noop { return } t.done = true @@ -870,7 +871,7 @@ func (t *TrackedRequestToken) doneIfNotMovedLocked(ctx context.Context) { // stillTracked returns true if no Done* method has been called. func (t *TrackedRequestToken) stillTracked() bool { - return !t.done + return !(t.done || t.noop) } // Move returns a new token which can untrack the request. The original token is @@ -879,11 +880,20 @@ func (t *TrackedRequestToken) Move(ctx context.Context) TrackedRequestToken { if t.done { log.Fatalf(ctx, "attempting to Move() after Done() call") } + if t.noop { + return *t + } cpy := *t t.done = true return cpy } +// SetNoop marks this token as not actually corresponding to a tracked request. +// Any operation on the token will be a no-op. +func (t *TrackedRequestToken) SetNoop() { + t.noop = true +} + // TrackEvaluatingRequest atomically starts tracking an evaluating request and // returns the minimum timestamp at which this request can write. The tracked // request is identified by its tentative write timestamp. After calling this, diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index a4b6e40cefac..cb41a78a8507 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -88,21 +88,28 @@ func (r *Replica) executeWriteBatch( minTS, untrack := r.store.cfg.ClosedTimestamp.Tracker.Track(ctx) defer untrack(ctx, 0, 0, 0) // covers all error returns below - // Start tracking this request. The act of tracking also gives us a closed - // timestamp, which we must ensure to evaluate above of. We're going to pass - // in minTS to applyTimestampCache(), which bumps us accordingly if necessary. - // We need to start tracking this request before we know the final write - // timestamp at which this request will evaluate because we need to atomically - // read the closed timestamp and start to be tracked. + // Start tracking this request if it is an MVCC write (i.e. if it's the kind + // of request that needs to obey the closed timestamp). The act of tracking + // also gives us a closed timestamp, which we must ensure to evaluate above + // of. We're going to pass in minTS to applyTimestampCache(), which bumps us + // accordingly if necessary. We need to start tracking this request before we + // know the final write timestamp at which this request will evaluate because + // we need to atomically read the closed timestamp and start to be tracked. // TODO(andrei): The timestamp cache (and also the "old closed timestamp // mechanism" in the form of minTS) might bump us above the timestamp at which // we're registering with the proposalBuf. In that case, this request will be // tracked at an unnecessarily low timestamp which can block the closing of // this low timestamp for no reason. We should refactor such that the request // starts being tracked after we apply the timestamp cache. - minTS2, tok := r.mu.proposalBuf.TrackEvaluatingRequest(ctx, ba.WriteTimestamp()) + var tok TrackedRequestToken + if ba.IsIntentWrite() { + var minTS2 hlc.Timestamp + minTS2, tok = r.mu.proposalBuf.TrackEvaluatingRequest(ctx, ba.WriteTimestamp()) + minTS.Forward(minTS2) + } else { + tok.SetNoop() + } defer tok.DoneIfNotMoved(ctx) - minTS.Forward(minTS2) // Examine the timestamp cache for preceding commands which require this // command to move its timestamp forward. Or, in the case of a transactional From 111347d50876cde8446864969f584b129d9e56f0 Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Thu, 8 Apr 2021 16:58:53 -0400 Subject: [PATCH 06/37] closedts: add comment to connection drop Release note: None --- pkg/kv/kvserver/closedts/sidetransport/receiver.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pkg/kv/kvserver/closedts/sidetransport/receiver.go b/pkg/kv/kvserver/closedts/sidetransport/receiver.go index 07b714c39e37..0d84751d5230 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/receiver.go +++ b/pkg/kv/kvserver/closedts/sidetransport/receiver.go @@ -127,6 +127,14 @@ func (s *Receiver) onRecvErr(ctx context.Context, nodeID roachpb.NodeID, err err log.VEventf(ctx, 2, "closed timestamps side-transport connection dropped from node: %d (%s)", nodeID, err) } if nodeID != 0 { + // Remove the connection from the map, awaiting a new conn to be opened by + // the remote node. Note that, in doing so, we lose all information about + // the ranges tracked by this connection. We could go through all of them + // and move their data to the replica, but that might be expensive. + // Alternatively, we could also do something else to not destroy the state + // of this connection. Note, though, that if any of these closed timestamps + // have been actually used to serve a read already, the info has been copied + // to the respective replica. delete(s.mu.conns, nodeID) } } From d56e8a38f70300e4436435d8528082e9a4a26024 Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Wed, 14 Apr 2021 14:45:32 -0400 Subject: [PATCH 07/37] kvserver: small closedts assignment refactor Make it clear that an else case is, in fact, an else case. Release note: None --- pkg/kv/kvserver/replica_proposal_buf.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 85deb72a6e99..a2dd209c7ef2 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -713,7 +713,6 @@ func (b *propBuf) assignClosedTimestampToProposalLocked( // Lease transfers behave like regular proposals. Note that transfers // carry a summary of the timestamp cache, so the new leaseholder will be // aware of all the reads performed by the previous leaseholder. - isBrandNewLeaseRequest := false if p.Request.IsLeaseRequest() { // We read the lease from the ReplicatedEvalResult, not from leaseReq, because the // former is more up to date, having been modified by the evaluation. @@ -723,12 +722,10 @@ func (b *propBuf) assignClosedTimestampToProposalLocked( if leaseExtension { return nil } - isBrandNewLeaseRequest = true // For brand new leases, we close the lease start time. Since this proposing // replica is not the leaseholder, the previous target is meaningless. closedTSTarget = newLease.Start.ToTimestamp() - } - if !isBrandNewLeaseRequest { + } else { lb := b.evalTracker.LowerBound(ctx) if !lb.IsEmpty() { // If the tracker told us that requests are currently evaluating at From 5c56d1930172e8614a6b1f2eca08198f90c05e8d Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Wed, 14 Apr 2021 14:51:04 -0400 Subject: [PATCH 08/37] kvserver: tighten comment Clarifiy a <= vs < in a closedts comment. Release note: None --- pkg/kv/kvserver/replica_proposal_buf.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index a2dd209c7ef2..df45a0078581 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -150,11 +150,12 @@ type propBuf struct { cnt propBufCnt arr propBufArray - // assignedClosedTimestamp is the largest "closed timestamp" - i.e. the largest - // timestamp that was communicated to other replicas as closed, representing a - // promise that this leaseholder will not evaluate writes below this timestamp - // any more. It is set when proposals are flushed from the buffer, and also - // by the side-transport which closes timestamps out of band. + // assignedClosedTimestamp is the largest "closed timestamp" - i.e. the + // largest timestamp that was communicated to other replicas as closed, + // representing a promise that this leaseholder will not evaluate writes with + // timestamp <= assignedClosedTimestamp any more. It is set when proposals are + // flushed from the buffer, and also by the side-transport which closes + // timestamps out of band. // // Note that this field is not used by the local replica (or by anybody) // directly to decide whether follower reads can be served. See From 26c871551c6a6692712b2f8ba41fd4104d0282b7 Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Wed, 14 Apr 2021 15:02:09 -0400 Subject: [PATCH 09/37] kvserver: add a closedts assertion Release note: None --- pkg/kv/kvserver/replica_proposal_buf.go | 15 +++++++++++++++ pkg/kv/kvserver/replica_proposal_buf_test.go | 17 +++++++++-------- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index df45a0078581..6ac466b08638 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -727,6 +727,21 @@ func (b *propBuf) assignClosedTimestampToProposalLocked( // replica is not the leaseholder, the previous target is meaningless. closedTSTarget = newLease.Start.ToTimestamp() } else { + // Sanity check that this command is not violating the closed timestamp. It + // must be writing at a timestamp above assignedClosedTimestamp + // (assignedClosedTimestamp represents the promise that this replica made + // through previous commands to not evaluate requests with lower + // timestamps); in other words, assignedClosedTimestamp was not supposed to + // have been incremented while requests with lower timestamps were + // evaluating (instead, assignedClosedTimestamp was supposed to have bumped + // the write timestamp of any request the began evaluating after it was + // set). + if p.Request.WriteTimestamp().Less(b.assignedClosedTimestamp) && p.Request.IsIntentWrite() { + return errors.AssertionFailedf("attempting to propose command writing below closed timestamp. "+ + "wts: %s < assigned closed: %s; ba: %s", + p.Request.WriteTimestamp(), b.assignedClosedTimestamp, p.Request) + } + lb := b.evalTracker.LowerBound(ctx) if !lb.IsEmpty() { // If the tracker told us that requests are currently evaluating at diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index f7dd181a69d9..b7e03de27c22 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -201,9 +201,10 @@ type proposalCreator struct { lease kvserverpb.LeaseStatus } -func (pc proposalCreator) newPutProposal() (*ProposalData, []byte) { +func (pc proposalCreator) newPutProposal(ts hlc.Timestamp) (*ProposalData, []byte) { var ba roachpb.BatchRequest ba.Add(&roachpb.PutRequest{}) + ba.Timestamp = ts return pc.newProposal(ba) } @@ -268,7 +269,7 @@ func TestProposalBuffer(t *testing.T) { if leaseReq { pd, data = pc.newLeaseProposal(roachpb.Lease{}) } else { - pd, data = pc.newPutProposal() + pd, data = pc.newPutProposal(hlc.Timestamp{}) } _, tok := b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) mlai, err := b.Insert(ctx, pd, data, tok) @@ -293,7 +294,7 @@ func TestProposalBuffer(t *testing.T) { // Insert another proposal. This causes the buffer to flush. Doing so // results in a lease applied index being skipped, which is harmless. // Remember that the lease request above did not receive a lease index. - pd, data := pc.newPutProposal() + pd, data := pc.newPutProposal(hlc.Timestamp{}) _, tok := b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) mlai, err := b.Insert(ctx, pd, data, tok) require.Nil(t, err) @@ -361,7 +362,7 @@ func TestProposalBufferConcurrentWithDestroy(t *testing.T) { for i := 0; i < concurrency; i++ { g.Go(func() error { for { - pd, data := pc.newPutProposal() + pd, data := pc.newPutProposal(hlc.Timestamp{}) _, tok := b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) mlai, err := b.Insert(ctx, pd, data, tok) if err != nil { @@ -429,7 +430,7 @@ func TestProposalBufferRegistersAllOnProposalError(t *testing.T) { num := propBufArrayMinSize toks := make([]TrackedRequestToken, num) for i := 0; i < num; i++ { - pd, data := pc.newPutProposal() + pd, data := pc.newPutProposal(hlc.Timestamp{}) _, toks[i] = b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) _, err := b.Insert(ctx, pd, data, toks[i]) require.Nil(t, err) @@ -474,7 +475,7 @@ func TestProposalBufferRegistrationWithInsertionErrors(t *testing.T) { if i%2 == 0 { pd, data = pc.newLeaseProposal(roachpb.Lease{}) } else { - pd, data = pc.newPutProposal() + pd, data = pc.newPutProposal(hlc.Timestamp{}) } _, toks1[i] = b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) _, err := b.Insert(ctx, pd, data, toks1[i]) @@ -493,7 +494,7 @@ func TestProposalBufferRegistrationWithInsertionErrors(t *testing.T) { if i%2 == 0 { pd, data = pc.newLeaseProposal(roachpb.Lease{}) } else { - pd, data = pc.newPutProposal() + pd, data = pc.newPutProposal(hlc.Timestamp{}) } _, toks2[i] = b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) _, err := b.Insert(ctx, pd, data, toks2[i]) @@ -871,7 +872,7 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { var data []byte switch tc.reqType { case regularWrite: - pd, data = pc.newPutProposal() + pd, data = pc.newPutProposal(now.ToTimestamp()) case newLease: pd, data = pc.newLeaseProposal(tc.lease) case leaseTransfer: From 4a31adecde3a5783819937eca5f8ee55a27c6357 Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Thu, 15 Apr 2021 11:49:02 -0400 Subject: [PATCH 10/37] kvserver: move submitProposalFilter down Make this testing filter run after a closed timestamp is assigned to the command to be filtered. I had to slightly change a test because, now, the filter doesn't run if the raft group is nil. Release note: None --- pkg/kv/kvserver/replica_proposal_buf.go | 22 ++++++++++---------- pkg/kv/kvserver/replica_proposal_buf_test.go | 2 ++ 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 6ac466b08638..134b8b4e11c7 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -595,17 +595,6 @@ func (b *propBuf) FlushLockedWithRaftGroup( // Exit the tracker. p.tok.doneIfNotMovedLocked(ctx) - // Potentially drop the proposal before passing it to etcd/raft, but - // only after performing necessary bookkeeping. - if filter := b.testing.submitProposalFilter; filter != nil { - if drop, err := filter(p); drop || err != nil { - if firstErr == nil { - firstErr = err - } - continue - } - } - // If we don't have a raft group or if the raft group has rejected one // of the proposals, we don't try to propose any more proposals. The // rest of the proposals will still be registered with the proposer, so @@ -628,6 +617,17 @@ func (b *propBuf) FlushLockedWithRaftGroup( } } + // Potentially drop the proposal before passing it to etcd/raft, but + // only after performing necessary bookkeeping. + if filter := b.testing.submitProposalFilter; filter != nil { + if drop, err := filter(p); drop || err != nil { + if firstErr == nil { + firstErr = err + } + continue + } + } + // Coordinate proposing the command to etcd/raft. if crt := p.command.ReplicatedEvalResult.ChangeReplicas; crt != nil { // Flush any previously batched (non-conf change) proposals to diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index b7e03de27c22..7046a2a008c8 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -421,7 +421,9 @@ func TestProposalBufferRegistersAllOnProposalError(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() + raft := &testProposerRaft{} var p testProposer + p.raftGroup = raft var b propBuf var pc proposalCreator clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) From 0fe64979622a6a94942bb6f497581d73ff47e137 Mon Sep 17 00:00:00 2001 From: MiguelNovelo <miguel.novelo@digitalonus.com> Date: Fri, 16 Apr 2021 08:17:11 -0500 Subject: [PATCH 11/37] roachtest: added hibernate ignorelist for 21.1 Previously, some random issues with hibernate were afecting roachtest This was inadequate because they are false failures To address this, this patch set these tests in ignore list Release note: none --- pkg/cmd/roachtest/hibernate_blocklist.go | 33 +++++++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/hibernate_blocklist.go b/pkg/cmd/roachtest/hibernate_blocklist.go index f2fe6790bfbe..2d5b5f72f7f9 100644 --- a/pkg/cmd/roachtest/hibernate_blocklist.go +++ b/pkg/cmd/roachtest/hibernate_blocklist.go @@ -18,7 +18,7 @@ var hibernateBlocklists = blocklistsForVersion{ {"v19.2", "hibernateBlockList19_2", hibernateBlockList19_2, "", nil}, {"v20.1", "hibernateBlockList20_1", hibernateBlockList20_1, "", nil}, {"v20.2", "hibernateBlockList20_2", hibernateBlockList20_2, "", nil}, - {"v21.1", "hibernateBlockList21_1", hibernateBlockList21_1, "", nil}, + {"v21.1", "hibernateBlockList21_1", hibernateBlockList21_1, "", hibernateIgnoreList21_1}, } var hibernateSpatialBlocklists = blocklistsForVersion{ @@ -32,6 +32,37 @@ var hibernateSpatialBlockList21_1 = blocklist{} var hibernateBlockList21_1 = blocklist{} +var hibernateIgnoreList21_1 = blocklist{ + "org.hibernate.userguide.hql.HQLTest.test_collection_index_operator_example_1": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_abs_function_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_aggregate_functions_example_1": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_aggregate_functions_example_2": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_aggregate_functions_example_3": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_all_subquery_comparison_qualifier_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_basic_usage_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_list_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_named_query_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_parameter_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_parameter_inferred_type_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_parameter_short_form_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_positional_parameter_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_scroll_open_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_scroll_projection_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_stream_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_stream_projection_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_api_unique_result_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_between_predicate_example_1": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_between_predicate_example_3": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_between_predicate_example_4": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_cast_function_example": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_collection_expressions_example_1": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_collection_expressions_example_10": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_collection_expressions_example_2": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_collection_expressions_example_3": "unknown", + "org.hibernate.userguide.hql.HQLTest.test_hql_collection_expressions_example_4": "unknown", +} + var hibernateBlockList20_2 = blocklist{} var hibernateBlockList20_1 = blocklist{ From 5f2c2965b9b5759389079f2a731ad481ba22218d Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Thu, 15 Apr 2021 20:27:17 -0400 Subject: [PATCH 12/37] kvserver: improve test utilities for lease info The TestServer/TestCluster have facilities for querying a range's lease. This patch addresses two problems with this functionality: 1) If the node being queried has a lease proposal in flight, the respective functions would return it at the detriment of the current lease. This behavior is dubious, and indeed not what a test that I'm writing wants. This patch adds a function that returns both the current and the prospective next lease. 2) The functions let you ask for the state of a particular node to be queried. This was buggy, though, because the LeaseInfoRequest could be transparently forwarded to a different node in case the queried node doesn't have a replica or has a learner replica. That's confusing as hell. This patch optionally dissallows that, giving the caller confidence that they got the state from the node it was interested in. Release note: None --- pkg/kv/kvserver/batcheval/cmd_lease_info.go | 2 + pkg/kv/kvserver/client_replica_test.go | 17 +- pkg/roachpb/api.pb.go | 1149 ++++++++++--------- pkg/roachpb/api.proto | 17 +- pkg/server/testserver.go | 64 +- pkg/testutils/testcluster/testcluster.go | 33 +- 6 files changed, 734 insertions(+), 548 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_info.go b/pkg/kv/kvserver/batcheval/cmd_lease_info.go index 1cdcb9238645..722a227b8c1a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_info.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_info.go @@ -40,8 +40,10 @@ func LeaseInfo( // If there's a lease request in progress, speculatively return that future // lease. reply.Lease = nextLease + reply.CurrentLease = &lease } else { reply.Lease = lease } + reply.EvaluatedBy = cArgs.EvalCtx.StoreID() return result.Result{}, nil } diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 06b593b361c4..9e0e2dffb00f 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -1424,10 +1424,11 @@ func TestLeaseNotUsedAfterRestart(t *testing.T) { // Test that a lease extension (a RequestLeaseRequest that doesn't change the // lease holder) is not blocked by ongoing reads. The test relies on the fact // that RequestLeaseRequest does not declare to touch the whole key span of the -// range, and thus don't conflict through the command queue with other reads. +// range, and thus don't conflict through latches with other reads. func TestLeaseExtensionNotBlockedByRead(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + ctx := context.Background() readBlocked := make(chan struct{}) cmdFilter := func(fArgs kvserverbase.FilterArgs) *roachpb.Error { if fArgs.Hdr.UserPriority == 42 { @@ -1449,7 +1450,7 @@ func TestLeaseExtensionNotBlockedByRead(t *testing.T) { }, }) s := srv.(*server.TestServer) - defer s.Stopper().Stop(context.Background()) + defer s.Stopper().Stop(ctx) store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) if err != nil { @@ -1465,7 +1466,7 @@ func TestLeaseExtensionNotBlockedByRead(t *testing.T) { Key: key, }, } - if _, pErr := kv.SendWrappedWith(context.Background(), s.DB().NonTransactionalSender(), + if _, pErr := kv.SendWrappedWith(ctx, s.DB().NonTransactionalSender(), roachpb.Header{UserPriority: 42}, &getReq); pErr != nil { errChan <- pErr.GoError() @@ -1502,21 +1503,21 @@ func TestLeaseExtensionNotBlockedByRead(t *testing.T) { } for { - curLease, _, err := s.GetRangeLease(context.Background(), key) + leaseInfo, _, err := s.GetRangeLease(ctx, key, server.AllowQueryToBeForwardedToDifferentNode) if err != nil { t.Fatal(err) } - leaseReq.PrevLease = curLease + leaseReq.PrevLease = leaseInfo.CurrentOrProspective() - _, pErr := kv.SendWrapped(context.Background(), s.DB().NonTransactionalSender(), &leaseReq) + _, pErr := kv.SendWrapped(ctx, s.DB().NonTransactionalSender(), &leaseReq) if _, ok := pErr.GetDetail().(*roachpb.AmbiguousResultError); ok { - log.Infof(context.Background(), "retrying lease after %s", pErr) + log.Infof(ctx, "retrying lease after %s", pErr) continue } if _, ok := pErr.GetDetail().(*roachpb.LeaseRejectedError); ok { // Lease rejected? Try again. The extension should work because // extending is idempotent (assuming the PrevLease matches). - log.Infof(context.Background(), "retrying lease after %s", pErr) + log.Infof(ctx, "retrying lease after %s", pErr) continue } if pErr != nil { diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index e97cac76e34e..3149d6e89818 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -3298,7 +3298,9 @@ var xxx_messageInfo_TransferLeaseRequest proto.InternalMessageInfo // LeaseInfoRequest is the argument to the LeaseInfo() method, for getting // information about a range's lease. // It's a point request, so it addresses one single range, and returns the lease -// currently in effect for that range. +// currently in effect for that range. This request is commonly set with +// ReadConsistency=INCONSISTENT in order for the request to be served by the +// node to whom the request was sent. type LeaseInfoRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` } @@ -3338,6 +3340,19 @@ type LeaseInfoResponse struct { // The last lease known by the replica serving the request. It can also be the // tentative future lease, if a lease transfer is in progress. Lease Lease `protobuf:"bytes,2,opt,name=lease,proto3" json:"lease"` + // current_lease is set if `lease` represents a tentative future lease. In + // that case, current_lease represents the lease that's currently in effect. + CurrentLease *Lease `protobuf:"bytes,3,opt,name=current_lease,json=currentLease,proto3" json:"current_lease,omitempty"` + // evaluated_by returns the store that evaluated this request. This + // corresponds to the leaseholder unless ReadConsistency=INCONSISTENT was + // used. The response reflects the evaluator's view of the lease. When the + // client cares to see a particular node's view, it can use this field to + // check whether the node it intended query (by sending the request to that + // node and using ReadConsistency=INCONSISTENT) indeed served it - it's + // possible that even if ReadConsistency=INCONSISTENT was used, the request is + // still not evaluated by the node it was sent to if that node's replica is a + // learner or the node doesn't have a replica at all. + EvaluatedBy StoreID `protobuf:"varint,4,opt,name=evaluated_by,json=evaluatedBy,proto3,casttype=StoreID" json:"evaluated_by,omitempty"` } func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } @@ -7311,528 +7326,529 @@ func init() { func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_e08772acc330f58b) } var fileDescriptor_e08772acc330f58b = []byte{ - // 8322 bytes of a gzipped FileDescriptorProto + // 8352 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0xc9, - 0x76, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0x50, 0xa2, 0x5a, 0xa5, 0xf9, 0xe1, 0x70, 0x77, 0x47, 0x33, - 0x3d, 0xff, 0x73, 0x77, 0xa9, 0x9d, 0x99, 0xbb, 0xd9, 0xf5, 0xce, 0x7a, 0xaf, 0x25, 0x8a, 0x33, - 0xa4, 0x34, 0xd2, 0x68, 0x9a, 0xd4, 0x0c, 0x76, 0x7d, 0x9d, 0x76, 0xab, 0xbb, 0x44, 0xf5, 0x15, - 0xd9, 0xcd, 0xe9, 0x6e, 0xea, 0x67, 0x81, 0x00, 0x71, 0x6c, 0x38, 0xf7, 0xc9, 0xb8, 0x01, 0x0c, - 0xf8, 0x1a, 0x0e, 0xe2, 0xeb, 0x5c, 0x23, 0x7e, 0x08, 0x90, 0x04, 0x48, 0x90, 0x3f, 0x24, 0xf6, - 0x4b, 0x80, 0x5c, 0x04, 0x4e, 0x7c, 0xfd, 0x66, 0x04, 0x88, 0x62, 0xeb, 0xe6, 0x21, 0x81, 0x83, - 0x20, 0xc8, 0x8b, 0x81, 0x7d, 0x08, 0x82, 0xfa, 0xe9, 0x3f, 0xb2, 0x49, 0x51, 0xb3, 0x7d, 0xe3, - 0x05, 0xfc, 0x22, 0xb1, 0x4f, 0xd5, 0x39, 0x5d, 0x75, 0xaa, 0xea, 0xd4, 0xf9, 0xaa, 0x4f, 0x55, - 0xc1, 0xbc, 0x6d, 0xa9, 0xda, 0x5e, 0x77, 0x67, 0x49, 0xed, 0x1a, 0xe5, 0xae, 0x6d, 0xb9, 0x16, - 0x9a, 0xd7, 0x2c, 0x6d, 0x9f, 0x92, 0xcb, 0x3c, 0xb1, 0x74, 0x7f, 0xff, 0x60, 0x69, 0xff, 0xc0, - 0xc1, 0xf6, 0x01, 0xb6, 0x97, 0x34, 0xcb, 0xd4, 0x7a, 0xb6, 0x8d, 0x4d, 0xed, 0x78, 0xa9, 0x6d, - 0x69, 0xfb, 0xf4, 0x8f, 0x61, 0xb6, 0x18, 0x7b, 0x34, 0xaf, 0x8d, 0x55, 0xdd, 0xe9, 0x75, 0x3a, - 0xaa, 0x7d, 0xbc, 0x64, 0x3b, 0xdd, 0x9d, 0x25, 0xfe, 0xc0, 0xf3, 0x22, 0xef, 0xed, 0xba, 0xea, - 0xaa, 0x9c, 0x76, 0xc1, 0xa3, 0x61, 0xdb, 0xb6, 0x6c, 0x87, 0x53, 0x2f, 0x79, 0xd4, 0x0e, 0x76, - 0xd5, 0x50, 0xee, 0xb7, 0x1c, 0xd7, 0xb2, 0xd5, 0x16, 0x5e, 0xc2, 0x66, 0xcb, 0x30, 0x31, 0xc9, - 0x70, 0xa0, 0x69, 0x3c, 0xf1, 0xed, 0xd8, 0xc4, 0x47, 0x3c, 0xb5, 0xd8, 0x73, 0x8d, 0xf6, 0xd2, - 0x5e, 0x5b, 0x5b, 0x72, 0x8d, 0x0e, 0x76, 0x5c, 0xb5, 0xd3, 0xf5, 0xaa, 0x40, 0x53, 0x5c, 0x5b, - 0xd5, 0x0c, 0xb3, 0xe5, 0xfd, 0xef, 0xee, 0x2c, 0xd9, 0x58, 0xb3, 0x6c, 0x1d, 0xeb, 0x8a, 0xd3, - 0x55, 0x4d, 0xaf, 0xb8, 0x2d, 0xab, 0x65, 0xd1, 0x9f, 0x4b, 0xe4, 0x17, 0xa7, 0x5e, 0x6d, 0x59, - 0x56, 0xab, 0x8d, 0x97, 0xe8, 0xd3, 0x4e, 0x6f, 0x77, 0x49, 0xef, 0xd9, 0xaa, 0x6b, 0x58, 0x9c, - 0x4b, 0xfa, 0xe7, 0x02, 0xcc, 0xca, 0xf8, 0x75, 0x0f, 0x3b, 0x6e, 0x0d, 0xab, 0x3a, 0xb6, 0xd1, - 0x15, 0x48, 0xef, 0xe3, 0xe3, 0x62, 0xfa, 0x9a, 0x70, 0x77, 0x66, 0x65, 0xfa, 0xcb, 0x93, 0xc5, - 0xf4, 0x3a, 0x3e, 0x96, 0x09, 0x0d, 0x5d, 0x83, 0x69, 0x6c, 0xea, 0x0a, 0x49, 0xce, 0x44, 0x93, - 0xa7, 0xb0, 0xa9, 0xaf, 0xe3, 0x63, 0xf4, 0x6d, 0xc8, 0x3a, 0x44, 0x9a, 0xa9, 0xe1, 0xe2, 0xe4, - 0x35, 0xe1, 0xee, 0xe4, 0xca, 0xcf, 0x7d, 0x79, 0xb2, 0xf8, 0x49, 0xcb, 0x70, 0xf7, 0x7a, 0x3b, - 0x65, 0xcd, 0xea, 0x2c, 0xf9, 0x6d, 0xaa, 0xef, 0x04, 0xbf, 0x97, 0xba, 0xfb, 0xad, 0xa5, 0x7e, - 0x1d, 0x95, 0x9b, 0x47, 0x66, 0x03, 0xbf, 0x96, 0x7d, 0x89, 0x6b, 0x99, 0xac, 0x20, 0xa6, 0xd6, - 0x32, 0xd9, 0x94, 0x98, 0x96, 0xfe, 0x28, 0x05, 0x05, 0x19, 0x3b, 0x5d, 0xcb, 0x74, 0x30, 0x2f, - 0xf9, 0xfb, 0x90, 0x76, 0x8f, 0x4c, 0x5a, 0xf2, 0xfc, 0xc3, 0xab, 0xe5, 0x81, 0xde, 0x53, 0x6e, - 0xda, 0xaa, 0xe9, 0xa8, 0x1a, 0xa9, 0xbe, 0x4c, 0xb2, 0xa2, 0x8f, 0x20, 0x6f, 0x63, 0xa7, 0xd7, - 0xc1, 0x54, 0x91, 0xb4, 0x52, 0xf9, 0x87, 0x97, 0x63, 0x38, 0x1b, 0x5d, 0xd5, 0x94, 0x81, 0xe5, - 0x25, 0xbf, 0x51, 0x03, 0x66, 0x39, 0xa7, 0x8d, 0x55, 0xc7, 0x32, 0x8b, 0xd3, 0xd7, 0x84, 0xbb, - 0x85, 0x87, 0xe5, 0x18, 0xde, 0x68, 0x29, 0xc9, 0x63, 0xaf, 0x83, 0x65, 0xca, 0x25, 0xcf, 0xd8, - 0xa1, 0x27, 0x74, 0x05, 0xb2, 0x66, 0xaf, 0x43, 0xf4, 0xeb, 0x50, 0xed, 0xa5, 0xe5, 0x69, 0xb3, - 0xd7, 0x59, 0xc7, 0xc7, 0x0e, 0x7a, 0x0b, 0x72, 0x24, 0x69, 0xe7, 0xd8, 0xc5, 0x4e, 0x31, 0x4b, - 0xd3, 0x48, 0xde, 0x15, 0xf2, 0x2c, 0x7d, 0x0a, 0x33, 0x61, 0xa9, 0x08, 0x41, 0x41, 0xae, 0x36, - 0xb6, 0x37, 0xaa, 0xca, 0xf6, 0xe6, 0xfa, 0xe6, 0xf3, 0x57, 0x9b, 0xe2, 0x04, 0xba, 0x00, 0x22, - 0xa7, 0xad, 0x57, 0x3f, 0x53, 0x9e, 0xd5, 0x37, 0xea, 0x4d, 0x51, 0x28, 0x65, 0xbe, 0xfb, 0xc3, - 0xab, 0x13, 0x6b, 0x99, 0xec, 0x94, 0x38, 0x2d, 0xfd, 0x50, 0x00, 0x78, 0x8a, 0x5d, 0xde, 0x1b, - 0xd0, 0x0a, 0x4c, 0xed, 0xd1, 0x12, 0x17, 0x05, 0xaa, 0x96, 0x6b, 0xb1, 0x55, 0x0b, 0xf5, 0x9c, - 0x95, 0xec, 0x8f, 0x4e, 0x16, 0x27, 0x7e, 0x7c, 0xb2, 0x28, 0xc8, 0x9c, 0x13, 0xbd, 0x80, 0xfc, - 0x3e, 0x3e, 0x56, 0xf8, 0xb8, 0x2c, 0xa6, 0xa8, 0x8e, 0xde, 0x0f, 0x09, 0xda, 0x3f, 0x28, 0x7b, - 0x43, 0xb4, 0x1c, 0x1a, 0xce, 0x65, 0xc2, 0x51, 0x6e, 0xb8, 0x36, 0x36, 0x5b, 0xee, 0x9e, 0x0c, - 0xfb, 0xf8, 0xf8, 0x19, 0x93, 0x21, 0xfd, 0x81, 0x00, 0x79, 0x5a, 0x4a, 0xa6, 0x54, 0x54, 0xe9, - 0x2b, 0xe6, 0xf5, 0x33, 0x5b, 0x20, 0xa6, 0x9c, 0x65, 0x98, 0x3c, 0x50, 0xdb, 0x3d, 0x4c, 0x4b, - 0x98, 0x7f, 0x58, 0x8c, 0x91, 0xf1, 0x92, 0xa4, 0xcb, 0x2c, 0x1b, 0x7a, 0x0c, 0x33, 0x86, 0xe9, - 0x62, 0xd3, 0x55, 0x18, 0x5b, 0xfa, 0x0c, 0xb6, 0x3c, 0xcb, 0x4d, 0x1f, 0xa4, 0x7f, 0x26, 0x00, - 0x6c, 0xf5, 0x12, 0xd5, 0xf3, 0x37, 0xc7, 0x2c, 0xff, 0x4a, 0x86, 0xb0, 0x7a, 0xb5, 0xb8, 0x04, - 0x53, 0x86, 0xd9, 0x36, 0x4c, 0x56, 0xfe, 0xac, 0xcc, 0x9f, 0xd0, 0x05, 0x98, 0xdc, 0x69, 0x1b, - 0xa6, 0x4e, 0xc7, 0x43, 0x56, 0x66, 0x0f, 0x92, 0x0c, 0x79, 0x5a, 0xea, 0x04, 0xf5, 0x2e, 0x9d, - 0xa4, 0xe0, 0x62, 0xc5, 0x32, 0x75, 0x83, 0x0c, 0x49, 0xb5, 0xfd, 0xb5, 0xd0, 0xca, 0x1a, 0x5c, - 0xd0, 0x71, 0xd7, 0xc6, 0x9a, 0xea, 0x62, 0x5d, 0xc1, 0x47, 0xdd, 0x31, 0xdb, 0x18, 0x05, 0x5c, - 0xd5, 0xa3, 0x2e, 0xa5, 0x91, 0x51, 0x4b, 0x04, 0xb0, 0x51, 0x3b, 0x45, 0x4c, 0xa6, 0x9c, 0xc5, - 0x47, 0x5d, 0x3a, 0x6a, 0xe3, 0xd5, 0x8c, 0xbe, 0x09, 0x97, 0xd5, 0x76, 0xdb, 0x3a, 0x54, 0x8c, - 0x5d, 0x45, 0xb7, 0xb0, 0xa3, 0x98, 0x96, 0xab, 0xe0, 0x23, 0xc3, 0x71, 0xa9, 0x49, 0xc8, 0xca, - 0x0b, 0x34, 0xb9, 0xbe, 0xbb, 0x6a, 0x61, 0x67, 0xd3, 0x72, 0xab, 0x24, 0x29, 0xd4, 0x94, 0xd3, - 0xe1, 0xa6, 0x94, 0x7e, 0x01, 0x2e, 0xf5, 0xeb, 0x37, 0xc9, 0xf6, 0xfb, 0x43, 0x01, 0x0a, 0x75, - 0xd3, 0x70, 0xbf, 0x16, 0x0d, 0xe7, 0xeb, 0x33, 0x1d, 0xd6, 0xe7, 0x7d, 0x10, 0x77, 0x55, 0xa3, - 0xfd, 0xdc, 0x6c, 0x5a, 0x9d, 0x1d, 0xc7, 0xb5, 0x4c, 0xec, 0x70, 0x85, 0x0f, 0xd0, 0xa5, 0x97, - 0x30, 0xe7, 0xd7, 0x26, 0x49, 0x35, 0xb9, 0x20, 0xd6, 0x4d, 0xcd, 0xc6, 0x1d, 0x6c, 0x26, 0xaa, - 0xa7, 0xb7, 0x21, 0x67, 0x78, 0x72, 0xa9, 0xae, 0xd2, 0x72, 0x40, 0x90, 0x7a, 0x30, 0x1f, 0x7a, - 0x6b, 0x92, 0xe6, 0x92, 0x4c, 0x46, 0xf8, 0x50, 0x09, 0xda, 0x88, 0x4c, 0x46, 0xf8, 0x90, 0x99, - 0xb7, 0x06, 0xcc, 0xae, 0xe2, 0x36, 0x76, 0x71, 0x82, 0x35, 0x95, 0xb6, 0xa1, 0xe0, 0x09, 0x4d, - 0xb2, 0x61, 0x7e, 0x43, 0x00, 0xc4, 0xe5, 0xaa, 0x66, 0x2b, 0xc9, 0x12, 0xa3, 0x45, 0xe2, 0x5a, - 0xb8, 0x3d, 0xdb, 0x64, 0xd3, 0x39, 0xeb, 0x93, 0xc0, 0x48, 0x74, 0x46, 0x0f, 0x86, 0x6c, 0x26, - 0x3c, 0x64, 0xb9, 0x7b, 0x73, 0x08, 0x0b, 0x91, 0x82, 0x25, 0xdb, 0x7c, 0x19, 0x5a, 0xa6, 0xd4, - 0xb5, 0x74, 0xd8, 0x87, 0xa3, 0x44, 0xe9, 0xfb, 0x02, 0xcc, 0x57, 0xda, 0x58, 0xb5, 0x13, 0xd7, - 0xc8, 0xb7, 0x20, 0xab, 0x63, 0x55, 0xa7, 0x55, 0x66, 0x03, 0xfb, 0x9d, 0x90, 0x14, 0xe2, 0xe9, - 0x96, 0xf7, 0xda, 0x5a, 0xb9, 0xe9, 0xf9, 0xc0, 0x7c, 0x74, 0xfb, 0x4c, 0xd2, 0x67, 0x80, 0xc2, - 0x25, 0x4b, 0xb2, 0x23, 0xfc, 0x6e, 0x0a, 0x90, 0x8c, 0x0f, 0xb0, 0xed, 0x26, 0x5e, 0xed, 0x55, - 0xc8, 0xbb, 0xaa, 0xdd, 0xc2, 0xae, 0x42, 0xbc, 0xfb, 0xf3, 0xd4, 0x1c, 0x18, 0x1f, 0x21, 0xa3, - 0x26, 0xdc, 0xc1, 0xa6, 0xba, 0xd3, 0xc6, 0x54, 0x8a, 0xb2, 0x63, 0xf5, 0x4c, 0x5d, 0x31, 0x5c, - 0x6c, 0xab, 0xae, 0x65, 0x2b, 0x56, 0xd7, 0x35, 0x3a, 0xc6, 0x17, 0xd4, 0xb1, 0xe7, 0x5d, 0xed, - 0x06, 0xcb, 0x4e, 0x98, 0x57, 0x48, 0xe6, 0x3a, 0xcf, 0xfb, 0x3c, 0x94, 0x15, 0x95, 0x61, 0xc1, - 0x68, 0x99, 0x96, 0x8d, 0x95, 0x96, 0xa6, 0xb8, 0x7b, 0x36, 0x76, 0xf6, 0xac, 0xb6, 0x37, 0x21, - 0xcd, 0xb3, 0xa4, 0xa7, 0x5a, 0xd3, 0x4b, 0x90, 0x3e, 0x87, 0x85, 0x88, 0x96, 0x92, 0x6c, 0x82, - 0xff, 0x2d, 0x40, 0xbe, 0xa1, 0xa9, 0x66, 0x92, 0xba, 0xff, 0x14, 0xf2, 0x8e, 0xa6, 0x9a, 0xca, - 0xae, 0x65, 0x77, 0x54, 0x97, 0xd6, 0xab, 0x10, 0xd1, 0xbd, 0xef, 0xdf, 0x6b, 0xaa, 0xf9, 0x84, - 0x66, 0x92, 0xc1, 0xf1, 0x7f, 0xf7, 0xfb, 0xaf, 0x93, 0x5f, 0xdd, 0x7f, 0x65, 0xc3, 0x7b, 0x2d, - 0x93, 0x4d, 0x8b, 0x19, 0xe9, 0x2f, 0x04, 0x98, 0x61, 0x55, 0x4e, 0x72, 0x78, 0x7f, 0x00, 0x19, - 0xdb, 0x3a, 0x64, 0xc3, 0x3b, 0xff, 0xf0, 0xad, 0x18, 0x11, 0xeb, 0xf8, 0x38, 0x3c, 0x7f, 0xd2, - 0xec, 0x68, 0x05, 0xb8, 0x97, 0xaa, 0x50, 0xee, 0xf4, 0xb8, 0xdc, 0xc0, 0xb8, 0x64, 0x22, 0xe3, - 0x0e, 0xcc, 0xed, 0xa8, 0xae, 0xb6, 0xa7, 0xd8, 0xbc, 0x90, 0x64, 0xae, 0x4d, 0xdf, 0x9d, 0x91, - 0x0b, 0x94, 0xec, 0x15, 0xdd, 0x21, 0x35, 0x67, 0xe3, 0xcd, 0xc1, 0x7f, 0xc5, 0xda, 0xfc, 0xff, - 0x0a, 0x7c, 0x0c, 0x79, 0x35, 0xff, 0xab, 0xd6, 0xf4, 0xbf, 0x99, 0x82, 0xcb, 0x95, 0x3d, 0xac, - 0xed, 0x57, 0x2c, 0xd3, 0x31, 0x1c, 0x97, 0xe8, 0x2e, 0xc9, 0xf6, 0x7f, 0x0b, 0x72, 0x87, 0x86, - 0xbb, 0xa7, 0xe8, 0xc6, 0xee, 0x2e, 0xb5, 0xb6, 0x59, 0x39, 0x4b, 0x08, 0xab, 0xc6, 0xee, 0x2e, - 0x7a, 0x04, 0x99, 0x8e, 0xa5, 0x33, 0x67, 0xbe, 0xf0, 0x70, 0x31, 0x46, 0x3c, 0x2d, 0x9a, 0xd3, - 0xeb, 0x6c, 0x58, 0x3a, 0x96, 0x69, 0x66, 0x74, 0x15, 0x40, 0x23, 0xd4, 0xae, 0x65, 0x98, 0x2e, - 0x37, 0x8e, 0x21, 0x0a, 0xaa, 0x41, 0xce, 0xc5, 0x76, 0xc7, 0x30, 0x55, 0x17, 0x17, 0x27, 0xa9, - 0xf2, 0x6e, 0xc6, 0x16, 0xbc, 0xdb, 0x36, 0x34, 0x75, 0x15, 0x3b, 0x9a, 0x6d, 0x74, 0x5d, 0xcb, - 0xe6, 0x5a, 0x0c, 0x98, 0xa5, 0x5f, 0xcb, 0x40, 0x71, 0x50, 0x37, 0x49, 0xf6, 0x90, 0x2d, 0x98, - 0xb2, 0xb1, 0xd3, 0x6b, 0xbb, 0xbc, 0x8f, 0x3c, 0x1c, 0xa6, 0x82, 0x98, 0x12, 0xd0, 0xa5, 0x8b, - 0xb6, 0xcb, 0x8b, 0xcd, 0xe5, 0x94, 0xfe, 0x8d, 0x00, 0x53, 0x2c, 0x01, 0x3d, 0x80, 0xac, 0x4d, - 0x26, 0x06, 0xc5, 0xd0, 0x69, 0x19, 0xd3, 0x2b, 0x97, 0x4e, 0x4f, 0x16, 0xa7, 0xe9, 0x64, 0x51, - 0x5f, 0xfd, 0x32, 0xf8, 0x29, 0x4f, 0xd3, 0x7c, 0x75, 0x9d, 0xb4, 0x96, 0xe3, 0xaa, 0xb6, 0x4b, - 0x17, 0x95, 0x52, 0x0c, 0x21, 0x51, 0xc2, 0x3a, 0x3e, 0x46, 0x6b, 0x30, 0xe5, 0xb8, 0xaa, 0xdb, - 0x73, 0x78, 0x7b, 0x9d, 0xab, 0xb0, 0x0d, 0xca, 0x29, 0x73, 0x09, 0xc4, 0xdd, 0xd2, 0xb1, 0xab, - 0x1a, 0x6d, 0xda, 0x80, 0x39, 0x99, 0x3f, 0x49, 0xbf, 0x25, 0xc0, 0x14, 0xcb, 0x8a, 0x2e, 0xc3, - 0x82, 0xbc, 0xbc, 0xf9, 0xb4, 0xaa, 0xd4, 0x37, 0x57, 0xab, 0xcd, 0xaa, 0xbc, 0x51, 0xdf, 0x5c, - 0x6e, 0x56, 0xc5, 0x09, 0x74, 0x09, 0x90, 0x97, 0x50, 0x79, 0xbe, 0xd9, 0xa8, 0x37, 0x9a, 0xd5, - 0xcd, 0xa6, 0x28, 0xd0, 0x35, 0x15, 0x4a, 0x0f, 0x51, 0x53, 0xe8, 0x26, 0x5c, 0xeb, 0xa7, 0x2a, - 0x8d, 0xe6, 0x72, 0xb3, 0xa1, 0x54, 0x1b, 0xcd, 0xfa, 0xc6, 0x72, 0xb3, 0xba, 0x2a, 0xa6, 0x47, - 0xe4, 0x22, 0x2f, 0x91, 0xe5, 0x6a, 0xa5, 0x29, 0x66, 0x24, 0x17, 0x2e, 0xca, 0x58, 0xb3, 0x3a, - 0xdd, 0x9e, 0x8b, 0x49, 0x29, 0x9d, 0x24, 0x47, 0xca, 0x65, 0x98, 0xd6, 0xed, 0x63, 0xc5, 0xee, - 0x99, 0x7c, 0x9c, 0x4c, 0xe9, 0xf6, 0xb1, 0xdc, 0x33, 0xa5, 0x7f, 0x22, 0xc0, 0xa5, 0xfe, 0xd7, - 0x26, 0xd9, 0x09, 0x5f, 0x40, 0x5e, 0xd5, 0x75, 0xac, 0x2b, 0x3a, 0x6e, 0xbb, 0x2a, 0x77, 0x89, - 0xee, 0x87, 0x24, 0xf1, 0xa5, 0xc0, 0xb2, 0xbf, 0x14, 0xb8, 0xf1, 0xb2, 0x52, 0xa1, 0x05, 0x59, - 0x25, 0x1c, 0x9e, 0xf9, 0xa1, 0x42, 0x28, 0x45, 0xfa, 0x9f, 0x19, 0x98, 0xad, 0x9a, 0x7a, 0xf3, - 0x28, 0xd1, 0xb9, 0xe4, 0x12, 0x4c, 0x69, 0x56, 0xa7, 0x63, 0xb8, 0x9e, 0x82, 0xd8, 0x13, 0xfa, - 0x99, 0x90, 0x2b, 0x9b, 0x1e, 0xc3, 0xa1, 0x0b, 0x9c, 0x58, 0xf4, 0x8b, 0x70, 0x99, 0x58, 0x4d, - 0xdb, 0x54, 0xdb, 0x0a, 0x93, 0xa6, 0xb8, 0xb6, 0xd1, 0x6a, 0x61, 0x9b, 0x2f, 0x3f, 0xde, 0x8d, - 0x29, 0x67, 0x9d, 0x73, 0x54, 0x28, 0x43, 0x93, 0xe5, 0x97, 0x2f, 0x1a, 0x71, 0x64, 0xf4, 0x09, - 0x00, 0x99, 0x8a, 0xe8, 0x92, 0xa6, 0xc3, 0xed, 0xd1, 0xb0, 0x35, 0x4d, 0xcf, 0x04, 0x11, 0x06, - 0xf2, 0xec, 0xa0, 0x17, 0x20, 0x1a, 0xa6, 0xb2, 0xdb, 0x36, 0x5a, 0x7b, 0xae, 0x72, 0x68, 0x1b, - 0x2e, 0x76, 0x8a, 0xf3, 0x54, 0x46, 0x5c, 0x53, 0x37, 0xf8, 0xd2, 0xac, 0xfe, 0x8a, 0xe4, 0xe4, - 0xd2, 0x0a, 0x86, 0xf9, 0x84, 0xf2, 0x53, 0xa2, 0x83, 0x96, 0x08, 0x14, 0x7a, 0xdd, 0x33, 0x6c, - 0xac, 0x3c, 0xe8, 0x6a, 0x74, 0x1d, 0x24, 0xbb, 0x52, 0x38, 0x3d, 0x59, 0x04, 0x99, 0x91, 0x1f, - 0x6c, 0x55, 0x08, 0x34, 0x62, 0xbf, 0xbb, 0x1a, 0x7a, 0x05, 0xf7, 0x42, 0x4b, 0x30, 0x64, 0x32, - 0xe7, 0x9a, 0x52, 0x5d, 0x65, 0xcf, 0x68, 0xed, 0x61, 0x5b, 0xf1, 0x57, 0xca, 0xe9, 0x62, 0x68, - 0x56, 0xbe, 0x19, 0x30, 0x54, 0x54, 0x93, 0x29, 0x64, 0xd9, 0xad, 0xd1, 0xcc, 0x7e, 0x33, 0x90, - 0xf6, 0xec, 0x5a, 0x86, 0x63, 0x99, 0xc5, 0x1c, 0x6b, 0x4f, 0xf6, 0x84, 0xee, 0x81, 0xe8, 0x1e, - 0x99, 0xca, 0x1e, 0x56, 0x6d, 0x77, 0x07, 0xab, 0x2e, 0x99, 0xf8, 0x81, 0xe6, 0x98, 0x73, 0x8f, - 0xcc, 0x5a, 0x88, 0xbc, 0x96, 0xc9, 0x4e, 0x8b, 0x59, 0xe9, 0xbf, 0x08, 0x50, 0xf0, 0xba, 0x5b, - 0x92, 0x23, 0xe3, 0x2e, 0x88, 0x96, 0x89, 0x95, 0xee, 0x9e, 0xea, 0x60, 0x5e, 0x69, 0x3e, 0xe1, - 0x14, 0x2c, 0x13, 0x6f, 0x11, 0x32, 0xab, 0x1b, 0xda, 0x82, 0x79, 0xc7, 0x55, 0x5b, 0x86, 0xd9, - 0x0a, 0xe9, 0x62, 0x72, 0x7c, 0x70, 0x21, 0x72, 0x6e, 0x9f, 0x1e, 0xf1, 0x52, 0xfe, 0x58, 0x80, - 0xf9, 0x65, 0xbd, 0x63, 0x98, 0x8d, 0x6e, 0xdb, 0x48, 0x74, 0xcd, 0xe2, 0x26, 0xe4, 0x1c, 0x22, - 0x33, 0x30, 0xf8, 0x01, 0x02, 0xcd, 0xd2, 0x14, 0x62, 0xf9, 0x9f, 0xc1, 0x1c, 0x3e, 0xea, 0x1a, - 0xec, 0x53, 0x05, 0x03, 0x4e, 0x99, 0xf1, 0xeb, 0x56, 0x08, 0x78, 0x49, 0x12, 0xaf, 0xd3, 0x67, - 0x80, 0xc2, 0x55, 0x4a, 0x12, 0xbb, 0x7c, 0x06, 0x0b, 0x54, 0xf4, 0xb6, 0xe9, 0x24, 0xac, 0x2f, - 0xe9, 0xe7, 0xe1, 0x42, 0x54, 0x74, 0x92, 0xe5, 0x7e, 0xc5, 0x5b, 0x79, 0x03, 0xdb, 0x89, 0x82, - 0x5e, 0x5f, 0xd7, 0x5c, 0x70, 0x92, 0x65, 0xfe, 0x15, 0x01, 0xae, 0x50, 0xd9, 0xf4, 0x6b, 0xce, - 0x2e, 0xb6, 0x9f, 0x61, 0xd5, 0x49, 0x14, 0xb1, 0xdf, 0x80, 0x29, 0x86, 0xbc, 0x69, 0xff, 0x9c, - 0x5c, 0xc9, 0x13, 0xcf, 0xa5, 0xe1, 0x5a, 0x36, 0xf1, 0x5c, 0x78, 0x92, 0xa4, 0x42, 0x29, 0xae, - 0x14, 0x49, 0xd6, 0xf4, 0xef, 0x09, 0x30, 0xcf, 0x9d, 0x46, 0xd2, 0x95, 0x2b, 0x7b, 0xc4, 0x67, - 0x42, 0x55, 0xc8, 0x6b, 0xf4, 0x97, 0xe2, 0x1e, 0x77, 0x31, 0x95, 0x5f, 0x18, 0xe5, 0x6f, 0x32, - 0xb6, 0xe6, 0x71, 0x17, 0x13, 0xa7, 0xd5, 0xfb, 0x4d, 0x14, 0x15, 0xaa, 0xe4, 0x48, 0x8f, 0x95, - 0x8e, 0x23, 0x9a, 0xd7, 0x73, 0xfd, 0xb8, 0x0e, 0xfe, 0x69, 0x9a, 0x2b, 0x81, 0xbd, 0x83, 0x67, - 0x4f, 0xd4, 0x47, 0xf9, 0x1c, 0x2e, 0x85, 0xa7, 0x82, 0x50, 0xc5, 0x53, 0xe7, 0xa8, 0x78, 0x68, - 0x45, 0x3f, 0xa0, 0xa2, 0xcf, 0x20, 0xb4, 0x66, 0xaf, 0xb0, 0x3a, 0x79, 0xe8, 0xe7, 0x3c, 0xea, - 0x98, 0x0f, 0xa4, 0x30, 0xba, 0x83, 0x2a, 0x90, 0xc5, 0x47, 0x5d, 0x45, 0xc7, 0x8e, 0xc6, 0x0d, - 0x97, 0x14, 0x27, 0x90, 0x14, 0x65, 0x00, 0x0f, 0x4c, 0xe3, 0xa3, 0x2e, 0x21, 0xa2, 0x6d, 0x32, - 0x15, 0x7b, 0xae, 0x02, 0x2d, 0xb6, 0x73, 0x36, 0xbc, 0x08, 0x7a, 0x0a, 0x17, 0x37, 0xe7, 0x7b, - 0x09, 0x4c, 0x84, 0xf4, 0x03, 0x01, 0xde, 0x8a, 0x6d, 0xb5, 0x24, 0x27, 0xb2, 0x4f, 0x20, 0x43, - 0x2b, 0x9f, 0x3a, 0x67, 0xe5, 0x29, 0x97, 0xf4, 0xdd, 0x14, 0x1f, 0xe3, 0x32, 0x6e, 0x5b, 0x44, - 0xb1, 0x89, 0xaf, 0xca, 0x3d, 0x87, 0xd9, 0x03, 0xcb, 0x25, 0x8e, 0x04, 0x6f, 0xf6, 0xd4, 0xb9, - 0x9b, 0x7d, 0x86, 0x0a, 0xf0, 0x5a, 0xfc, 0x25, 0xcc, 0x9b, 0x96, 0xa9, 0x44, 0x85, 0x9e, 0xbf, - 0x2f, 0xcd, 0x99, 0x96, 0xf9, 0x32, 0x24, 0xd7, 0xb7, 0x33, 0x7d, 0x9a, 0x48, 0xd2, 0xce, 0x7c, - 0x4f, 0x80, 0x05, 0xdf, 0xc7, 0x49, 0xd8, 0x83, 0xfe, 0x00, 0xd2, 0xa6, 0x75, 0x78, 0x9e, 0x55, - 0x4f, 0x92, 0x9f, 0xcc, 0x7a, 0xd1, 0x12, 0x25, 0x59, 0xdf, 0x7f, 0x9b, 0x82, 0xdc, 0xd3, 0x4a, - 0x92, 0xb5, 0xfc, 0x84, 0xaf, 0xa8, 0xb3, 0xf6, 0x8e, 0xeb, 0xed, 0xfe, 0xfb, 0xca, 0x4f, 0x2b, - 0xeb, 0xf8, 0xd8, 0xeb, 0xed, 0x84, 0x0b, 0x2d, 0x43, 0x2e, 0xba, 0xf6, 0x3a, 0xa6, 0xa6, 0x02, - 0xae, 0x12, 0x86, 0x49, 0x2a, 0xd7, 0x8b, 0xde, 0x10, 0x62, 0xa2, 0x37, 0xc8, 0x6b, 0x7c, 0x4f, - 0x31, 0x75, 0x9e, 0xd7, 0x84, 0x5c, 0xc4, 0x49, 0x71, 0x4a, 0x7a, 0x01, 0x40, 0xaa, 0x93, 0x64, - 0x93, 0xfc, 0x6a, 0x1a, 0x0a, 0x5b, 0x3d, 0x67, 0x2f, 0xe1, 0xde, 0x57, 0x01, 0xe8, 0xf6, 0x1c, - 0x8a, 0x17, 0x8e, 0x4c, 0x5e, 0xe7, 0x33, 0x02, 0x43, 0xbc, 0x4a, 0x33, 0xbe, 0xe6, 0x91, 0x89, - 0x6a, 0x5c, 0x08, 0x56, 0x82, 0xe8, 0x92, 0x1b, 0xa3, 0xc0, 0x6a, 0xf3, 0xc8, 0xdc, 0xc0, 0x3e, - 0x4a, 0x65, 0x92, 0x30, 0x91, 0xf4, 0x09, 0x4c, 0x93, 0x07, 0xc5, 0xb5, 0xce, 0xd3, 0xcc, 0x53, - 0x84, 0xa7, 0x69, 0xa1, 0xc7, 0x90, 0x63, 0xdc, 0x64, 0xf6, 0x9b, 0xa2, 0xb3, 0x5f, 0x5c, 0x5d, - 0xb8, 0x1a, 0xe9, 0xbc, 0x97, 0xa5, 0xac, 0x64, 0xae, 0xbb, 0x00, 0x93, 0xbb, 0x96, 0xad, 0x79, - 0xdf, 0x87, 0xd9, 0x03, 0x6b, 0xcf, 0xb5, 0x4c, 0x36, 0x2b, 0xe6, 0xd6, 0x32, 0xd9, 0x9c, 0x08, - 0xd2, 0x6f, 0x09, 0x30, 0xe7, 0x37, 0x44, 0x92, 0x13, 0x42, 0x25, 0xa2, 0xc5, 0xf3, 0x37, 0x05, - 0x51, 0xa0, 0xf4, 0xef, 0xa9, 0x47, 0xa4, 0x59, 0x07, 0xb4, 0x65, 0x92, 0xec, 0x29, 0x8f, 0x59, - 0xec, 0x50, 0xea, 0xbc, 0xad, 0x4b, 0xc3, 0x88, 0x1e, 0xc0, 0x05, 0xa3, 0x43, 0xec, 0xb9, 0xe1, - 0xb6, 0x8f, 0x39, 0x6c, 0x73, 0xb1, 0xf7, 0x21, 0x7a, 0x21, 0x48, 0xab, 0x78, 0x49, 0xd2, 0xef, - 0xd2, 0x05, 0xf0, 0xa0, 0x26, 0x49, 0xaa, 0xba, 0x0e, 0xb3, 0x36, 0x13, 0x4d, 0xdc, 0x9a, 0x73, - 0x6a, 0x7b, 0xc6, 0x67, 0x25, 0x0a, 0xff, 0x9d, 0x14, 0xcc, 0xbd, 0xe8, 0x61, 0xfb, 0xf8, 0xeb, - 0xa4, 0xee, 0xdb, 0x30, 0x77, 0xa8, 0x1a, 0xae, 0xb2, 0x6b, 0xd9, 0x4a, 0xaf, 0xab, 0xab, 0xae, - 0x17, 0xc0, 0x32, 0x4b, 0xc8, 0x4f, 0x2c, 0x7b, 0x9b, 0x12, 0x11, 0x06, 0xb4, 0x6f, 0x5a, 0x87, - 0xa6, 0x42, 0xc8, 0x14, 0x28, 0x1f, 0x99, 0x7c, 0x55, 0x7a, 0xe5, 0xc3, 0xff, 0x7c, 0xb2, 0xf8, - 0x68, 0xac, 0xb0, 0x34, 0x1a, 0x82, 0xd7, 0xeb, 0x19, 0x7a, 0x79, 0x7b, 0xbb, 0xbe, 0x2a, 0x8b, - 0x54, 0xe4, 0x2b, 0x26, 0xb1, 0x79, 0x64, 0x3a, 0xd2, 0x3f, 0x48, 0x81, 0x18, 0xe8, 0x28, 0xc9, - 0x86, 0xac, 0x42, 0xfe, 0x75, 0x0f, 0xdb, 0xc6, 0x1b, 0x34, 0x23, 0x70, 0x46, 0x62, 0x76, 0xee, - 0xc3, 0xbc, 0x7b, 0x64, 0x2a, 0x2c, 0x68, 0x90, 0xc5, 0x92, 0x78, 0x31, 0x10, 0x73, 0x2e, 0x29, - 0x33, 0xa1, 0xd3, 0x38, 0x12, 0x07, 0x7d, 0x0e, 0x33, 0x11, 0x6d, 0xa5, 0xbf, 0x9a, 0xb6, 0xf2, - 0x87, 0x21, 0x45, 0xfd, 0x81, 0x00, 0x88, 0x2a, 0xaa, 0xce, 0x3e, 0x1b, 0x7c, 0x5d, 0xfa, 0xd3, - 0x5d, 0x10, 0x69, 0x88, 0xa7, 0x62, 0xec, 0x2a, 0x1d, 0xc3, 0x71, 0x0c, 0xb3, 0xc5, 0x3b, 0x54, - 0x81, 0xd2, 0xeb, 0xbb, 0x1b, 0x8c, 0x2a, 0xfd, 0x0d, 0x58, 0x88, 0x54, 0x20, 0xc9, 0xc6, 0xbe, - 0x0e, 0x33, 0xbb, 0xec, 0xab, 0x2e, 0x15, 0xce, 0x57, 0x1c, 0xf3, 0x94, 0xc6, 0xde, 0x27, 0xfd, - 0x79, 0x0a, 0x2e, 0xc8, 0xd8, 0xb1, 0xda, 0x07, 0x38, 0x79, 0x15, 0xd6, 0x80, 0x7f, 0xce, 0x51, - 0xde, 0x48, 0x93, 0x39, 0xc6, 0xcc, 0xa6, 0xb9, 0xe8, 0xb2, 0xfd, 0xcd, 0xd1, 0x3d, 0x76, 0x70, - 0xa1, 0x9e, 0xaf, 0xd1, 0x65, 0x22, 0x6b, 0x74, 0x16, 0xcc, 0xb1, 0x0f, 0xd2, 0xba, 0xe2, 0xe0, - 0xd7, 0x66, 0xaf, 0xe3, 0x81, 0xa1, 0xf2, 0xa8, 0x42, 0xd6, 0x19, 0x4b, 0x03, 0xbf, 0xde, 0xec, - 0x75, 0xa8, 0xef, 0xbc, 0x72, 0x89, 0x94, 0xf7, 0xf4, 0x64, 0xb1, 0x10, 0x49, 0x73, 0xe4, 0x82, - 0xe1, 0x3f, 0x13, 0xe9, 0xd2, 0xb7, 0xe1, 0x62, 0x9f, 0xb2, 0x93, 0xf4, 0x78, 0xfe, 0x75, 0x1a, - 0xae, 0x44, 0xc5, 0x27, 0x0d, 0x71, 0xbe, 0xee, 0x0d, 0x5a, 0x83, 0xd9, 0x8e, 0x61, 0xbe, 0xd9, - 0xea, 0xe5, 0x4c, 0xc7, 0x30, 0x83, 0x65, 0xdd, 0x98, 0xae, 0x31, 0xf5, 0x53, 0xed, 0x1a, 0x2a, - 0x94, 0xe2, 0xda, 0x2e, 0xc9, 0xfe, 0xf1, 0x5d, 0x01, 0x66, 0x92, 0x5e, 0x96, 0x7b, 0xb3, 0xc0, - 0x3a, 0xa9, 0x09, 0xb3, 0x3f, 0x85, 0x75, 0xbc, 0xdf, 0x11, 0x00, 0x35, 0xed, 0x9e, 0x49, 0x40, - 0xed, 0x33, 0xab, 0x95, 0x64, 0x35, 0x2f, 0xc0, 0xa4, 0x61, 0xea, 0xf8, 0x88, 0x56, 0x33, 0x23, - 0xb3, 0x87, 0xc8, 0xd7, 0xc9, 0xf4, 0x58, 0x5f, 0x27, 0xa5, 0xcf, 0x61, 0x21, 0x52, 0xc4, 0x24, - 0xeb, 0xff, 0x3f, 0x52, 0xb0, 0xc0, 0x2b, 0x92, 0xf8, 0x0a, 0xe6, 0x37, 0x61, 0xb2, 0x4d, 0x64, - 0x8e, 0x68, 0x67, 0xfa, 0x4e, 0xaf, 0x9d, 0x69, 0x66, 0xf4, 0xb3, 0x00, 0x5d, 0x1b, 0x1f, 0x28, - 0x8c, 0x35, 0x3d, 0x16, 0x6b, 0x8e, 0x70, 0x50, 0x02, 0xfa, 0xbe, 0x00, 0x73, 0x64, 0x40, 0x77, - 0x6d, 0xab, 0x6b, 0x39, 0xc4, 0x67, 0x71, 0xc6, 0x83, 0x39, 0x2f, 0x4e, 0x4f, 0x16, 0x67, 0x37, - 0x0c, 0x73, 0x8b, 0x33, 0x36, 0x1b, 0x63, 0xef, 0x19, 0xf0, 0x76, 0x4e, 0x94, 0x2b, 0x6d, 0x4b, - 0xdb, 0x0f, 0xbe, 0xb7, 0x11, 0xcb, 0xe2, 0x8b, 0x73, 0xa4, 0x3f, 0x12, 0xe0, 0xc2, 0x4f, 0x6d, - 0xb9, 0xf8, 0x2f, 0x43, 0xd9, 0xd2, 0x4b, 0x10, 0xe9, 0x8f, 0xba, 0xb9, 0x6b, 0x25, 0xb9, 0x70, - 0xff, 0x6b, 0x02, 0xcc, 0x87, 0x04, 0x27, 0xe9, 0xe0, 0xbc, 0x91, 0x9e, 0xa4, 0x9f, 0x27, 0x2e, - 0x4f, 0x78, 0x94, 0x24, 0x39, 0x06, 0xff, 0x45, 0x0a, 0x2e, 0x55, 0xd8, 0x67, 0x6e, 0x2f, 0xee, - 0x23, 0xc9, 0x9e, 0x51, 0x84, 0xe9, 0x03, 0x6c, 0x3b, 0x86, 0xc5, 0xa6, 0xdf, 0x59, 0xd9, 0x7b, - 0x44, 0x25, 0xc8, 0x3a, 0xa6, 0xda, 0x75, 0xf6, 0x2c, 0xef, 0xfb, 0x9e, 0xff, 0xec, 0xc7, 0xa8, - 0x4c, 0xbe, 0x79, 0x8c, 0xca, 0xd4, 0xe8, 0x18, 0x95, 0xe9, 0xaf, 0x10, 0xa3, 0xc2, 0x3f, 0xa6, - 0xfd, 0x07, 0x01, 0x2e, 0x0f, 0x68, 0x2e, 0xc9, 0xde, 0xf2, 0x1d, 0xc8, 0x6b, 0x5c, 0x30, 0x31, - 0xd8, 0xec, 0x4b, 0x61, 0x9d, 0x64, 0x7b, 0x43, 0x2c, 0x72, 0x7a, 0xb2, 0x08, 0x5e, 0x51, 0xeb, - 0xab, 0x5c, 0x39, 0xe4, 0xb7, 0x2e, 0xfd, 0xf2, 0x2c, 0xcc, 0x55, 0x8f, 0xd8, 0x2a, 0x79, 0x83, - 0xb9, 0x09, 0xe8, 0x09, 0x64, 0xbb, 0xb6, 0x75, 0x60, 0x78, 0xd5, 0x28, 0x44, 0x02, 0x14, 0xbc, - 0x6a, 0xf4, 0x71, 0x6d, 0x71, 0x0e, 0xd9, 0xe7, 0x45, 0x4d, 0xc8, 0x3d, 0xb3, 0x34, 0xb5, 0xfd, - 0xc4, 0x68, 0x7b, 0x3d, 0xff, 0xfd, 0xb3, 0x05, 0x95, 0x7d, 0x9e, 0x2d, 0xd5, 0xdd, 0xf3, 0x1a, - 0xc1, 0x27, 0xa2, 0x3a, 0x64, 0x6b, 0xae, 0xdb, 0x25, 0x89, 0xdc, 0x76, 0xdc, 0x19, 0x43, 0x28, - 0x61, 0xf1, 0xa2, 0x6a, 0x3d, 0x76, 0xd4, 0x84, 0xf9, 0xa7, 0x74, 0x8f, 0x58, 0xa5, 0x6d, 0xf5, - 0xf4, 0x8a, 0x65, 0xee, 0x1a, 0x2d, 0x6e, 0xb7, 0x6f, 0x8f, 0x21, 0xf3, 0x69, 0xa5, 0x21, 0x0f, - 0x0a, 0x40, 0xcb, 0x90, 0x6d, 0x3c, 0xe2, 0xc2, 0x98, 0x5f, 0x77, 0x6b, 0x0c, 0x61, 0x8d, 0x47, - 0xb2, 0xcf, 0x86, 0xd6, 0x20, 0xbf, 0xfc, 0x45, 0xcf, 0xc6, 0x5c, 0xca, 0xd4, 0xd0, 0xe8, 0x88, - 0x7e, 0x29, 0x94, 0x4b, 0x0e, 0x33, 0xa3, 0x06, 0x14, 0x5e, 0x59, 0xf6, 0x7e, 0xdb, 0x52, 0xbd, - 0x1a, 0x4e, 0x53, 0x71, 0xdf, 0x18, 0x43, 0x9c, 0xc7, 0x28, 0xf7, 0x89, 0x40, 0xdf, 0x86, 0x39, - 0xd2, 0x18, 0x4d, 0x75, 0xa7, 0xed, 0x15, 0x32, 0x4b, 0xa5, 0xbe, 0x3b, 0x86, 0x54, 0x9f, 0xd3, - 0x5b, 0xf8, 0xef, 0x13, 0x55, 0x92, 0x61, 0x36, 0xd2, 0x09, 0x10, 0x82, 0x4c, 0x97, 0xb4, 0xb7, - 0x40, 0xe3, 0x97, 0xe8, 0x6f, 0xf4, 0x1e, 0x4c, 0x9b, 0x96, 0x8e, 0xbd, 0x11, 0x32, 0xbb, 0x72, - 0xe1, 0xf4, 0x64, 0x71, 0x6a, 0xd3, 0xd2, 0x99, 0x47, 0xc3, 0x7f, 0xc9, 0x53, 0x24, 0x53, 0x5d, - 0x2f, 0x5d, 0x83, 0x0c, 0x69, 0x77, 0x62, 0x98, 0x76, 0x54, 0x07, 0x6f, 0xdb, 0x06, 0x97, 0xe6, - 0x3d, 0x96, 0xfe, 0x71, 0x0a, 0x52, 0x8d, 0x47, 0xc4, 0x67, 0xdf, 0xe9, 0x69, 0xfb, 0xd8, 0xe5, - 0xe9, 0xfc, 0x89, 0xfa, 0xf2, 0x36, 0xde, 0x35, 0x98, 0x6b, 0x95, 0x93, 0xf9, 0x13, 0x7a, 0x07, - 0x40, 0xd5, 0x34, 0xec, 0x38, 0x8a, 0xb7, 0x77, 0x30, 0x27, 0xe7, 0x18, 0x65, 0x1d, 0x1f, 0x13, - 0x36, 0x07, 0x6b, 0x36, 0x76, 0xbd, 0xe0, 0x2b, 0xf6, 0x44, 0xd8, 0x5c, 0xdc, 0xe9, 0x2a, 0xae, - 0xb5, 0x8f, 0x4d, 0xda, 0x4f, 0x72, 0xc4, 0xd4, 0x74, 0xba, 0x4d, 0x42, 0x20, 0x56, 0x12, 0x9b, - 0x7a, 0x60, 0xd2, 0x72, 0xb2, 0xff, 0x4c, 0x44, 0xda, 0xb8, 0x65, 0xf0, 0x9d, 0x77, 0x39, 0x99, - 0x3f, 0x11, 0x2d, 0xa9, 0x3d, 0x77, 0x8f, 0xb6, 0x44, 0x4e, 0xa6, 0xbf, 0xd1, 0x6d, 0x98, 0x63, - 0xf1, 0x9a, 0x0a, 0x36, 0x35, 0x85, 0x1a, 0xd7, 0x1c, 0x4d, 0x9e, 0x65, 0xe4, 0xaa, 0xa9, 0x11, - 0x53, 0x8a, 0x1e, 0x01, 0x27, 0x28, 0xfb, 0x1d, 0x87, 0xe8, 0x14, 0x48, 0xae, 0x95, 0xb9, 0xd3, - 0x93, 0xc5, 0x7c, 0x83, 0x26, 0xac, 0x6f, 0x34, 0xea, 0xab, 0x72, 0x9e, 0xe5, 0x5a, 0xef, 0x38, - 0x75, 0xbd, 0xf4, 0xeb, 0x02, 0xa4, 0x9f, 0x56, 0x1a, 0xe7, 0x56, 0x99, 0x57, 0xd0, 0x74, 0xa8, - 0xa0, 0x77, 0x60, 0x6e, 0xc7, 0x68, 0xb7, 0x0d, 0xb3, 0x45, 0xbc, 0xa8, 0xef, 0x60, 0xcd, 0x53, - 0x58, 0x81, 0x93, 0xb7, 0x18, 0x15, 0x5d, 0x83, 0xbc, 0x66, 0x63, 0x1d, 0x9b, 0xae, 0xa1, 0xb6, - 0x1d, 0xae, 0xb9, 0x30, 0xa9, 0xf4, 0x4b, 0x02, 0x4c, 0xd2, 0x11, 0x80, 0xde, 0x86, 0x9c, 0x66, - 0x99, 0xae, 0x6a, 0x98, 0xdc, 0x94, 0xe5, 0xe4, 0x80, 0x30, 0xb4, 0x78, 0xd7, 0x61, 0x46, 0xd5, - 0x34, 0xab, 0x67, 0xba, 0x8a, 0xa9, 0x76, 0x30, 0x2f, 0x66, 0x9e, 0xd3, 0x36, 0xd5, 0x0e, 0x46, - 0x8b, 0xe0, 0x3d, 0xfa, 0x5b, 0x42, 0x73, 0x32, 0x70, 0xd2, 0x3a, 0x3e, 0x2e, 0xfd, 0x3b, 0x01, - 0xb2, 0xde, 0x98, 0x21, 0xc5, 0x68, 0x61, 0x93, 0x05, 0xa1, 0x7b, 0xc5, 0xf0, 0x09, 0xfd, 0x53, - 0x65, 0x2e, 0x98, 0x2a, 0x2f, 0xc0, 0xa4, 0x4b, 0x86, 0x05, 0x2f, 0x01, 0x7b, 0xa0, 0xeb, 0xd9, - 0x6d, 0xb5, 0xc5, 0x96, 0xf3, 0x72, 0x32, 0x7b, 0x20, 0x95, 0xe1, 0x61, 0xbf, 0x4c, 0x23, 0xfc, - 0x89, 0x94, 0x94, 0x05, 0xa7, 0xee, 0xe0, 0x96, 0x61, 0xd2, 0xbe, 0x94, 0x96, 0x81, 0x92, 0x56, - 0x08, 0x05, 0xbd, 0x05, 0x39, 0x96, 0x01, 0x9b, 0x3a, 0xed, 0x50, 0x69, 0x39, 0x4b, 0x09, 0x55, - 0x53, 0x2f, 0x61, 0xc8, 0xf9, 0x83, 0x93, 0x34, 0x5b, 0xcf, 0xf1, 0x15, 0x49, 0x7f, 0xa3, 0xf7, - 0xe1, 0xc2, 0xeb, 0x9e, 0xda, 0x36, 0x76, 0xe9, 0x4a, 0x1d, 0x8d, 0xd2, 0xa7, 0x3a, 0x63, 0x35, - 0x41, 0x7e, 0x1a, 0x95, 0x40, 0x55, 0xe7, 0x8d, 0xe5, 0x74, 0x30, 0x96, 0xa5, 0xdf, 0x13, 0x60, - 0x9e, 0x85, 0x51, 0xb1, 0x80, 0xd9, 0xe4, 0xfc, 0x90, 0x8f, 0x21, 0xa7, 0xab, 0xae, 0xca, 0x36, - 0xb9, 0xa6, 0x46, 0x6e, 0x72, 0xf5, 0x37, 0x5d, 0xa8, 0xae, 0x4a, 0x37, 0xba, 0x22, 0xc8, 0x90, - 0xdf, 0x6c, 0x3f, 0xb0, 0x4c, 0x7f, 0x4b, 0x9f, 0x01, 0x0a, 0x17, 0x34, 0x49, 0x8f, 0xec, 0x1e, - 0x5c, 0x24, 0xba, 0xae, 0x9a, 0x9a, 0x7d, 0xdc, 0x75, 0x0d, 0xcb, 0x7c, 0x4e, 0xff, 0x3a, 0x48, - 0x0c, 0x7d, 0xd8, 0xa2, 0xdf, 0xb3, 0xa4, 0xdf, 0x9f, 0x82, 0xd9, 0xea, 0x51, 0xd7, 0xb2, 0x13, - 0x5d, 0x06, 0x5b, 0x81, 0x69, 0xbe, 0x52, 0x30, 0xe2, 0xdb, 0x75, 0x9f, 0x31, 0xf7, 0x3e, 0xdc, - 0x73, 0x46, 0xb4, 0x02, 0xc0, 0x82, 0x5a, 0x69, 0xe0, 0x52, 0xfa, 0x1c, 0x9f, 0xda, 0x28, 0x1b, - 0xdd, 0xf0, 0xb1, 0x09, 0xf9, 0xce, 0x81, 0xa6, 0x29, 0xbb, 0x46, 0xdb, 0xe5, 0xb1, 0x81, 0xf1, - 0x61, 0xec, 0x1b, 0x2f, 0x2b, 0x95, 0x27, 0x34, 0x13, 0x8b, 0xa9, 0x0b, 0x9e, 0x65, 0x20, 0x12, - 0xd8, 0x6f, 0xf4, 0x2e, 0xf0, 0xcd, 0x47, 0x8a, 0xe3, 0x6d, 0x25, 0x5c, 0x99, 0x3d, 0x3d, 0x59, - 0xcc, 0xc9, 0x94, 0xda, 0x68, 0x34, 0xe5, 0x1c, 0xcb, 0xd0, 0x70, 0x5c, 0x74, 0x03, 0x66, 0xad, - 0x8e, 0xe1, 0x2a, 0x9e, 0x93, 0xc4, 0x3d, 0xca, 0x19, 0x42, 0xf4, 0x9c, 0xa8, 0xf3, 0xec, 0x49, - 0x99, 0x1e, 0x7f, 0x4f, 0xca, 0xdf, 0x16, 0xe0, 0x12, 0x57, 0xa4, 0xb2, 0x43, 0xe3, 0xf0, 0xd5, - 0xb6, 0xe1, 0x1e, 0x2b, 0xfb, 0x07, 0xc5, 0x2c, 0xf5, 0x5b, 0x7f, 0x26, 0xb6, 0x41, 0x42, 0xfd, - 0xa0, 0xec, 0x35, 0xcb, 0xf1, 0x33, 0xce, 0xbc, 0x7e, 0x50, 0x35, 0x5d, 0xfb, 0x78, 0xe5, 0xf2, - 0xe9, 0xc9, 0xe2, 0xc2, 0x60, 0xea, 0x4b, 0x79, 0xc1, 0x19, 0x64, 0x41, 0x35, 0x00, 0xec, 0xf7, - 0x43, 0x3a, 0x63, 0xc4, 0xfb, 0x1f, 0xb1, 0x1d, 0x56, 0x0e, 0xf1, 0xa2, 0xbb, 0x20, 0xf2, 0x3d, - 0x40, 0xbb, 0x46, 0x1b, 0x2b, 0x8e, 0xf1, 0x05, 0xa6, 0x73, 0x4b, 0x5a, 0x2e, 0x30, 0x3a, 0x11, - 0xd1, 0x30, 0xbe, 0xc0, 0xa5, 0xef, 0x40, 0x71, 0x58, 0xe9, 0xc3, 0x43, 0x20, 0xc7, 0x3e, 0xe9, - 0x7e, 0x14, 0x5d, 0xcf, 0x19, 0xa3, 0xab, 0xf2, 0x35, 0x9d, 0x8f, 0x53, 0x1f, 0x09, 0xd2, 0x3f, - 0x4c, 0xc1, 0xec, 0x4a, 0xaf, 0xbd, 0xff, 0xbc, 0xdb, 0x60, 0x87, 0x21, 0x10, 0x33, 0xc8, 0x0c, - 0x05, 0x29, 0xa0, 0xc0, 0xcc, 0x20, 0xb5, 0x04, 0xc6, 0x17, 0x98, 0x4c, 0x4e, 0xa1, 0x70, 0x19, - 0xbe, 0xcf, 0x80, 0xd6, 0x21, 0x20, 0xd3, 0xad, 0x00, 0x1f, 0x41, 0x31, 0x94, 0x91, 0x2e, 0xbe, - 0x28, 0xd8, 0x74, 0x6d, 0x03, 0xb3, 0x05, 0xc4, 0xb4, 0x1c, 0x8a, 0xe9, 0xa9, 0x93, 0xe4, 0x2a, - 0x4b, 0x45, 0x4d, 0x98, 0x21, 0x19, 0x8f, 0x15, 0x3a, 0x85, 0x78, 0x0b, 0xbc, 0x0f, 0x62, 0xaa, - 0x15, 0x29, 0x77, 0x99, 0xea, 0xa7, 0x42, 0x79, 0xe8, 0x4f, 0x39, 0x8f, 0x03, 0x4a, 0xe9, 0x53, - 0x10, 0xfb, 0x33, 0x84, 0x75, 0x99, 0x61, 0xba, 0xbc, 0x10, 0xd6, 0x65, 0x3a, 0xa4, 0xa7, 0xb5, - 0x4c, 0x36, 0x23, 0x4e, 0x4a, 0x7f, 0x96, 0x86, 0x82, 0xd7, 0xcd, 0x92, 0x04, 0x3a, 0x2b, 0x30, - 0x49, 0x3a, 0x85, 0x17, 0x81, 0x72, 0x7b, 0x44, 0xef, 0xe6, 0x91, 0xed, 0xa4, 0xb3, 0x78, 0x20, - 0x99, 0xb2, 0x26, 0x61, 0x70, 0x4a, 0xbf, 0x94, 0x82, 0x0c, 0xc5, 0x16, 0x0f, 0x20, 0x43, 0x27, - 0x0a, 0x61, 0x9c, 0x89, 0x82, 0x66, 0xf5, 0xa7, 0xb3, 0x54, 0xc8, 0x35, 0x25, 0x3e, 0xdf, 0x9e, - 0xfa, 0xc1, 0x83, 0x87, 0xd4, 0xd8, 0xcc, 0xc8, 0xfc, 0x09, 0xad, 0xd0, 0xd0, 0x28, 0xcb, 0x76, - 0xb1, 0xce, 0x7d, 0xfa, 0x6b, 0x67, 0xb5, 0xaf, 0x37, 0x29, 0x79, 0x7c, 0xe8, 0x0a, 0xa4, 0x89, - 0x15, 0x9b, 0x66, 0x51, 0x0e, 0xa7, 0x27, 0x8b, 0x69, 0x62, 0xbf, 0x08, 0x0d, 0x2d, 0x41, 0x3e, - 0x6a, 0x32, 0x88, 0x07, 0x47, 0x0d, 0x63, 0x68, 0xb8, 0x43, 0xdb, 0x1f, 0x5a, 0x0c, 0xcf, 0xf2, - 0x36, 0xfe, 0x5f, 0x19, 0x98, 0xad, 0x77, 0x92, 0x9e, 0x52, 0x96, 0xa3, 0x2d, 0x1c, 0x07, 0x84, - 0x22, 0x2f, 0x8d, 0x69, 0xe0, 0xc8, 0x0c, 0x9e, 0x3e, 0xdf, 0x0c, 0xfe, 0x29, 0xf5, 0xa2, 0x59, - 0xd7, 0x98, 0x1a, 0xbf, 0x6b, 0x4c, 0x63, 0x53, 0xa7, 0x33, 0x51, 0x9d, 0x78, 0xda, 0xfc, 0x4c, - 0x8a, 0xf4, 0x10, 0xcc, 0x14, 0x2d, 0x3f, 0xf5, 0x73, 0x64, 0xc2, 0x13, 0xec, 0x15, 0xa1, 0x91, - 0x2e, 0x51, 0x8b, 0x3a, 0xfd, 0xe6, 0x16, 0xb5, 0xe4, 0xf2, 0xce, 0xfa, 0x31, 0xa4, 0x75, 0xc3, - 0x6b, 0x9c, 0xf1, 0xa7, 0x6a, 0xc2, 0x74, 0x46, 0xaf, 0xcd, 0x84, 0x7b, 0x2d, 0xeb, 0x25, 0xa5, - 0x3a, 0x40, 0x50, 0x37, 0x74, 0x0d, 0xa6, 0xac, 0xb6, 0xee, 0x6d, 0x76, 0x99, 0x5d, 0xc9, 0x9d, - 0x9e, 0x2c, 0x4e, 0x3e, 0x6f, 0xeb, 0xf5, 0x55, 0x79, 0xd2, 0x6a, 0xeb, 0x75, 0x9d, 0x1e, 0xe8, - 0x81, 0x0f, 0x15, 0x3f, 0x12, 0x6e, 0x46, 0x9e, 0x36, 0xf1, 0xe1, 0x2a, 0x76, 0x34, 0xde, 0xe1, - 0x7e, 0x5b, 0x80, 0x82, 0xa7, 0xbb, 0x64, 0x8d, 0x4a, 0xd6, 0xe8, 0xf0, 0x41, 0x96, 0x3e, 0xdf, - 0x20, 0xf3, 0xf8, 0xf8, 0x46, 0xe4, 0x5f, 0x11, 0x78, 0x6c, 0x73, 0x43, 0x53, 0x5d, 0xe2, 0x54, - 0x24, 0x38, 0x30, 0xee, 0x81, 0x68, 0xab, 0xa6, 0x6e, 0x75, 0x8c, 0x2f, 0x30, 0x5b, 0x08, 0x75, - 0xf8, 0x67, 0xcf, 0x39, 0x9f, 0x4e, 0x57, 0xfd, 0x1c, 0xe9, 0xbf, 0x0b, 0x3c, 0x0e, 0xda, 0x2f, - 0x46, 0xb2, 0xc1, 0x29, 0x79, 0xfe, 0x69, 0xc0, 0xdc, 0xb5, 0xbc, 0x30, 0xae, 0xb7, 0x87, 0x05, - 0x2d, 0xd6, 0xcd, 0x5d, 0xcb, 0xfb, 0xcc, 0x6e, 0x7b, 0x04, 0xa7, 0xf4, 0x73, 0x30, 0x49, 0x93, - 0xdf, 0xc0, 0x80, 0xfa, 0xb1, 0xf7, 0x44, 0xe3, 0x7f, 0x9a, 0x82, 0x9b, 0xb4, 0xaa, 0x2f, 0xb1, - 0x6d, 0xec, 0x1e, 0x6f, 0xd9, 0x96, 0x8b, 0x35, 0x17, 0xeb, 0xc1, 0xea, 0x76, 0x82, 0x4d, 0xa0, - 0x43, 0x8e, 0xc7, 0x05, 0x18, 0x3a, 0x3f, 0xd4, 0xe7, 0xe9, 0x57, 0x5b, 0x64, 0xcb, 0xb2, 0x78, - 0x82, 0xfa, 0xaa, 0x9c, 0x65, 0x92, 0xeb, 0x3a, 0x5a, 0x86, 0x5c, 0xd7, 0xab, 0xc6, 0xb9, 0x42, - 0xcf, 0x7c, 0x2e, 0xb4, 0x0e, 0x73, 0xbc, 0xa0, 0x6a, 0xdb, 0x38, 0xc0, 0x8a, 0xea, 0x9e, 0x67, - 0x9e, 0x9b, 0x65, 0xbc, 0xcb, 0x84, 0x75, 0xd9, 0x95, 0xfe, 0x4e, 0x06, 0x6e, 0x9d, 0xa1, 0xe2, - 0x24, 0xbb, 0x57, 0x09, 0xb2, 0x07, 0xe4, 0x45, 0x06, 0xaf, 0x7d, 0x56, 0xf6, 0x9f, 0xd1, 0x4e, - 0xc4, 0x59, 0xda, 0x55, 0x8d, 0x36, 0x71, 0xae, 0x58, 0xb0, 0xef, 0xf0, 0x70, 0xc2, 0xf8, 0xe0, - 0xd9, 0x90, 0x5b, 0xf5, 0x84, 0x0a, 0xa2, 0xd9, 0x1c, 0xf4, 0x5d, 0x01, 0x4a, 0xec, 0x85, 0x2c, - 0xe2, 0xb4, 0xef, 0x35, 0x19, 0xfa, 0x9a, 0xd5, 0x98, 0xd7, 0x8c, 0xa5, 0xa3, 0x72, 0xe8, 0x5d, - 0xbc, 0x20, 0xc5, 0xf0, 0xdb, 0xc2, 0x45, 0x29, 0xfd, 0x86, 0x00, 0xf9, 0x10, 0x01, 0xdd, 0x1e, - 0xd8, 0x32, 0x98, 0x3f, 0x8d, 0xdb, 0x27, 0x78, 0x6b, 0x60, 0x9f, 0xe0, 0x4a, 0xf6, 0xcb, 0x93, - 0xc5, 0x8c, 0xcc, 0xf6, 0x8d, 0x78, 0x3b, 0x06, 0xaf, 0x07, 0x27, 0x54, 0xa5, 0xfb, 0x32, 0x79, - 0x47, 0x54, 0xd1, 0x85, 0x23, 0xd5, 0xfb, 0x1c, 0x4d, 0x17, 0x8e, 0xc8, 0x93, 0xf4, 0x9b, 0x29, - 0x98, 0x5f, 0xd6, 0xf5, 0x46, 0x83, 0x5b, 0xf8, 0xe4, 0xc6, 0x98, 0x07, 0xa1, 0x53, 0x01, 0x84, - 0x46, 0xef, 0x01, 0xd2, 0x0d, 0x87, 0x9d, 0xf4, 0xe2, 0xec, 0xa9, 0xba, 0x75, 0x18, 0x44, 0x9d, - 0xcc, 0x7b, 0x29, 0x0d, 0x2f, 0x01, 0x35, 0x80, 0x62, 0x39, 0xc5, 0x71, 0x55, 0xff, 0xab, 0xda, - 0xad, 0xb1, 0x36, 0xcc, 0x31, 0x90, 0xe7, 0x3f, 0xca, 0x39, 0x22, 0x87, 0xfe, 0x24, 0xa8, 0xc4, - 0x20, 0x8d, 0xe2, 0x2a, 0xaa, 0xe3, 0x6d, 0xf5, 0x62, 0x67, 0xcc, 0x14, 0x18, 0x7d, 0xd9, 0x61, - 0x3b, 0xb8, 0xd8, 0x76, 0x8e, 0x40, 0x35, 0x49, 0x02, 0xfe, 0xbf, 0x2f, 0x40, 0x41, 0xc6, 0xbb, - 0x36, 0x76, 0x12, 0x5d, 0xf2, 0x78, 0x02, 0x33, 0x36, 0x93, 0xaa, 0xec, 0xda, 0x56, 0xe7, 0x3c, - 0xb6, 0x22, 0xcf, 0x19, 0x9f, 0xd8, 0x56, 0x87, 0x9b, 0xe4, 0x97, 0x30, 0xe7, 0x97, 0x31, 0xc9, - 0xca, 0xff, 0x1e, 0xdd, 0x0c, 0xce, 0x04, 0x27, 0x1d, 0xfe, 0x91, 0xac, 0x06, 0xe8, 0x67, 0xb8, - 0x70, 0x41, 0x93, 0x54, 0xc3, 0x7f, 0x13, 0xa0, 0xd0, 0xe8, 0xed, 0xb0, 0x13, 0xcc, 0x92, 0xd3, - 0x40, 0x15, 0x72, 0x6d, 0xbc, 0xeb, 0x2a, 0x6f, 0xb4, 0x11, 0x21, 0x4b, 0x58, 0xe9, 0x36, 0x8c, - 0xa7, 0x00, 0x36, 0xdd, 0x0d, 0x49, 0xe5, 0xa4, 0xcf, 0x29, 0x27, 0x47, 0x79, 0x09, 0x99, 0xcc, - 0x3a, 0x73, 0x7e, 0x35, 0x93, 0x9c, 0x5f, 0x5e, 0x45, 0xac, 0x43, 0xfa, 0x3c, 0xd6, 0x61, 0x9e, - 0x47, 0xbc, 0xc4, 0x5b, 0x88, 0x32, 0x2c, 0x50, 0xb7, 0x4c, 0x51, 0xbb, 0xdd, 0xb6, 0xe1, 0x81, - 0x79, 0x6a, 0x7f, 0x32, 0xf2, 0x3c, 0x4d, 0x5a, 0x66, 0x29, 0x14, 0xc6, 0xa3, 0x5f, 0x15, 0x60, - 0x66, 0xd7, 0xc6, 0xf8, 0x0b, 0xac, 0x50, 0x93, 0x3c, 0x5e, 0x48, 0xcf, 0x2a, 0x29, 0xc3, 0x57, - 0xfe, 0xe4, 0x9f, 0x67, 0x2f, 0x6e, 0x90, 0xf7, 0xa2, 0x4d, 0x10, 0xb5, 0x36, 0x0b, 0x42, 0xf0, - 0xc3, 0x8b, 0xce, 0x81, 0x7d, 0xe6, 0x18, 0x73, 0x10, 0x61, 0xf4, 0x82, 0x0c, 0x26, 0x55, 0x57, - 0xf8, 0xa9, 0x91, 0x1c, 0xba, 0x94, 0x87, 0x9c, 0x0a, 0x11, 0x3a, 0x6c, 0xb2, 0x2c, 0x63, 0x55, - 0xe7, 0x1e, 0x36, 0x19, 0x57, 0xfe, 0x03, 0x1f, 0x57, 0xaf, 0x60, 0x9e, 0xf6, 0x9b, 0xa4, 0x37, - 0x77, 0x4b, 0x3f, 0x4c, 0x01, 0x0a, 0x4b, 0xfe, 0xe9, 0xf5, 0xb7, 0x54, 0x72, 0xfd, 0xed, 0x5d, - 0x40, 0x2c, 0x6e, 0xd5, 0x51, 0xba, 0xd8, 0x56, 0x1c, 0xac, 0x59, 0xfc, 0x3c, 0x2f, 0x41, 0x16, - 0x79, 0xca, 0x16, 0xb6, 0x1b, 0x94, 0x8e, 0x96, 0x01, 0x02, 0xaf, 0x9d, 0x4f, 0x8a, 0xe3, 0x38, - 0xed, 0x39, 0xdf, 0x69, 0x97, 0xbe, 0x27, 0x40, 0x61, 0xc3, 0x68, 0xd9, 0x6a, 0xa2, 0xc7, 0x55, - 0xa1, 0x8f, 0xa3, 0x5f, 0x33, 0xf2, 0x0f, 0x4b, 0x71, 0x91, 0x56, 0x2c, 0x87, 0x07, 0xb7, 0x39, - 0x03, 0x99, 0x6b, 0xfc, 0x12, 0x25, 0x69, 0x64, 0xff, 0x63, 0x09, 0x66, 0x78, 0xb9, 0xb7, 0x4d, - 0xc3, 0x32, 0xd1, 0x03, 0x48, 0xb7, 0xf8, 0xd7, 0xaa, 0x7c, 0xec, 0xca, 0x72, 0x70, 0x18, 0x64, - 0x6d, 0x42, 0x26, 0x79, 0x09, 0x4b, 0xb7, 0xe7, 0xc6, 0x78, 0xf0, 0xc1, 0xe6, 0x83, 0x30, 0x4b, - 0xb7, 0xe7, 0xa2, 0x06, 0xcc, 0x69, 0xc1, 0x09, 0x74, 0x0a, 0x61, 0x4f, 0x0f, 0xc5, 0xfd, 0xb1, - 0x67, 0x01, 0xd6, 0x26, 0xe4, 0x82, 0x16, 0x49, 0x40, 0x95, 0xf0, 0xc1, 0x67, 0x99, 0x81, 0xc8, - 0xc6, 0x60, 0xdb, 0x7c, 0xf4, 0xd0, 0xb5, 0xda, 0x44, 0xe8, 0x7c, 0x34, 0xf4, 0x31, 0x4c, 0xe9, - 0xf4, 0x88, 0x2d, 0x6e, 0xa5, 0xe2, 0x1a, 0x3a, 0x72, 0x92, 0x59, 0x6d, 0x42, 0xe6, 0x1c, 0x68, - 0x0d, 0x66, 0xd8, 0x2f, 0xe6, 0x43, 0x73, 0xdb, 0x72, 0x6b, 0xb8, 0x84, 0xd0, 0xec, 0x5e, 0x9b, - 0x90, 0xf3, 0x7a, 0x40, 0x45, 0x4f, 0x21, 0xaf, 0xb5, 0xb1, 0x6a, 0x73, 0x51, 0xb7, 0x87, 0x6e, - 0xc7, 0x1c, 0x38, 0x96, 0xab, 0x36, 0x21, 0x83, 0xe6, 0x13, 0x49, 0xa1, 0x6c, 0x7a, 0x3a, 0x13, - 0x97, 0xf4, 0xfe, 0xd0, 0x42, 0x0d, 0x1e, 0x75, 0x55, 0xa3, 0xb3, 0xbe, 0x4f, 0x45, 0xdf, 0x84, - 0x8c, 0xa3, 0xa9, 0xde, 0x1a, 0xcd, 0xd5, 0x21, 0xc7, 0xe7, 0x04, 0xcc, 0x34, 0x37, 0x7a, 0xcc, - 0xdc, 0x6f, 0xf7, 0xc8, 0x5b, 0x2e, 0x8f, 0xd3, 0x69, 0xe4, 0x98, 0x06, 0xa2, 0x53, 0x4c, 0x09, - 0x44, 0x0f, 0x2a, 0xc1, 0x1b, 0x0a, 0xdd, 0xe8, 0x4c, 0xd7, 0xc7, 0xe3, 0xf5, 0x30, 0xb0, 0x31, - 0xbd, 0x46, 0xcf, 0x82, 0xf0, 0x88, 0x68, 0x03, 0x66, 0x99, 0xa0, 0x1e, 0xdb, 0x33, 0x5d, 0x5c, - 0x1a, 0x1a, 0xcd, 0x10, 0xb3, 0x6b, 0xbb, 0x36, 0x21, 0xcf, 0xa8, 0x21, 0x72, 0x50, 0xae, 0x0e, - 0xb6, 0x5b, 0xb8, 0x98, 0x1f, 0x5d, 0xae, 0x70, 0xcc, 0xa6, 0x5f, 0x2e, 0x4a, 0x44, 0xbf, 0x08, - 0x17, 0x98, 0x20, 0x97, 0x87, 0xa2, 0xf1, 0xe0, 0xaf, 0x77, 0x86, 0x06, 0x0d, 0x0c, 0xdd, 0xe7, - 0x5c, 0x9b, 0x90, 0x91, 0x3a, 0x90, 0x88, 0x34, 0xb8, 0xc8, 0xde, 0xc0, 0x37, 0xca, 0xda, 0x7c, - 0x6f, 0x67, 0xf1, 0x06, 0x7d, 0xc5, 0x7b, 0xc3, 0x5e, 0x11, 0xbb, 0x7f, 0xb7, 0x36, 0x21, 0x2f, - 0xa8, 0x83, 0xa9, 0x41, 0x35, 0x6c, 0xbe, 0x25, 0x91, 0x77, 0xb7, 0xf7, 0x46, 0x57, 0x23, 0x6e, - 0x2b, 0xa7, 0x5f, 0x8d, 0x48, 0x22, 0x69, 0x40, 0xff, 0x28, 0x06, 0xda, 0x99, 0x66, 0x86, 0x36, - 0x60, 0xcc, 0xbe, 0x45, 0xd2, 0x80, 0x7b, 0x21, 0x32, 0x2a, 0x43, 0xaa, 0xa5, 0x15, 0x67, 0x87, - 0xce, 0x0f, 0xfe, 0xde, 0xbc, 0xda, 0x84, 0x9c, 0x6a, 0x69, 0xe8, 0x53, 0xc8, 0xb2, 0x8d, 0x56, - 0x47, 0x66, 0xb1, 0x30, 0xd4, 0xe0, 0x46, 0xb7, 0xab, 0xd5, 0x26, 0x64, 0xba, 0xb7, 0x8b, 0x77, - 0x64, 0xbe, 0x89, 0x86, 0x8a, 0x28, 0x8f, 0xd8, 0x5f, 0xdd, 0xb7, 0x95, 0x89, 0x74, 0x18, 0xdb, - 0x27, 0xa2, 0x2d, 0x28, 0xd8, 0x2c, 0xcc, 0xd8, 0xdb, 0x14, 0x20, 0x0e, 0x8d, 0xf5, 0x89, 0xdb, - 0x17, 0x50, 0xa3, 0x0b, 0x1f, 0x21, 0x3a, 0x69, 0xbb, 0xa8, 0x44, 0xde, 0x76, 0xf3, 0x43, 0xdb, - 0x6e, 0x68, 0x8c, 0x3a, 0x69, 0x3b, 0x7b, 0x20, 0x11, 0x7d, 0x08, 0x93, 0x6c, 0x9c, 0x20, 0x2a, - 0x32, 0x2e, 0x7c, 0xad, 0x6f, 0x88, 0xb0, 0xfc, 0xc4, 0x7a, 0xb9, 0x3c, 0xd6, 0x56, 0x69, 0x5b, - 0xad, 0xe2, 0xc2, 0x50, 0xeb, 0x35, 0x18, 0x35, 0x4c, 0xac, 0x97, 0x1b, 0x50, 0x49, 0x07, 0xb2, - 0x59, 0x0a, 0x1f, 0x62, 0x17, 0x86, 0x76, 0xa0, 0x98, 0x10, 0xdc, 0x1a, 0xdd, 0x05, 0x15, 0x90, - 0x7d, 0xc3, 0xea, 0x60, 0x85, 0x1a, 0xc5, 0x8b, 0xa3, 0x0d, 0x6b, 0xe4, 0x4c, 0x33, 0xdf, 0xb0, - 0x32, 0x2a, 0x7a, 0x09, 0x22, 0x3f, 0x58, 0x27, 0xf8, 0xb8, 0x7a, 0x89, 0xca, 0xbb, 0x17, 0x3b, - 0x21, 0xc6, 0x05, 0x27, 0xd6, 0x88, 0x87, 0x1a, 0x4d, 0x41, 0x9f, 0xc1, 0x3c, 0x95, 0xa7, 0x68, - 0xc1, 0x59, 0x48, 0xc5, 0xe2, 0xc0, 0xc9, 0x3a, 0xc3, 0x8f, 0x4d, 0xf2, 0x24, 0x8b, 0x5a, 0x5f, - 0x12, 0x19, 0x0f, 0x86, 0x69, 0xb8, 0x74, 0xee, 0x2e, 0x0d, 0x1d, 0x0f, 0xd1, 0x73, 0x60, 0xc9, - 0x78, 0x30, 0x18, 0x85, 0x74, 0xe3, 0x3e, 0x8b, 0xf7, 0xf6, 0xd0, 0x6e, 0x3c, 0xc4, 0xd8, 0xcd, - 0xba, 0x11, 0x3b, 0xb7, 0x0a, 0xc0, 0x70, 0x09, 0xf5, 0xfc, 0xae, 0x0e, 0x75, 0x00, 0xfa, 0x43, - 0x64, 0x89, 0x03, 0xd0, 0xf6, 0x68, 0x64, 0x9c, 0xd2, 0x55, 0x0f, 0x85, 0x46, 0x7a, 0x14, 0x17, - 0x87, 0x8e, 0xd3, 0x81, 0xa8, 0x0c, 0x32, 0x4e, 0x0f, 0x7d, 0x22, 0xf1, 0x24, 0xd8, 0xa7, 0xa9, - 0xe2, 0xb5, 0xe1, 0xb3, 0x5e, 0xf8, 0x0b, 0x35, 0x9d, 0xf5, 0x28, 0x81, 0xf0, 0xb2, 0x15, 0xf7, - 0xa2, 0x34, 0x94, 0x37, 0xf2, 0x75, 0x85, 0xf0, 0x32, 0x0e, 0xb4, 0x0c, 0x39, 0xe2, 0x14, 0x1f, - 0x53, 0x33, 0x73, 0x7d, 0x28, 0x30, 0xed, 0xdb, 0xc0, 0x57, 0x9b, 0x90, 0xb3, 0xaf, 0x39, 0x89, - 0x74, 0x6d, 0x26, 0x82, 0x1b, 0x98, 0xfb, 0x43, 0xbb, 0xf6, 0xe0, 0xce, 0x2d, 0xd2, 0xb5, 0x5f, - 0x07, 0xd4, 0x60, 0xde, 0x75, 0xd8, 0x1a, 0x7d, 0xf1, 0xe6, 0xe8, 0x79, 0x37, 0xfa, 0x45, 0xc1, - 0x9f, 0x77, 0x39, 0x99, 0xcd, 0xbb, 0xba, 0xe2, 0x38, 0x2c, 0xfc, 0xe7, 0xd6, 0x88, 0x79, 0xb7, - 0x6f, 0xd5, 0x8e, 0xcd, 0xbb, 0x7a, 0x83, 0x71, 0x12, 0x17, 0xd4, 0xf6, 0x4e, 0xb3, 0xe2, 0x98, - 0xe5, 0xce, 0x50, 0x17, 0x34, 0xf6, 0xb8, 0x2d, 0xe2, 0x82, 0xda, 0x91, 0x04, 0xf4, 0xb3, 0x30, - 0xcd, 0x57, 0x49, 0x8a, 0x77, 0x47, 0x38, 0xe5, 0xe1, 0x85, 0x2d, 0x32, 0x26, 0x38, 0x0f, 0xb3, - 0x50, 0x6c, 0x75, 0x86, 0x59, 0xe0, 0x7b, 0x23, 0x2c, 0xd4, 0xc0, 0x02, 0x11, 0xb3, 0x50, 0x01, - 0x99, 0x94, 0xc6, 0x61, 0x2b, 0x0b, 0xc5, 0x6f, 0x0c, 0x2d, 0x4d, 0x74, 0x89, 0x85, 0x94, 0x86, - 0xf3, 0xd0, 0x19, 0x8b, 0x3a, 0x0c, 0x4c, 0x3b, 0xef, 0x0e, 0x9f, 0xb1, 0xfa, 0xb1, 0x6a, 0xcd, - 0xfb, 0x06, 0xc2, 0xb4, 0xf2, 0xb7, 0x04, 0xb8, 0xc6, 0xfa, 0x00, 0x5d, 0x01, 0x3e, 0x56, 0xfc, - 0x05, 0xfc, 0x10, 0x10, 0x7f, 0x40, 0xc5, 0x7f, 0x78, 0xfe, 0xf5, 0x66, 0xef, 0x8d, 0xef, 0xa8, - 0xa3, 0xf2, 0x11, 0x65, 0x74, 0x18, 0x82, 0x2a, 0x3e, 0x1c, 0xaa, 0x8c, 0x28, 0xea, 0x23, 0xca, - 0xe0, 0x3c, 0x2b, 0xd3, 0x3c, 0x10, 0xc0, 0xdf, 0x17, 0x3d, 0x27, 0x8a, 0x6b, 0x99, 0xec, 0x65, - 0xb1, 0xb8, 0x96, 0xc9, 0x5e, 0x11, 0x4b, 0x6b, 0x99, 0xec, 0x5b, 0xe2, 0xdb, 0xd2, 0x3f, 0x2a, - 0xc1, 0xac, 0x07, 0xba, 0x18, 0xa0, 0x7a, 0x18, 0x06, 0x54, 0x57, 0x87, 0x01, 0x2a, 0x0e, 0xd3, - 0x38, 0xa2, 0x7a, 0x18, 0x46, 0x54, 0x57, 0x87, 0x21, 0xaa, 0x80, 0x87, 0x40, 0xaa, 0xe6, 0x30, - 0x48, 0x75, 0x6f, 0x0c, 0x48, 0xe5, 0x8b, 0xea, 0xc7, 0x54, 0xab, 0x83, 0x98, 0xea, 0xe6, 0x68, - 0x4c, 0xe5, 0x8b, 0x0a, 0x81, 0xaa, 0xc7, 0x7d, 0xa0, 0xea, 0xfa, 0x08, 0x50, 0xe5, 0xf3, 0x7b, - 0xa8, 0x6a, 0x3d, 0x16, 0x55, 0xdd, 0x3e, 0x0b, 0x55, 0xf9, 0x72, 0x22, 0xb0, 0xaa, 0x16, 0x07, - 0xab, 0x6e, 0x9d, 0x01, 0xab, 0x7c, 0x51, 0x61, 0x5c, 0xb5, 0x1e, 0x8b, 0xab, 0x6e, 0x9f, 0x85, - 0xab, 0x82, 0x62, 0x85, 0x81, 0xd5, 0x07, 0x11, 0x60, 0xb5, 0x38, 0x14, 0x58, 0xf9, 0xdc, 0x0c, - 0x59, 0x7d, 0xd2, 0x8f, 0xac, 0xae, 0x8f, 0x40, 0x56, 0x81, 0x62, 0x39, 0xb4, 0xaa, 0xc5, 0x41, - 0xab, 0x5b, 0x67, 0x40, 0xab, 0x40, 0x17, 0x21, 0x6c, 0xb5, 0x19, 0x8f, 0xad, 0xee, 0x9c, 0x89, - 0xad, 0x7c, 0x69, 0x51, 0x70, 0x55, 0x8b, 0x03, 0x57, 0xb7, 0xce, 0x00, 0x57, 0x7d, 0x25, 0x63, - 0xe8, 0x4a, 0x1d, 0x89, 0xae, 0xde, 0x1b, 0x13, 0x5d, 0xf9, 0xa2, 0xe3, 0xe0, 0x95, 0x3e, 0x1a, - 0x5e, 0x95, 0xc7, 0x85, 0x57, 0xfe, 0x4b, 0x62, 0xf1, 0x95, 0x3a, 0x12, 0x5f, 0xbd, 0x37, 0x26, - 0xbe, 0xea, 0xab, 0x48, 0x14, 0x60, 0x6d, 0xc6, 0x03, 0xac, 0x3b, 0x67, 0x02, 0xac, 0xa0, 0x15, - 0x23, 0x08, 0x6b, 0x29, 0x84, 0xb0, 0xde, 0x19, 0x82, 0xb0, 0x7c, 0x56, 0x02, 0xb1, 0xbe, 0x35, - 0x00, 0xb1, 0xa4, 0x51, 0x10, 0xcb, 0xe7, 0xf5, 0x31, 0x56, 0x2d, 0x0e, 0x63, 0xdd, 0x3a, 0x03, - 0x63, 0x05, 0xfd, 0x26, 0x04, 0xb2, 0x5e, 0x0c, 0x01, 0x59, 0x77, 0xcf, 0x06, 0x59, 0xbe, 0xbc, - 0x3e, 0x94, 0xa5, 0x8e, 0x44, 0x59, 0xef, 0x8d, 0x89, 0xb2, 0x82, 0x16, 0x8c, 0x81, 0x59, 0x1f, - 0x45, 0x61, 0xd6, 0xb5, 0xe1, 0x30, 0xcb, 0x17, 0xc3, 0x71, 0xd6, 0x7a, 0x2c, 0xce, 0xba, 0x7d, - 0x16, 0xce, 0x0a, 0xac, 0x59, 0x18, 0x68, 0x6d, 0xc6, 0x03, 0xad, 0x3b, 0x67, 0x02, 0xad, 0xa0, - 0x23, 0x45, 0x90, 0xd6, 0x7a, 0x2c, 0xd2, 0xba, 0x7d, 0x16, 0xd2, 0xea, 0x33, 0xb5, 0x1c, 0x6a, - 0xbd, 0x1a, 0x0a, 0xb5, 0xee, 0x8f, 0x03, 0xb5, 0x7c, 0xa1, 0x03, 0x58, 0xeb, 0xf3, 0xe1, 0x58, - 0xeb, 0x1b, 0xe7, 0x38, 0xa2, 0x36, 0x16, 0x6c, 0x7d, 0x6b, 0x00, 0x6c, 0x49, 0xa3, 0xc0, 0x56, - 0x30, 0x32, 0x3c, 0xb4, 0x55, 0x8d, 0xc1, 0x46, 0x37, 0x47, 0x63, 0xa3, 0x60, 0x22, 0x0f, 0xc0, - 0x51, 0x2d, 0x0e, 0x1c, 0xdd, 0x3a, 0x03, 0x1c, 0x05, 0x03, 0x2c, 0x84, 0x8e, 0x1e, 0xf7, 0xa1, - 0xa3, 0xeb, 0x67, 0x46, 0x38, 0x86, 0xe0, 0xd1, 0xe3, 0x3e, 0x78, 0x74, 0x7d, 0x04, 0x3c, 0x0a, - 0x98, 0x39, 0x3e, 0x5a, 0x19, 0xc4, 0x47, 0x37, 0x46, 0xe2, 0x23, 0x5f, 0x42, 0x00, 0x90, 0xd6, - 0x63, 0x01, 0xd2, 0xed, 0xb3, 0x00, 0x52, 0xd0, 0x23, 0xc3, 0x08, 0x69, 0x33, 0x1e, 0x21, 0xdd, - 0x39, 0x13, 0x21, 0xf5, 0xcd, 0x9e, 0x1e, 0x44, 0xaa, 0xc5, 0x41, 0xa4, 0x5b, 0x67, 0x40, 0xa4, - 0xf0, 0xec, 0xe9, 0x63, 0xa4, 0xe6, 0x30, 0x8c, 0x74, 0x6f, 0x0c, 0x8c, 0x14, 0xf8, 0x94, 0x7d, - 0x20, 0xe9, 0xd3, 0x7e, 0x90, 0x24, 0x8d, 0x02, 0x49, 0x41, 0x5f, 0xf6, 0x50, 0xd2, 0x66, 0x3c, - 0x4a, 0xba, 0x73, 0x26, 0x4a, 0x0a, 0x9b, 0x97, 0x10, 0x4c, 0xfa, 0xb4, 0x1f, 0x26, 0x49, 0xa3, - 0x60, 0x52, 0x50, 0x1e, 0x0f, 0x27, 0xd5, 0xe2, 0x70, 0xd2, 0xad, 0x33, 0x70, 0x52, 0x68, 0xd6, - 0x09, 0x80, 0xd2, 0x2f, 0x8f, 0x0f, 0x94, 0x3e, 0x7a, 0xd3, 0xc0, 0x9c, 0xb3, 0x91, 0xd2, 0xa7, - 0xfd, 0x48, 0x49, 0x1a, 0x85, 0x94, 0x02, 0x7d, 0x9c, 0x0f, 0x2a, 0xad, 0x65, 0xb2, 0x6f, 0x8b, - 0xef, 0x48, 0x7f, 0x3e, 0x05, 0x53, 0x35, 0x2f, 0x22, 0x36, 0x74, 0x14, 0x99, 0xf0, 0x26, 0x47, - 0x91, 0xa1, 0x55, 0xd2, 0xb5, 0xa8, 0xc7, 0x74, 0xf6, 0x01, 0x96, 0x83, 0x47, 0x2c, 0x72, 0xd6, - 0x37, 0x38, 0x13, 0x00, 0x7d, 0x00, 0xb3, 0x3d, 0x07, 0xdb, 0x4a, 0xd7, 0x36, 0x2c, 0xdb, 0x70, - 0xd9, 0xbe, 0x27, 0x61, 0x45, 0xfc, 0xf2, 0x64, 0x71, 0x66, 0xdb, 0xc1, 0xf6, 0x16, 0xa7, 0xcb, - 0x33, 0xbd, 0xd0, 0x93, 0x77, 0x39, 0xdd, 0xe4, 0xf8, 0x97, 0xd3, 0xbd, 0x00, 0x91, 0x7e, 0x73, - 0x0e, 0x4f, 0x32, 0xec, 0xd8, 0xaf, 0xf8, 0xf9, 0x90, 0xee, 0x4b, 0xf4, 0x72, 0xd2, 0xe3, 0xbf, - 0xe6, 0xec, 0x28, 0x11, 0x35, 0x80, 0x1e, 0xc8, 0xa3, 0x74, 0xad, 0xb6, 0xa1, 0x1d, 0x53, 0xdf, - 0x21, 0x7a, 0xaa, 0xfa, 0xc8, 0xbb, 0x0d, 0x5e, 0xa9, 0x86, 0xbb, 0x45, 0x39, 0x65, 0x38, 0xf4, - 0x7f, 0xa3, 0x07, 0x70, 0xb1, 0xa3, 0x1e, 0xd1, 0xd0, 0x64, 0xc5, 0x73, 0x06, 0x68, 0xb8, 0x30, - 0xbb, 0xa6, 0x0e, 0x75, 0xd4, 0x23, 0x7a, 0x7d, 0x1e, 0x4b, 0xa2, 0x77, 0xdf, 0x5c, 0x87, 0x19, - 0xbe, 0x1f, 0x82, 0x5d, 0x8d, 0x35, 0x47, 0x73, 0xf2, 0x7b, 0x52, 0xd8, 0xed, 0x58, 0xb7, 0xa0, - 0xa0, 0x1b, 0x8e, 0x6b, 0x98, 0x9a, 0xcb, 0x4f, 0xb2, 0x66, 0x47, 0x36, 0xcf, 0x7a, 0x54, 0x76, - 0x5c, 0x75, 0x13, 0xe6, 0xb5, 0xb6, 0xe1, 0xbb, 0x58, 0x6c, 0xd2, 0x9b, 0x1f, 0xda, 0x97, 0x2b, - 0x34, 0x6f, 0xff, 0x07, 0xe1, 0x39, 0x2d, 0x4a, 0x46, 0x15, 0x98, 0x6b, 0xa9, 0x2e, 0x3e, 0x54, - 0x8f, 0x15, 0x6f, 0x7b, 0x65, 0x9e, 0x1e, 0x05, 0xfb, 0xd6, 0xe9, 0xc9, 0xe2, 0xec, 0x53, 0x96, - 0x34, 0xb0, 0xcb, 0x72, 0xb6, 0x15, 0x4a, 0xd0, 0xd1, 0x1d, 0x98, 0x53, 0x9d, 0x63, 0x53, 0xa3, - 0x0d, 0x88, 0x4d, 0xa7, 0xe7, 0x50, 0x0f, 0x39, 0x2b, 0x17, 0x28, 0xb9, 0xe2, 0x51, 0xd1, 0x63, - 0x28, 0xf1, 0x0b, 0x2b, 0x0e, 0x55, 0x5b, 0x57, 0x68, 0xa3, 0x07, 0xc3, 0x43, 0xa4, 0x3c, 0x97, - 0xd9, 0x05, 0x15, 0x24, 0x03, 0x69, 0xe9, 0xf0, 0xa9, 0xcd, 0xd3, 0x62, 0x76, 0x2d, 0x93, 0x05, - 0x31, 0xbf, 0x96, 0xc9, 0xce, 0x88, 0xb3, 0x6b, 0x99, 0x6c, 0x41, 0x9c, 0x93, 0x7e, 0x5d, 0x80, - 0x99, 0xc8, 0x46, 0xb2, 0xc7, 0x7d, 0x1f, 0x91, 0xaf, 0xc4, 0x3b, 0xfb, 0xc3, 0x42, 0xd8, 0xb3, - 0xbc, 0x69, 0xbd, 0x28, 0xf6, 0xc5, 0xe1, 0x2e, 0x1e, 0x5d, 0x0d, 0xf1, 0xc2, 0x68, 0x3c, 0xb6, - 0x8f, 0x33, 0xdf, 0xff, 0xc1, 0xe2, 0x84, 0xf4, 0x17, 0x19, 0x98, 0x8d, 0x6e, 0x1b, 0xab, 0xf7, - 0x95, 0x2b, 0xce, 0xb8, 0x47, 0x38, 0xca, 0x23, 0xae, 0xda, 0xc9, 0x05, 0xb7, 0x4f, 0xb0, 0x62, - 0x5e, 0x1b, 0xf1, 0xa9, 0x3c, 0x5c, 0xce, 0x80, 0xb1, 0xf4, 0x9f, 0xd2, 0xbe, 0x9d, 0x2a, 0xc3, - 0x24, 0x3d, 0x01, 0x8a, 0x17, 0xad, 0xd8, 0x3f, 0x52, 0x88, 0xe7, 0x42, 0xd2, 0x65, 0x96, 0x8d, - 0xd8, 0xb5, 0xe6, 0x1b, 0x1d, 0xb1, 0x18, 0x98, 0xe4, 0xf3, 0x5f, 0x62, 0xd9, 0x63, 0x47, 0x6c, - 0xfe, 0x7f, 0x0c, 0xb5, 0x21, 0xef, 0x43, 0xbf, 0x00, 0x73, 0x9a, 0xd5, 0x6e, 0xb3, 0x39, 0x8b, - 0x8d, 0xd0, 0xc1, 0x43, 0x77, 0x68, 0x11, 0xf8, 0xbd, 0xa5, 0x65, 0xff, 0xfe, 0xd2, 0xb2, 0xcc, - 0xef, 0x2f, 0x0d, 0xc5, 0x41, 0x17, 0x7c, 0x61, 0x6c, 0x60, 0xf7, 0x85, 0x64, 0x4f, 0xbf, 0x49, - 0x48, 0x36, 0x0b, 0xb2, 0xe7, 0x3d, 0xef, 0x8f, 0x05, 0x1e, 0x10, 0xf3, 0xcc, 0xb2, 0xf6, 0x7b, - 0x7e, 0x10, 0x75, 0x29, 0x7c, 0x60, 0x66, 0x10, 0x2d, 0x4a, 0xb7, 0x04, 0xc5, 0x59, 0xe0, 0xd4, - 0x57, 0xb3, 0xc0, 0xd7, 0x61, 0xa6, 0x6b, 0xe3, 0x5d, 0xec, 0x6a, 0x7b, 0x8a, 0xd9, 0xeb, 0xf0, - 0xfd, 0x50, 0x79, 0x8f, 0xb6, 0xd9, 0xeb, 0xa0, 0x7b, 0x20, 0xfa, 0x59, 0x38, 0x9c, 0xf1, 0x4e, - 0x6b, 0xf3, 0xe8, 0x1c, 0xfc, 0x48, 0xff, 0x47, 0x80, 0x85, 0x48, 0x9d, 0xf8, 0x98, 0x5a, 0x83, - 0xbc, 0xee, 0xcf, 0x79, 0x4e, 0x51, 0x38, 0x67, 0x1c, 0x71, 0x98, 0x19, 0x29, 0x70, 0xc9, 0x7b, - 0x2d, 0xbd, 0xb1, 0x21, 0x10, 0x9b, 0x3a, 0xa7, 0xd8, 0x8b, 0x81, 0x9c, 0xd5, 0xd0, 0x0b, 0xfc, - 0x41, 0x96, 0x1e, 0x6b, 0x90, 0x49, 0xbf, 0x2d, 0x80, 0x48, 0x5f, 0xf0, 0x04, 0x63, 0x3d, 0x11, - 0xeb, 0xe6, 0x05, 0xec, 0xa7, 0xc6, 0xdf, 0xf1, 0x14, 0xb9, 0x65, 0x26, 0x1d, 0xbd, 0x65, 0x46, - 0xfa, 0x81, 0x00, 0x05, 0xbf, 0x84, 0xec, 0x26, 0xc8, 0x11, 0xe7, 0xb2, 0xbe, 0xd9, 0x6d, 0x87, - 0xde, 0xf9, 0x31, 0x63, 0x5d, 0x4e, 0x19, 0x3e, 0x3f, 0x86, 0xdd, 0xd2, 0xf7, 0x77, 0xbd, 0x9e, - 0x43, 0x8a, 0x58, 0x09, 0xce, 0x09, 0x79, 0x83, 0xcd, 0x5f, 0x32, 0xbd, 0x44, 0xd7, 0x6a, 0x1f, - 0xb0, 0x23, 0x7f, 0xc6, 0x32, 0x7b, 0x88, 0x87, 0x81, 0x01, 0x5f, 0xf8, 0xd0, 0x9b, 0x0d, 0x7a, - 0xbd, 0x2e, 0xfb, 0xed, 0x48, 0x4f, 0x42, 0x0a, 0xa4, 0x8d, 0x4f, 0xb4, 0x34, 0x96, 0x29, 0xf6, - 0xb4, 0xc4, 0xfa, 0xca, 0x1f, 0x86, 0x5b, 0xa2, 0x7a, 0x40, 0x30, 0xd8, 0x23, 0x48, 0x1f, 0xa8, - 0xed, 0x51, 0x91, 0x54, 0x91, 0x96, 0x93, 0x49, 0x6e, 0xf4, 0x24, 0x72, 0xbc, 0x4a, 0x6a, 0xf8, - 0xaa, 0xc4, 0xa0, 0x4a, 0x23, 0xc7, 0xb0, 0x7c, 0x18, 0xed, 0xeb, 0x23, 0x5f, 0x1f, 0xee, 0xf4, - 0x1f, 0x67, 0x7e, 0xf4, 0x83, 0x45, 0x41, 0xfa, 0x04, 0x90, 0x8c, 0x1d, 0xec, 0xbe, 0xe8, 0x59, - 0x76, 0x70, 0x54, 0x4d, 0x7f, 0x0c, 0xfd, 0x64, 0x7c, 0x0c, 0xbd, 0x74, 0x11, 0x16, 0x22, 0xdc, - 0xcc, 0x58, 0x48, 0x1f, 0xc2, 0x95, 0xa7, 0x96, 0xe3, 0x18, 0x5d, 0x02, 0x7c, 0xe8, 0xa8, 0x24, - 0x53, 0x8b, 0x6f, 0x1e, 0xb3, 0x5d, 0x8a, 0x35, 0x4d, 0x66, 0x46, 0x72, 0xb2, 0xff, 0x2c, 0xfd, - 0xbe, 0x00, 0x97, 0x07, 0x39, 0x99, 0x96, 0xe3, 0xf6, 0xaa, 0x4e, 0x6b, 0x56, 0x70, 0xb4, 0xe1, - 0xd9, 0xbd, 0xd5, 0xcb, 0x4e, 0x1c, 0x29, 0xfe, 0x4e, 0xa5, 0xa3, 0x52, 0xf3, 0xc1, 0xf7, 0xcd, - 0x17, 0x38, 0x79, 0x83, 0x51, 0x03, 0x4b, 0x92, 0x19, 0xcf, 0x92, 0x34, 0x61, 0x6e, 0xcd, 0x32, - 0x4c, 0xe2, 0xaf, 0x79, 0xf5, 0x5d, 0x86, 0xc2, 0x8e, 0x61, 0xaa, 0xf6, 0xb1, 0xe2, 0x05, 0xf0, - 0x09, 0x67, 0x05, 0xf0, 0xc9, 0xb3, 0x8c, 0x83, 0x3f, 0x4a, 0x3f, 0x16, 0x40, 0x0c, 0xc4, 0x72, - 0x8b, 0xfc, 0x2e, 0x80, 0xd6, 0xee, 0x39, 0x2e, 0xb6, 0xbd, 0x56, 0x9a, 0x61, 0x91, 0xf9, 0x15, - 0x46, 0xad, 0xaf, 0xca, 0x39, 0x9e, 0xa1, 0xae, 0xa3, 0x1b, 0xd1, 0x63, 0x3d, 0x26, 0x57, 0xe0, - 0x74, 0xe0, 0x30, 0x0f, 0xd2, 0xec, 0x8e, 0x6b, 0xd9, 0x3e, 0x76, 0xe1, 0xcd, 0xee, 0x5d, 0x54, - 0x40, 0x77, 0xa3, 0x63, 0xba, 0xf9, 0xa6, 0x40, 0xdc, 0x85, 0x03, 0xec, 0x57, 0x29, 0x73, 0x76, - 0x95, 0x18, 0x87, 0x57, 0xa5, 0x7f, 0x25, 0xc0, 0x5c, 0x85, 0xb5, 0x86, 0xdf, 0xc2, 0x23, 0x2c, - 0xda, 0x2a, 0x64, 0xdd, 0x23, 0x53, 0xe9, 0x60, 0xff, 0x72, 0x9f, 0x73, 0x9c, 0x3b, 0x38, 0xed, - 0xb2, 0x47, 0x7a, 0x5f, 0x24, 0xbf, 0xac, 0x9c, 0x0f, 0x97, 0x2b, 0x65, 0x76, 0x9b, 0x79, 0xd9, - 0xbb, 0xcd, 0xbc, 0xbc, 0xca, 0x33, 0x30, 0xa3, 0xfe, 0xfd, 0xff, 0xba, 0x28, 0xc8, 0x3e, 0x13, - 0x9b, 0xf7, 0xef, 0x37, 0x48, 0xaf, 0x1f, 0x98, 0x99, 0x51, 0x01, 0x20, 0x74, 0x6b, 0x13, 0xbf, - 0x1f, 0x7b, 0x79, 0x55, 0xd9, 0xde, 0xac, 0x3c, 0xdf, 0xd8, 0xa8, 0x37, 0x9b, 0xd5, 0x55, 0x51, - 0x40, 0x22, 0xcc, 0x44, 0xee, 0x7c, 0x4a, 0xb1, 0x1b, 0xb3, 0xef, 0xff, 0x35, 0x80, 0xe0, 0xfa, - 0x38, 0x22, 0x6b, 0xbd, 0xfa, 0x99, 0xf2, 0x72, 0xf9, 0xd9, 0x76, 0xb5, 0x21, 0x4e, 0x20, 0x04, - 0x85, 0x95, 0xe5, 0x66, 0xa5, 0xa6, 0xc8, 0xd5, 0xc6, 0xd6, 0xf3, 0xcd, 0x46, 0xd5, 0xbb, 0x69, - 0xfb, 0xfe, 0x2a, 0xcc, 0x84, 0x0f, 0x6f, 0x42, 0x0b, 0x30, 0x57, 0xa9, 0x55, 0x2b, 0xeb, 0xca, - 0xcb, 0xfa, 0xb2, 0xf2, 0x62, 0xbb, 0xba, 0x5d, 0x15, 0x27, 0x68, 0xd1, 0x28, 0xf1, 0xc9, 0xf6, - 0xb3, 0x67, 0xa2, 0x80, 0xe6, 0x20, 0xcf, 0x9e, 0xe9, 0xfd, 0x50, 0x62, 0xea, 0xfe, 0x06, 0xe4, - 0x43, 0xa7, 0x3c, 0x93, 0xd7, 0x6d, 0x6d, 0x37, 0x6a, 0x4a, 0xb3, 0xbe, 0x51, 0x6d, 0x34, 0x97, - 0x37, 0xb6, 0x98, 0x0c, 0x4a, 0x5b, 0x5e, 0x79, 0x2e, 0x37, 0x45, 0xc1, 0x7f, 0x6e, 0x3e, 0xdf, - 0xae, 0xd4, 0xbc, 0x6a, 0x48, 0x99, 0x6c, 0x5a, 0x4c, 0xdf, 0xff, 0x9b, 0x02, 0x5c, 0x1e, 0x72, - 0x90, 0x11, 0xca, 0xc3, 0xf4, 0xb6, 0x49, 0x8f, 0xbc, 0x15, 0x27, 0xd0, 0x6c, 0xe8, 0x2c, 0x23, - 0x51, 0x40, 0x59, 0x76, 0x9a, 0x8c, 0x98, 0x42, 0x53, 0x90, 0x6a, 0x3c, 0x12, 0xd3, 0xa4, 0xa4, - 0xa1, 0xa3, 0x80, 0xc4, 0x0c, 0xca, 0xf1, 0x43, 0x48, 0xc4, 0x49, 0x34, 0x13, 0x9c, 0x05, 0x22, - 0x4e, 0x11, 0x51, 0xfe, 0x99, 0x1a, 0xe2, 0xf4, 0xfd, 0xeb, 0x10, 0x3a, 0xb7, 0x00, 0x01, 0x4c, - 0x3d, 0x53, 0x5d, 0xec, 0xb8, 0xe2, 0x04, 0x9a, 0x86, 0xf4, 0x72, 0xbb, 0x2d, 0x0a, 0x0f, 0xff, - 0x65, 0x06, 0xb2, 0xde, 0x3d, 0x48, 0xe8, 0x19, 0x4c, 0xb2, 0xc5, 0xc5, 0xc5, 0xe1, 0x68, 0x81, - 0x0e, 0xe8, 0xd2, 0xb5, 0xb3, 0xe0, 0x84, 0x34, 0x81, 0xfe, 0x3a, 0xe4, 0x43, 0x5e, 0x14, 0x1a, - 0xba, 0xa0, 0x13, 0xf1, 0x1c, 0x4b, 0xb7, 0xcf, 0xca, 0xe6, 0xcb, 0x7f, 0x05, 0x39, 0xdf, 0xaa, - 0xa3, 0x1b, 0xa3, 0x6c, 0xbe, 0x27, 0x7b, 0xf4, 0xc4, 0x40, 0xc6, 0x9f, 0x34, 0xf1, 0xbe, 0x80, - 0x6c, 0x40, 0x83, 0x06, 0x18, 0xc5, 0x85, 0x71, 0x0d, 0xb5, 0xf0, 0xa5, 0xfb, 0x63, 0xe5, 0x0e, - 0xde, 0x49, 0x94, 0x15, 0xcc, 0x22, 0xf1, 0xca, 0x1a, 0x98, 0xa3, 0xe2, 0x95, 0x15, 0x33, 0x19, - 0x4d, 0xa0, 0x17, 0x90, 0x21, 0xd6, 0x13, 0xc5, 0xf9, 0x95, 0x7d, 0xd6, 0xba, 0x74, 0x63, 0x64, - 0x1e, 0x4f, 0xe4, 0xca, 0xbd, 0x1f, 0xfd, 0xd9, 0xd5, 0x89, 0x1f, 0x9d, 0x5e, 0x15, 0x7e, 0x7c, - 0x7a, 0x55, 0xf8, 0x93, 0xd3, 0xab, 0xc2, 0x9f, 0x9e, 0x5e, 0x15, 0xbe, 0xf7, 0x93, 0xab, 0x13, - 0x3f, 0xfe, 0xc9, 0xd5, 0x89, 0x3f, 0xf9, 0xc9, 0xd5, 0x89, 0xcf, 0xa7, 0x39, 0xf7, 0xce, 0x14, - 0x35, 0x2d, 0x8f, 0xfe, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x92, 0x15, 0x80, 0x99, 0x8f, 0x82, - 0x00, 0x00, + 0x76, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0x51, 0x69, 0x7e, 0x38, 0xdc, 0xdd, 0xd1, 0x4c, + 0xcf, 0xff, 0xdc, 0x5d, 0x6a, 0x67, 0xe6, 0x6e, 0x76, 0xbd, 0xb3, 0xde, 0x6b, 0x89, 0xe2, 0x0c, + 0x29, 0x8d, 0x34, 0x9a, 0x26, 0x35, 0x83, 0x5d, 0x5f, 0xa7, 0xdd, 0xea, 0x2e, 0x51, 0x7d, 0x45, + 0x76, 0x73, 0xba, 0x9b, 0xfa, 0x59, 0x20, 0x40, 0x1c, 0x1b, 0xce, 0x7d, 0x0a, 0x6e, 0x00, 0x03, + 0xbe, 0x86, 0x83, 0xf8, 0x3a, 0xd7, 0x88, 0x1f, 0x02, 0x24, 0x01, 0x12, 0xe4, 0x0f, 0x89, 0xfd, + 0x12, 0x20, 0x17, 0x81, 0x13, 0x5f, 0xbf, 0x19, 0x01, 0xa2, 0xd8, 0xba, 0x79, 0x48, 0xe0, 0x20, + 0x08, 0xf2, 0x62, 0x60, 0x81, 0x04, 0x41, 0xfd, 0xf4, 0x1f, 0xd9, 0xa4, 0xa8, 0xd9, 0xbe, 0xf1, + 0x02, 0x7e, 0x91, 0x58, 0xa7, 0xea, 0x9c, 0xae, 0x3a, 0x55, 0x75, 0xea, 0x7c, 0xd5, 0xa7, 0xaa, + 0x61, 0xde, 0x32, 0x15, 0x75, 0xaf, 0xb7, 0xb3, 0xa4, 0xf4, 0xf4, 0x4a, 0xcf, 0x32, 0x1d, 0x13, + 0xcd, 0xab, 0xa6, 0xba, 0x4f, 0xc9, 0x15, 0x9e, 0x59, 0xbe, 0xbf, 0x7f, 0xb0, 0xb4, 0x7f, 0x60, + 0x63, 0xeb, 0x00, 0x5b, 0x4b, 0xaa, 0x69, 0xa8, 0x7d, 0xcb, 0xc2, 0x86, 0x7a, 0xbc, 0xd4, 0x31, + 0xd5, 0x7d, 0xfa, 0x47, 0x37, 0xda, 0x8c, 0x3d, 0x5c, 0xd6, 0xc2, 0x8a, 0x66, 0xf7, 0xbb, 0x5d, + 0xc5, 0x3a, 0x5e, 0xb2, 0xec, 0xde, 0xce, 0x12, 0x4f, 0xf0, 0xb2, 0xc8, 0x7d, 0xba, 0xa6, 0x38, + 0x0a, 0xa7, 0x5d, 0x70, 0x69, 0xd8, 0xb2, 0x4c, 0xcb, 0xe6, 0xd4, 0x4b, 0x2e, 0xb5, 0x8b, 0x1d, + 0x25, 0x50, 0xfa, 0x2d, 0xdb, 0x31, 0x2d, 0xa5, 0x8d, 0x97, 0xb0, 0xd1, 0xd6, 0x0d, 0x4c, 0x0a, + 0x1c, 0xa8, 0x2a, 0xcf, 0x7c, 0x3b, 0x32, 0xf3, 0x11, 0xcf, 0x2d, 0xf5, 0x1d, 0xbd, 0xb3, 0xb4, + 0xd7, 0x51, 0x97, 0x1c, 0xbd, 0x8b, 0x6d, 0x47, 0xe9, 0xf6, 0xdc, 0x26, 0xd0, 0x1c, 0xc7, 0x52, + 0x54, 0xdd, 0x68, 0xbb, 0xff, 0x7b, 0x3b, 0x4b, 0x16, 0x56, 0x4d, 0x4b, 0xc3, 0x9a, 0x6c, 0xf7, + 0x14, 0xc3, 0xad, 0x6e, 0xdb, 0x6c, 0x9b, 0xf4, 0xe7, 0x12, 0xf9, 0xc5, 0xa9, 0x57, 0xdb, 0xa6, + 0xd9, 0xee, 0xe0, 0x25, 0x9a, 0xda, 0xe9, 0xef, 0x2e, 0x69, 0x7d, 0x4b, 0x71, 0x74, 0x93, 0x73, + 0x89, 0xff, 0x4c, 0x80, 0x9c, 0x84, 0x5f, 0xf7, 0xb1, 0xed, 0xd4, 0xb1, 0xa2, 0x61, 0x0b, 0x5d, + 0x81, 0xe4, 0x3e, 0x3e, 0x2e, 0x25, 0xaf, 0x09, 0x77, 0xe7, 0x56, 0x66, 0xbf, 0x3c, 0x59, 0x4c, + 0xae, 0xe3, 0x63, 0x89, 0xd0, 0xd0, 0x35, 0x98, 0xc5, 0x86, 0x26, 0x93, 0xec, 0x54, 0x38, 0x7b, + 0x06, 0x1b, 0xda, 0x3a, 0x3e, 0x46, 0xdf, 0x86, 0xb4, 0x4d, 0xa4, 0x19, 0x2a, 0x2e, 0x4d, 0x5f, + 0x13, 0xee, 0x4e, 0xaf, 0xfc, 0xdc, 0x97, 0x27, 0x8b, 0x9f, 0xb4, 0x75, 0x67, 0xaf, 0xbf, 0x53, + 0x51, 0xcd, 0xee, 0x92, 0xd7, 0xa7, 0xda, 0x8e, 0xff, 0x7b, 0xa9, 0xb7, 0xdf, 0x5e, 0x1a, 0xd4, + 0x51, 0xa5, 0x75, 0x64, 0x34, 0xf1, 0x6b, 0xc9, 0x93, 0xb8, 0x96, 0x4a, 0x0b, 0xc5, 0xc4, 0x5a, + 0x2a, 0x9d, 0x28, 0x26, 0xc5, 0x3f, 0x4c, 0x40, 0x5e, 0xc2, 0x76, 0xcf, 0x34, 0x6c, 0xcc, 0x6b, + 0xfe, 0x3e, 0x24, 0x9d, 0x23, 0x83, 0xd6, 0x3c, 0xfb, 0xf0, 0x6a, 0x65, 0x68, 0xf4, 0x54, 0x5a, + 0x96, 0x62, 0xd8, 0x8a, 0x4a, 0x9a, 0x2f, 0x91, 0xa2, 0xe8, 0x23, 0xc8, 0x5a, 0xd8, 0xee, 0x77, + 0x31, 0x55, 0x24, 0x6d, 0x54, 0xf6, 0xe1, 0xe5, 0x08, 0xce, 0x66, 0x4f, 0x31, 0x24, 0x60, 0x65, + 0xc9, 0x6f, 0xd4, 0x84, 0x1c, 0xe7, 0xb4, 0xb0, 0x62, 0x9b, 0x46, 0x69, 0xf6, 0x9a, 0x70, 0x37, + 0xff, 0xb0, 0x12, 0xc1, 0x1b, 0xae, 0x25, 0x49, 0xf6, 0xbb, 0x58, 0xa2, 0x5c, 0xd2, 0x9c, 0x15, + 0x48, 0xa1, 0x2b, 0x90, 0x36, 0xfa, 0x5d, 0xa2, 0x5f, 0x9b, 0x6a, 0x2f, 0x29, 0xcd, 0x1a, 0xfd, + 0xee, 0x3a, 0x3e, 0xb6, 0xd1, 0x5b, 0x90, 0x21, 0x59, 0x3b, 0xc7, 0x0e, 0xb6, 0x4b, 0x69, 0x9a, + 0x47, 0xca, 0xae, 0x90, 0xb4, 0xf8, 0x29, 0xcc, 0x05, 0xa5, 0x22, 0x04, 0x79, 0xa9, 0xd6, 0xdc, + 0xde, 0xa8, 0xc9, 0xdb, 0x9b, 0xeb, 0x9b, 0xcf, 0x5f, 0x6d, 0x16, 0xa7, 0xd0, 0x05, 0x28, 0x72, + 0xda, 0x7a, 0xed, 0x33, 0xf9, 0x59, 0x63, 0xa3, 0xd1, 0x2a, 0x0a, 0xe5, 0xd4, 0x77, 0x7f, 0x78, + 0x75, 0x6a, 0x2d, 0x95, 0x9e, 0x29, 0xce, 0x8a, 0x3f, 0x14, 0x00, 0x9e, 0x62, 0x87, 0x8f, 0x06, + 0xb4, 0x02, 0x33, 0x7b, 0xb4, 0xc6, 0x25, 0x81, 0xaa, 0xe5, 0x5a, 0x64, 0xd3, 0x02, 0x23, 0x67, + 0x25, 0xfd, 0xa3, 0x93, 0xc5, 0xa9, 0x1f, 0x9f, 0x2c, 0x0a, 0x12, 0xe7, 0x44, 0x2f, 0x20, 0xbb, + 0x8f, 0x8f, 0x65, 0x3e, 0x2f, 0x4b, 0x09, 0xaa, 0xa3, 0xf7, 0x03, 0x82, 0xf6, 0x0f, 0x2a, 0xee, + 0x14, 0xad, 0x04, 0xa6, 0x73, 0x85, 0x70, 0x54, 0x9a, 0x8e, 0x85, 0x8d, 0xb6, 0xb3, 0x27, 0xc1, + 0x3e, 0x3e, 0x7e, 0xc6, 0x64, 0x88, 0xbf, 0x2f, 0x40, 0x96, 0xd6, 0x92, 0x29, 0x15, 0x55, 0x07, + 0xaa, 0x79, 0xfd, 0xcc, 0x1e, 0x88, 0xa8, 0x67, 0x05, 0xa6, 0x0f, 0x94, 0x4e, 0x1f, 0xd3, 0x1a, + 0x66, 0x1f, 0x96, 0x22, 0x64, 0xbc, 0x24, 0xf9, 0x12, 0x2b, 0x86, 0x1e, 0xc3, 0x9c, 0x6e, 0x38, + 0xd8, 0x70, 0x64, 0xc6, 0x96, 0x3c, 0x83, 0x2d, 0xcb, 0x4a, 0xd3, 0x84, 0xf8, 0x4f, 0x05, 0x80, + 0xad, 0x7e, 0xac, 0x7a, 0xfe, 0xe6, 0x84, 0xf5, 0x5f, 0x49, 0x11, 0x56, 0xb7, 0x15, 0x97, 0x60, + 0x46, 0x37, 0x3a, 0xba, 0xc1, 0xea, 0x9f, 0x96, 0x78, 0x0a, 0x5d, 0x80, 0xe9, 0x9d, 0x8e, 0x6e, + 0x68, 0x74, 0x3e, 0xa4, 0x25, 0x96, 0x10, 0x25, 0xc8, 0xd2, 0x5a, 0xc7, 0xa8, 0x77, 0xf1, 0x24, + 0x01, 0x17, 0xab, 0xa6, 0xa1, 0xe9, 0x64, 0x4a, 0x2a, 0x9d, 0xaf, 0x85, 0x56, 0xd6, 0xe0, 0x82, + 0x86, 0x7b, 0x16, 0x56, 0x15, 0x07, 0x6b, 0x32, 0x3e, 0xea, 0x4d, 0xd8, 0xc7, 0xc8, 0xe7, 0xaa, + 0x1d, 0xf5, 0x28, 0x8d, 0xcc, 0x5a, 0x22, 0x80, 0xcd, 0xda, 0x19, 0x62, 0x32, 0xa5, 0x34, 0x3e, + 0xea, 0xd1, 0x59, 0x1b, 0xad, 0x66, 0xf4, 0x4d, 0xb8, 0xac, 0x74, 0x3a, 0xe6, 0xa1, 0xac, 0xef, + 0xca, 0x9a, 0x89, 0x6d, 0xd9, 0x30, 0x1d, 0x19, 0x1f, 0xe9, 0xb6, 0x43, 0x4d, 0x42, 0x5a, 0x5a, + 0xa0, 0xd9, 0x8d, 0xdd, 0x55, 0x13, 0xdb, 0x9b, 0xa6, 0x53, 0x23, 0x59, 0x81, 0xae, 0x9c, 0x0d, + 0x76, 0xa5, 0xf8, 0x0b, 0x70, 0x69, 0x50, 0xbf, 0x71, 0xf6, 0xdf, 0x1f, 0x08, 0x90, 0x6f, 0x18, + 0xba, 0xf3, 0xb5, 0xe8, 0x38, 0x4f, 0x9f, 0xc9, 0xa0, 0x3e, 0xef, 0x43, 0x71, 0x57, 0xd1, 0x3b, + 0xcf, 0x8d, 0x96, 0xd9, 0xdd, 0xb1, 0x1d, 0xd3, 0xc0, 0x36, 0x57, 0xf8, 0x10, 0x5d, 0x7c, 0x09, + 0x05, 0xaf, 0x35, 0x71, 0xaa, 0xc9, 0x81, 0x62, 0xc3, 0x50, 0x2d, 0xdc, 0xc5, 0x46, 0xac, 0x7a, + 0x7a, 0x1b, 0x32, 0xba, 0x2b, 0x97, 0xea, 0x2a, 0x29, 0xf9, 0x04, 0xb1, 0x0f, 0xf3, 0x81, 0xa7, + 0xc6, 0x69, 0x2e, 0xc9, 0x62, 0x84, 0x0f, 0x65, 0xbf, 0x8f, 0xc8, 0x62, 0x84, 0x0f, 0x99, 0x79, + 0x6b, 0x42, 0x6e, 0x15, 0x77, 0xb0, 0x83, 0x63, 0x6c, 0xa9, 0xb8, 0x0d, 0x79, 0x57, 0x68, 0x9c, + 0x1d, 0xf3, 0xeb, 0x02, 0x20, 0x2e, 0x57, 0x31, 0xda, 0x71, 0xd6, 0x18, 0x2d, 0x12, 0xd7, 0xc2, + 0xe9, 0x5b, 0x06, 0x5b, 0xce, 0xd9, 0x98, 0x04, 0x46, 0xa2, 0x2b, 0xba, 0x3f, 0x65, 0x53, 0xc1, + 0x29, 0xcb, 0xdd, 0x9b, 0x43, 0x58, 0x08, 0x55, 0x2c, 0xde, 0xee, 0x4b, 0xd1, 0x3a, 0x25, 0xae, + 0x25, 0x83, 0x3e, 0x1c, 0x25, 0x8a, 0xdf, 0x17, 0x60, 0xbe, 0xda, 0xc1, 0x8a, 0x15, 0xbb, 0x46, + 0xbe, 0x05, 0x69, 0x0d, 0x2b, 0x1a, 0x6d, 0x32, 0x9b, 0xd8, 0xef, 0x04, 0xa4, 0x10, 0x4f, 0xb7, + 0xb2, 0xd7, 0x51, 0x2b, 0x2d, 0xd7, 0x07, 0xe6, 0xb3, 0xdb, 0x63, 0x12, 0x3f, 0x03, 0x14, 0xac, + 0x59, 0x9c, 0x03, 0xe1, 0x77, 0x12, 0x80, 0x24, 0x7c, 0x80, 0x2d, 0x27, 0xf6, 0x66, 0xaf, 0x42, + 0xd6, 0x51, 0xac, 0x36, 0x76, 0x64, 0xe2, 0xdd, 0x9f, 0xa7, 0xe5, 0xc0, 0xf8, 0x08, 0x19, 0xb5, + 0xe0, 0x0e, 0x36, 0x94, 0x9d, 0x0e, 0xa6, 0x52, 0xe4, 0x1d, 0xb3, 0x6f, 0x68, 0xb2, 0xee, 0x60, + 0x4b, 0x71, 0x4c, 0x4b, 0x36, 0x7b, 0x8e, 0xde, 0xd5, 0xbf, 0xa0, 0x8e, 0x3d, 0x1f, 0x6a, 0x37, + 0x58, 0x71, 0xc2, 0xbc, 0x42, 0x0a, 0x37, 0x78, 0xd9, 0xe7, 0x81, 0xa2, 0xa8, 0x02, 0x0b, 0x7a, + 0xdb, 0x30, 0x2d, 0x2c, 0xb7, 0x55, 0xd9, 0xd9, 0xb3, 0xb0, 0xbd, 0x67, 0x76, 0xdc, 0x05, 0x69, + 0x9e, 0x65, 0x3d, 0x55, 0x5b, 0x6e, 0x86, 0xf8, 0x39, 0x2c, 0x84, 0xb4, 0x14, 0x67, 0x17, 0xfc, + 0x2f, 0x01, 0xb2, 0x4d, 0x55, 0x31, 0xe2, 0xd4, 0xfd, 0xa7, 0x90, 0xb5, 0x55, 0xc5, 0x90, 0x77, + 0x4d, 0xab, 0xab, 0x38, 0xb4, 0x5d, 0xf9, 0x90, 0xee, 0x3d, 0xff, 0x5e, 0x55, 0x8c, 0x27, 0xb4, + 0x90, 0x04, 0xb6, 0xf7, 0x7b, 0xd0, 0x7f, 0x9d, 0xfe, 0xea, 0xfe, 0x2b, 0x9b, 0xde, 0x6b, 0xa9, + 0x74, 0xb2, 0x98, 0x12, 0xff, 0x5c, 0x80, 0x39, 0xd6, 0xe4, 0x38, 0xa7, 0xf7, 0x07, 0x90, 0xb2, + 0xcc, 0x43, 0x36, 0xbd, 0xb3, 0x0f, 0xdf, 0x8a, 0x10, 0xb1, 0x8e, 0x8f, 0x83, 0xeb, 0x27, 0x2d, + 0x8e, 0x56, 0x80, 0x7b, 0xa9, 0x32, 0xe5, 0x4e, 0x4e, 0xca, 0x0d, 0x8c, 0x4b, 0x22, 0x32, 0xee, + 0x40, 0x61, 0x47, 0x71, 0xd4, 0x3d, 0xd9, 0xe2, 0x95, 0x24, 0x6b, 0x6d, 0xf2, 0xee, 0x9c, 0x94, + 0xa7, 0x64, 0xb7, 0xea, 0x36, 0x69, 0x39, 0x9b, 0x6f, 0x36, 0xfe, 0x4b, 0xd6, 0xe7, 0xff, 0x57, + 0xe0, 0x73, 0xc8, 0x6d, 0xf9, 0x5f, 0xb6, 0xae, 0xff, 0x8d, 0x04, 0x5c, 0xae, 0xee, 0x61, 0x75, + 0xbf, 0x6a, 0x1a, 0xb6, 0x6e, 0x3b, 0x44, 0x77, 0x71, 0xf6, 0xff, 0x5b, 0x90, 0x39, 0xd4, 0x9d, + 0x3d, 0x59, 0xd3, 0x77, 0x77, 0xa9, 0xb5, 0x4d, 0x4b, 0x69, 0x42, 0x58, 0xd5, 0x77, 0x77, 0xd1, + 0x23, 0x48, 0x75, 0x4d, 0x8d, 0x39, 0xf3, 0xf9, 0x87, 0x8b, 0x11, 0xe2, 0x69, 0xd5, 0xec, 0x7e, + 0x77, 0xc3, 0xd4, 0xb0, 0x44, 0x0b, 0xa3, 0xab, 0x00, 0x2a, 0xa1, 0xf6, 0x4c, 0xdd, 0x70, 0xb8, + 0x71, 0x0c, 0x50, 0x50, 0x1d, 0x32, 0x0e, 0xb6, 0xba, 0xba, 0xa1, 0x38, 0xb8, 0x34, 0x4d, 0x95, + 0x77, 0x33, 0xb2, 0xe2, 0xbd, 0x8e, 0xae, 0x2a, 0xab, 0xd8, 0x56, 0x2d, 0xbd, 0xe7, 0x98, 0x16, + 0xd7, 0xa2, 0xcf, 0x2c, 0xfe, 0xad, 0x14, 0x94, 0x86, 0x75, 0x13, 0xe7, 0x08, 0xd9, 0x82, 0x19, + 0x0b, 0xdb, 0xfd, 0x8e, 0xc3, 0xc7, 0xc8, 0xc3, 0x51, 0x2a, 0x88, 0xa8, 0x01, 0xdd, 0xba, 0xe8, + 0x38, 0xbc, 0xda, 0x5c, 0x4e, 0xf9, 0x5f, 0x0b, 0x30, 0xc3, 0x32, 0xd0, 0x03, 0x48, 0x5b, 0x64, + 0x61, 0x90, 0x75, 0x8d, 0xd6, 0x31, 0xb9, 0x72, 0xe9, 0xf4, 0x64, 0x71, 0x96, 0x2e, 0x16, 0x8d, + 0xd5, 0x2f, 0xfd, 0x9f, 0xd2, 0x2c, 0x2d, 0xd7, 0xd0, 0x48, 0x6f, 0xd9, 0x8e, 0x62, 0x39, 0x74, + 0x53, 0x29, 0xc1, 0x10, 0x12, 0x25, 0xac, 0xe3, 0x63, 0xb4, 0x06, 0x33, 0xb6, 0xa3, 0x38, 0x7d, + 0x9b, 0xf7, 0xd7, 0xb9, 0x2a, 0xdb, 0xa4, 0x9c, 0x12, 0x97, 0x40, 0xdc, 0x2d, 0x0d, 0x3b, 0x8a, + 0xde, 0xa1, 0x1d, 0x98, 0x91, 0x78, 0x4a, 0xfc, 0x4d, 0x01, 0x66, 0x58, 0x51, 0x74, 0x19, 0x16, + 0xa4, 0xe5, 0xcd, 0xa7, 0x35, 0xb9, 0xb1, 0xb9, 0x5a, 0x6b, 0xd5, 0xa4, 0x8d, 0xc6, 0xe6, 0x72, + 0xab, 0x56, 0x9c, 0x42, 0x97, 0x00, 0xb9, 0x19, 0xd5, 0xe7, 0x9b, 0xcd, 0x46, 0xb3, 0x55, 0xdb, + 0x6c, 0x15, 0x05, 0xba, 0xa7, 0x42, 0xe9, 0x01, 0x6a, 0x02, 0xdd, 0x84, 0x6b, 0x83, 0x54, 0xb9, + 0xd9, 0x5a, 0x6e, 0x35, 0xe5, 0x5a, 0xb3, 0xd5, 0xd8, 0x58, 0x6e, 0xd5, 0x56, 0x8b, 0xc9, 0x31, + 0xa5, 0xc8, 0x43, 0x24, 0xa9, 0x56, 0x6d, 0x15, 0x53, 0xa2, 0x03, 0x17, 0x25, 0xac, 0x9a, 0xdd, + 0x5e, 0xdf, 0xc1, 0xa4, 0x96, 0x76, 0x9c, 0x33, 0xe5, 0x32, 0xcc, 0x6a, 0xd6, 0xb1, 0x6c, 0xf5, + 0x0d, 0x3e, 0x4f, 0x66, 0x34, 0xeb, 0x58, 0xea, 0x1b, 0xe2, 0x3f, 0x16, 0xe0, 0xd2, 0xe0, 0x63, + 0xe3, 0x1c, 0x84, 0x2f, 0x20, 0xab, 0x68, 0x1a, 0xd6, 0x64, 0x0d, 0x77, 0x1c, 0x85, 0xbb, 0x44, + 0xf7, 0x03, 0x92, 0xf8, 0x56, 0x60, 0xc5, 0xdb, 0x0a, 0xdc, 0x78, 0x59, 0xad, 0xd2, 0x8a, 0xac, + 0x12, 0x0e, 0xd7, 0xfc, 0x50, 0x21, 0x94, 0x22, 0xfe, 0x8f, 0x14, 0xe4, 0x6a, 0x86, 0xd6, 0x3a, + 0x8a, 0x75, 0x2d, 0xb9, 0x04, 0x33, 0xaa, 0xd9, 0xed, 0xea, 0x8e, 0xab, 0x20, 0x96, 0x42, 0x3f, + 0x13, 0x70, 0x65, 0x93, 0x13, 0x38, 0x74, 0xbe, 0x13, 0x8b, 0x7e, 0x11, 0x2e, 0x13, 0xab, 0x69, + 0x19, 0x4a, 0x47, 0x66, 0xd2, 0x64, 0xc7, 0xd2, 0xdb, 0x6d, 0x6c, 0xf1, 0xed, 0xc7, 0xbb, 0x11, + 0xf5, 0x6c, 0x70, 0x8e, 0x2a, 0x65, 0x68, 0xb1, 0xf2, 0xd2, 0x45, 0x3d, 0x8a, 0x8c, 0x3e, 0x01, + 0x20, 0x4b, 0x11, 0xdd, 0xd2, 0xb4, 0xb9, 0x3d, 0x1a, 0xb5, 0xa7, 0xe9, 0x9a, 0x20, 0xc2, 0x40, + 0xd2, 0x36, 0x7a, 0x01, 0x45, 0xdd, 0x90, 0x77, 0x3b, 0x7a, 0x7b, 0xcf, 0x91, 0x0f, 0x2d, 0xdd, + 0xc1, 0x76, 0x69, 0x9e, 0xca, 0x88, 0xea, 0xea, 0x26, 0xdf, 0x9a, 0xd5, 0x5e, 0x91, 0x92, 0x5c, + 0x5a, 0x5e, 0x37, 0x9e, 0x50, 0x7e, 0x4a, 0xb4, 0xd1, 0x12, 0x81, 0x42, 0xaf, 0xfb, 0xba, 0x85, + 0xe5, 0x07, 0x3d, 0x95, 0xee, 0x83, 0xa4, 0x57, 0xf2, 0xa7, 0x27, 0x8b, 0x20, 0x31, 0xf2, 0x83, + 0xad, 0x2a, 0x81, 0x46, 0xec, 0x77, 0x4f, 0x45, 0xaf, 0xe0, 0x5e, 0x60, 0x0b, 0x86, 0x2c, 0xe6, + 0x5c, 0x53, 0x8a, 0x23, 0xef, 0xe9, 0xed, 0x3d, 0x6c, 0xc9, 0xde, 0x4e, 0x39, 0xdd, 0x0c, 0x4d, + 0x4b, 0x37, 0x7d, 0x86, 0xaa, 0x62, 0x30, 0x85, 0x2c, 0x3b, 0x75, 0x5a, 0xd8, 0xeb, 0x06, 0xd2, + 0x9f, 0x3d, 0x53, 0xb7, 0x4d, 0xa3, 0x94, 0x61, 0xfd, 0xc9, 0x52, 0xe8, 0x1e, 0x14, 0x9d, 0x23, + 0x43, 0xde, 0xc3, 0x8a, 0xe5, 0xec, 0x60, 0xc5, 0x21, 0x0b, 0x3f, 0xd0, 0x12, 0x05, 0xe7, 0xc8, + 0xa8, 0x07, 0xc8, 0x6b, 0xa9, 0xf4, 0x6c, 0x31, 0x2d, 0xfe, 0x67, 0x01, 0xf2, 0xee, 0x70, 0x8b, + 0x73, 0x66, 0xdc, 0x85, 0xa2, 0x69, 0x60, 0xb9, 0xb7, 0xa7, 0xd8, 0x98, 0x37, 0x9a, 0x2f, 0x38, + 0x79, 0xd3, 0xc0, 0x5b, 0x84, 0xcc, 0xda, 0x86, 0xb6, 0x60, 0xde, 0x76, 0x94, 0xb6, 0x6e, 0xb4, + 0x03, 0xba, 0x98, 0x9e, 0x1c, 0x5c, 0x14, 0x39, 0xb7, 0x47, 0x0f, 0x79, 0x29, 0x7f, 0x24, 0xc0, + 0xfc, 0xb2, 0xd6, 0xd5, 0x8d, 0x66, 0xaf, 0xa3, 0xc7, 0xba, 0x67, 0x71, 0x13, 0x32, 0x36, 0x91, + 0xe9, 0x1b, 0x7c, 0x1f, 0x81, 0xa6, 0x69, 0x0e, 0xb1, 0xfc, 0xcf, 0xa0, 0x80, 0x8f, 0x7a, 0x3a, + 0x7b, 0x55, 0xc1, 0x80, 0x53, 0x6a, 0xf2, 0xb6, 0xe5, 0x7d, 0x5e, 0x92, 0xc5, 0xdb, 0xf4, 0x19, + 0xa0, 0x60, 0x93, 0xe2, 0xc4, 0x2e, 0x9f, 0xc1, 0x02, 0x15, 0xbd, 0x6d, 0xd8, 0x31, 0xeb, 0x4b, + 0xfc, 0x79, 0xb8, 0x10, 0x16, 0x1d, 0x67, 0xbd, 0x5f, 0xf1, 0x5e, 0xde, 0xc0, 0x56, 0xac, 0xa0, + 0xd7, 0xd3, 0x35, 0x17, 0x1c, 0x67, 0x9d, 0x7f, 0x45, 0x80, 0x2b, 0x54, 0x36, 0x7d, 0x9b, 0xb3, + 0x8b, 0xad, 0x67, 0x58, 0xb1, 0x63, 0x45, 0xec, 0x37, 0x60, 0x86, 0x21, 0x6f, 0x3a, 0x3e, 0xa7, + 0x57, 0xb2, 0xc4, 0x73, 0x69, 0x3a, 0xa6, 0x45, 0x3c, 0x17, 0x9e, 0x25, 0x2a, 0x50, 0x8e, 0xaa, + 0x45, 0x9c, 0x2d, 0xfd, 0xbb, 0x02, 0xcc, 0x73, 0xa7, 0x91, 0x0c, 0xe5, 0xea, 0x1e, 0xf1, 0x99, + 0x50, 0x0d, 0xb2, 0x2a, 0xfd, 0x25, 0x3b, 0xc7, 0x3d, 0x4c, 0xe5, 0xe7, 0xc7, 0xf9, 0x9b, 0x8c, + 0xad, 0x75, 0xdc, 0xc3, 0xc4, 0x69, 0x75, 0x7f, 0x13, 0x45, 0x05, 0x1a, 0x39, 0xd6, 0x63, 0xa5, + 0xf3, 0x88, 0x96, 0x75, 0x5d, 0x3f, 0xae, 0x83, 0x7f, 0x92, 0xe4, 0x4a, 0x60, 0xcf, 0xe0, 0xc5, + 0x63, 0xf5, 0x51, 0x3e, 0x87, 0x4b, 0xc1, 0xa5, 0x20, 0xd0, 0xf0, 0xc4, 0x39, 0x1a, 0x1e, 0xd8, + 0xd1, 0xf7, 0xa9, 0xe8, 0x33, 0x08, 0xec, 0xd9, 0xcb, 0xac, 0x4d, 0x2e, 0xfa, 0x39, 0x8f, 0x3a, + 0xe6, 0x7d, 0x29, 0x8c, 0x6e, 0xa3, 0x2a, 0xa4, 0xf1, 0x51, 0x4f, 0xd6, 0xb0, 0xad, 0x72, 0xc3, + 0x25, 0x46, 0x09, 0x24, 0x55, 0x19, 0xc2, 0x03, 0xb3, 0xf8, 0xa8, 0x47, 0x88, 0x68, 0x9b, 0x2c, + 0xc5, 0xae, 0xab, 0x40, 0xab, 0x6d, 0x9f, 0x0d, 0x2f, 0xfc, 0x91, 0xc2, 0xc5, 0x15, 0x3c, 0x2f, + 0x81, 0x89, 0x10, 0x7f, 0x20, 0xc0, 0x5b, 0x91, 0xbd, 0x16, 0xe7, 0x42, 0xf6, 0x09, 0xa4, 0x68, + 0xe3, 0x13, 0xe7, 0x6c, 0x3c, 0xe5, 0x12, 0xbf, 0x9b, 0xe0, 0x73, 0x5c, 0xc2, 0x1d, 0x93, 0x28, + 0x36, 0xf6, 0x5d, 0xb9, 0xe7, 0x90, 0x3b, 0x30, 0x1d, 0xe2, 0x48, 0xf0, 0x6e, 0x4f, 0x9c, 0xbb, + 0xdb, 0xe7, 0xa8, 0x00, 0xb7, 0xc7, 0x5f, 0xc2, 0xbc, 0x61, 0x1a, 0x72, 0x58, 0xe8, 0xf9, 0xc7, + 0x52, 0xc1, 0x30, 0x8d, 0x97, 0x01, 0xb9, 0x9e, 0x9d, 0x19, 0xd0, 0x44, 0x9c, 0x76, 0xe6, 0x7b, + 0x02, 0x2c, 0x78, 0x3e, 0x4e, 0xcc, 0x1e, 0xf4, 0x07, 0x90, 0x34, 0xcc, 0xc3, 0xf3, 0xec, 0x7a, + 0x92, 0xf2, 0x64, 0xd5, 0x0b, 0xd7, 0x28, 0xce, 0xf6, 0xfe, 0x9b, 0x04, 0x64, 0x9e, 0x56, 0xe3, + 0x6c, 0xe5, 0x27, 0x7c, 0x47, 0x9d, 0xf5, 0x77, 0xd4, 0x68, 0xf7, 0x9e, 0x57, 0x79, 0x5a, 0x5d, + 0xc7, 0xc7, 0xee, 0x68, 0x27, 0x5c, 0x68, 0x19, 0x32, 0xe1, 0xbd, 0xd7, 0x09, 0x35, 0xe5, 0x73, + 0x95, 0x31, 0x4c, 0x53, 0xb9, 0x6e, 0xf4, 0x86, 0x10, 0x11, 0xbd, 0x41, 0x1e, 0xe3, 0x79, 0x8a, + 0x89, 0xf3, 0x3c, 0x26, 0xe0, 0x22, 0x4e, 0x17, 0x67, 0xc4, 0x17, 0x00, 0xa4, 0x39, 0x71, 0x76, + 0xc9, 0xaf, 0x26, 0x21, 0xbf, 0xd5, 0xb7, 0xf7, 0x62, 0x1e, 0x7d, 0x55, 0x80, 0x5e, 0xdf, 0xa6, + 0x78, 0xe1, 0xc8, 0xe0, 0x6d, 0x3e, 0x23, 0x30, 0xc4, 0x6d, 0x34, 0xe3, 0x6b, 0x1d, 0x19, 0xa8, + 0xce, 0x85, 0x60, 0xd9, 0x8f, 0x2e, 0xb9, 0x31, 0x0e, 0xac, 0xb6, 0x8e, 0x8c, 0x0d, 0xec, 0xa1, + 0x54, 0x26, 0x09, 0x13, 0x49, 0x9f, 0xc0, 0x2c, 0x49, 0xc8, 0x8e, 0x79, 0x9e, 0x6e, 0x9e, 0x21, + 0x3c, 0x2d, 0x13, 0x3d, 0x86, 0x0c, 0xe3, 0x26, 0xab, 0xdf, 0x0c, 0x5d, 0xfd, 0xa2, 0xda, 0xc2, + 0xd5, 0x48, 0xd7, 0xbd, 0x34, 0x65, 0x25, 0x6b, 0xdd, 0x05, 0x98, 0xde, 0x35, 0x2d, 0xd5, 0x7d, + 0x3f, 0xcc, 0x12, 0xac, 0x3f, 0xd7, 0x52, 0xe9, 0x74, 0x31, 0xb3, 0x96, 0x4a, 0x67, 0x8a, 0x20, + 0xfe, 0xa6, 0x00, 0x05, 0xaf, 0x23, 0xe2, 0x5c, 0x10, 0xaa, 0x21, 0x2d, 0x9e, 0xbf, 0x2b, 0x88, + 0x02, 0xc5, 0x7f, 0x47, 0x3d, 0x22, 0xd5, 0x3c, 0xa0, 0x3d, 0x13, 0xe7, 0x48, 0x79, 0xcc, 0x62, + 0x87, 0x12, 0xe7, 0xed, 0x5d, 0x1a, 0x46, 0xf4, 0x00, 0x2e, 0xe8, 0x5d, 0x62, 0xcf, 0x75, 0xa7, + 0x73, 0xcc, 0x61, 0x9b, 0x83, 0xdd, 0x17, 0xd1, 0x0b, 0x7e, 0x5e, 0xd5, 0xcd, 0x12, 0x7f, 0x87, + 0x6e, 0x80, 0xfb, 0x2d, 0x89, 0x53, 0xd5, 0x0d, 0xc8, 0x59, 0x4c, 0x34, 0x71, 0x6b, 0xce, 0xa9, + 0xed, 0x39, 0x8f, 0x95, 0x28, 0xfc, 0xb7, 0x13, 0x50, 0x78, 0xd1, 0xc7, 0xd6, 0xf1, 0xd7, 0x49, + 0xdd, 0xb7, 0xa1, 0x70, 0xa8, 0xe8, 0x8e, 0xbc, 0x6b, 0x5a, 0x72, 0xbf, 0xa7, 0x29, 0x8e, 0x1b, + 0xc0, 0x92, 0x23, 0xe4, 0x27, 0xa6, 0xb5, 0x4d, 0x89, 0x08, 0x03, 0xda, 0x37, 0xcc, 0x43, 0x43, + 0x26, 0x64, 0x0a, 0x94, 0x8f, 0x0c, 0xbe, 0x2b, 0xbd, 0xf2, 0xe1, 0x7f, 0x3a, 0x59, 0x7c, 0x34, + 0x51, 0x58, 0x1a, 0x0d, 0xc1, 0xeb, 0xf7, 0x75, 0xad, 0xb2, 0xbd, 0xdd, 0x58, 0x95, 0x8a, 0x54, + 0xe4, 0x2b, 0x26, 0xb1, 0x75, 0x64, 0xd8, 0xe2, 0xdf, 0x4f, 0x40, 0xd1, 0xd7, 0x51, 0x9c, 0x1d, + 0x59, 0x83, 0xec, 0xeb, 0x3e, 0xb6, 0xf4, 0x37, 0xe8, 0x46, 0xe0, 0x8c, 0xc4, 0xec, 0xdc, 0x87, + 0x79, 0xe7, 0xc8, 0x90, 0x59, 0xd0, 0x20, 0x8b, 0x25, 0x71, 0x63, 0x20, 0x0a, 0x0e, 0xa9, 0x33, + 0xa1, 0xd3, 0x38, 0x12, 0x1b, 0x7d, 0x0e, 0x73, 0x21, 0x6d, 0x25, 0xbf, 0x9a, 0xb6, 0xb2, 0x87, + 0x01, 0x45, 0xfd, 0xbe, 0x00, 0x88, 0x2a, 0xaa, 0xc1, 0x5e, 0x1b, 0x7c, 0x5d, 0xc6, 0xd3, 0x5d, + 0x28, 0xd2, 0x10, 0x4f, 0x59, 0xdf, 0x95, 0xbb, 0xba, 0x6d, 0xeb, 0x46, 0x9b, 0x0f, 0xa8, 0x3c, + 0xa5, 0x37, 0x76, 0x37, 0x18, 0x55, 0xfc, 0x6b, 0xb0, 0x10, 0x6a, 0x40, 0x9c, 0x9d, 0x7d, 0x1d, + 0xe6, 0x76, 0xd9, 0x5b, 0x5d, 0x2a, 0x9c, 0xef, 0x38, 0x66, 0x29, 0x8d, 0x3d, 0x4f, 0xfc, 0xb3, + 0x04, 0x5c, 0x90, 0xb0, 0x6d, 0x76, 0x0e, 0x70, 0xfc, 0x2a, 0xac, 0x03, 0x7f, 0x9d, 0x23, 0xbf, + 0x91, 0x26, 0x33, 0x8c, 0x99, 0x2d, 0x73, 0xe1, 0x6d, 0xfb, 0x9b, 0xe3, 0x47, 0xec, 0xf0, 0x46, + 0x3d, 0xdf, 0xa3, 0x4b, 0x85, 0xf6, 0xe8, 0x4c, 0x28, 0xb0, 0x17, 0xd2, 0x9a, 0x6c, 0xe3, 0xd7, + 0x46, 0xbf, 0xeb, 0x82, 0xa1, 0xca, 0xb8, 0x4a, 0x36, 0x18, 0x4b, 0x13, 0xbf, 0xde, 0xec, 0x77, + 0xa9, 0xef, 0xbc, 0x72, 0x89, 0xd4, 0xf7, 0xf4, 0x64, 0x31, 0x1f, 0xca, 0xb3, 0xa5, 0xbc, 0xee, + 0xa5, 0x89, 0x74, 0xf1, 0xdb, 0x70, 0x71, 0x40, 0xd9, 0x71, 0x7a, 0x3c, 0xff, 0x2a, 0x09, 0x57, + 0xc2, 0xe2, 0xe3, 0x86, 0x38, 0x5f, 0xf7, 0x0e, 0xad, 0x43, 0xae, 0xab, 0x1b, 0x6f, 0xb6, 0x7b, + 0x39, 0xd7, 0xd5, 0x0d, 0x7f, 0x5b, 0x37, 0x62, 0x68, 0xcc, 0xfc, 0x54, 0x87, 0x86, 0x02, 0xe5, + 0xa8, 0xbe, 0x8b, 0x73, 0x7c, 0x7c, 0x57, 0x80, 0xb9, 0xb8, 0xb7, 0xe5, 0xde, 0x2c, 0xb0, 0x4e, + 0x6c, 0x41, 0xee, 0xa7, 0xb0, 0x8f, 0xf7, 0xdb, 0x02, 0xa0, 0x96, 0xd5, 0x37, 0x08, 0xa8, 0x7d, + 0x66, 0xb6, 0xe3, 0x6c, 0xe6, 0x05, 0x98, 0xd6, 0x0d, 0x0d, 0x1f, 0xd1, 0x66, 0xa6, 0x24, 0x96, + 0x08, 0xbd, 0x9d, 0x4c, 0x4e, 0xf4, 0x76, 0x52, 0xfc, 0x1c, 0x16, 0x42, 0x55, 0x8c, 0xb3, 0xfd, + 0xff, 0x3d, 0x01, 0x0b, 0xbc, 0x21, 0xb1, 0xef, 0x60, 0x7e, 0x13, 0xa6, 0x3b, 0x44, 0xe6, 0x98, + 0x7e, 0xa6, 0xcf, 0x74, 0xfb, 0x99, 0x16, 0x46, 0x3f, 0x0b, 0xd0, 0xb3, 0xf0, 0x81, 0xcc, 0x58, + 0x93, 0x13, 0xb1, 0x66, 0x08, 0x07, 0x25, 0xa0, 0xef, 0x0b, 0x50, 0x20, 0x13, 0xba, 0x67, 0x99, + 0x3d, 0xd3, 0x26, 0x3e, 0x8b, 0x3d, 0x19, 0xcc, 0x79, 0x71, 0x7a, 0xb2, 0x98, 0xdb, 0xd0, 0x8d, + 0x2d, 0xce, 0xd8, 0x6a, 0x4e, 0x7c, 0x66, 0xc0, 0x3d, 0x39, 0x51, 0xa9, 0x76, 0x4c, 0x75, 0xdf, + 0x7f, 0xdf, 0x46, 0x2c, 0x8b, 0x27, 0xce, 0x16, 0xff, 0x50, 0x80, 0x0b, 0x3f, 0xb5, 0xed, 0xe2, + 0xbf, 0x08, 0x65, 0x8b, 0x2f, 0xa1, 0x48, 0x7f, 0x34, 0x8c, 0x5d, 0x33, 0xce, 0x8d, 0xfb, 0xff, + 0x23, 0xc0, 0x7c, 0x40, 0x70, 0x9c, 0x0e, 0xce, 0x9b, 0xea, 0x29, 0xc7, 0x22, 0x6c, 0x9c, 0xc9, + 0x54, 0x25, 0xcd, 0xf1, 0xe2, 0x6c, 0x50, 0x56, 0x60, 0x0e, 0x13, 0x2b, 0x46, 0xb7, 0x78, 0x77, + 0xd8, 0xb9, 0x95, 0x81, 0x1d, 0xfd, 0xac, 0x57, 0x60, 0xe5, 0x58, 0xfc, 0x79, 0xe2, 0x61, 0x05, + 0x27, 0x65, 0x9c, 0x53, 0xfe, 0x9f, 0x27, 0xe0, 0x52, 0x95, 0xbd, 0x55, 0x77, 0xc3, 0x4c, 0xe2, + 0x1c, 0x88, 0x25, 0x98, 0x3d, 0xc0, 0x96, 0xad, 0x9b, 0x6c, 0xb5, 0xcf, 0x49, 0x6e, 0x12, 0x95, + 0x21, 0x6d, 0x1b, 0x4a, 0xcf, 0xde, 0x33, 0xdd, 0xd7, 0x89, 0x5e, 0xda, 0x0b, 0x89, 0x99, 0x7e, + 0xf3, 0x90, 0x98, 0x99, 0xf1, 0x21, 0x31, 0xb3, 0x5f, 0x21, 0x24, 0x86, 0xbf, 0xbb, 0xfb, 0xf7, + 0x02, 0x5c, 0x1e, 0xd2, 0x5c, 0x9c, 0x83, 0xf3, 0x3b, 0x90, 0x55, 0xb9, 0x60, 0xb2, 0x3e, 0xb0, + 0x17, 0x93, 0x0d, 0x52, 0xec, 0x0d, 0xa1, 0xcf, 0xe9, 0xc9, 0x22, 0xb8, 0x55, 0x6d, 0xac, 0x72, + 0xe5, 0x90, 0xdf, 0x9a, 0xf8, 0xcb, 0x39, 0x28, 0xd4, 0x8e, 0xd8, 0xa6, 0x7c, 0x93, 0x79, 0x25, + 0xe8, 0x09, 0xa4, 0x7b, 0x96, 0x79, 0xa0, 0xbb, 0xcd, 0xc8, 0x87, 0xe2, 0x21, 0xdc, 0x66, 0x0c, + 0x70, 0x6d, 0x71, 0x0e, 0xc9, 0xe3, 0x45, 0x2d, 0xc8, 0x3c, 0x33, 0x55, 0xa5, 0xf3, 0x44, 0xef, + 0xb8, 0x13, 0xed, 0xfd, 0xb3, 0x05, 0x55, 0x3c, 0x9e, 0x2d, 0xc5, 0xd9, 0x73, 0x3b, 0xc1, 0x23, + 0xa2, 0x06, 0xa4, 0xeb, 0x8e, 0xd3, 0x23, 0x99, 0x7c, 0xfe, 0xdd, 0x99, 0x40, 0x28, 0x61, 0x71, + 0x83, 0x78, 0x5d, 0x76, 0xd4, 0x82, 0xf9, 0xa7, 0xf4, 0x48, 0x5a, 0xb5, 0x63, 0xf6, 0xb5, 0xaa, + 0x69, 0xec, 0xea, 0x6d, 0xbe, 0x4c, 0xdc, 0x9e, 0x40, 0xe6, 0xd3, 0x6a, 0x53, 0x1a, 0x16, 0x80, + 0x96, 0x21, 0xdd, 0x7c, 0xc4, 0x85, 0x31, 0x37, 0xf2, 0xd6, 0x04, 0xc2, 0x9a, 0x8f, 0x24, 0x8f, + 0x0d, 0xad, 0x41, 0x76, 0xf9, 0x8b, 0xbe, 0x85, 0xb9, 0x94, 0x99, 0x91, 0xc1, 0x18, 0x83, 0x52, + 0x28, 0x97, 0x14, 0x64, 0x46, 0x4d, 0xc8, 0xbf, 0x32, 0xad, 0xfd, 0x8e, 0xa9, 0xb8, 0x2d, 0x9c, + 0xa5, 0xe2, 0xbe, 0x31, 0x81, 0x38, 0x97, 0x51, 0x1a, 0x10, 0x81, 0xbe, 0x0d, 0x05, 0xd2, 0x19, + 0x2d, 0x65, 0xa7, 0xe3, 0x56, 0x32, 0x4d, 0xa5, 0xbe, 0x3b, 0x81, 0x54, 0x8f, 0xd3, 0x7d, 0xcf, + 0x30, 0x20, 0xaa, 0x2c, 0x41, 0x2e, 0x34, 0x08, 0x10, 0x82, 0x54, 0x8f, 0xf4, 0xb7, 0x40, 0xc3, + 0xa5, 0xe8, 0x6f, 0xf4, 0x1e, 0xcc, 0x1a, 0xa6, 0x86, 0xdd, 0x19, 0x92, 0x5b, 0xb9, 0x70, 0x7a, + 0xb2, 0x38, 0xb3, 0x69, 0x6a, 0xcc, 0x81, 0xe2, 0xbf, 0xa4, 0x19, 0x52, 0xa8, 0xa1, 0x95, 0xaf, + 0x41, 0x8a, 0xf4, 0x3b, 0x31, 0x4c, 0x3b, 0x8a, 0x8d, 0xb7, 0x2d, 0x9d, 0x4b, 0x73, 0x93, 0xe5, + 0x7f, 0x94, 0x80, 0x44, 0xf3, 0x11, 0x81, 0x08, 0x3b, 0x7d, 0x75, 0x1f, 0x3b, 0x3c, 0x9f, 0xa7, + 0x28, 0x74, 0xb0, 0xf0, 0xae, 0xce, 0x3c, 0xb9, 0x8c, 0xc4, 0x53, 0xe8, 0x1d, 0x00, 0x45, 0x55, + 0xb1, 0x6d, 0xcb, 0xee, 0x51, 0xc5, 0x8c, 0x94, 0x61, 0x94, 0x75, 0x7c, 0x4c, 0xd8, 0x6c, 0xac, + 0x5a, 0xd8, 0x71, 0x63, 0xbd, 0x58, 0x8a, 0xb0, 0x39, 0xb8, 0xdb, 0x93, 0x1d, 0x73, 0x1f, 0x1b, + 0x74, 0x9c, 0x64, 0x88, 0xa9, 0xe9, 0xf6, 0x5a, 0x84, 0x40, 0xac, 0x24, 0x36, 0x34, 0xdf, 0xa4, + 0x65, 0x24, 0x2f, 0x4d, 0x44, 0x5a, 0xb8, 0xad, 0xf3, 0x83, 0x7e, 0x19, 0x89, 0xa7, 0x88, 0x96, + 0x94, 0xbe, 0xb3, 0x47, 0x7b, 0x22, 0x23, 0xd1, 0xdf, 0xe8, 0x36, 0x14, 0x58, 0x78, 0xa8, 0x8c, + 0x0d, 0x55, 0xa6, 0xc6, 0x35, 0x43, 0xb3, 0x73, 0x8c, 0x5c, 0x33, 0x54, 0x62, 0x4a, 0xd1, 0x23, + 0xe0, 0x04, 0x79, 0xbf, 0x6b, 0x13, 0x9d, 0x02, 0x29, 0xb5, 0x52, 0x38, 0x3d, 0x59, 0xcc, 0x36, + 0x69, 0xc6, 0xfa, 0x46, 0x93, 0x2c, 0x50, 0xac, 0xd4, 0x7a, 0xd7, 0x6e, 0x68, 0xe5, 0x5f, 0x13, + 0x20, 0xf9, 0xb4, 0xda, 0x3c, 0xb7, 0xca, 0xdc, 0x8a, 0x26, 0x03, 0x15, 0xbd, 0x03, 0x85, 0x1d, + 0xbd, 0xd3, 0xd1, 0x8d, 0x36, 0x71, 0xda, 0xbe, 0x83, 0x55, 0x57, 0x61, 0x79, 0x4e, 0xde, 0x62, + 0x54, 0x74, 0x0d, 0xb2, 0xaa, 0x85, 0x35, 0x6c, 0x38, 0xba, 0xd2, 0xb1, 0xb9, 0xe6, 0x82, 0xa4, + 0xf2, 0x2f, 0x09, 0x30, 0x4d, 0x67, 0x00, 0x7a, 0x1b, 0x32, 0xaa, 0x69, 0x38, 0x8a, 0x6e, 0x70, + 0x53, 0x96, 0x91, 0x7c, 0xc2, 0xc8, 0xea, 0x5d, 0x87, 0x39, 0x45, 0x55, 0xcd, 0xbe, 0xe1, 0xc8, + 0x86, 0xd2, 0xc5, 0xbc, 0x9a, 0x59, 0x4e, 0xdb, 0x54, 0xba, 0x18, 0x2d, 0x82, 0x9b, 0xf4, 0x4e, + 0xa0, 0x66, 0x24, 0xe0, 0xa4, 0x75, 0x7c, 0x5c, 0xfe, 0xb7, 0x02, 0xa4, 0xdd, 0x39, 0x43, 0xaa, + 0xd1, 0xc6, 0x06, 0x8b, 0x79, 0x77, 0xab, 0xe1, 0x11, 0x06, 0x97, 0xca, 0x8c, 0xbf, 0x54, 0x5e, + 0x80, 0x69, 0x87, 0x4c, 0x0b, 0x5e, 0x03, 0x96, 0xa0, 0xdb, 0xe7, 0x1d, 0xa5, 0xcd, 0x76, 0x0f, + 0x33, 0x12, 0x4b, 0x90, 0xc6, 0xf0, 0x28, 0x63, 0xa6, 0x11, 0x9e, 0x22, 0x35, 0x65, 0xb1, 0xb0, + 0x3b, 0xb8, 0xad, 0x1b, 0x74, 0x2c, 0x25, 0x25, 0xa0, 0xa4, 0x15, 0x42, 0x41, 0x6f, 0x41, 0x86, + 0x15, 0xc0, 0x86, 0x46, 0x07, 0x54, 0x52, 0x4a, 0x53, 0x42, 0xcd, 0xd0, 0xca, 0x18, 0x32, 0xde, + 0xe4, 0x24, 0xdd, 0xd6, 0xb7, 0x3d, 0x45, 0xd2, 0xdf, 0xe8, 0x7d, 0xb8, 0xf0, 0xba, 0xaf, 0x74, + 0xf4, 0x5d, 0xba, 0x31, 0x48, 0x0f, 0x05, 0x50, 0x9d, 0xb1, 0x96, 0x20, 0x2f, 0x8f, 0x4a, 0xa0, + 0xaa, 0x73, 0xe7, 0x72, 0xd2, 0x9f, 0xcb, 0xe2, 0xef, 0x0a, 0x30, 0xcf, 0xa2, 0xb6, 0x58, 0x7c, + 0x6e, 0x7c, 0x7e, 0xc8, 0xc7, 0x90, 0xd1, 0x14, 0x47, 0x61, 0x67, 0x6a, 0x13, 0x63, 0xcf, 0xd4, + 0x7a, 0x67, 0x3c, 0x14, 0x47, 0xa1, 0xe7, 0x6a, 0x11, 0xa4, 0xc8, 0x6f, 0x76, 0xfc, 0x58, 0xa2, + 0xbf, 0xc5, 0xcf, 0x00, 0x05, 0x2b, 0x1a, 0xa7, 0x47, 0x76, 0x0f, 0x2e, 0x12, 0x5d, 0xd7, 0x0c, + 0xd5, 0x3a, 0xee, 0x39, 0xba, 0x69, 0x3c, 0xa7, 0x7f, 0x6d, 0x54, 0x0c, 0xbc, 0x47, 0xa3, 0xaf, + 0xcf, 0xc4, 0xdf, 0x9b, 0x81, 0x5c, 0xed, 0xa8, 0x67, 0x5a, 0xb1, 0xee, 0xba, 0xad, 0xc0, 0x2c, + 0xdf, 0x98, 0x18, 0xf3, 0xaa, 0x7c, 0xc0, 0x98, 0xbb, 0x71, 0x02, 0x9c, 0x11, 0xad, 0x00, 0xb0, + 0x18, 0x5a, 0x1a, 0x27, 0x95, 0x3c, 0xc7, 0x9b, 0x3d, 0xca, 0x46, 0xcf, 0x97, 0x6c, 0x42, 0xb6, + 0x7b, 0xa0, 0xaa, 0xf2, 0xae, 0xde, 0x71, 0x78, 0x28, 0x62, 0x74, 0xd4, 0xfc, 0xc6, 0xcb, 0x6a, + 0xf5, 0x09, 0x2d, 0xc4, 0x42, 0xf8, 0xfc, 0xb4, 0x04, 0x44, 0x02, 0xfb, 0x8d, 0xde, 0x05, 0x7e, + 0xd6, 0x49, 0xb6, 0xdd, 0x93, 0x8b, 0x2b, 0xb9, 0xd3, 0x93, 0xc5, 0x8c, 0x44, 0xa9, 0xcd, 0x66, + 0x4b, 0xca, 0xb0, 0x02, 0x4d, 0xdb, 0x41, 0x37, 0x20, 0x67, 0x76, 0x75, 0x47, 0x76, 0x9d, 0x24, + 0xee, 0x51, 0xce, 0x11, 0xa2, 0xeb, 0x44, 0x9d, 0xe7, 0x08, 0xcc, 0xec, 0xe4, 0x47, 0x60, 0xfe, + 0xa6, 0x00, 0x97, 0xb8, 0x22, 0xe5, 0x1d, 0x1a, 0xf6, 0xaf, 0x74, 0x74, 0xe7, 0x58, 0xde, 0x3f, + 0x28, 0xa5, 0xa9, 0xdf, 0xfa, 0x33, 0x91, 0x1d, 0x12, 0x18, 0x07, 0x15, 0xb7, 0x5b, 0x8e, 0x9f, + 0x71, 0xe6, 0xf5, 0x83, 0x9a, 0xe1, 0x58, 0xc7, 0x2b, 0x97, 0x4f, 0x4f, 0x16, 0x17, 0x86, 0x73, + 0x5f, 0x4a, 0x0b, 0xf6, 0x30, 0x0b, 0xaa, 0x03, 0x60, 0x6f, 0x1c, 0xd2, 0x15, 0x23, 0xda, 0xff, + 0x88, 0x1c, 0xb0, 0x52, 0x80, 0x17, 0xdd, 0x85, 0x22, 0x3f, 0x72, 0xb4, 0xab, 0x77, 0xb0, 0x6c, + 0xeb, 0x5f, 0x60, 0xba, 0xb6, 0x24, 0xa5, 0x3c, 0xa3, 0x13, 0x11, 0x4d, 0xfd, 0x0b, 0x5c, 0xfe, + 0x0e, 0x94, 0x46, 0xd5, 0x3e, 0x38, 0x05, 0x32, 0xec, 0x0d, 0xf2, 0x47, 0xe1, 0xed, 0xa3, 0x09, + 0x86, 0x2a, 0xdf, 0x42, 0xfa, 0x38, 0xf1, 0x91, 0x20, 0xfe, 0x83, 0x04, 0xe4, 0x56, 0xfa, 0x9d, + 0xfd, 0xe7, 0xbd, 0x26, 0xbb, 0x7b, 0x81, 0x98, 0x41, 0x66, 0x28, 0x48, 0x05, 0x05, 0x66, 0x06, + 0xa9, 0x25, 0xd0, 0xbf, 0xc0, 0x64, 0x71, 0x0a, 0x44, 0xe7, 0xf0, 0x63, 0x0d, 0xb4, 0x0d, 0x3e, + 0x99, 0x9e, 0x3c, 0xf8, 0x08, 0x4a, 0x81, 0x82, 0x74, 0xaf, 0x47, 0xc6, 0x86, 0x63, 0xe9, 0x98, + 0xed, 0x57, 0x26, 0xa5, 0x40, 0x08, 0x51, 0x83, 0x64, 0xd7, 0x58, 0x2e, 0x6a, 0xc1, 0x1c, 0x29, + 0x78, 0x2c, 0xd3, 0x25, 0xc4, 0xdd, 0x4f, 0x7e, 0x10, 0xd1, 0xac, 0x50, 0xbd, 0x2b, 0x54, 0x3f, + 0x55, 0xca, 0x43, 0x7f, 0x4a, 0x59, 0xec, 0x53, 0xca, 0x9f, 0x42, 0x71, 0xb0, 0x40, 0x50, 0x97, + 0x29, 0xa6, 0xcb, 0x0b, 0x41, 0x5d, 0x26, 0x03, 0x7a, 0x5a, 0x4b, 0xa5, 0x53, 0xc5, 0x69, 0xf1, + 0x4f, 0x93, 0x90, 0x77, 0x87, 0x59, 0x9c, 0x40, 0x67, 0x05, 0xa6, 0xc9, 0xa0, 0x70, 0x03, 0x5e, + 0x6e, 0x8f, 0x19, 0xdd, 0x3c, 0x90, 0x9e, 0x0c, 0x16, 0x17, 0x93, 0x53, 0xd6, 0x38, 0x0c, 0x4e, + 0xf9, 0x97, 0x12, 0x90, 0xa2, 0xd8, 0xe2, 0x01, 0xa4, 0xe8, 0x42, 0x21, 0x4c, 0xb2, 0x50, 0xd0, + 0xa2, 0xde, 0x72, 0x96, 0x08, 0xb8, 0xa6, 0xc4, 0xe7, 0xdb, 0x53, 0x3e, 0x78, 0xf0, 0x90, 0x1a, + 0x9b, 0x39, 0x89, 0xa7, 0xd0, 0x0a, 0x8d, 0xc4, 0x32, 0x2d, 0x07, 0x6b, 0xdc, 0xa7, 0xbf, 0x76, + 0x56, 0xff, 0xba, 0x8b, 0x92, 0xcb, 0x87, 0xae, 0x40, 0x92, 0x58, 0xb1, 0x59, 0x16, 0x54, 0x71, + 0x7a, 0xb2, 0x98, 0x24, 0xf6, 0x8b, 0xd0, 0xd0, 0x12, 0x64, 0xc3, 0x26, 0x83, 0x78, 0x70, 0xd4, + 0x30, 0x06, 0xa6, 0x3b, 0x74, 0xbc, 0xa9, 0xc5, 0xf0, 0x2c, 0xef, 0xe3, 0xff, 0x99, 0x82, 0x5c, + 0xa3, 0x1b, 0xf7, 0x92, 0xb2, 0x1c, 0xee, 0xe1, 0x28, 0x20, 0x14, 0x7a, 0x68, 0x44, 0x07, 0x87, + 0x56, 0xf0, 0xe4, 0xf9, 0x56, 0xf0, 0x4f, 0xa9, 0x17, 0xcd, 0x86, 0xc6, 0xcc, 0xe4, 0x43, 0x63, + 0x16, 0x1b, 0x1a, 0x5d, 0x89, 0x1a, 0xc4, 0xd3, 0xe6, 0x57, 0x60, 0x24, 0x47, 0x60, 0xa6, 0x70, + 0xfd, 0xa9, 0x9f, 0x23, 0x11, 0x1e, 0xff, 0x68, 0x0a, 0x0d, 0xac, 0x09, 0x5b, 0xd4, 0xd9, 0x37, + 0xb7, 0xa8, 0x65, 0x87, 0x0f, 0xd6, 0x8f, 0x21, 0xa9, 0xe9, 0x6e, 0xe7, 0x4c, 0xbe, 0x54, 0x13, + 0xa6, 0x33, 0x46, 0x6d, 0x2a, 0x38, 0x6a, 0xd9, 0x28, 0x29, 0x37, 0x00, 0xfc, 0xb6, 0xa1, 0x6b, + 0x30, 0x63, 0x76, 0x34, 0xf7, 0x6c, 0x4d, 0x6e, 0x25, 0x73, 0x7a, 0xb2, 0x38, 0xfd, 0xbc, 0xa3, + 0x35, 0x56, 0xa5, 0x69, 0xb3, 0xa3, 0x35, 0x34, 0x7a, 0x7f, 0x08, 0x3e, 0x94, 0xbd, 0xc0, 0xbb, + 0x39, 0x69, 0xd6, 0xc0, 0x87, 0xab, 0xd8, 0x56, 0xf9, 0x80, 0xfb, 0x2d, 0x01, 0xf2, 0xae, 0xee, + 0xe2, 0x35, 0x2a, 0x69, 0xbd, 0xcb, 0x27, 0x59, 0xf2, 0x7c, 0x93, 0xcc, 0xe5, 0xe3, 0xe7, 0x9e, + 0x7f, 0x45, 0xe0, 0xa1, 0xd4, 0x4d, 0x55, 0x71, 0x88, 0x53, 0x11, 0xe3, 0xc4, 0xb8, 0x07, 0x45, + 0x4b, 0x31, 0x34, 0xb3, 0xab, 0x7f, 0x81, 0xd9, 0x66, 0xa2, 0xcd, 0xdf, 0xb2, 0x16, 0x3c, 0x3a, + 0xdd, 0xf5, 0xb3, 0xc5, 0xff, 0x26, 0xf0, 0xb0, 0x6b, 0xaf, 0x1a, 0xf1, 0xc6, 0xc2, 0x64, 0xf9, + 0x9b, 0x08, 0x63, 0xd7, 0x74, 0xa3, 0xc6, 0xde, 0x1e, 0x15, 0x23, 0xd9, 0x30, 0x76, 0x4d, 0xf7, + 0xad, 0xbe, 0xe5, 0x12, 0xec, 0xf2, 0xcf, 0xc1, 0x34, 0xcd, 0x7e, 0x03, 0x03, 0xea, 0x85, 0xfa, + 0x13, 0x8d, 0xff, 0x49, 0x02, 0x6e, 0xd2, 0xa6, 0xbe, 0xc4, 0x96, 0xbe, 0x7b, 0xbc, 0x65, 0x99, + 0x0e, 0x56, 0x1d, 0xac, 0xf9, 0x9b, 0xe9, 0x31, 0x76, 0x81, 0x06, 0x19, 0x1e, 0x86, 0xa0, 0x6b, + 0xfc, 0x0e, 0xa1, 0xa7, 0x5f, 0x6d, 0x93, 0x2d, 0xcd, 0xc2, 0x17, 0x1a, 0xab, 0x52, 0x9a, 0x49, + 0x6e, 0x68, 0x68, 0x19, 0x32, 0x3d, 0xb7, 0x19, 0xe7, 0x8a, 0x74, 0xf3, 0xb8, 0xd0, 0x3a, 0x14, + 0x78, 0x45, 0x95, 0x8e, 0x7e, 0x80, 0x65, 0xc5, 0x39, 0xcf, 0x3a, 0x97, 0x63, 0xbc, 0xcb, 0x84, + 0x75, 0xd9, 0x11, 0xff, 0x76, 0x0a, 0x6e, 0x9d, 0xa1, 0xe2, 0x38, 0x87, 0x57, 0x19, 0xd2, 0x07, + 0xe4, 0x41, 0x3a, 0x6f, 0x7d, 0x5a, 0xf2, 0xd2, 0x68, 0x27, 0xe4, 0x2c, 0xed, 0x2a, 0x7a, 0x87, + 0x38, 0x57, 0x2c, 0xb6, 0x78, 0x74, 0xf4, 0x62, 0x74, 0xac, 0x6e, 0xc0, 0xad, 0x7a, 0x42, 0x05, + 0xd1, 0x62, 0x36, 0xfa, 0xae, 0x00, 0x65, 0xf6, 0x40, 0x16, 0xe0, 0x3a, 0xf0, 0x98, 0x14, 0x7d, + 0xcc, 0x6a, 0xc4, 0x63, 0x26, 0xd2, 0x51, 0x25, 0xf0, 0x2c, 0x5e, 0x91, 0x52, 0xf0, 0x69, 0xc1, + 0xaa, 0x94, 0x7f, 0x5d, 0x80, 0x6c, 0x80, 0x80, 0x6e, 0x0f, 0x9d, 0x50, 0xcc, 0x9e, 0x46, 0x1d, + 0x4b, 0xbc, 0x35, 0x74, 0x2c, 0x71, 0x25, 0xfd, 0xe5, 0xc9, 0x62, 0x4a, 0x62, 0xc7, 0x54, 0xdc, + 0x03, 0x8a, 0xd7, 0xfd, 0x0b, 0xb1, 0x92, 0x03, 0x85, 0xdc, 0x1b, 0xb1, 0xe8, 0xc6, 0x91, 0xe2, + 0xbe, 0xfd, 0xa6, 0x1b, 0x47, 0x24, 0x25, 0xfe, 0x46, 0x02, 0xe6, 0x97, 0x35, 0xad, 0xd9, 0xe4, + 0x16, 0x3e, 0xbe, 0x39, 0xe6, 0x42, 0xe8, 0x84, 0x0f, 0xa1, 0xd1, 0x7b, 0x80, 0x34, 0xdd, 0x66, + 0x17, 0xcb, 0xd8, 0x7b, 0x8a, 0x66, 0x1e, 0xfa, 0x41, 0x2e, 0xf3, 0x6e, 0x4e, 0xd3, 0xcd, 0x40, + 0x4d, 0xa0, 0x58, 0x4e, 0xb6, 0x1d, 0xc5, 0x7b, 0x89, 0x77, 0x6b, 0xa2, 0xf3, 0x79, 0x0c, 0xe4, + 0x79, 0x49, 0x29, 0x43, 0xe4, 0xd0, 0x9f, 0x04, 0x95, 0xe8, 0xa4, 0x53, 0x1c, 0x59, 0xb1, 0xdd, + 0x93, 0x65, 0xec, 0x4a, 0x9b, 0x3c, 0xa3, 0x2f, 0xdb, 0xec, 0xc0, 0x18, 0x3b, 0x3d, 0xe2, 0xab, + 0x26, 0x4e, 0xc0, 0xff, 0xf7, 0x04, 0xc8, 0x4b, 0x78, 0xd7, 0xc2, 0x76, 0xac, 0x5b, 0x1e, 0x4f, + 0x60, 0xce, 0x62, 0x52, 0xe5, 0x5d, 0xcb, 0xec, 0x9e, 0xc7, 0x56, 0x64, 0x39, 0xe3, 0x13, 0xcb, + 0xec, 0x72, 0x93, 0xfc, 0x12, 0x0a, 0x5e, 0x1d, 0xe3, 0x6c, 0xfc, 0xef, 0xd2, 0xb3, 0xe7, 0x4c, + 0x70, 0xdc, 0xd1, 0x26, 0xf1, 0x6a, 0x80, 0xbe, 0x86, 0x0b, 0x56, 0x34, 0x4e, 0x35, 0xfc, 0x57, + 0x01, 0xf2, 0xcd, 0xfe, 0x0e, 0xbb, 0x30, 0x2d, 0x3e, 0x0d, 0xd4, 0x20, 0xd3, 0xc1, 0xbb, 0x8e, + 0xfc, 0x46, 0xe7, 0x1e, 0xd2, 0x84, 0x95, 0x9e, 0xfa, 0x78, 0x0a, 0x60, 0xd1, 0xc3, 0x97, 0x54, + 0x4e, 0xf2, 0x9c, 0x72, 0x32, 0x94, 0x97, 0x90, 0xc9, 0xaa, 0x53, 0xf0, 0x9a, 0x19, 0xe7, 0xfa, + 0xf2, 0x2a, 0x64, 0x1d, 0x92, 0xe7, 0xb1, 0x0e, 0xf3, 0x3c, 0xc0, 0x26, 0xda, 0x42, 0x54, 0x60, + 0x81, 0xba, 0x65, 0xb2, 0xd2, 0xeb, 0x75, 0x74, 0x17, 0xcc, 0x53, 0xfb, 0x93, 0x92, 0xe6, 0x69, + 0xd6, 0x32, 0xcb, 0xa1, 0x30, 0x1e, 0xfd, 0xaa, 0x00, 0x73, 0xbb, 0x16, 0xc6, 0x5f, 0x60, 0x99, + 0x9a, 0xe4, 0xc9, 0x22, 0x88, 0x56, 0x49, 0x1d, 0xbe, 0x72, 0x84, 0x41, 0x96, 0x3d, 0xb8, 0x49, + 0x9e, 0x8b, 0x36, 0xa1, 0xa8, 0x76, 0x58, 0xcc, 0x83, 0x17, 0xcd, 0x74, 0x0e, 0xec, 0x53, 0x60, + 0xcc, 0x7e, 0x40, 0xd3, 0x0b, 0x32, 0x99, 0x14, 0x4d, 0xe6, 0x97, 0x54, 0x72, 0xe8, 0x52, 0x19, + 0x71, 0x09, 0x45, 0xe0, 0x6e, 0xcb, 0x8a, 0x84, 0x15, 0x8d, 0x7b, 0xd8, 0x64, 0x5e, 0x79, 0x09, + 0x3e, 0xaf, 0x5e, 0xc1, 0x3c, 0x1d, 0x37, 0x71, 0x9f, 0x25, 0x17, 0x7f, 0x98, 0x00, 0x14, 0x94, + 0xfc, 0xd3, 0x1b, 0x6f, 0x89, 0xf8, 0xc6, 0xdb, 0xbb, 0x80, 0x58, 0x98, 0xac, 0x2d, 0xf7, 0xb0, + 0x25, 0xdb, 0x58, 0x35, 0xf9, 0xf5, 0x61, 0x82, 0x54, 0xe4, 0x39, 0x5b, 0xd8, 0x6a, 0x52, 0x3a, + 0x5a, 0x06, 0xf0, 0xbd, 0x76, 0xbe, 0x28, 0x4e, 0xe2, 0xb4, 0x67, 0x3c, 0xa7, 0x5d, 0xfc, 0x9e, + 0x00, 0xf9, 0x0d, 0xbd, 0x6d, 0x29, 0xb1, 0xde, 0x8e, 0x85, 0x3e, 0x0e, 0xbf, 0xcd, 0xc8, 0x3e, + 0x2c, 0x47, 0x05, 0x76, 0xb1, 0x12, 0x2e, 0xdc, 0xe6, 0x0c, 0x64, 0xad, 0xf1, 0x6a, 0x14, 0xa7, + 0x91, 0xfd, 0x0f, 0x65, 0x98, 0xe3, 0xf5, 0xde, 0x36, 0x74, 0xd3, 0x40, 0x0f, 0x20, 0xd9, 0xe6, + 0x6f, 0xab, 0xb2, 0x91, 0x3b, 0xcb, 0xfe, 0xdd, 0x93, 0xf5, 0x29, 0x89, 0x94, 0x25, 0x2c, 0xbd, + 0xbe, 0x13, 0xe1, 0xc1, 0xfb, 0x67, 0x1d, 0x82, 0x2c, 0xbd, 0xbe, 0x83, 0x9a, 0x50, 0x50, 0xfd, + 0x0b, 0xef, 0x64, 0xc2, 0x9e, 0x1c, 0x89, 0xfb, 0x23, 0xaf, 0x1e, 0xac, 0x4f, 0x49, 0x79, 0x35, + 0x94, 0x81, 0xaa, 0xc1, 0x7b, 0xd6, 0x52, 0x43, 0x81, 0x94, 0xfe, 0x29, 0xfd, 0xf0, 0x1d, 0x6f, + 0xf5, 0xa9, 0xc0, 0x75, 0x6c, 0xe8, 0x63, 0x98, 0xd1, 0xe8, 0x8d, 0x5e, 0xdc, 0x4a, 0x45, 0x75, + 0x74, 0xe8, 0xe2, 0xb4, 0xfa, 0x94, 0xc4, 0x39, 0xd0, 0x1a, 0xcc, 0xb1, 0x5f, 0xcc, 0x87, 0xe6, + 0xb6, 0xe5, 0xd6, 0x68, 0x09, 0x81, 0xd5, 0xbd, 0x3e, 0x25, 0x65, 0x35, 0x9f, 0x8a, 0x9e, 0x42, + 0x56, 0xed, 0x60, 0xc5, 0xe2, 0xa2, 0x6e, 0x8f, 0x3c, 0xfd, 0x39, 0x74, 0x0b, 0x58, 0x7d, 0x4a, + 0x02, 0xd5, 0x23, 0x92, 0x4a, 0x59, 0xf4, 0x32, 0x28, 0x2e, 0xe9, 0xfd, 0x91, 0x95, 0x1a, 0xbe, + 0x59, 0xab, 0x4e, 0x57, 0x7d, 0x8f, 0x8a, 0xbe, 0x09, 0x29, 0x5b, 0x55, 0xdc, 0x3d, 0x9a, 0xab, + 0x23, 0x6e, 0xeb, 0xf1, 0x99, 0x69, 0x69, 0xf4, 0x98, 0xb9, 0xdf, 0xce, 0x91, 0xbb, 0x5d, 0x1e, + 0xa5, 0xd3, 0xd0, 0xad, 0x10, 0x44, 0xa7, 0x98, 0x12, 0x88, 0x1e, 0x14, 0x82, 0x37, 0x64, 0x7a, + 0xae, 0x9a, 0xee, 0x8f, 0x47, 0xeb, 0x61, 0xe8, 0x1c, 0x7c, 0x9d, 0x5e, 0x3d, 0xe1, 0x12, 0xd1, + 0x06, 0xe4, 0x98, 0xa0, 0x3e, 0x3b, 0xa2, 0x5d, 0x5a, 0x1a, 0x19, 0xcd, 0x10, 0x71, 0x48, 0xbc, + 0x3e, 0x25, 0xcd, 0x29, 0x01, 0xb2, 0x5f, 0xaf, 0x2e, 0xb6, 0xda, 0xb8, 0x94, 0x1d, 0x5f, 0xaf, + 0x60, 0x88, 0xa8, 0x57, 0x2f, 0x4a, 0x44, 0xbf, 0x08, 0x17, 0x98, 0x20, 0x87, 0x47, 0xbe, 0xf1, + 0x00, 0xaa, 0x77, 0x46, 0x06, 0x0d, 0x8c, 0x3c, 0x56, 0x5d, 0x9f, 0x92, 0x90, 0x32, 0x94, 0x89, + 0x54, 0xb8, 0xc8, 0x9e, 0xc0, 0xcf, 0xe5, 0x5a, 0xfc, 0x28, 0x69, 0xe9, 0x06, 0x7d, 0xc4, 0x7b, + 0xa3, 0x1e, 0x11, 0x79, 0x5c, 0xb8, 0x3e, 0x25, 0x2d, 0x28, 0xc3, 0xb9, 0x7e, 0x33, 0x2c, 0x7e, + 0x02, 0x92, 0x0f, 0xb7, 0xf7, 0xc6, 0x37, 0x23, 0xea, 0xe4, 0xa8, 0xd7, 0x8c, 0x50, 0x26, 0xe9, + 0x40, 0xef, 0xe6, 0x07, 0x3a, 0x98, 0xe6, 0x46, 0x76, 0x60, 0xc4, 0x31, 0x49, 0xd2, 0x81, 0x7b, + 0x01, 0x32, 0xaa, 0x40, 0xa2, 0xad, 0x96, 0x72, 0x23, 0xd7, 0x07, 0xef, 0x28, 0x60, 0x7d, 0x4a, + 0x4a, 0xb4, 0x55, 0xf4, 0x29, 0xa4, 0xd9, 0xb9, 0xae, 0x23, 0xa3, 0x94, 0x1f, 0x69, 0x70, 0xc3, + 0xa7, 0xe3, 0xea, 0x53, 0x12, 0x3d, 0x4a, 0xc6, 0x07, 0x32, 0x3f, 0xb3, 0x43, 0x45, 0x54, 0xc6, + 0x1c, 0xe7, 0x1e, 0x38, 0x39, 0x45, 0x06, 0x8c, 0xe5, 0x11, 0xd1, 0x16, 0xe4, 0x2d, 0x16, 0xd5, + 0xec, 0x9e, 0x41, 0x28, 0x8e, 0x8c, 0xf5, 0x89, 0x3a, 0x86, 0x50, 0xa7, 0x1b, 0x1f, 0x01, 0x3a, + 0xe9, 0xbb, 0xb0, 0x44, 0xde, 0x77, 0xf3, 0x23, 0xfb, 0x6e, 0x64, 0x48, 0x3c, 0xe9, 0x3b, 0x6b, + 0x28, 0x13, 0x7d, 0x08, 0xd3, 0x6c, 0x9e, 0x20, 0x2a, 0x32, 0x2a, 0x7c, 0x6d, 0x60, 0x8a, 0xb0, + 0xf2, 0xc4, 0x7a, 0x39, 0x3c, 0xb4, 0x57, 0xee, 0x98, 0xed, 0xd2, 0xc2, 0x48, 0xeb, 0x35, 0x1c, + 0xa4, 0x4c, 0xac, 0x97, 0xe3, 0x53, 0xc9, 0x00, 0xb2, 0x58, 0x0e, 0x9f, 0x62, 0x17, 0x46, 0x0e, + 0xa0, 0x88, 0x88, 0xdf, 0x3a, 0x3d, 0x74, 0xe5, 0x93, 0x3d, 0xc3, 0x6a, 0x63, 0x99, 0x1a, 0xc5, + 0x8b, 0xe3, 0x0d, 0x6b, 0xe8, 0x0a, 0x35, 0xcf, 0xb0, 0x32, 0x2a, 0x7a, 0x09, 0x45, 0x7e, 0x8f, + 0x8f, 0xff, 0x72, 0xf5, 0x12, 0x95, 0x77, 0x2f, 0x72, 0x41, 0x8c, 0x0a, 0x4e, 0xac, 0x13, 0x0f, + 0x35, 0x9c, 0x83, 0x3e, 0x83, 0x79, 0x2a, 0x4f, 0x56, 0xfd, 0xab, 0x97, 0x4a, 0xa5, 0xa1, 0x8b, + 0x7c, 0x46, 0xdf, 0xd2, 0xe4, 0x4a, 0x2e, 0xaa, 0x03, 0x59, 0x64, 0x3e, 0xe8, 0x86, 0xee, 0xd0, + 0xb5, 0xbb, 0x3c, 0x72, 0x3e, 0x84, 0xaf, 0x9d, 0x25, 0xf3, 0x41, 0x67, 0x14, 0x32, 0x8c, 0x07, + 0x2c, 0xde, 0xdb, 0x23, 0x87, 0xf1, 0x08, 0x63, 0x97, 0x73, 0x42, 0x76, 0x6e, 0x15, 0x80, 0xe1, + 0x12, 0xea, 0xf9, 0x5d, 0x1d, 0xe9, 0x00, 0x0c, 0x46, 0xe4, 0x12, 0x07, 0xa0, 0xe3, 0xd2, 0xc8, + 0x3c, 0xa5, 0xbb, 0x1e, 0x32, 0x8d, 0xf4, 0x28, 0x2d, 0x8e, 0x9c, 0xa7, 0x43, 0x51, 0x19, 0x64, + 0x9e, 0x1e, 0x7a, 0x44, 0xe2, 0x49, 0xb0, 0x57, 0x53, 0xa5, 0x6b, 0xa3, 0x57, 0xbd, 0xe0, 0x1b, + 0x6a, 0xba, 0xea, 0x51, 0x02, 0xe1, 0x65, 0x3b, 0xee, 0x25, 0x71, 0x24, 0x6f, 0xe8, 0xed, 0x0a, + 0xe1, 0x65, 0x1c, 0x68, 0x19, 0x32, 0xc4, 0x29, 0x3e, 0xa6, 0x66, 0xe6, 0xfa, 0x48, 0x60, 0x3a, + 0x70, 0x5e, 0xb0, 0x3e, 0x25, 0xa5, 0x5f, 0x73, 0x12, 0x19, 0xda, 0x4c, 0x04, 0x37, 0x30, 0xf7, + 0x47, 0x0e, 0xed, 0xe1, 0x83, 0x62, 0x64, 0x68, 0xbf, 0xf6, 0xa9, 0xfe, 0xba, 0x6b, 0xb3, 0x3d, + 0xfa, 0xd2, 0xcd, 0xf1, 0xeb, 0x6e, 0xf8, 0x8d, 0x82, 0xb7, 0xee, 0x72, 0x32, 0x5b, 0x77, 0x35, + 0xd9, 0xb6, 0x59, 0xf8, 0xcf, 0xad, 0x31, 0xeb, 0xee, 0xc0, 0xae, 0x1d, 0x5b, 0x77, 0xb5, 0x26, + 0xe3, 0x24, 0x2e, 0xa8, 0xe5, 0x5e, 0x9e, 0xc5, 0x31, 0xcb, 0x9d, 0x91, 0x2e, 0x68, 0xe4, 0xed, + 0x5e, 0xc4, 0x05, 0xb5, 0x42, 0x19, 0xe8, 0x67, 0x61, 0x96, 0xef, 0x92, 0x94, 0xee, 0x8e, 0x71, + 0xca, 0x83, 0x1b, 0x5b, 0x64, 0x4e, 0x70, 0x1e, 0x66, 0xa1, 0xd8, 0xee, 0x0c, 0xb3, 0xc0, 0xf7, + 0xc6, 0x58, 0xa8, 0xa1, 0x0d, 0x22, 0x66, 0xa1, 0x7c, 0x32, 0xa9, 0x8d, 0xcd, 0x76, 0x16, 0x4a, + 0xdf, 0x18, 0x59, 0x9b, 0xf0, 0x16, 0x0b, 0xa9, 0x0d, 0xe7, 0xa1, 0x2b, 0x16, 0x75, 0x18, 0x98, + 0x76, 0xde, 0x1d, 0xbd, 0x62, 0x0d, 0x62, 0xd5, 0xba, 0xfb, 0x0e, 0x84, 0x69, 0xe5, 0x6f, 0x08, + 0x70, 0x8d, 0x8d, 0x01, 0xba, 0x03, 0x7c, 0x2c, 0x7b, 0x1b, 0xf8, 0x01, 0x20, 0xfe, 0x80, 0x8a, + 0xff, 0xf0, 0xfc, 0xfb, 0xcd, 0xee, 0x13, 0xdf, 0x51, 0xc6, 0x95, 0x23, 0xca, 0xe8, 0x32, 0x04, + 0x55, 0x7a, 0x38, 0x52, 0x19, 0x61, 0xd4, 0x47, 0x94, 0xc1, 0x79, 0x56, 0x66, 0x79, 0x20, 0x80, + 0x77, 0x0c, 0xbb, 0x50, 0x2c, 0xae, 0xa5, 0xd2, 0x97, 0x8b, 0xa5, 0xb5, 0x54, 0xfa, 0x4a, 0xb1, + 0xbc, 0x96, 0x4a, 0xbf, 0x55, 0x7c, 0x5b, 0xfc, 0x87, 0x65, 0xc8, 0xb9, 0xa0, 0x8b, 0x01, 0xaa, + 0x87, 0x41, 0x40, 0x75, 0x75, 0x14, 0xa0, 0xe2, 0x30, 0x8d, 0x23, 0xaa, 0x87, 0x41, 0x44, 0x75, + 0x75, 0x14, 0xa2, 0xf2, 0x79, 0x08, 0xa4, 0x6a, 0x8d, 0x82, 0x54, 0xf7, 0x26, 0x80, 0x54, 0x9e, + 0xa8, 0x41, 0x4c, 0xb5, 0x3a, 0x8c, 0xa9, 0x6e, 0x8e, 0xc7, 0x54, 0x9e, 0xa8, 0x00, 0xa8, 0x7a, + 0x3c, 0x00, 0xaa, 0xae, 0x8f, 0x01, 0x55, 0x1e, 0xbf, 0x8b, 0xaa, 0xd6, 0x23, 0x51, 0xd5, 0xed, + 0xb3, 0x50, 0x95, 0x27, 0x27, 0x04, 0xab, 0xea, 0x51, 0xb0, 0xea, 0xd6, 0x19, 0xb0, 0xca, 0x13, + 0x15, 0xc4, 0x55, 0xeb, 0x91, 0xb8, 0xea, 0xf6, 0x59, 0xb8, 0xca, 0xaf, 0x56, 0x10, 0x58, 0x7d, + 0x10, 0x02, 0x56, 0x8b, 0x23, 0x81, 0x95, 0xc7, 0xcd, 0x90, 0xd5, 0x27, 0x83, 0xc8, 0xea, 0xfa, + 0x18, 0x64, 0xe5, 0x2b, 0x96, 0x43, 0xab, 0x7a, 0x14, 0xb4, 0xba, 0x75, 0x06, 0xb4, 0xf2, 0x75, + 0x11, 0xc0, 0x56, 0x9b, 0xd1, 0xd8, 0xea, 0xce, 0x99, 0xd8, 0xca, 0x93, 0x16, 0x06, 0x57, 0xf5, + 0x28, 0x70, 0x75, 0xeb, 0x0c, 0x70, 0x35, 0x50, 0x33, 0x86, 0xae, 0x94, 0xb1, 0xe8, 0xea, 0xbd, + 0x09, 0xd1, 0x95, 0x27, 0x3a, 0x0a, 0x5e, 0x69, 0xe3, 0xe1, 0x55, 0x65, 0x52, 0x78, 0xe5, 0x3d, + 0x24, 0x12, 0x5f, 0x29, 0x63, 0xf1, 0xd5, 0x7b, 0x13, 0xe2, 0xab, 0x81, 0x86, 0x84, 0x01, 0xd6, + 0x66, 0x34, 0xc0, 0xba, 0x73, 0x26, 0xc0, 0xf2, 0x7b, 0x31, 0x84, 0xb0, 0x96, 0x02, 0x08, 0xeb, + 0x9d, 0x11, 0x08, 0xcb, 0x63, 0x25, 0x10, 0xeb, 0x5b, 0x43, 0x10, 0x4b, 0x1c, 0x07, 0xb1, 0x3c, + 0x5e, 0x0f, 0x63, 0xd5, 0xa3, 0x30, 0xd6, 0xad, 0x33, 0x30, 0x96, 0x3f, 0x6e, 0x02, 0x20, 0xeb, + 0xc5, 0x08, 0x90, 0x75, 0xf7, 0x6c, 0x90, 0xe5, 0xc9, 0x1b, 0x40, 0x59, 0xca, 0x58, 0x94, 0xf5, + 0xde, 0x84, 0x28, 0xcb, 0xef, 0xc1, 0x08, 0x98, 0xf5, 0x51, 0x18, 0x66, 0x5d, 0x1b, 0x0d, 0xb3, + 0x3c, 0x31, 0x1c, 0x67, 0xad, 0x47, 0xe2, 0xac, 0xdb, 0x67, 0xe1, 0x2c, 0xdf, 0x9a, 0x05, 0x81, + 0xd6, 0x66, 0x34, 0xd0, 0xba, 0x73, 0x26, 0xd0, 0xf2, 0x07, 0x52, 0x08, 0x69, 0xad, 0x47, 0x22, + 0xad, 0xdb, 0x67, 0x21, 0xad, 0x01, 0x53, 0xcb, 0xa1, 0xd6, 0xab, 0x91, 0x50, 0xeb, 0xfe, 0x24, + 0x50, 0xcb, 0x13, 0x3a, 0x84, 0xb5, 0x3e, 0x1f, 0x8d, 0xb5, 0xbe, 0x71, 0x8e, 0x1b, 0x71, 0x23, + 0xc1, 0xd6, 0xb7, 0x86, 0xc0, 0x96, 0x38, 0x0e, 0x6c, 0xf9, 0x33, 0xc3, 0x45, 0x5b, 0xb5, 0x08, + 0x6c, 0x74, 0x73, 0x3c, 0x36, 0xf2, 0x17, 0x72, 0x1f, 0x1c, 0xd5, 0xa3, 0xc0, 0xd1, 0xad, 0x33, + 0xc0, 0x91, 0x3f, 0xc1, 0x02, 0xe8, 0xe8, 0xf1, 0x00, 0x3a, 0xba, 0x7e, 0x66, 0x84, 0x63, 0x00, + 0x1e, 0x3d, 0x1e, 0x80, 0x47, 0xd7, 0xc7, 0xc0, 0x23, 0x9f, 0x99, 0xe3, 0xa3, 0x95, 0x61, 0x7c, + 0x74, 0x63, 0x2c, 0x3e, 0xf2, 0x24, 0xf8, 0x00, 0x69, 0x3d, 0x12, 0x20, 0xdd, 0x3e, 0x0b, 0x20, + 0xf9, 0x23, 0x32, 0x88, 0x90, 0x36, 0xa3, 0x11, 0xd2, 0x9d, 0x33, 0x11, 0xd2, 0xc0, 0xea, 0xe9, + 0x42, 0xa4, 0x7a, 0x14, 0x44, 0xba, 0x75, 0x06, 0x44, 0x0a, 0xae, 0x9e, 0x1e, 0x46, 0x6a, 0x8d, + 0xc2, 0x48, 0xf7, 0x26, 0xc0, 0x48, 0xbe, 0x4f, 0x39, 0x00, 0x92, 0x3e, 0x1d, 0x04, 0x49, 0xe2, + 0x38, 0x90, 0xe4, 0x8f, 0x65, 0x17, 0x25, 0x6d, 0x46, 0xa3, 0xa4, 0x3b, 0x67, 0xa2, 0xa4, 0xa0, + 0x79, 0x09, 0xc0, 0xa4, 0x4f, 0x07, 0x61, 0x92, 0x38, 0x0e, 0x26, 0xf9, 0xf5, 0x71, 0x71, 0x52, + 0x3d, 0x0a, 0x27, 0xdd, 0x3a, 0x03, 0x27, 0x05, 0x56, 0x1d, 0x1f, 0x28, 0xfd, 0xf2, 0xe4, 0x40, + 0xe9, 0xa3, 0x37, 0x0d, 0xcc, 0x39, 0x1b, 0x29, 0x7d, 0x3a, 0x88, 0x94, 0xc4, 0x71, 0x48, 0xc9, + 0xd7, 0xc7, 0xf9, 0xa0, 0xd2, 0x5a, 0x2a, 0xfd, 0x76, 0xf1, 0x1d, 0xf1, 0xcf, 0x66, 0x60, 0xa6, + 0xee, 0x46, 0xc4, 0x06, 0x6e, 0x3e, 0x13, 0xde, 0xe4, 0xe6, 0x33, 0xb4, 0x4a, 0x86, 0x16, 0xf5, + 0x98, 0xce, 0xbe, 0x2f, 0x73, 0xf8, 0x46, 0x47, 0xce, 0xfa, 0x06, 0x57, 0x10, 0xa0, 0x0f, 0x20, + 0xd7, 0xb7, 0xb1, 0x25, 0xf7, 0x2c, 0xdd, 0xb4, 0x74, 0x87, 0x9d, 0x7b, 0x12, 0x56, 0x8a, 0x5f, + 0x9e, 0x2c, 0xce, 0x6d, 0xdb, 0xd8, 0xda, 0xe2, 0x74, 0x69, 0xae, 0x1f, 0x48, 0xb9, 0xdf, 0xc2, + 0x9b, 0x9e, 0xfc, 0x5b, 0x78, 0x2f, 0xa0, 0x48, 0xdf, 0x39, 0x07, 0x17, 0x19, 0x76, 0xcb, 0x58, + 0xf4, 0x7a, 0x48, 0xcf, 0x25, 0xba, 0x25, 0xe9, 0x6d, 0x63, 0x05, 0x2b, 0x4c, 0x44, 0x4d, 0xa0, + 0xf7, 0xff, 0xc8, 0x3d, 0xb3, 0xa3, 0xab, 0xc7, 0xd4, 0x77, 0x08, 0x5f, 0xe2, 0x3e, 0xf6, 0x53, + 0x0a, 0xaf, 0x14, 0xdd, 0xd9, 0xa2, 0x9c, 0x12, 0x1c, 0x7a, 0xbf, 0xd1, 0x03, 0xb8, 0xd8, 0x55, + 0x8e, 0x68, 0x68, 0xb2, 0xec, 0x3a, 0x03, 0x34, 0x5c, 0x98, 0x7d, 0x15, 0x0f, 0x75, 0x95, 0x23, + 0xfa, 0xb5, 0x3e, 0x96, 0x45, 0x3f, 0xb5, 0x73, 0x1d, 0xe6, 0xf8, 0x79, 0x08, 0xf6, 0x25, 0xae, + 0x02, 0x2d, 0xc9, 0x3f, 0xcb, 0xc2, 0x3e, 0xc6, 0x75, 0x0b, 0xf2, 0x9a, 0x6e, 0x3b, 0xba, 0xa1, + 0x3a, 0xfc, 0xe2, 0x6c, 0x76, 0x43, 0x74, 0xce, 0xa5, 0xb2, 0xdb, 0xb1, 0x5b, 0x30, 0xaf, 0x76, + 0x74, 0xcf, 0xc5, 0x62, 0x8b, 0xde, 0xfc, 0xc8, 0xb1, 0x5c, 0xa5, 0x65, 0x07, 0x5f, 0x08, 0x17, + 0xd4, 0x30, 0x19, 0x55, 0xa1, 0xd0, 0x56, 0x1c, 0x7c, 0xa8, 0x1c, 0xcb, 0xee, 0xf1, 0xca, 0x2c, + 0x3d, 0xa7, 0xfe, 0xd6, 0xe9, 0xc9, 0x62, 0xee, 0x29, 0xcb, 0x1a, 0x3a, 0x65, 0x99, 0x6b, 0x07, + 0x32, 0x34, 0x74, 0x07, 0x0a, 0x8a, 0x7d, 0x6c, 0xa8, 0xb4, 0x03, 0xb1, 0x61, 0xf7, 0x6d, 0xea, + 0x21, 0xa7, 0xa5, 0x3c, 0x25, 0x57, 0x5d, 0x2a, 0x7a, 0x0c, 0x65, 0xfe, 0x7d, 0x8c, 0x43, 0xc5, + 0xd2, 0x64, 0xda, 0xe9, 0xfe, 0xf4, 0x28, 0x52, 0x9e, 0xcb, 0xec, 0x7b, 0x18, 0xa4, 0x00, 0xe9, + 0xe9, 0xe0, 0x25, 0xd1, 0xb3, 0xc5, 0xf4, 0x5a, 0x2a, 0x0d, 0xc5, 0xec, 0x5a, 0x2a, 0x3d, 0x57, + 0xcc, 0xad, 0xa5, 0xd2, 0xf9, 0x62, 0x41, 0xfc, 0x35, 0x01, 0xe6, 0x42, 0x07, 0xc9, 0x1e, 0x0f, + 0xbc, 0x44, 0xbe, 0x12, 0xed, 0xec, 0x8f, 0x0a, 0x61, 0x4f, 0xf3, 0xae, 0x75, 0xa3, 0xd8, 0x17, + 0x47, 0xbb, 0x78, 0x74, 0x37, 0xc4, 0x0d, 0xa3, 0x71, 0xd9, 0x3e, 0x4e, 0x7d, 0xff, 0x07, 0x8b, + 0x53, 0xe2, 0x9f, 0xa7, 0x20, 0x17, 0x3e, 0x36, 0xd6, 0x18, 0xa8, 0x57, 0x94, 0x71, 0x0f, 0x71, + 0x54, 0xc6, 0x7c, 0xd9, 0x27, 0xe3, 0x7f, 0xec, 0x82, 0x55, 0xf3, 0xda, 0x98, 0x57, 0xe5, 0xc1, + 0x7a, 0xfa, 0x8c, 0xe5, 0xff, 0x98, 0xf4, 0xec, 0x54, 0x05, 0xa6, 0xe9, 0x85, 0x53, 0xbc, 0x6a, + 0xa5, 0xc1, 0x99, 0x42, 0x3c, 0x17, 0x92, 0x2f, 0xb1, 0x62, 0xc4, 0xae, 0xb5, 0xde, 0xe8, 0x46, + 0x47, 0xdf, 0x24, 0x9f, 0xff, 0x9b, 0x99, 0x7d, 0x76, 0xa3, 0xe7, 0xff, 0xc7, 0x50, 0x1b, 0xf2, + 0x3c, 0xf4, 0x0b, 0x50, 0x50, 0xcd, 0x4e, 0x87, 0xad, 0x59, 0x6c, 0x86, 0x0e, 0xdf, 0xf1, 0x43, + 0xab, 0xc0, 0x3f, 0x93, 0x5a, 0xf1, 0x3e, 0x97, 0x5a, 0x91, 0xf8, 0xe7, 0x52, 0x03, 0x71, 0xd0, + 0x79, 0x4f, 0x18, 0x9b, 0xd8, 0x03, 0x21, 0xd9, 0xb3, 0x6f, 0x12, 0x92, 0xcd, 0x82, 0xec, 0xf9, + 0xc8, 0xfb, 0x23, 0x81, 0x07, 0xc4, 0x3c, 0x33, 0xcd, 0xfd, 0xbe, 0x17, 0x44, 0x5d, 0x0e, 0xde, + 0xcf, 0xe9, 0x47, 0x8b, 0xd2, 0x23, 0x41, 0x51, 0x16, 0x38, 0xf1, 0xd5, 0x2c, 0xf0, 0x75, 0x98, + 0xeb, 0x59, 0x78, 0x17, 0x3b, 0xea, 0x9e, 0x6c, 0xf4, 0xbb, 0xfc, 0x3c, 0x54, 0xd6, 0xa5, 0x6d, + 0xf6, 0xbb, 0xe8, 0x1e, 0x14, 0xbd, 0x22, 0x1c, 0xce, 0xb8, 0x97, 0xc3, 0xb9, 0x74, 0x0e, 0x7e, + 0xc4, 0xff, 0x2d, 0xc0, 0x42, 0xa8, 0x4d, 0x7c, 0x4e, 0xad, 0x41, 0x56, 0xf3, 0xd6, 0x3c, 0xbb, + 0x24, 0x9c, 0x33, 0x8e, 0x38, 0xc8, 0x8c, 0x64, 0xb8, 0xe4, 0x3e, 0x96, 0x7e, 0x20, 0xc2, 0x17, + 0x9b, 0x38, 0xa7, 0xd8, 0x8b, 0xbe, 0x9c, 0xd5, 0xc0, 0x03, 0xbc, 0x49, 0x96, 0x9c, 0x68, 0x92, + 0x89, 0xbf, 0x25, 0x40, 0x91, 0x3e, 0xe0, 0x09, 0xc6, 0x5a, 0x2c, 0xd6, 0xcd, 0x0d, 0xd8, 0x4f, + 0x4c, 0x7e, 0xe2, 0x29, 0xf4, 0x51, 0x9b, 0x64, 0xf8, 0xa3, 0x36, 0xe2, 0x0f, 0x04, 0xc8, 0x7b, + 0x35, 0x64, 0x1f, 0x9e, 0x1c, 0x73, 0x0d, 0xec, 0x9b, 0x7d, 0x5c, 0xd1, 0xbd, 0xae, 0x66, 0xa2, + 0x6f, 0x61, 0x06, 0xaf, 0xab, 0x61, 0x1f, 0x05, 0xfc, 0x3b, 0xee, 0xc8, 0x21, 0x55, 0xac, 0xfa, + 0xf7, 0x84, 0xbc, 0xc1, 0xe1, 0x2f, 0x89, 0x7e, 0xb3, 0xd7, 0xec, 0x1c, 0xb0, 0x1b, 0x86, 0x26, + 0x32, 0x7b, 0x88, 0x87, 0x81, 0x01, 0xdf, 0xf8, 0xd0, 0x5a, 0x4d, 0xfa, 0x35, 0x5f, 0xf6, 0xdb, + 0x16, 0x9f, 0x04, 0x14, 0x48, 0x3b, 0x9f, 0x68, 0x69, 0x22, 0x53, 0xec, 0x6a, 0x89, 0x8d, 0x95, + 0x3f, 0x08, 0xf6, 0x44, 0xed, 0x80, 0x60, 0xb0, 0x47, 0x90, 0x3c, 0x50, 0x3a, 0xe3, 0x22, 0xa9, + 0x42, 0x3d, 0x27, 0x91, 0xd2, 0xe8, 0x49, 0xe8, 0x7a, 0x95, 0xc4, 0xe8, 0x5d, 0x89, 0x61, 0x95, + 0x86, 0xae, 0x61, 0xf9, 0x30, 0x3c, 0xd6, 0xc7, 0x3e, 0x3e, 0x38, 0xe8, 0x3f, 0x4e, 0xfd, 0xe8, + 0x07, 0x8b, 0x82, 0xf8, 0x09, 0x20, 0x09, 0xdb, 0xd8, 0x79, 0xd1, 0x37, 0x2d, 0xff, 0xaa, 0x9a, + 0xc1, 0x18, 0xfa, 0xe9, 0xe8, 0x18, 0x7a, 0xf1, 0x22, 0x2c, 0x84, 0xb8, 0x99, 0xb1, 0x10, 0x3f, + 0x84, 0x2b, 0x4f, 0x4d, 0xdb, 0xd6, 0x7b, 0x04, 0xf8, 0xd0, 0x59, 0x49, 0x96, 0x16, 0xcf, 0x3c, + 0xa6, 0x7b, 0x14, 0x6b, 0x1a, 0xcc, 0x8c, 0x64, 0x24, 0x2f, 0x2d, 0xfe, 0x9e, 0x00, 0x97, 0x87, + 0x39, 0x99, 0x96, 0xa3, 0xce, 0xaa, 0xce, 0xaa, 0xa6, 0x7f, 0x93, 0xe2, 0xd9, 0xa3, 0xd5, 0x2d, + 0x4e, 0x1c, 0x29, 0xfe, 0x4c, 0xb9, 0xab, 0x50, 0xf3, 0xc1, 0xcf, 0xcd, 0xe7, 0x39, 0x79, 0x83, + 0x51, 0x7d, 0x4b, 0x92, 0x9a, 0xcc, 0x92, 0xb4, 0xa0, 0xb0, 0x66, 0xea, 0x06, 0xf1, 0xd7, 0xdc, + 0xf6, 0x2e, 0x43, 0x7e, 0x47, 0x37, 0x14, 0xeb, 0x58, 0x76, 0x03, 0xf8, 0x84, 0xb3, 0x02, 0xf8, + 0xa4, 0x1c, 0xe3, 0xe0, 0x49, 0xf1, 0xc7, 0x02, 0x14, 0x7d, 0xb1, 0xdc, 0x22, 0xbf, 0x0b, 0xa0, + 0x76, 0xfa, 0xb6, 0x83, 0x2d, 0xb7, 0x97, 0xe6, 0x58, 0x64, 0x7e, 0x95, 0x51, 0x1b, 0xab, 0x52, + 0x86, 0x17, 0x68, 0x68, 0xe8, 0x46, 0xf8, 0x5a, 0x8f, 0xe9, 0x15, 0x38, 0x1d, 0xba, 0xcc, 0x83, + 0x74, 0xbb, 0xed, 0x98, 0x96, 0x87, 0x5d, 0x78, 0xb7, 0xbb, 0xb7, 0x28, 0xd1, 0xd3, 0xe8, 0x98, + 0x1e, 0xbe, 0xc9, 0x13, 0x77, 0xe1, 0x00, 0x7b, 0x4d, 0x4a, 0x9d, 0xdd, 0x24, 0xc6, 0xe1, 0x36, + 0xe9, 0x5f, 0x0a, 0x50, 0xa8, 0xb2, 0xde, 0xf0, 0x7a, 0x78, 0x8c, 0x45, 0x5b, 0x85, 0xb4, 0x73, + 0x64, 0xc8, 0x5d, 0xec, 0x7d, 0x4b, 0xe8, 0x1c, 0xd7, 0x1c, 0xce, 0x3a, 0x2c, 0x49, 0x3f, 0x4f, + 0xc9, 0xbf, 0x8d, 0xce, 0xa7, 0xcb, 0x95, 0x0a, 0xfb, 0x78, 0x7a, 0xc5, 0xfd, 0x78, 0x7a, 0x65, + 0x95, 0x17, 0x60, 0x46, 0xfd, 0xfb, 0xff, 0x65, 0x51, 0x90, 0x3c, 0x26, 0xb6, 0xee, 0xdf, 0x6f, + 0x92, 0x51, 0x3f, 0xb4, 0x32, 0xa3, 0x3c, 0x40, 0xe0, 0x23, 0x51, 0xfc, 0x73, 0xdc, 0xcb, 0xab, + 0xf2, 0xf6, 0x66, 0xf5, 0xf9, 0xc6, 0x46, 0xa3, 0xd5, 0xaa, 0xad, 0x16, 0x05, 0x54, 0x84, 0xb9, + 0xd0, 0x27, 0xa6, 0x12, 0xec, 0x03, 0xdd, 0xf7, 0xff, 0x0a, 0x80, 0xff, 0xb5, 0x3a, 0x22, 0x6b, + 0xbd, 0xf6, 0x99, 0xfc, 0x72, 0xf9, 0xd9, 0x76, 0xad, 0x59, 0x9c, 0x42, 0x08, 0xf2, 0x2b, 0xcb, + 0xad, 0x6a, 0x5d, 0x96, 0x6a, 0xcd, 0xad, 0xe7, 0x9b, 0xcd, 0x9a, 0xfb, 0x61, 0xef, 0xfb, 0xab, + 0x30, 0x17, 0xbc, 0xbc, 0x09, 0x2d, 0x40, 0xa1, 0x5a, 0xaf, 0x55, 0xd7, 0xe5, 0x97, 0x8d, 0x65, + 0xf9, 0xc5, 0x76, 0x6d, 0xbb, 0x56, 0x9c, 0xa2, 0x55, 0xa3, 0xc4, 0x27, 0xdb, 0xcf, 0x9e, 0x15, + 0x05, 0x54, 0x80, 0x2c, 0x4b, 0xd3, 0xcf, 0x51, 0x15, 0x13, 0xf7, 0x37, 0x20, 0x1b, 0xb8, 0x54, + 0x9a, 0x3c, 0x6e, 0x6b, 0xbb, 0x59, 0x97, 0x5b, 0x8d, 0x8d, 0x5a, 0xb3, 0xb5, 0xbc, 0xb1, 0xc5, + 0x64, 0x50, 0xda, 0xf2, 0xca, 0x73, 0xa9, 0x55, 0x14, 0xbc, 0x74, 0xeb, 0xf9, 0x76, 0xb5, 0xee, + 0x36, 0x43, 0x4c, 0xa5, 0x93, 0xc5, 0xe4, 0xfd, 0xbf, 0x2e, 0xc0, 0xe5, 0x11, 0x17, 0x19, 0xa1, + 0x2c, 0xcc, 0x6e, 0x1b, 0xf4, 0x86, 0xdd, 0xe2, 0x14, 0xca, 0x05, 0xee, 0x32, 0x2a, 0x0a, 0x28, + 0xcd, 0x6e, 0x93, 0x29, 0x26, 0xd0, 0x0c, 0x24, 0x9a, 0x8f, 0x8a, 0x49, 0x52, 0xd3, 0xc0, 0x55, + 0x40, 0xc5, 0x14, 0xca, 0xf0, 0x4b, 0x48, 0x8a, 0xd3, 0x68, 0xce, 0xbf, 0x0b, 0xa4, 0x38, 0x43, + 0x44, 0x79, 0x77, 0x6a, 0x14, 0x67, 0xef, 0x5f, 0x87, 0xc0, 0xbd, 0x05, 0x08, 0x60, 0xe6, 0x99, + 0xe2, 0x60, 0xdb, 0x29, 0x4e, 0xa1, 0x59, 0x48, 0x2e, 0x77, 0x3a, 0x45, 0xe1, 0xe1, 0xbf, 0x48, + 0x41, 0xda, 0xfd, 0xec, 0x12, 0x7a, 0x06, 0xd3, 0x6c, 0x73, 0x71, 0x71, 0x34, 0x5a, 0xa0, 0x13, + 0xba, 0x7c, 0xed, 0x2c, 0x38, 0x21, 0x4e, 0xa1, 0xbf, 0x0a, 0xd9, 0x80, 0x17, 0x85, 0x46, 0x6e, + 0xe8, 0x84, 0x3c, 0xc7, 0xf2, 0xed, 0xb3, 0x8a, 0x79, 0xf2, 0x5f, 0x41, 0xc6, 0xb3, 0xea, 0xe8, + 0xc6, 0x38, 0x9b, 0xef, 0xca, 0x1e, 0xbf, 0x30, 0x90, 0xf9, 0x27, 0x4e, 0xbd, 0x2f, 0x20, 0x0b, + 0xd0, 0xb0, 0x01, 0x46, 0x51, 0x61, 0x5c, 0x23, 0x2d, 0x7c, 0xf9, 0xfe, 0x44, 0xa5, 0xfd, 0x67, + 0x12, 0x65, 0xf9, 0xab, 0x48, 0xb4, 0xb2, 0x86, 0xd6, 0xa8, 0x68, 0x65, 0x45, 0x2c, 0x46, 0x53, + 0xe8, 0x05, 0xa4, 0x88, 0xf5, 0x44, 0x51, 0x7e, 0xe5, 0x80, 0xb5, 0x2e, 0xdf, 0x18, 0x5b, 0xc6, + 0x15, 0xb9, 0x72, 0xef, 0x47, 0x7f, 0x7a, 0x75, 0xea, 0x47, 0xa7, 0x57, 0x85, 0x1f, 0x9f, 0x5e, + 0x15, 0xfe, 0xf8, 0xf4, 0xaa, 0xf0, 0x27, 0xa7, 0x57, 0x85, 0xef, 0xfd, 0xe4, 0xea, 0xd4, 0x8f, + 0x7f, 0x72, 0x75, 0xea, 0x8f, 0x7f, 0x72, 0x75, 0xea, 0xf3, 0x59, 0xce, 0xbd, 0x33, 0x43, 0x4d, + 0xcb, 0xa3, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0xca, 0x8f, 0x18, 0xcf, 0xfe, 0x82, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -11480,6 +11496,23 @@ func (m *LeaseInfoResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.EvaluatedBy != 0 { + i = encodeVarintApi(dAtA, i, uint64(m.EvaluatedBy)) + i-- + dAtA[i] = 0x20 + } + if m.CurrentLease != nil { + { + size, err := m.CurrentLease.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } { size, err := m.Lease.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -16645,12 +16678,12 @@ func (m *ContentionEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n279, err279 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) - if err279 != nil { - return 0, err279 + n280, err280 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) + if err280 != nil { + return 0, err280 } - i -= n279 - i = encodeVarintApi(dAtA, i, uint64(n279)) + i -= n280 + i = encodeVarintApi(dAtA, i, uint64(n280)) i-- dAtA[i] = 0x1a { @@ -17824,6 +17857,13 @@ func (m *LeaseInfoResponse) Size() (n int) { n += 1 + l + sovApi(uint64(l)) l = m.Lease.Size() n += 1 + l + sovApi(uint64(l)) + if m.CurrentLease != nil { + l = m.CurrentLease.Size() + n += 1 + l + sovApi(uint64(l)) + } + if m.EvaluatedBy != 0 { + n += 1 + sovApi(uint64(m.EvaluatedBy)) + } return n } @@ -28882,6 +28922,61 @@ func (m *LeaseInfoResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CurrentLease", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CurrentLease == nil { + m.CurrentLease = &Lease{} + } + if err := m.CurrentLease.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EvaluatedBy", wireType) + } + m.EvaluatedBy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.EvaluatedBy |= StoreID(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 58e112b7ffd6..11a3291daef2 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1195,7 +1195,9 @@ message TransferLeaseRequest { // LeaseInfoRequest is the argument to the LeaseInfo() method, for getting // information about a range's lease. // It's a point request, so it addresses one single range, and returns the lease -// currently in effect for that range. +// currently in effect for that range. This request is commonly set with +// ReadConsistency=INCONSISTENT in order for the request to be served by the +// node to whom the request was sent. message LeaseInfoRequest{ RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; } @@ -1205,6 +1207,19 @@ message LeaseInfoResponse{ // The last lease known by the replica serving the request. It can also be the // tentative future lease, if a lease transfer is in progress. Lease lease = 2 [(gogoproto.nullable) = false]; + // current_lease is set if `lease` represents a tentative future lease. In + // that case, current_lease represents the lease that's currently in effect. + Lease current_lease = 3; + // evaluated_by returns the store that evaluated this request. This + // corresponds to the leaseholder unless ReadConsistency=INCONSISTENT was + // used. The response reflects the evaluator's view of the lease. When the + // client cares to see a particular node's view, it can use this field to + // check whether the node it intended query (by sending the request to that + // node and using ReadConsistency=INCONSISTENT) indeed served it - it's + // possible that even if ReadConsistency=INCONSISTENT was used, the request is + // still not evaluated by the node it was sent to if that node's replica is a + // learner or the node doesn't have a replica at all. + int32 evaluated_by = 4 [(gogoproto.casttype) = "StoreID"]; } // A RequestLeaseResponse is the response to a RequestLease() or TransferLease() diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 9730c22ec305..cdcef72f58d3 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -1238,13 +1238,46 @@ func (ts *TestServer) SplitRange( return ts.SplitRangeWithExpiration(splitKey, hlc.MaxTimestamp) } -// GetRangeLease returns the current lease for the range containing key, and a -// timestamp taken from the node. +// LeaseInfo describes a range's current and potentially future lease. +type LeaseInfo struct { + cur, next roachpb.Lease +} + +// Current returns the range's current lease. +func (l LeaseInfo) Current() roachpb.Lease { + return l.cur +} + +// CurrentOrProspective returns the range's potential next lease, if a lease +// request is in progress, or the current lease otherwise. +func (l LeaseInfo) CurrentOrProspective() roachpb.Lease { + if !l.next.Empty() { + return l.next + } + return l.cur +} + +// LeaseInfoOpt enumerates options for GetRangeLease. +type LeaseInfoOpt int + +const ( + // AllowQueryToBeForwardedToDifferentNode specifies that, if the current node + // doesn't have a voter replica, the lease info can come from a different + // node. + AllowQueryToBeForwardedToDifferentNode LeaseInfoOpt = iota + // QueryLocalNodeOnly specifies that an error should be returned if the node + // is not able to serve the lease query (because it doesn't have a voting + // replica). + QueryLocalNodeOnly +) + +// GetRangeLease returns information on the lease for the range containing key, and a +// timestamp taken from the node. The lease is returned regardless of its status. // -// The lease is returned regardless of its status. +// queryPolicy specifies if its OK to forward the request to a different node. func (ts *TestServer) GetRangeLease( - ctx context.Context, key roachpb.Key, -) (_ roachpb.Lease, now hlc.ClockTimestamp, _ error) { + ctx context.Context, key roachpb.Key, queryPolicy LeaseInfoOpt, +) (_ LeaseInfo, now hlc.ClockTimestamp, _ error) { leaseReq := roachpb.LeaseInfoRequest{ RequestHeader: roachpb.RequestHeader{ Key: key, @@ -1262,10 +1295,25 @@ func (ts *TestServer) GetRangeLease( &leaseReq, ) if pErr != nil { - return roachpb.Lease{}, hlc.ClockTimestamp{}, pErr.GoError() + return LeaseInfo{}, hlc.ClockTimestamp{}, pErr.GoError() + } + // Adapt the LeaseInfoResponse format to LeaseInfo. + resp := leaseResp.(*roachpb.LeaseInfoResponse) + if queryPolicy == QueryLocalNodeOnly && resp.EvaluatedBy != ts.GetFirstStoreID() { + // TODO(andrei): Figure out how to deal with nodes with multiple stores. + // This API API should permit addressing the query to a particular store. + return LeaseInfo{}, hlc.ClockTimestamp{}, errors.Errorf( + "request not evaluated locally; evaluated by s%d instead of local s%d", + resp.EvaluatedBy, ts.GetFirstStoreID()) + } + var l LeaseInfo + if resp.CurrentLease != nil { + l.cur = *resp.CurrentLease + l.next = resp.Lease + } else { + l.cur = resp.Lease } - return leaseResp.(*roachpb.LeaseInfoResponse).Lease, ts.Clock().NowAsClockTimestamp(), nil - + return l, ts.Clock().NowAsClockTimestamp(), nil } // ExecutorConfig is part of the TestServerInterface. diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index db812d184c35..e06bcc71bef5 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -696,7 +696,8 @@ func (tc *TestCluster) WaitForVoters( // startKey is start key of range. // // waitForVoter indicates that the method should wait until the targets are full -// voters in the range. +// voters in the range (and they also know that they're voters - i.e. the +// respective replica has caught up with the config change). // // targets are replication target for change replica. func (tc *TestCluster) waitForNewReplicas( @@ -953,29 +954,53 @@ func (tc *TestCluster) MoveRangeLeaseNonCooperatively( // FindRangeLease is similar to FindRangeLeaseHolder but returns a Lease proto // without verifying if the lease is still active. Instead, it returns a time- // stamp taken off the queried node's clock. +// +// DEPRECATED - use FindRangeLeaseEx instead. func (tc *TestCluster) FindRangeLease( rangeDesc roachpb.RangeDescriptor, hint *roachpb.ReplicationTarget, ) (_ roachpb.Lease, now hlc.ClockTimestamp, _ error) { + l, now, err := tc.FindRangeLeaseEx(context.TODO(), rangeDesc, hint) + if err != nil { + return roachpb.Lease{}, hlc.ClockTimestamp{}, err + } + return l.CurrentOrProspective(), now, err +} + +// FindRangeLeaseEx returns information about a range's lease. As opposed to +// FindRangeLeaseHolder, it doesn't check the validity of the lease; instead it +// returns a timestamp from a node's clock. +// +// If hint is not nil, the respective node will be queried. If that node doesn't +// have a replica able to serve a LeaseInfoRequest, an error will be returned. +// If hint is nil, the first node is queried. In either case, if the returned +// lease is not valid, it's possible that the returned lease information is +// stale - i.e. there might be a newer lease unbeknownst to the queried node. +func (tc *TestCluster) FindRangeLeaseEx( + ctx context.Context, rangeDesc roachpb.RangeDescriptor, hint *roachpb.ReplicationTarget, +) (_ server.LeaseInfo, now hlc.ClockTimestamp, _ error) { + var queryPolicy server.LeaseInfoOpt if hint != nil { var ok bool if _, ok = rangeDesc.GetReplicaDescriptor(hint.StoreID); !ok { - return roachpb.Lease{}, hlc.ClockTimestamp{}, errors.Errorf( + return server.LeaseInfo{}, hlc.ClockTimestamp{}, errors.Errorf( "bad hint: %+v; store doesn't have a replica of the range", hint) } + queryPolicy = server.QueryLocalNodeOnly } else { hint = &roachpb.ReplicationTarget{ NodeID: rangeDesc.Replicas().Descriptors()[0].NodeID, StoreID: rangeDesc.Replicas().Descriptors()[0].StoreID} + queryPolicy = server.AllowQueryToBeForwardedToDifferentNode } // Find the server indicated by the hint and send a LeaseInfoRequest through // it. hintServer, err := tc.findMemberServer(hint.StoreID) if err != nil { - return roachpb.Lease{}, hlc.ClockTimestamp{}, errors.Wrapf(err, "bad hint: %+v; no such node", hint) + return server.LeaseInfo{}, hlc.ClockTimestamp{}, errors.Wrapf(err, "bad hint: %+v; no such node", hint) } - return hintServer.GetRangeLease(context.TODO(), rangeDesc.StartKey.AsRawKey()) + return hintServer.GetRangeLease(ctx, rangeDesc.StartKey.AsRawKey(), queryPolicy) } // FindRangeLeaseHolder is part of TestClusterInterface. From edc4b5372bac83d16d8040eea9cdff8b73f081fc Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Thu, 8 Apr 2021 20:25:04 -0400 Subject: [PATCH 13/37] kvserver: fix propagation of closedts from side trans to replica state A replica maintains a local copy of the closed timestamp receiver's info for the respective closed timestamp, for querying performance. The idea was that when the replica's local info is to stale, we go to the receiver, and on the way back update the replica. We were failing to update the replica because a defer was capturing by value a variable, missing its subsequent update. This patch fixes it by introducing more encapsulation on the replica's relationship with the receiver, simplifying the code. Release note: None --- pkg/kv/kvserver/BUILD.bazel | 1 + pkg/kv/kvserver/replica.go | 39 ++--- pkg/kv/kvserver/replica_closedts.go | 130 +++++++++++++--- .../replica_closedts_internal_test.go | 139 ++++++++++++++++++ pkg/kv/kvserver/replica_follower_read.go | 119 +++------------ pkg/kv/kvserver/replica_init.go | 3 + pkg/kv/kvserver/replica_send.go | 7 +- pkg/testutils/localtestcluster/BUILD.bazel | 1 + .../localtestcluster/local_test_cluster.go | 2 + 9 files changed, 286 insertions(+), 155 deletions(-) create mode 100644 pkg/kv/kvserver/replica_closedts_internal_test.go diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 8c0034adab20..92f54e6de9d4 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -244,6 +244,7 @@ go_test( "replica_application_cmd_buf_test.go", "replica_application_state_machine_test.go", "replica_batch_updates_test.go", + "replica_closedts_internal_test.go", "replica_closedts_test.go", "replica_command_test.go", "replica_consistency_test.go", diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 179e6e3f38a8..865175233bb4 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -262,6 +262,18 @@ type Replica struct { // metrics about it. tenantLimiter tenantrate.Limiter + // sideTransportClosedTimestamp encapsulates state related to the closed + // timestamp's information about the range. Note that the + // sideTransportClosedTimestamp does not incorporate the closed timestamp + // information carried by Raft commands. That can be found in + // r.mu.state.RaftClosedTimestamp. Generally, the Raft state should be queried + // in parallel with the side transport state to determine an up to date closed + // timestamp (i.e. the maximum across the two). For a given LAI, the side + // transport closed timestamp will always lead the Raft closed timestamp. + // Across LAIs, the larger LAI will always include the larger closed + // timestamp, independent of the source. + sideTransportClosedTimestamp sidetransportAccess + mu struct { // Protects all fields in the mu struct. syncutil.RWMutex @@ -406,25 +418,6 @@ type Replica struct { // The minimum allowed ID for this replica. Initialized from // RangeTombstone.NextReplicaID. tombstoneMinReplicaID roachpb.ReplicaID - // sideTransportClosedTimestamp stores the closed timestamp that was - // communicated by the side transport. The replica can use it if it has - // applied all the commands with indexes <= sideTransportCloseTimestampLAI. - // Note that there's also state.RaftClosedTimestamp, which might be higher - // than this closed timestamp. The maximum across the two can be used. - // - // TODO(andrei): actually implement and reference also the global storage - // for side-transport closed timestamps. - // - // TODO(andrei): document here and probably elsewhere the relationship - // between the sideTransportClosedTimestamp and the raftClosedTimestamp. - // Specifically that for a given LAI, the side transport closed timestamp - // will always lead the raft closed timestamp, but that across LAIs, the - // larger LAI will always include the larger closed timestamp, independent - // of the source. - sideTransportClosedTimestamp hlc.Timestamp - // sideTransportCloseTimestampLAI is the lease-applied index associated - // with sideTransportClosedTimestamp. - sideTransportCloseTimestampLAI ctpb.LAI // The ID of the leader replica within the Raft group. Used to determine // when the leadership changes. @@ -1233,10 +1226,6 @@ func (r *Replica) checkExecutionCanProceed( r.maybeExtendLeaseAsync(ctx, st) } }() - var update replicaUpdate - // When we're done, apply the update (if any) after releasing r.mu. - defer update.apply(ctx, r) - now := r.Clock().NowAsClockTimestamp() rSpan, err := keys.Range(ba.Requests) if err != nil { @@ -1292,9 +1281,7 @@ func (r *Replica) checkExecutionCanProceed( // If not, can we serve this request on a follower? // TODO(nvanbenschoten): once we make this check cheaper // than leaseGoodToGoRLocked, invert these checks. - var ok bool - ok, update = r.canServeFollowerReadRLocked(ctx, ba, err) - if !ok { + if !r.canServeFollowerReadRLocked(ctx, ba, err) { return st, err } err = nil // ignore error diff --git a/pkg/kv/kvserver/replica_closedts.go b/pkg/kv/kvserver/replica_closedts.go index 4a7ebd11bd4b..d33293034508 100644 --- a/pkg/kv/kvserver/replica_closedts.go +++ b/pkg/kv/kvserver/replica_closedts.go @@ -15,10 +15,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) // EmitMLAI registers the replica's last assigned max lease index with the @@ -144,8 +146,8 @@ func (r *Replica) BumpSideTransportClosed( // Update the replica directly since there's no side-transport connection to // the local node. - r.mu.sideTransportClosedTimestamp = target - r.mu.sideTransportCloseTimestampLAI = lai + r.sideTransportClosedTimestamp.forward(ctx, target, lai) + return true, lai, policy, desc } @@ -163,35 +165,117 @@ func (r *Replica) closedTimestampTargetRLocked() hlc.Timestamp { ) } -// ForwardSideTransportClosedTimestamp forwards -// r.mu.sideTransportClosedTimestamp. It is called by the closed timestamp -// side-transport receiver. +// ForwardSideTransportClosedTimestamp forwards the side-transport closed +// timestamp. It is called by the closed timestamp side-transport receiver. func (r *Replica) ForwardSideTransportClosedTimestamp( - ctx context.Context, closedTS hlc.Timestamp, lai ctpb.LAI, + ctx context.Context, closed hlc.Timestamp, lai ctpb.LAI, ) { - r.mu.Lock() - defer r.mu.Unlock() + r.sideTransportClosedTimestamp.forward(ctx, closed, lai) +} + +// sidetransportAccess encapsulates state related to the closed timestamp's +// information about the range. It stores a potentially stale closed timestamp +// directly and, when that's not sufficient for a caller, it delegates to the +// sidetransport.Receiver for more up to date information and updates the local +// state. The idea is that the local state is cheap to access and acts as a +// cache. +// +// Note that the sidetransportAccess does not incorporate the closed timestamp +// information carried by Raft commands. That can be found in +// r.mu.state.RaftClosedTimestamp. Generally, the Raft state should be queried +// in parallel with the side transport state to determine an up to date closed +// timestamp (i.e. the maximum across the two). For a given LAI, the side +// transport closed timestamp will always lead the Raft closed timestamp. Across +// LAIs, the larger LAI will always include the larger closed timestamp, +// independent of the source. +type sidetransportAccess struct { + rangeID roachpb.RangeID + receiver sidetransportReceiver + mu struct { + syncutil.RWMutex + // closedTimestamp is the cached info about the closed timestamp that was + // communicated by the side transport. The replica can use it if it has + // applied commands up to (and including) lai. + closedTimestamp hlc.Timestamp + lai ctpb.LAI + } +} + +// sidetransportReceiver abstracts *sidetransport.Receiver. +type sidetransportReceiver interface { + GetClosedTimestamp( + ctx context.Context, rangeID roachpb.RangeID, leaseholderNode roachpb.NodeID, + ) (hlc.Timestamp, ctpb.LAI) +} + +func (st *sidetransportAccess) init(receiver *sidetransport.Receiver, rangeID roachpb.RangeID) { + if receiver != nil { + // Avoid st.receiver becoming a typed nil. + st.receiver = receiver + } + st.rangeID = rangeID +} - if r.mu.sideTransportClosedTimestamp.Forward(closedTS) { - if r.mu.sideTransportCloseTimestampLAI > lai { +// forward bumps the local closed timestamp info. +func (st *sidetransportAccess) forward(ctx context.Context, closed hlc.Timestamp, lai ctpb.LAI) { + st.mu.Lock() + defer st.mu.Unlock() + if st.mu.closedTimestamp.Forward(closed) { + if st.mu.lai > lai { log.Fatalf(ctx, "received side-transport notification with higher closed timestamp "+ "but lower LAI: r%d current LAI: %d received LAI: %d", - r.RangeID, r.mu.sideTransportCloseTimestampLAI, lai) + st.rangeID, st.mu.lai, lai) } - r.mu.sideTransportCloseTimestampLAI = lai + st.mu.lai = lai } } -// getSideTransportClosedTimestamp returns the replica's information about the -// timestamp that was closed by the side-transport. Note that this not include -// r.mu.state.RaftClosedTimestamp. Also note that this might not be the highest -// closed timestamp communicated by the side-transport - the -// ClosedTimestampReceiver should be checked too if an up-to-date value is -// required. +// get returns the closed timestamp that the side transport knows for the range. +// leaseholder is the known leaseholder for the range. appliedLAI is the LAI +// that the replica has caught up to. sufficient, if not empty, is a hint +// indicating that any lower or equal closed timestamp suffices; the caller +// doesn't need the highest closed timestamp necessarily. // -// It's the responsibility of the caller to check the returned LAI against the -// replica's applied LAI. If the returned LAI hasn't applied, the closed -// timestamp cannot be used. -func (r *Replica) getSideTransportClosedTimestampRLocked() (closedTS hlc.Timestamp, lai ctpb.LAI) { - return r.mu.sideTransportClosedTimestamp, r.mu.sideTransportCloseTimestampLAI +// Returns an empty timestamp if no closed timestamp is known. +// +// get can be called without holding replica.mu. This means that a caller can +// pass an appliedLAI that's lower than what a previous caller passed in. That's +// fine, except the second caller might get an empty result. +func (st *sidetransportAccess) get( + ctx context.Context, leaseholder roachpb.NodeID, appliedLAI ctpb.LAI, sufficient hlc.Timestamp, +) hlc.Timestamp { + st.mu.RLock() + closed := st.mu.closedTimestamp + lai := st.mu.lai + st.mu.RUnlock() + + // The local replica hasn't caught up to the closed timestamp we have stored, + // so what we have stored is not usable. There's no point in going to the + // receiver, as that one can only have an even higher LAI. + if appliedLAI < lai { + return hlc.Timestamp{} + } + + // If the local info is enough to satisfy sufficient, we're done. + if !sufficient.IsEmpty() && sufficient.LessEq(closed) { + return closed + } + + // Check with the receiver. + + // Some tests don't have the receiver set. + if st.receiver == nil { + return closed + } + + receiverClosed, receiverLAI := st.receiver.GetClosedTimestamp(ctx, st.rangeID, leaseholder) + if receiverClosed.IsEmpty() || appliedLAI < receiverLAI { + return closed + } + + // Update the local closed timestamp info. + if closed.Forward(receiverClosed) { + st.forward(ctx, closed, receiverLAI) + } + return closed } diff --git a/pkg/kv/kvserver/replica_closedts_internal_test.go b/pkg/kv/kvserver/replica_closedts_internal_test.go new file mode 100644 index 000000000000..a8b998709e9e --- /dev/null +++ b/pkg/kv/kvserver/replica_closedts_internal_test.go @@ -0,0 +1,139 @@ +// Copyright 2021 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 kvserver + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +func TestSideTransportClosed(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + ts1 := hlc.Timestamp{WallTime: 1} + ts2 := hlc.Timestamp{WallTime: 2} + ts3 := hlc.Timestamp{WallTime: 3} + + tests := []struct { + name string + applied ctpb.LAI + localClosed hlc.Timestamp + localLAI ctpb.LAI + receiverClosed hlc.Timestamp + receiverLAI ctpb.LAI + sufficient hlc.Timestamp + + expectedLocalUpdate bool + expectedClosed hlc.Timestamp + }{ + { + name: "all empty", + expectedClosed: hlc.Timestamp{}, + expectedLocalUpdate: false, + }, + { + name: "only local", + applied: 100, + localClosed: ts1, + localLAI: 1, + expectedClosed: ts1, + }, + { + name: "only receiver", + applied: 100, + receiverClosed: ts1, + receiverLAI: 1, + expectedClosed: ts1, + expectedLocalUpdate: true, + }, + { + name: "local sufficient", + applied: 100, + localClosed: ts1, + localLAI: 1, + receiverClosed: ts2, + receiverLAI: 2, + // The caller won't need a closed timestamp > ts1, so we expect the + // receiver to not be consulted. + sufficient: ts1, + expectedClosed: ts1, + expectedLocalUpdate: false, + }, + { + name: "local insufficient", + applied: 100, + localClosed: ts1, + localLAI: 1, + receiverClosed: ts2, + receiverLAI: 2, + sufficient: ts3, + expectedClosed: ts2, + expectedLocalUpdate: true, + }, + { + name: "replication not caught up", + applied: 0, + localClosed: ts1, + localLAI: 1, + receiverClosed: ts2, + receiverLAI: 2, + sufficient: ts3, + // We expect no usable closed timestamp to be returned. And also we expect + // the local state to not be updated because the LAI from the receiver has + // not been applied by the replica. + expectedClosed: hlc.Timestamp{}, + expectedLocalUpdate: false, + }, + } + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + r := mockReceiver{ + closed: tc.receiverClosed, + lai: tc.receiverLAI, + } + var s sidetransportAccess + s.receiver = &r + s.mu.closedTimestamp = tc.localClosed + s.mu.lai = tc.localLAI + closed := s.get(ctx, roachpb.NodeID(1), tc.applied, tc.sufficient) + require.Equal(t, tc.expectedClosed, closed) + if tc.expectedLocalUpdate { + require.Equal(t, tc.receiverClosed, s.mu.closedTimestamp) + require.Equal(t, tc.receiverLAI, s.mu.lai) + } else { + require.Equal(t, tc.localClosed, s.mu.closedTimestamp) + require.Equal(t, tc.localLAI, s.mu.lai) + } + }) + } +} + +type mockReceiver struct { + closed hlc.Timestamp + lai ctpb.LAI +} + +var _ sidetransportReceiver = &mockReceiver{} + +func (r *mockReceiver) GetClosedTimestamp( + ctx context.Context, rangeID roachpb.RangeID, leaseholderNode roachpb.NodeID, +) (hlc.Timestamp, ctpb.LAI) { + return r.closed, r.lai +} diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index c5577161799f..2577442f11e6 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -52,22 +52,6 @@ func BatchCanBeEvaluatedOnFollower(ba roachpb.BatchRequest) bool { return ba.Txn != nil && ba.IsAllTransactional() && ba.IsReadOnly() && !ba.IsLocking() } -// replicaUpdate contains updates to be applied to a replica. It's intended to -// be returned by functions holding r.mu in reader mode, to be applied later -// when the mutex can be taken in write mode. -type replicaUpdate struct { - sideTransportClosedTimestamp hlc.Timestamp - sideTransportClosedLAI ctpb.LAI -} - -// apply copies the information into the replica. This cannot be called with r.mu held. -func (u replicaUpdate) apply(ctx context.Context, r *Replica) { - if u == (replicaUpdate{}) { - return - } - r.ForwardSideTransportClosedTimestamp(ctx, u.sideTransportClosedTimestamp, u.sideTransportClosedLAI) -} - // canServeFollowerReadRLocked tests, when a range lease could not be acquired, // whether the batch can be served as a follower read despite the error. Only // non-locking, read-only requests can be served as follower reads. The batch @@ -75,7 +59,7 @@ func (u replicaUpdate) apply(ctx context.Context, r *Replica) { // accepted as a follower read. func (r *Replica) canServeFollowerReadRLocked( ctx context.Context, ba *roachpb.BatchRequest, err error, -) (bool, replicaUpdate) { +) bool { var lErr *roachpb.NotLeaseHolderError eligible := errors.As(err, &lErr) && lErr.LeaseHolder != nil && lErr.Lease.Type() == roachpb.LeaseEpoch && @@ -84,23 +68,23 @@ func (r *Replica) canServeFollowerReadRLocked( if !eligible { // We couldn't do anything with the error, propagate it. - return false, replicaUpdate{} + return false } repDesc, err := r.getReplicaDescriptorRLocked() if err != nil { - return false, replicaUpdate{} + return false } switch typ := repDesc.GetType(); typ { case roachpb.VOTER_FULL, roachpb.VOTER_INCOMING, roachpb.NON_VOTER: default: log.Eventf(ctx, "%s replicas cannot serve follower reads", typ) - return false, replicaUpdate{} + return false } requiredFrontier := ba.Txn.RequiredFrontier() - maxClosed, _, update := r.maxClosedRLocked(ctx, requiredFrontier /* sufficient */) + maxClosed, _ := r.maxClosedRLocked(ctx, requiredFrontier /* sufficient */) canServeFollowerRead := requiredFrontier.LessEq(maxClosed) tsDiff := requiredFrontier.GoTime().Sub(maxClosed.GoTime()) if !canServeFollowerRead { @@ -124,7 +108,7 @@ func (r *Replica) canServeFollowerReadRLocked( r.store.cfg.ClosedTimestamp.Storage.(*ctstorage.MultiStorage).StringForNodes(lErr.LeaseHolder.NodeID), ) } - return false, update + return false } // This replica can serve this read! @@ -133,7 +117,7 @@ func (r *Replica) canServeFollowerReadRLocked( // serve reads for that and smaller timestamps forever. log.Eventf(ctx, "%s; query timestamp below closed timestamp by %s", kvbase.FollowerReadServingMsg, -tsDiff) r.store.metrics.FollowerReadsCount.Inc(1) - return true, update + return true } // maxClosed returns the maximum closed timestamp for this range. @@ -152,10 +136,8 @@ func (r *Replica) canServeFollowerReadRLocked( // mechanism. func (r *Replica) maxClosed(ctx context.Context) (_ hlc.Timestamp, ok bool) { r.mu.RLock() - res, ok, update := r.maxClosedRLocked(ctx, hlc.Timestamp{} /* sufficient */) - r.mu.RUnlock() - update.apply(ctx, r) - return res, ok + defer r.mu.RUnlock() + return r.maxClosedRLocked(ctx, hlc.Timestamp{} /* sufficient */) } // maxClosedRLocked is like maxClosed, except that it requires r.mu to be @@ -166,33 +148,18 @@ func (r *Replica) maxClosed(ctx context.Context) (_ hlc.Timestamp, ok bool) { // we can avoid consulting the ClosedTimestampReceiver. func (r *Replica) maxClosedRLocked( ctx context.Context, sufficient hlc.Timestamp, -) (_ hlc.Timestamp, ok bool, _ replicaUpdate) { +) (_ hlc.Timestamp, ok bool) { appliedLAI := ctpb.LAI(r.mu.state.LeaseAppliedIndex) lease := r.mu.state.Lease initialMaxClosed := r.mu.initialMaxClosed - replicaStateClosed := r.mu.state.RaftClosedTimestamp - // Consider the timestamp closed through the side-transport. Such a timestamp - // can be in two places: - // - r.mu.sideTransportClosedTimestamp - // - in the sidetransport.Receiver - // We check the former here. We check the latter further down, only if we have - // to. - var sideTransportClosed hlc.Timestamp - sideTransportClosedMaybe, minLAI := r.getSideTransportClosedTimestampRLocked() - // We can use sideTransportClosedMaybe if we've applied at least up to minLAI. - // The replica could in theory maintain more information about what lower - // timestamps the side transport had closed with lower LAIs, but we don't - // bother. - replicationBehind := appliedLAI < minLAI - if !replicationBehind { - sideTransportClosed = sideTransportClosedMaybe - } + raftClosed := r.mu.state.RaftClosedTimestamp + sideTransportClosed := r.sideTransportClosedTimestamp.get(ctx, lease.Replica.NodeID, appliedLAI, sufficient) // TODO(andrei): In 21.1 we added support for closed timestamps on ranges with // expiration-based leases. Once the old closed timestamp transport is gone in // 21.2, this can go away. if lease.Expiration != nil { - return hlc.Timestamp{}, false, replicaUpdate{} + return hlc.Timestamp{}, false } // Look at the legacy closed timestamp propagation mechanism. maxClosed := r.store.cfg.ClosedTimestamp.Provider.MaxClosed( @@ -204,45 +171,15 @@ func (r *Replica) maxClosedRLocked( // timestamp. Otherwise, ignore it. We expect to delete this code soon, but // we keep it around for now to avoid a regression in follower read // availability in mixed v20.2/v21.1 clusters. - if replicaStateClosed.IsEmpty() { + if raftClosed.IsEmpty() { maxClosed.Forward(lease.Start.ToTimestamp()) } // Look at the "new" closed timestamp propagation mechanism. - maxClosed.Forward(replicaStateClosed) + maxClosed.Forward(raftClosed) maxClosed.Forward(sideTransportClosed) - // If the closed timestamp we know so far is sufficient, we return early - // without consulting the ClosedTimestampReceiver. - if !sufficient.IsEmpty() && sufficient.LessEq(maxClosed) { - return maxClosed, true, replicaUpdate{} - } - - // We now look at sidetransport.Receiver, unless replicationBehind was set; - // the LAIs in the Receiver are >= the one returned by - // getSideTransportClosedTimestampRLocked(), so there's no point in even - // checking. - var update replicaUpdate - // In some tests the lease can be empty, or the ClosedTimestampReceiver might - // not be set. - if !replicationBehind && !lease.Empty() && r.store.cfg.ClosedTimestampReceiver != nil { - otherSideTransportClosed, otherSideTransportLAI := - r.store.cfg.ClosedTimestampReceiver.GetClosedTimestamp(ctx, r.RangeID, lease.Replica.NodeID) - if appliedLAI < otherSideTransportLAI { - otherSideTransportClosed = hlc.Timestamp{} - } - // If otherSideTransportClosed ends up winning, we return it in update so - // that the caller copies it into the Replica. Hopefully, future calls with - // `sufficient` set don't need to go to the Receiver for a while. - if maxClosed.Forward(otherSideTransportClosed) { - update = replicaUpdate{ - sideTransportClosedTimestamp: otherSideTransportClosed, - sideTransportClosedLAI: otherSideTransportLAI, - } - } - } - - return maxClosed, true, update + return maxClosed, true } // ClosedTimestampV2 returns the closed timestamp. Unlike MaxClosedTimestamp, it @@ -257,26 +194,8 @@ func (r *Replica) maxClosedRLocked( // timestamp mechanism is deleted. At that point, the two should be equivalent. func (r *Replica) ClosedTimestampV2(ctx context.Context) hlc.Timestamp { r.mu.RLock() - defer r.mu.RUnlock() - appliedLAI := ctpb.LAI(r.mu.state.LeaseAppliedIndex) - - closed := r.mu.state.RaftClosedTimestamp - sideTransportClosedMaybe, minLAI := r.getSideTransportClosedTimestampRLocked() - replicationBehind := appliedLAI < minLAI - if !replicationBehind { - closed.Forward(sideTransportClosedMaybe) - } - - // Tests might not be configured with a receiver. - if receiver := r.store.cfg.ClosedTimestampReceiver; receiver != nil { - otherSideTransportClosed, otherSideTransportLAI := - r.store.cfg.ClosedTimestampReceiver.GetClosedTimestamp(ctx, r.RangeID, r.mu.state.Lease.Replica.NodeID) - replicationBehind = appliedLAI < otherSideTransportLAI - if !replicationBehind { - closed.Forward(otherSideTransportClosed) - } - } - - return closed + leaseholder := r.mu.state.Lease.Replica.NodeID + r.mu.RUnlock() + return r.sideTransportClosedTimestamp.get(ctx, leaseholder, appliedLAI, hlc.Timestamp{} /* sufficient */) } diff --git a/pkg/kv/kvserver/replica_init.go b/pkg/kv/kvserver/replica_init.go index c64027597c26..7e010130e18e 100644 --- a/pkg/kv/kvserver/replica_init.go +++ b/pkg/kv/kvserver/replica_init.go @@ -216,6 +216,9 @@ func (r *Replica) loadRaftMuLockedReplicaMuLocked(desc *roachpb.RangeDescriptor) return errors.Wrap(err, "while initializing sideloaded storage") } r.assertStateRaftMuLockedReplicaMuRLocked(ctx, r.store.Engine()) + + r.sideTransportClosedTimestamp.init(r.store.cfg.ClosedTimestampReceiver, desc.RangeID) + return nil } diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 8e05b7f547d3..863c70e8f977 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -466,14 +466,9 @@ func (r *Replica) handleInvalidLeaseError( // and leases and address the TODO in checkExecutionCanProceed to check the // closed timestamp before consulting the lease. - var update replicaUpdate - defer update.apply(ctx, r) - r.mu.RLock() defer r.mu.RUnlock() - var ok bool - ok, update = r.canServeFollowerReadRLocked(ctx, ba, pErr.GoError()) - if ok { + if r.canServeFollowerReadRLocked(ctx, ba, pErr.GoError()) { // Follower read possible. Retry command. return nil } diff --git a/pkg/testutils/localtestcluster/BUILD.bazel b/pkg/testutils/localtestcluster/BUILD.bazel index c54475c63c10..bb3b681d6d55 100644 --- a/pkg/testutils/localtestcluster/BUILD.bazel +++ b/pkg/testutils/localtestcluster/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver", + "//pkg/kv/kvserver/closedts/sidetransport", "//pkg/kv/kvserver/liveness", "//pkg/roachpb", "//pkg/rpc", diff --git a/pkg/testutils/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index 103d7ea7369f..a7a3823b5669 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -184,6 +185,7 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto /* deterministic */ false, ) cfg.Transport = transport + cfg.ClosedTimestampReceiver = sidetransport.NewReceiver(nc, ltc.stopper, ltc.Stores, nil /* testingKnobs */) ctx := context.TODO() if err := kvserver.WriteClusterVersion(ctx, ltc.Eng, clusterversion.TestingClusterVersion); err != nil { From b5849f3a7a66d02f37f3a2f4a77895908bf17ebd Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich <yahor@cockroachlabs.com> Date: Wed, 14 Apr 2021 17:17:43 -0700 Subject: [PATCH 14/37] flowinfra: remove remnants of RunSyncFlow DistSQL RPC We recently removed `RunSyncFlow` RPC call of DistSQL, but a couple of things were missed. This commit cleans up the outbox based on that removal, removes `SetupSyncFlow` method as well as adds some comments. Release note: None --- pkg/sql/distsql/BUILD.bazel | 3 +- pkg/sql/distsql/server.go | 29 +-- ...test.go => setup_flow_after_drain_test.go} | 30 +-- pkg/sql/execinfrapb/api.pb.go | 190 ++++++------------ pkg/sql/execinfrapb/api.proto | 6 +- pkg/sql/flowinfra/outbox.go | 32 +-- pkg/sql/physicalplan/aggregator_funcs_test.go | 5 +- 7 files changed, 96 insertions(+), 199 deletions(-) rename pkg/sql/distsql/{sync_flow_after_drain_test.go => setup_flow_after_drain_test.go} (73%) diff --git a/pkg/sql/distsql/BUILD.bazel b/pkg/sql/distsql/BUILD.bazel index a7eddc3eb452..a6e8aba27eae 100644 --- a/pkg/sql/distsql/BUILD.bazel +++ b/pkg/sql/distsql/BUILD.bazel @@ -42,7 +42,7 @@ go_test( "columnar_utils_test.go", "inbound_test.go", "main_test.go", - "sync_flow_after_drain_test.go", + "setup_flow_after_drain_test.go", "vectorized_panic_propagation_test.go", ], embed = [":distsql"], @@ -83,6 +83,7 @@ go_test( "//pkg/util", "//pkg/util/hlc", "//pkg/util/leaktest", + "//pkg/util/log", "//pkg/util/netutil", "//pkg/util/randutil", "//pkg/util/stop", diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index 0f7d9f246aa8..70d80aafc925 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -439,26 +439,6 @@ func newFlow( return rowflow.NewRowBasedFlow(base) } -// SetupSyncFlow sets up a synchronous flow, connecting the sync response -// output stream to the given RowReceiver. The flow is not started. The flow -// will be associated with the given context. -// Note: the returned context contains a span that must be finished through -// Flow.Cleanup. -func (ds *ServerImpl) SetupSyncFlow( - ctx context.Context, - parentMonitor *mon.BytesMonitor, - req *execinfrapb.SetupFlowRequest, - output execinfra.RowReceiver, -) (context.Context, flowinfra.Flow, error) { - ctx, f, err := ds.setupFlow( - ds.AnnotateCtx(ctx), tracing.SpanFromContext(ctx), parentMonitor, req, output, LocalState{}, - ) - if err != nil { - return nil, nil, err - } - return ctx, f, err -} - // LocalState carries information that is required to set up a flow with wrapped // planNodes. type LocalState struct { @@ -483,9 +463,12 @@ type LocalState struct { LocalProcs []execinfra.LocalProcessor } -// SetupLocalSyncFlow sets up a synchronous flow on the current (planning) node. -// It's used by the gateway node to set up the flows local to it. -// It's the same as SetupSyncFlow except it takes the localState. +// SetupLocalSyncFlow sets up a synchronous flow on the current (planning) node, +// connecting the sync response output stream to the given RowReceiver. It's +// used by the gateway node to set up the flows local to it. The flow is not +// started. The flow will be associated with the given context. +// Note: the returned context contains a span that must be finished through +// Flow.Cleanup. func (ds *ServerImpl) SetupLocalSyncFlow( ctx context.Context, parentMonitor *mon.BytesMonitor, diff --git a/pkg/sql/distsql/sync_flow_after_drain_test.go b/pkg/sql/distsql/setup_flow_after_drain_test.go similarity index 73% rename from pkg/sql/distsql/sync_flow_after_drain_test.go rename to pkg/sql/distsql/setup_flow_after_drain_test.go index 1b19be321cf1..0faef3ddc843 100644 --- a/pkg/sql/distsql/sync_flow_after_drain_test.go +++ b/pkg/sql/distsql/setup_flow_after_drain_test.go @@ -18,18 +18,17 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/testutils/distsqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" ) -// Test that we can register send a sync flow to the distSQLSrv after the -// FlowRegistry is draining and the we can also clean that flow up (the flow -// will get a draining error). This used to crash. -func TestSyncFlowAfterDrain(t *testing.T) { +// Test that we can send a setup flow request to the distSQLSrv after the +// FlowRegistry is draining. +func TestSetupFlowAfterDrain(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) ctx := context.Background() // We'll create a server just so that we can extract its distsql ServerConfig, @@ -68,22 +67,13 @@ func TestSyncFlowAfterDrain(t *testing.T) { }, } - types := make([]*types.T, 0) - rb := distsqlutils.NewRowBuffer(types, nil /* rows */, distsqlutils.RowBufferArgs{}) - ctx, flow, err := distSQLSrv.SetupSyncFlow(ctx, distSQLSrv.memMonitor, &req, rb) + // We expect to see an error in the response. + resp, err := distSQLSrv.SetupFlow(ctx, &req) if err != nil { t.Fatal(err) } - if err := flow.Start(ctx, func() {}); err != nil { - t.Fatal(err) - } - flow.Wait() - _, meta := rb.Next() - if meta == nil { - t.Fatal("expected draining err, got no meta") - } - if !testutils.IsError(meta.Err, "the registry is draining") { - t.Fatalf("expected draining err, got: %v", meta.Err) + respErr := resp.Error.ErrorDetail(ctx) + if !testutils.IsError(respErr, "the registry is draining") { + t.Fatalf("expected draining err, got: %v", respErr) } - flow.Cleanup(ctx) } diff --git a/pkg/sql/execinfrapb/api.pb.go b/pkg/sql/execinfrapb/api.pb.go index 394b40b51da5..024aefb732d9 100644 --- a/pkg/sql/execinfrapb/api.pb.go +++ b/pkg/sql/execinfrapb/api.pb.go @@ -201,9 +201,6 @@ type ConsumerSignal struct { // the producer to push whatever trailing metadata it has and close its // stream. DrainRequest *DrainRequest `protobuf:"bytes,1,opt,name=drain_request,json=drainRequest" json:"drain_request,omitempty"` - // Used in the RunSyncFlow case; the first message on the client stream must - // contain this message. - SetupFlowRequest *SetupFlowRequest `protobuf:"bytes,2,opt,name=setup_flow_request,json=setupFlowRequest" json:"setup_flow_request,omitempty"` // Consumer->Producer handshake messages. See message definition. Handshake *ConsumerHandshake `protobuf:"bytes,3,opt,name=handshake" json:"handshake,omitempty"` } @@ -333,67 +330,65 @@ func init() { func init() { proto.RegisterFile("sql/execinfrapb/api.proto", fileDescriptor_0204a3f70d678e66) } var fileDescriptor_0204a3f70d678e66 = []byte{ - // 947 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xc1, 0x8e, 0xe3, 0x44, - 0x13, 0x8e, 0x93, 0x4e, 0xd2, 0xe9, 0x64, 0xb2, 0x9e, 0xd6, 0xe8, 0x97, 0xff, 0x1c, 0x92, 0xc8, - 0x5a, 0x20, 0x80, 0x70, 0x96, 0x2c, 0x5c, 0x80, 0x0b, 0x99, 0x00, 0x1b, 0xef, 0xce, 0x8a, 0x75, - 0x66, 0x57, 0x88, 0x03, 0x56, 0xc7, 0xee, 0x38, 0xd6, 0x38, 0x6e, 0x8f, 0xbb, 0xbd, 0x13, 0xde, - 0x80, 0xe3, 0x8a, 0x27, 0xe0, 0xc0, 0xc3, 0xcc, 0x71, 0x6f, 0xac, 0x38, 0x0c, 0x90, 0x79, 0x0b, - 0x0e, 0x08, 0xd9, 0xb1, 0x27, 0x9e, 0x0c, 0x89, 0xe0, 0xe6, 0xae, 0xfa, 0xbe, 0xaf, 0xab, 0xaa, - 0xab, 0xca, 0xe8, 0xff, 0xfc, 0xdc, 0xeb, 0xd3, 0x25, 0xb5, 0x5c, 0x7f, 0x16, 0x92, 0x60, 0xda, - 0x27, 0x81, 0xab, 0x05, 0x21, 0x13, 0x0c, 0x2b, 0x16, 0xb3, 0xce, 0x42, 0x46, 0xac, 0xb9, 0xc6, - 0xcf, 0x3d, 0xcd, 0x76, 0xb9, 0xe0, 0xe7, 0x5e, 0x18, 0xf9, 0xad, 0x23, 0x87, 0x39, 0x2c, 0x01, - 0xf5, 0xe3, 0xaf, 0x35, 0xbe, 0xd5, 0x71, 0x18, 0x73, 0x3c, 0xda, 0x4f, 0x4e, 0xd3, 0x68, 0xd6, - 0x17, 0xee, 0x82, 0x72, 0x41, 0x16, 0x41, 0x0a, 0xc0, 0x89, 0x58, 0x30, 0xed, 0xdb, 0x44, 0x90, - 0xd4, 0xd6, 0xda, 0xbe, 0x3f, 0xe7, 0xeb, 0x6e, 0xfb, 0x82, 0x90, 0x59, 0x94, 0x73, 0x16, 0xf2, - 0x14, 0x71, 0x3f, 0x46, 0x70, 0xca, 0xb9, 0xcb, 0xfc, 0x98, 0x18, 0x4c, 0xb3, 0x93, 0xb9, 0xd1, - 0x51, 0xff, 0x2a, 0x22, 0x79, 0x42, 0x45, 0x14, 0x7c, 0xe9, 0xb1, 0x0b, 0x83, 0x9e, 0x47, 0x94, - 0x0b, 0xfc, 0x1c, 0x1d, 0x79, 0x94, 0xcc, 0x4c, 0xb1, 0xf4, 0x4d, 0xd7, 0x0f, 0x22, 0x61, 0x72, - 0x41, 0x04, 0x55, 0xaa, 0x5d, 0xa9, 0x57, 0x1f, 0xdc, 0xd7, 0x36, 0xc9, 0xa7, 0x51, 0x6b, 0x4f, - 0x28, 0x99, 0x9d, 0x2e, 0xfd, 0x71, 0x0c, 0x9e, 0xc4, 0x58, 0xe3, 0xd0, 0xdb, 0x36, 0xe1, 0x07, - 0xa8, 0xfa, 0x92, 0x86, 0x71, 0x04, 0x4a, 0xb9, 0x2b, 0xf5, 0x0e, 0x86, 0xff, 0xbb, 0xbc, 0xea, - 0x14, 0xfe, 0xbc, 0xea, 0x34, 0x47, 0x2e, 0x17, 0x93, 0x67, 0x4f, 0x5e, 0xac, 0xbd, 0x46, 0x06, - 0xc3, 0x9f, 0x21, 0x30, 0xf3, 0xd8, 0x85, 0x52, 0x4a, 0x2e, 0x56, 0xb5, 0x5d, 0x55, 0xd7, 0xe2, - 0xe8, 0x27, 0x01, 0xb5, 0x86, 0x20, 0x96, 0x34, 0x12, 0x16, 0x3e, 0x41, 0x75, 0xfa, 0x92, 0x78, - 0xc7, 0xcc, 0x17, 0x74, 0x29, 0x94, 0x4a, 0x22, 0xf2, 0xd6, 0x6e, 0x91, 0x2f, 0x36, 0xe0, 0x54, - 0x27, 0xcf, 0xc7, 0x6d, 0x54, 0x3d, 0x0d, 0x89, 0x45, 0x1f, 0xbf, 0x50, 0x60, 0x57, 0xea, 0xc1, - 0x14, 0x93, 0x19, 0x71, 0x0f, 0x35, 0x8e, 0x99, 0xe7, 0x51, 0x2b, 0x49, 0x97, 0x2b, 0xb5, 0x1c, - 0xe8, 0x96, 0x47, 0x07, 0x50, 0x92, 0x8b, 0x3a, 0x80, 0x45, 0xb9, 0xa4, 0x5e, 0x4b, 0x08, 0x66, - 0xd1, 0xe3, 0x0f, 0x51, 0x35, 0x8e, 0xdc, 0x74, 0x6d, 0x45, 0xea, 0x4a, 0xbd, 0xc6, 0x50, 0x89, - 0xd9, 0xbf, 0x5e, 0x75, 0x2a, 0x31, 0x64, 0x3c, 0x5a, 0xdd, 0x7c, 0x19, 0x95, 0x18, 0x38, 0xb6, - 0xf1, 0x73, 0x54, 0x75, 0x88, 0xa0, 0x17, 0xe4, 0xfb, 0xa4, 0x4a, 0xe5, 0xe1, 0xa7, 0x69, 0x51, - 0x1f, 0x3a, 0xae, 0x98, 0x47, 0x53, 0xcd, 0x62, 0x8b, 0xfe, 0x4d, 0xca, 0xf6, 0x74, 0xf3, 0xdd, - 0x0f, 0xce, 0x9c, 0x7e, 0xf6, 0x80, 0x4f, 0x99, 0x4d, 0xc7, 0x23, 0x23, 0xd3, 0xc2, 0x27, 0x08, - 0x6d, 0x3a, 0x4a, 0x29, 0x76, 0x4b, 0xbd, 0xfa, 0xe0, 0x9d, 0xdd, 0xa5, 0xfb, 0x3a, 0xc3, 0xe6, - 0x1e, 0x21, 0x27, 0xa0, 0xfe, 0x58, 0x44, 0xf5, 0x5c, 0x79, 0xf1, 0x47, 0x08, 0x73, 0xb1, 0x10, - 0xa7, 0xd9, 0x14, 0x3c, 0x25, 0x3e, 0xe3, 0x49, 0xce, 0xa5, 0x94, 0xfd, 0x0f, 0x7e, 0x3c, 0x40, - 0x87, 0x62, 0xe9, 0x6f, 0x91, 0x8a, 0x39, 0xd2, 0x5d, 0x37, 0x7e, 0x86, 0x1a, 0xf9, 0xb6, 0x57, - 0xee, 0x25, 0x5d, 0xd0, 0xdb, 0x4a, 0xe5, 0xd6, 0x9c, 0x68, 0x93, 0xf5, 0x69, 0x44, 0x04, 0xc9, - 0x1a, 0x81, 0x6f, 0x4c, 0x3a, 0x80, 0x25, 0x19, 0xe8, 0x00, 0x02, 0xb9, 0xac, 0x03, 0x58, 0x96, - 0x2b, 0x3a, 0x80, 0x15, 0xb9, 0xaa, 0x03, 0x58, 0x95, 0xa1, 0x0e, 0x20, 0x94, 0x6b, 0x3a, 0x80, - 0x35, 0x19, 0xe9, 0x00, 0x22, 0xb9, 0xae, 0x03, 0x58, 0x97, 0x1b, 0x3a, 0x80, 0x0d, 0xf9, 0x40, - 0x07, 0xf0, 0x40, 0x6e, 0xea, 0x00, 0x36, 0xe5, 0x7b, 0xea, 0x57, 0xa8, 0x39, 0x71, 0x17, 0x81, - 0x47, 0x0d, 0xca, 0x03, 0xe6, 0x73, 0x8a, 0x3f, 0x46, 0x65, 0x1a, 0x86, 0x2c, 0x4c, 0x2a, 0x51, - 0x1f, 0x74, 0xf6, 0xf4, 0x6a, 0x0c, 0x33, 0xd6, 0x68, 0xf5, 0x87, 0x22, 0x6a, 0x1e, 0x33, 0x9f, - 0x47, 0x0b, 0x1a, 0x4e, 0x5c, 0xc7, 0x27, 0x1e, 0x7e, 0x8c, 0x0e, 0xec, 0x90, 0xb8, 0xbe, 0x19, - 0xae, 0x67, 0x3a, 0x55, 0x7c, 0x7b, 0xb7, 0xe2, 0x28, 0x86, 0xa7, 0x1b, 0xc0, 0x68, 0xd8, 0xb9, - 0x13, 0xfe, 0x06, 0x61, 0x1e, 0xef, 0x08, 0x33, 0x69, 0xce, 0x4c, 0xb1, 0x98, 0x28, 0xbe, 0xb7, - 0x5b, 0x71, 0x7b, 0xaf, 0x18, 0x32, 0xdf, 0xde, 0x34, 0x63, 0x54, 0x9b, 0x13, 0xdf, 0xe6, 0x73, - 0x72, 0x46, 0xd3, 0x29, 0x7f, 0x7f, 0xb7, 0x60, 0x96, 0xe3, 0xa3, 0x8c, 0x62, 0x6c, 0xd8, 0x9f, - 0x80, 0xcb, 0x9f, 0x3a, 0x92, 0xda, 0x44, 0x8d, 0x7c, 0x22, 0xea, 0xcf, 0x45, 0x74, 0x78, 0x87, - 0x86, 0x1f, 0x22, 0x6c, 0xa5, 0x46, 0x93, 0x5b, 0x73, 0x6a, 0x47, 0x1e, 0x5d, 0x8f, 0x5c, 0x36, - 0xb0, 0x87, 0x99, 0x7f, 0x92, 0xb9, 0xf1, 0x77, 0xa8, 0x75, 0x87, 0x64, 0xda, 0x94, 0xd8, 0x9e, - 0xeb, 0xd3, 0xb4, 0x1a, 0x2d, 0x6d, 0xbd, 0xe8, 0xb5, 0x6c, 0xd1, 0x6b, 0x37, 0xed, 0x38, 0x04, - 0xaf, 0x7e, 0xeb, 0x48, 0x86, 0xb2, 0x2d, 0x3c, 0x4a, 0x15, 0xf2, 0xeb, 0xb1, 0xf4, 0xef, 0xd6, - 0xe3, 0x23, 0x74, 0xb4, 0x70, 0x7d, 0x93, 0x58, 0x16, 0x0d, 0x04, 0xb5, 0xcd, 0x8c, 0x0e, 0xf6, - 0xd2, 0xf1, 0xc2, 0xf5, 0x3f, 0x4f, 0x29, 0xa9, 0x6d, 0xf0, 0x8b, 0x84, 0xaa, 0x29, 0x0c, 0x5b, - 0xa8, 0x76, 0xf3, 0x72, 0xf8, 0x3f, 0x3c, 0x6f, 0xab, 0xb7, 0x07, 0x7b, 0xab, 0xcf, 0xd5, 0x02, - 0x76, 0x10, 0x4a, 0xb6, 0x9e, 0x08, 0x29, 0x59, 0xe0, 0x77, 0xf7, 0x6e, 0x16, 0x3b, 0xb2, 0x68, - 0x78, 0x42, 0x39, 0x27, 0x0e, 0xdd, 0x77, 0xc9, 0xed, 0x11, 0x50, 0x0b, 0x3d, 0xe9, 0x81, 0x34, - 0xfc, 0xe0, 0xf2, 0x8f, 0x76, 0xe1, 0x72, 0xd5, 0x96, 0x5e, 0xaf, 0xda, 0xd2, 0x9b, 0x55, 0x5b, - 0xfa, 0x7d, 0xd5, 0x96, 0x5e, 0x5d, 0xb7, 0x0b, 0xaf, 0xaf, 0xdb, 0x85, 0x37, 0xd7, 0xed, 0xc2, - 0xb7, 0xf5, 0xdc, 0x6f, 0xf4, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0xa0, 0x01, 0x02, 0x07, 0xf4, - 0x07, 0x00, 0x00, + // 928 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0x41, 0x93, 0xe2, 0x44, + 0x14, 0x26, 0xd0, 0x40, 0xd3, 0x30, 0x6c, 0xa6, 0x6b, 0xca, 0x8a, 0x1c, 0x80, 0xa2, 0x56, 0x45, + 0x2d, 0xc3, 0xca, 0xea, 0x45, 0xbd, 0xc8, 0xa0, 0x2e, 0xd9, 0x9d, 0x2d, 0x37, 0xcc, 0xee, 0xc1, + 0x83, 0xa9, 0x26, 0x69, 0x42, 0x6a, 0x92, 0x74, 0x26, 0xdd, 0xd9, 0xc1, 0x7f, 0xb1, 0xe5, 0x2f, + 0xf0, 0xe0, 0x4f, 0xf0, 0x47, 0xcc, 0x71, 0x6f, 0x6e, 0x79, 0x18, 0x95, 0xf9, 0x17, 0x1e, 0x2c, + 0x2b, 0x21, 0x19, 0x32, 0x8c, 0x50, 0x7b, 0x4b, 0xbf, 0xf7, 0x7d, 0x5f, 0xbf, 0xf7, 0xfa, 0xbd, + 0x17, 0xf4, 0x2e, 0x3f, 0x77, 0x07, 0x74, 0x49, 0x4d, 0xc7, 0x9f, 0x87, 0x24, 0x98, 0x0d, 0x48, + 0xe0, 0xa8, 0x41, 0xc8, 0x04, 0xc3, 0x8a, 0xc9, 0xcc, 0xb3, 0x90, 0x11, 0x73, 0xa1, 0xf2, 0x73, + 0x57, 0xb5, 0x1c, 0x2e, 0xf8, 0xb9, 0x1b, 0x46, 0x7e, 0xeb, 0xc8, 0x66, 0x36, 0x4b, 0x40, 0x83, + 0xf8, 0x6b, 0x8d, 0x6f, 0x75, 0x6c, 0xc6, 0x6c, 0x97, 0x0e, 0x92, 0xd3, 0x2c, 0x9a, 0x0f, 0x84, + 0xe3, 0x51, 0x2e, 0x88, 0x17, 0xa4, 0x00, 0x9c, 0x88, 0x05, 0xb3, 0x81, 0x45, 0x04, 0x49, 0x6d, + 0xad, 0xed, 0xfb, 0x73, 0xbe, 0xee, 0xb6, 0x2f, 0x08, 0x99, 0x49, 0x39, 0x67, 0x21, 0x4f, 0x11, + 0xf7, 0x63, 0x04, 0xa7, 0x9c, 0x3b, 0xcc, 0x8f, 0x89, 0xc1, 0x2c, 0x3b, 0x19, 0x1b, 0x9d, 0xde, + 0xbf, 0x45, 0x24, 0x4f, 0xa9, 0x88, 0x82, 0x6f, 0x5d, 0x76, 0xa1, 0xd3, 0xf3, 0x88, 0x72, 0x81, + 0x9f, 0xa3, 0x23, 0x97, 0x92, 0xb9, 0x21, 0x96, 0xbe, 0xe1, 0xf8, 0x41, 0x24, 0x0c, 0x2e, 0x88, + 0xa0, 0x4a, 0xb5, 0x2b, 0xf5, 0xeb, 0xc3, 0xfb, 0xea, 0x26, 0xf9, 0x34, 0x6a, 0xf5, 0x09, 0x25, + 0xf3, 0xd3, 0xa5, 0x3f, 0x89, 0xc1, 0xd3, 0x18, 0xab, 0x1f, 0xba, 0xdb, 0x26, 0xfc, 0x00, 0x55, + 0x5f, 0xd2, 0x30, 0x8e, 0x40, 0x29, 0x77, 0xa5, 0xfe, 0xc1, 0xe8, 0x9d, 0xcb, 0xab, 0x4e, 0xe1, + 0x9f, 0xab, 0x4e, 0x73, 0xec, 0x70, 0x31, 0x7d, 0xf6, 0xe4, 0xc5, 0xda, 0xab, 0x67, 0x30, 0xfc, + 0x15, 0x02, 0x73, 0x97, 0x5d, 0x28, 0xa5, 0xe4, 0xe2, 0x9e, 0xba, 0xab, 0xea, 0x6a, 0x1c, 0xfd, + 0x34, 0xa0, 0xe6, 0x08, 0xc4, 0x92, 0x7a, 0xc2, 0xc2, 0x27, 0xa8, 0x4e, 0x5f, 0x12, 0xf7, 0x98, + 0xf9, 0x82, 0x2e, 0x85, 0x52, 0x49, 0x44, 0xde, 0xdb, 0x2d, 0xf2, 0xcd, 0x06, 0x9c, 0xea, 0xe4, + 0xf9, 0xb8, 0x8d, 0xaa, 0xa7, 0x21, 0x31, 0xe9, 0xe3, 0x17, 0x0a, 0xec, 0x4a, 0x7d, 0x98, 0x62, + 0x32, 0x23, 0xee, 0xa3, 0xc6, 0x31, 0x73, 0x5d, 0x6a, 0x26, 0xe9, 0x72, 0xa5, 0x96, 0x03, 0xdd, + 0xf2, 0x68, 0x00, 0x4a, 0x72, 0x51, 0x03, 0xb0, 0x28, 0x97, 0x7a, 0xd7, 0x12, 0x82, 0x59, 0xf4, + 0xf8, 0x53, 0x54, 0x8d, 0x23, 0x37, 0x1c, 0x4b, 0x91, 0xba, 0x52, 0xbf, 0x31, 0x52, 0x62, 0xf6, + 0x1f, 0x57, 0x9d, 0x4a, 0x0c, 0x99, 0x8c, 0x57, 0x37, 0x5f, 0x7a, 0x25, 0x06, 0x4e, 0x2c, 0xfc, + 0x1c, 0x55, 0x6d, 0x22, 0xe8, 0x05, 0xf9, 0x29, 0xa9, 0x52, 0x79, 0xf4, 0x65, 0x5a, 0xd4, 0x87, + 0xb6, 0x23, 0x16, 0xd1, 0x4c, 0x35, 0x99, 0x37, 0xb8, 0x49, 0xd9, 0x9a, 0x6d, 0xbe, 0x07, 0xc1, + 0x99, 0x3d, 0xc8, 0x1e, 0xf0, 0x29, 0xb3, 0xe8, 0x64, 0xac, 0x67, 0x5a, 0xf8, 0x04, 0xa1, 0x4d, + 0x47, 0x29, 0xc5, 0x6e, 0xa9, 0x5f, 0x1f, 0x7e, 0xb0, 0xbb, 0x74, 0xdf, 0x67, 0xd8, 0xdc, 0x23, + 0xe4, 0x04, 0x7a, 0x3f, 0x17, 0x51, 0x3d, 0x57, 0x5e, 0xfc, 0x19, 0xc2, 0x5c, 0x78, 0xe2, 0x34, + 0x9b, 0x82, 0xa7, 0xc4, 0x67, 0x3c, 0xc9, 0xb9, 0x94, 0xb2, 0xff, 0xc7, 0x8f, 0x87, 0xe8, 0x50, + 0x2c, 0xfd, 0x2d, 0x52, 0x31, 0x47, 0xba, 0xeb, 0xc6, 0xcf, 0x50, 0x23, 0xdf, 0xf6, 0xca, 0xbd, + 0xa4, 0x0b, 0xfa, 0x5b, 0xa9, 0xdc, 0x9a, 0x13, 0x75, 0xba, 0x3e, 0x8d, 0x89, 0x20, 0x59, 0x23, + 0xf0, 0x8d, 0x49, 0x03, 0xb0, 0x24, 0x03, 0x0d, 0x40, 0x20, 0x97, 0x35, 0x00, 0xcb, 0x72, 0x45, + 0x03, 0xb0, 0x22, 0x57, 0x35, 0x00, 0xab, 0x32, 0xd4, 0x00, 0x84, 0x72, 0x4d, 0x03, 0xb0, 0x26, + 0x23, 0x0d, 0x40, 0x24, 0xd7, 0x35, 0x00, 0xeb, 0x72, 0x43, 0x03, 0xb0, 0x21, 0x1f, 0x68, 0x00, + 0x1e, 0xc8, 0x4d, 0x0d, 0xc0, 0xa6, 0x7c, 0xaf, 0xf7, 0x1d, 0x6a, 0x4e, 0x1d, 0x2f, 0x70, 0xa9, + 0x4e, 0x79, 0xc0, 0x7c, 0x4e, 0xf1, 0xe7, 0xa8, 0x4c, 0xc3, 0x90, 0x85, 0x49, 0x25, 0xea, 0xc3, + 0xce, 0x9e, 0x5e, 0x8d, 0x61, 0xfa, 0x1a, 0xdd, 0xfb, 0x4d, 0x42, 0xcd, 0x63, 0xe6, 0xf3, 0xc8, + 0xa3, 0xe1, 0xd4, 0xb1, 0x7d, 0xe2, 0xe2, 0xc7, 0xe8, 0xc0, 0x0a, 0x89, 0xe3, 0x1b, 0xe1, 0x7a, + 0xa6, 0x53, 0xc5, 0xf7, 0x77, 0x2b, 0x8e, 0x63, 0x78, 0xba, 0x01, 0xf4, 0x86, 0x95, 0x3b, 0xe1, + 0x09, 0xaa, 0x2d, 0x88, 0x6f, 0xf1, 0x05, 0x39, 0xa3, 0xe9, 0x2c, 0x7e, 0xbc, 0x5b, 0x28, 0x8b, + 0xe4, 0x51, 0x46, 0xd1, 0x37, 0xec, 0x2f, 0xc0, 0xe5, 0x2f, 0x1d, 0x29, 0x6d, 0xfd, 0x26, 0x6a, + 0xe4, 0x2f, 0xed, 0xfd, 0x5a, 0x44, 0x87, 0x77, 0xc8, 0xf8, 0x21, 0xc2, 0x66, 0x6a, 0x34, 0xb8, + 0xb9, 0xa0, 0x56, 0xe4, 0xd2, 0xf5, 0x78, 0x64, 0xc3, 0x75, 0x98, 0xf9, 0xa7, 0x99, 0x1b, 0xff, + 0x88, 0x5a, 0x77, 0x48, 0x86, 0x45, 0x89, 0xe5, 0x3a, 0x3e, 0x4d, 0x5a, 0xa6, 0x3e, 0x6c, 0xa9, + 0xeb, 0xa5, 0xac, 0x66, 0x4b, 0x59, 0xbd, 0x69, 0x9d, 0x11, 0x78, 0xf5, 0x67, 0x47, 0xd2, 0x95, + 0x6d, 0xe1, 0x71, 0xaa, 0x90, 0x5f, 0x65, 0xa5, 0xb7, 0x5b, 0x65, 0x8f, 0xd0, 0x91, 0xe7, 0xf8, + 0x06, 0x31, 0x4d, 0x1a, 0x08, 0x6a, 0x19, 0x19, 0x1d, 0xec, 0xa5, 0x63, 0xcf, 0xf1, 0xbf, 0x4e, + 0x29, 0xa9, 0x6d, 0xf8, 0xbb, 0x84, 0xaa, 0x29, 0x0c, 0x9b, 0xa8, 0x76, 0xb3, 0xbd, 0xf1, 0x47, + 0xbb, 0xdf, 0x64, 0x7b, 0xc5, 0xb7, 0xfa, 0x7b, 0xb0, 0xb7, 0x7a, 0xb2, 0x57, 0xc0, 0x36, 0x42, + 0xc9, 0x86, 0x12, 0x21, 0x25, 0x1e, 0xfe, 0x70, 0xef, 0x16, 0xb0, 0x22, 0x93, 0x86, 0x27, 0x94, + 0x73, 0x62, 0xd3, 0x7d, 0x97, 0xdc, 0x6e, 0xd7, 0x5e, 0xa1, 0x2f, 0x3d, 0x90, 0x46, 0x9f, 0x5c, + 0xfe, 0xdd, 0x2e, 0x5c, 0xae, 0xda, 0xd2, 0xeb, 0x55, 0x5b, 0x7a, 0xb3, 0x6a, 0x4b, 0x7f, 0xad, + 0xda, 0xd2, 0xab, 0xeb, 0x76, 0xe1, 0xf5, 0x75, 0xbb, 0xf0, 0xe6, 0xba, 0x5d, 0xf8, 0xa1, 0x9e, + 0xfb, 0xe5, 0xfd, 0x17, 0x00, 0x00, 0xff, 0xff, 0x35, 0x00, 0x11, 0x7b, 0xa0, 0x07, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -799,18 +794,6 @@ func (m *ConsumerSignal) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x1a } - if m.SetupFlowRequest != nil { - { - size, err := m.SetupFlowRequest.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintApi(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } if m.DrainRequest != nil { { size, err := m.DrainRequest.MarshalToSizedBuffer(dAtA[:i]) @@ -876,12 +859,12 @@ func (m *ConsumerHandshake) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x18 if m.ConsumerScheduleDeadline != nil { - n9, err9 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ConsumerScheduleDeadline, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.ConsumerScheduleDeadline):]) - if err9 != nil { - return 0, err9 + n8, err8 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ConsumerScheduleDeadline, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.ConsumerScheduleDeadline):]) + if err8 != nil { + return 0, err8 } - i -= n9 - i = encodeVarintApi(dAtA, i, uint64(n9)) + i -= n8 + i = encodeVarintApi(dAtA, i, uint64(n8)) i-- dAtA[i] = 0x12 } @@ -981,10 +964,6 @@ func (m *ConsumerSignal) Size() (n int) { l = m.DrainRequest.Size() n += 1 + l + sovApi(uint64(l)) } - if m.SetupFlowRequest != nil { - l = m.SetupFlowRequest.Size() - n += 1 + l + sovApi(uint64(l)) - } if m.Handshake != nil { l = m.Handshake.Size() n += 1 + l + sovApi(uint64(l)) @@ -1027,9 +1006,6 @@ func (this *ConsumerSignal) GetValue() interface{} { if this.DrainRequest != nil { return this.DrainRequest } - if this.SetupFlowRequest != nil { - return this.SetupFlowRequest - } if this.Handshake != nil { return this.Handshake } @@ -1040,8 +1016,6 @@ func (this *ConsumerSignal) SetValue(value interface{}) bool { switch vt := value.(type) { case *DrainRequest: this.DrainRequest = vt - case *SetupFlowRequest: - this.SetupFlowRequest = vt case *ConsumerHandshake: this.Handshake = vt default: @@ -1668,42 +1642,6 @@ func (m *ConsumerSignal) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SetupFlowRequest", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthApi - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthApi - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.SetupFlowRequest == nil { - m.SetupFlowRequest = &SetupFlowRequest{} - } - if err := m.SetupFlowRequest.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Handshake", wireType) diff --git a/pkg/sql/execinfrapb/api.proto b/pkg/sql/execinfrapb/api.proto index a251a20dbd9a..4989d50113a1 100644 --- a/pkg/sql/execinfrapb/api.proto +++ b/pkg/sql/execinfrapb/api.proto @@ -88,12 +88,10 @@ message ConsumerSignal { // stream. optional DrainRequest drain_request = 1; - // Used in the RunSyncFlow case; the first message on the client stream must - // contain this message. - optional SetupFlowRequest setup_flow_request = 2; - // Consumer->Producer handshake messages. See message definition. optional ConsumerHandshake handshake = 3; + + reserved 2; } message DrainRequest { diff --git a/pkg/sql/flowinfra/outbox.go b/pkg/sql/flowinfra/outbox.go index 296aeac908f9..5585f079bcdb 100644 --- a/pkg/sql/flowinfra/outbox.go +++ b/pkg/sql/flowinfra/outbox.go @@ -337,9 +337,7 @@ func (m *Outbox) mainLoop(ctx context.Context) error { // RPCs that have this node as consumer to return errors. m.flowCtxCancel() // The consumer either doesn't care any more (it returned from the - // FlowStream RPC with an error if the outbox established the stream or - // it canceled the client context if the consumer established the - // stream through a RunSyncFlow RPC), or there was a communication error + // FlowStream RPC with an error), or there was a communication error // and the stream is dead. In any case, the stream has been closed and // the consumer will not consume more rows from this outbox. Make sure // the stream is not used any more. @@ -384,30 +382,23 @@ func (m *Outbox) listenForDrainSignalFromConsumer(ctx context.Context) (<-chan d stream := m.stream if err := m.flowCtx.Cfg.Stopper.RunAsyncTask(ctx, "drain", func(ctx context.Context) { - sendDrainSignal := func(drainRequested bool, err error) bool { + sendDrainSignal := func(drainRequested bool, err error) (shouldExit bool) { select { case ch <- drainSignal{drainRequested: drainRequested, err: err}: - return true + return false case <-ctx.Done(): - // Listening for consumer signals has been canceled. This generally - // means that the main outbox routine is no longer listening to these - // signals but, in the RunSyncFlow case, it may also mean that the - // client (the consumer) has canceled the RPC. In that case, the main - // routine is still listening (and this branch of the select has been - // randomly selected; the other was also available), so we have to - // notify it. Thus, we attempt sending again. - select { - case ch <- drainSignal{drainRequested: drainRequested, err: err}: - return true - default: - return false - } + // Listening for consumer signals has been canceled indicating + // that the main outbox routine is no longer listening to these + // signals. + return true } } for { signal, err := stream.Recv() if err == io.EOF { + // io.EOF indicates graceful completion of the stream, so we + // don't use io.EOF as an error. sendDrainSignal(false, nil) return } @@ -417,12 +408,9 @@ func (m *Outbox) listenForDrainSignalFromConsumer(ctx context.Context) (<-chan d } switch { case signal.DrainRequest != nil: - if !sendDrainSignal(true, nil) { + if shouldExit := sendDrainSignal(true, nil); shouldExit { return } - case signal.SetupFlowRequest != nil: - log.Fatalf(ctx, "unexpected SetupFlowRequest.\n"+ - "This SyncFlow specific message should have been handled in RunSyncFlow.") case signal.Handshake != nil: log.Eventf(ctx, "consumer sent handshake.\nConsuming flow scheduled: %t", signal.Handshake.ConsumerScheduled) diff --git a/pkg/sql/physicalplan/aggregator_funcs_test.go b/pkg/sql/physicalplan/aggregator_funcs_test.go index 7b882cc089e1..c881bbb90ea1 100644 --- a/pkg/sql/physicalplan/aggregator_funcs_test.go +++ b/pkg/sql/physicalplan/aggregator_funcs_test.go @@ -75,14 +75,13 @@ func runTestFlow( var rowBuf distsqlutils.RowBuffer - ctx, flow, err := distSQLSrv.SetupSyncFlow(context.Background(), distSQLSrv.ParentMemoryMonitor, &req, &rowBuf) + ctx, flow, err := distSQLSrv.SetupLocalSyncFlow(context.Background(), distSQLSrv.ParentMemoryMonitor, &req, &rowBuf, distsql.LocalState{}) if err != nil { t.Fatal(err) } - if err := flow.Start(ctx, func() {}); err != nil { + if err := flow.Run(ctx, func() {}); err != nil { t.Fatal(err) } - flow.Wait() flow.Cleanup(ctx) if !rowBuf.ProducerClosed() { From adbacbfe9d42f874455823a8e1f2737cad4e5c8c Mon Sep 17 00:00:00 2001 From: Rebecca Taft <becca@cockroachlabs.com> Date: Sun, 18 Apr 2021 17:22:23 -0500 Subject: [PATCH 15/37] opt: reduce cost of locality optimized anti join This commit reduces the cost of locality optimized anti join to to ensure that we always choose it over a non-locality-optimized anti join when possible. Informs #63735 Release note (performance improvement): Adjusted the estimated cost of locality optimized anti joins in the optimizer so that they are always chosen over non-locality-optimized anti joins when possible. This makes it more likely that queries involving anti joins (such as inserts with foreign key checks) can avoid visting remote regions. This results in lower latency. --- .../testdata/logic_test/multi_region_tpcc | 246 ++++++++++++++++++ pkg/sql/opt/xform/coster.go | 4 +- pkg/sql/opt/xform/testdata/coster/zone | 4 +- 3 files changed, 250 insertions(+), 4 deletions(-) create mode 100644 pkg/ccl/logictestccl/testdata/logic_test/multi_region_tpcc diff --git a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_tpcc b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_tpcc new file mode 100644 index 000000000000..300dc6c252f2 --- /dev/null +++ b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_tpcc @@ -0,0 +1,246 @@ +# LogicTest: multiregion-9node-3region-3azs + +statement ok +CREATE DATABASE tpcc PRIMARY REGION "ca-central-1" REGIONS "ap-southeast-2", "us-east-1"; +USE tpcc + +# NB: This doesn't include the foreign key reference to warehouse since we +# don't need the warehouse table for this test. +statement ok +CREATE TABLE district ( + d_id INT8 NOT NULL, + d_w_id INT8 NOT NULL, + d_name VARCHAR(10) NOT NULL, + d_street_1 VARCHAR(20) NOT NULL, + d_street_2 VARCHAR(20) NOT NULL, + d_city VARCHAR(20) NOT NULL, + d_state CHAR(2) NOT NULL, + d_zip CHAR(9) NOT NULL, + d_tax DECIMAL(4,4) NOT NULL, + d_ytd DECIMAL(12,2) NOT NULL, + d_next_o_id INT8 NOT NULL, + crdb_region crdb_internal_region NOT VISIBLE NOT NULL AS (CASE WHEN d_w_id BETWEEN 0:::INT8 AND 1665:::INT8 THEN 'ap-southeast-2':::crdb_internal_region WHEN d_w_id BETWEEN 1666:::INT8 AND 3332:::INT8 THEN 'us-east-1':::crdb_internal_region WHEN d_w_id BETWEEN 3333:::INT8 AND 4999:::INT8 THEN 'ca-central-1':::crdb_internal_region END) STORED, + CONSTRAINT "primary" PRIMARY KEY (d_w_id ASC, d_id ASC), + FAMILY "primary" (d_id, d_w_id, d_name, d_street_1, d_street_2, d_city, d_state, d_zip, d_tax, d_ytd, d_next_o_id, crdb_region) +) LOCALITY REGIONAL BY ROW + +statement ok +CREATE TABLE customer ( + c_id INT8 NOT NULL, + c_d_id INT8 NOT NULL, + c_w_id INT8 NOT NULL, + c_first VARCHAR(16) NOT NULL, + c_middle CHAR(2) NOT NULL, + c_last VARCHAR(16) NOT NULL, + c_street_1 VARCHAR(20) NOT NULL, + c_street_2 VARCHAR(20) NOT NULL, + c_city VARCHAR(20) NOT NULL, + c_state CHAR(2) NOT NULL, + c_zip CHAR(9) NOT NULL, + c_phone CHAR(16) NOT NULL, + c_since TIMESTAMP NOT NULL, + c_credit CHAR(2) NOT NULL, + c_credit_lim DECIMAL(12,2) NOT NULL, + c_discount DECIMAL(4,4) NOT NULL, + c_balance DECIMAL(12,2) NOT NULL, + c_ytd_payment DECIMAL(12,2) NOT NULL, + c_payment_cnt INT8 NOT NULL, + c_delivery_cnt INT8 NOT NULL, + c_data VARCHAR(500) NOT NULL, + crdb_region crdb_internal_region NOT VISIBLE NOT NULL AS (CASE WHEN c_w_id BETWEEN 0:::INT8 AND 1665:::INT8 THEN 'ap-southeast-2':::crdb_internal_region WHEN c_w_id BETWEEN 1666:::INT8 AND 3332:::INT8 THEN 'us-east-1':::crdb_internal_region WHEN c_w_id BETWEEN 3333:::INT8 AND 4999:::INT8 THEN 'ca-central-1':::crdb_internal_region END) STORED, + CONSTRAINT "primary" PRIMARY KEY (c_w_id ASC, c_d_id ASC, c_id ASC), + CONSTRAINT fk_c_w_id_ref_district FOREIGN KEY (c_w_id, c_d_id) REFERENCES district(d_w_id, d_id) NOT VALID, + INDEX customer_idx (c_w_id ASC, c_d_id ASC, c_last ASC, c_first ASC), + FAMILY "primary" (c_id, c_d_id, c_w_id, c_first, c_middle, c_last, c_street_1, c_street_2, c_city, c_state, c_zip, c_phone, c_since, c_credit, c_credit_lim, c_discount, c_balance, c_ytd_payment, c_payment_cnt, c_delivery_cnt, c_data, crdb_region) +) LOCALITY REGIONAL BY ROW + +statement ok +CREATE TABLE history ( + rowid UUID NOT NULL DEFAULT gen_random_uuid(), + h_c_id INT8 NOT NULL, + h_c_d_id INT8 NOT NULL, + h_c_w_id INT8 NOT NULL, + h_d_id INT8 NOT NULL, + h_w_id INT8 NOT NULL, + h_date TIMESTAMP NULL, + h_amount DECIMAL(6,2) NULL, + h_data VARCHAR(24) NULL, + crdb_region crdb_internal_region NOT VISIBLE NOT NULL AS (CASE WHEN h_w_id BETWEEN 0:::INT8 AND 1665:::INT8 THEN 'ap-southeast-2':::crdb_internal_region WHEN h_w_id BETWEEN 1666:::INT8 AND 3332:::INT8 THEN 'us-east-1':::crdb_internal_region WHEN h_w_id BETWEEN 3333:::INT8 AND 4999:::INT8 THEN 'ca-central-1':::crdb_internal_region END) STORED, + CONSTRAINT "primary" PRIMARY KEY (h_w_id ASC, rowid ASC), + CONSTRAINT fk_h_c_w_id_ref_customer FOREIGN KEY (h_c_w_id, h_c_d_id, h_c_id) REFERENCES customer(c_w_id, c_d_id, c_id) NOT VALID, + CONSTRAINT fk_h_w_id_ref_district FOREIGN KEY (h_w_id, h_d_id) REFERENCES district(d_w_id, d_id) NOT VALID, + FAMILY "primary" (rowid, h_c_id, h_c_d_id, h_c_w_id, h_d_id, h_w_id, h_date, h_amount, h_data, crdb_region) +) LOCALITY REGIONAL BY ROW + +statement ok +ALTER TABLE district INJECT STATISTICS '[ + { + "columns": [ + "d_w_id" + ], + "created_at": "2021-04-13 19:54:56.008454", + "distinct_count": 5004, + "name": "__auto__", + "null_count": 0, + "row_count": 50000 + }, + { + "columns": [ + "d_id" + ], + "created_at": "2021-04-13 19:54:56.008454", + "distinct_count": 10, + "name": "__auto__", + "null_count": 0, + "row_count": 50000 + } +]' + +statement ok +ALTER TABLE customer INJECT STATISTICS '[ + { + "columns": [ + "c_d_id" + ], + "created_at": "2021-04-13 20:35:46.476858", + "distinct_count": 10, + "name": "__auto__", + "null_count": 0, + "row_count": 150000000 + }, + { + "columns": [ + "c_w_id" + ], + "created_at": "2021-04-13 20:35:46.476858", + "distinct_count": 4998, + "name": "__auto__", + "null_count": 0, + "row_count": 150000000 + }, + { + "columns": [ + "c_id" + ], + "created_at": "2021-04-13 20:35:46.476858", + "distinct_count": 2999, + "name": "__auto__", + "null_count": 0, + "row_count": 150000000 + } +]' + +statement ok +ALTER TABLE history INJECT STATISTICS '[ + { + "columns": [ + "h_w_id" + ], + "created_at": "2021-04-13 20:58:06.757925", + "distinct_count": 4998, + "name": "__auto__", + "null_count": 0, + "row_count": 150000000 + }, + { + "columns": [ + "h_c_id" + ], + "created_at": "2021-04-13 20:58:06.757925", + "distinct_count": 2999, + "name": "__auto__", + "null_count": 0, + "row_count": 150000000 + }, + { + "columns": [ + "h_c_d_id" + ], + "created_at": "2021-04-13 20:58:06.757925", + "distinct_count": 10, + "name": "__auto__", + "null_count": 0, + "row_count": 150000000 + }, + { + "columns": [ + "h_c_w_id" + ], + "created_at": "2021-04-13 20:58:06.757925", + "distinct_count": 4998, + "name": "__auto__", + "null_count": 0, + "row_count": 150000000 + }, + { + "columns": [ + "h_d_id" + ], + "created_at": "2021-04-13 20:58:06.757925", + "distinct_count": 10, + "name": "__auto__", + "null_count": 0, + "row_count": 150000000 + } +]' + +# Regression test for #63735. Ensure that we choose locality optimized anti +# joins for the foreign key checks. +query T +EXPLAIN INSERT +INTO + history (h_c_id, h_c_d_id, h_c_w_id, h_d_id, h_w_id, h_amount, h_date, h_data) +VALUES + (2057, 4, 3, 4, 3, 2100.9, '2021-04-15 15:22:14', '9 zmssaF9m') +---- +distribution: local +vectorized: true +· +• root +│ +├── • insert +│ │ into: history(rowid, h_c_id, h_c_d_id, h_c_w_id, h_d_id, h_w_id, h_date, h_amount, h_data, crdb_region) +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • values +│ size: 11 columns, 1 row +│ +├── • constraint-check +│ │ +│ └── • error if rows +│ │ +│ └── • lookup join (anti) +│ │ estimated row count: 0 +│ │ table: customer@primary +│ │ equality cols are key +│ │ lookup condition: (((column3 = c_w_id) AND (column2 = c_d_id)) AND (column1 = c_id)) AND (crdb_region IN ('ca-central-1', 'us-east-1')) +│ │ +│ └── • lookup join (anti) +│ │ estimated row count: 1 +│ │ table: customer@primary +│ │ equality cols are key +│ │ lookup condition: (((column3 = c_w_id) AND (column2 = c_d_id)) AND (column1 = c_id)) AND (crdb_region = 'ap-southeast-2') +│ │ +│ └── • scan buffer +│ label: buffer 1 +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • lookup join (anti) + │ estimated row count: 0 + │ table: district@primary + │ equality cols are key + │ lookup condition: ((column5 = d_w_id) AND (column4 = d_id)) AND (crdb_region IN ('ca-central-1', 'us-east-1')) + │ + └── • lookup join (anti) + │ estimated row count: 1 + │ table: district@primary + │ equality cols are key + │ lookup condition: ((column5 = d_w_id) AND (column4 = d_id)) AND (crdb_region = 'ap-southeast-2') + │ + └── • scan buffer + label: buffer 1 diff --git a/pkg/sql/opt/xform/coster.go b/pkg/sql/opt/xform/coster.go index 51dbb0e9d3e4..f7b5ae8b563c 100644 --- a/pkg/sql/opt/xform/coster.go +++ b/pkg/sql/opt/xform/coster.go @@ -872,13 +872,13 @@ func (c *coster) computeIndexLookupJoinCost( cost *= preferLookupJoinFactor } - // If this lookup join is locality optimized, divide the cost by two in order to make + // If this lookup join is locality optimized, divide the cost by 2.5 in order to make // the total cost of the two lookup joins in the locality optimized plan less than // the cost of the single lookup join in the non-locality optimized plan. // TODO(rytaft): This is hacky. We should really be making this determination // based on the latency between regions. if localityOptimized { - cost /= 2 + cost /= 2.5 } return cost } diff --git a/pkg/sql/opt/xform/testdata/coster/zone b/pkg/sql/opt/xform/testdata/coster/zone index dccdccd216f4..e40bc35eca30 100644 --- a/pkg/sql/opt/xform/testdata/coster/zone +++ b/pkg/sql/opt/xform/testdata/coster/zone @@ -755,7 +755,7 @@ anti-join (lookup abc_part@bc_idx [as=a2]) │ └── a2.r:6 = 'west' [outer=(6), constraints=(/6: [/'west' - /'west']; tight), fd=()-->(6)] ├── cardinality: [0 - 1] ├── stats: [rows=1e-10] - ├── cost: 23.4031483 + ├── cost: 19.7431618 ├── key: () ├── fd: ()-->(1-4) ├── anti-join (lookup abc_part@bc_idx [as=a2]) @@ -766,7 +766,7 @@ anti-join (lookup abc_part@bc_idx [as=a2]) │ │ └── a2.r:6 = 'east' [outer=(6), constraints=(/6: [/'east' - /'east']; tight), fd=()-->(6)] │ ├── cardinality: [0 - 1] │ ├── stats: [rows=0.900900001, distinct(1)=0.89738934, null(1)=0, distinct(2)=0.900900001, null(2)=0, distinct(3)=0.900900001, null(3)=0, distinct(4)=0.900900001, null(4)=0] - │ ├── cost: 14.2891619 + │ ├── cost: 12.4499727 │ ├── key: () │ ├── fd: ()-->(1-4) │ ├── locality-optimized-search From e184d516149399d74804395407f624a9eca46f97 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss <knz@thaumogen.net> Date: Mon, 19 Apr 2021 11:57:27 +0200 Subject: [PATCH 16/37] build: update the go version requirement for `make` The builder image already requires go 1.15.10. This patch modifies the check for a non-builder `make` command to require at least the same version. Release note: None --- build/go-version-check.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/build/go-version-check.sh b/build/go-version-check.sh index b285997526d0..719eabe48b59 100755 --- a/build/go-version-check.sh +++ b/build/go-version-check.sh @@ -7,7 +7,8 @@ required_version_major=1 minimum_version_minor=15 -minimum_version_15_patch=3 +minimum_version_15_patch=10 # update to 11 when issue #63836 is addressed +minimum_version_16_patch=3 go=${1-go} From 576dd39114de6becd307f31fcab095faddd32fd2 Mon Sep 17 00:00:00 2001 From: Andrew Werner <awerner32@gmail.com> Date: Mon, 19 Apr 2021 09:12:20 -0400 Subject: [PATCH 17/37] jobs: skip flakey test that should be removed This tests seems to flake but exercises old code. It is scheduled for removal in #61417 which didn't make the 21.1 release. Touches #63842. Release note: None --- pkg/jobs/registry_external_test.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pkg/jobs/registry_external_test.go b/pkg/jobs/registry_external_test.go index 6c310851abba..3cfeb3f61089 100644 --- a/pkg/jobs/registry_external_test.go +++ b/pkg/jobs/registry_external_test.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -80,6 +81,11 @@ func TestRegistryResumeExpiredLease(t *testing.T) { defer log.Scope(t).Close(t) defer jobs.ResetConstructors()() + // This test exercises code that has not been in use since the 20.1->20.2 + // mixed version state. It is scheduled for removal and thus not worth + // de-flaking. + skip.WithIssue(t, 63842) + ctx := context.Background() ver201 := cluster.MakeTestingClusterSettingsWithVersions( From cb3e9ce3c0c448a16ba587a223bfefd2b882b59b Mon Sep 17 00:00:00 2001 From: Paul Bardea <pbardea@gmail.com> Date: Tue, 13 Apr 2021 10:53:49 -0400 Subject: [PATCH 18/37] backupccl: make job system table restoration idempotent Previously, custom implementations of restoring system tables during cluster restore may have not been idempotent. As such, a map was used to track when particular system tables had been restored. This was fragile. This change updates the system table restoration logic to be idempotent for all custom implementation (only the jobs table needed updating). Release note: None --- .../backupccl/full_cluster_backup_restore_test.go | 9 +++++---- pkg/ccl/backupccl/restore_job.go | 14 +------------- pkg/ccl/backupccl/system_schema.go | 2 +- 3 files changed, 7 insertions(+), 18 deletions(-) diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index d7942827e984..18623cd4821b 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -596,8 +596,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { // This test retries the job (by injected a retry error) after restoring a // every system table that has a custom restore function. This tried to tease // out any errors that may occur if some of the system table restoration - // functions are not idempotent (e.g. jobs table), but are retried by the - // restore anyway. + // functions are not idempotent. t.Run("retry-during-custom-system-table-restore", func(t *testing.T) { defer jobs.TestingSetAdoptAndCancelIntervals(100*time.Millisecond, 100*time.Millisecond)() @@ -612,14 +611,16 @@ func TestClusterRestoreFailCleanup(t *testing.T) { _, tcRestore, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupEmptyCluster() - // Inject a retry error + // Inject a retry error, that returns once. + alreadyErrored := false for _, server := range tcRestore.Servers { registry := server.JobRegistry().(*jobs.Registry) registry.TestingResumerCreationKnobs = map[jobspb.Type]func(raw jobs.Resumer) jobs.Resumer{ jobspb.TypeRestore: func(raw jobs.Resumer) jobs.Resumer { r := raw.(*restoreResumer) r.testingKnobs.duringSystemTableRestoration = func(systemTableName string) error { - if systemTableName == customRestoreSystemTable { + if !alreadyErrored && systemTableName == customRestoreSystemTable { + alreadyErrored = true return jobs.NewRetryJobError("injected error") } return nil diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 39f477939aae..94a05db965f6 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -2425,10 +2425,6 @@ func (r *restoreResumer) restoreSystemTables( tables []catalog.TableDescriptor, ) error { tempSystemDBID := getTempSystemDBID(restoreDetails) - details := r.job.Details().(jobspb.RestoreDetails) - if details.SystemTablesRestored == nil { - details.SystemTablesRestored = make(map[string]bool) - } // Iterate through all the tables that we're restoring, and if it was restored // to the temporary system DB then copy it's data over to the real system @@ -2438,10 +2434,6 @@ func (r *restoreResumer) restoreSystemTables( continue } systemTableName := table.GetName() - if details.SystemTablesRestored[systemTableName] { - // We've already restored this table. - continue - } if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { txn.SetDebugName("system-restore-txn") @@ -2462,11 +2454,7 @@ func (r *restoreResumer) restoreSystemTables( if err != nil { return errors.Wrapf(err, "restoring system table %s", systemTableName) } - - // System table restoration may not be idempotent, so we need to keep - // track of what we've restored. - details.SystemTablesRestored[systemTableName] = true - return r.job.SetDetails(ctx, txn, details) + return nil }); err != nil { return err } diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go index 1153b3b6cb75..4bc3b3311d21 100644 --- a/pkg/ccl/backupccl/system_schema.go +++ b/pkg/ccl/backupccl/system_schema.go @@ -105,7 +105,7 @@ func jobsRestoreFunc( // When restoring jobs, don't clear the existing table. - restoreQuery := fmt.Sprintf("INSERT INTO system.%s (SELECT * FROM %s);", + restoreQuery := fmt.Sprintf("INSERT INTO system.%s (SELECT * FROM %s) ON CONFLICT DO NOTHING;", systemTableName, tempTableName) opName := systemTableName + "-data-insert" if _, err := executor.Exec(ctx, opName, txn, restoreQuery); err != nil { From 1c882a03c85a3f37736ed910a621d1fbfa54a19b Mon Sep 17 00:00:00 2001 From: Paul Bardea <pbardea@gmail.com> Date: Fri, 26 Mar 2021 03:44:15 +0000 Subject: [PATCH 19/37] backupccl: reset restored jobs during cluster restore Previously, jobs were restored without modification during cluster restore. Due to a recently discovered bug where backup may miss non-transactional writes written to offline spans by these jobs, their progress may no longer be accurate on the restored cluster. IMPORT and RESTORE jobs perform non-transactional writes that may be missed. When a cluster RESTORE brings back these OFFLINE tables, it will also bring back its associated job. To ensure the underlying data in these tables is correct, the jobs are now set in a reverting state so that they can clean up after themselves. In-progress schema change jobs that are affected, will fail upon validation. Release note (bug fix): Fix a bug where restored jobs may have assumed to have made progress that was not captured in the backup. The restored jobs are now either canceled cluster restore. --- .../full_cluster_backup_restore_test.go | 59 +- pkg/ccl/backupccl/restoration_data.go | 18 + pkg/ccl/backupccl/restore_job.go | 49 +- pkg/ccl/backupccl/system_schema.go | 82 +++ pkg/jobs/jobspb/jobs.pb.go | 512 +++++++++--------- pkg/jobs/jobspb/jobs.proto | 13 +- 6 files changed, 448 insertions(+), 285 deletions(-) diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index 18623cd4821b..3a5b082760ec 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -630,10 +630,9 @@ func TestClusterRestoreFailCleanup(t *testing.T) { } } - // The initial restore will fail, and restart. + // The initial restore will return an error, and restart. sqlDBRestore.ExpectErr(t, `injected error: restarting in background`, `RESTORE FROM $1`, LocalFoo) - // Expect the job to succeed. If the job fails, it's likely due to - // attempting to restore the same system table data twice. + // Expect the restore to succeed. sqlDBRestore.CheckQueryResultsRetry(t, `SELECT count(*) FROM [SHOW JOBS] WHERE job_type = 'RESTORE' AND status = 'succeeded'`, [][]string{{"1"}}) @@ -842,9 +841,7 @@ func TestReintroduceOfflineSpans(t *testing.T) { const numAccounts = 1000 ctx, _, srcDB, tempDir, cleanupSrc := backupRestoreTestSetupWithParams(t, singleNode, numAccounts, InitManualReplication, params) - _, _, destDB, cleanupDst := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupSrc() - defer cleanupDst() dbBackupLoc := "nodelocal://0/my_db_backup" clusterBackupLoc := "nodelocal://0/my_cluster_backup" @@ -868,7 +865,10 @@ func TestReintroduceOfflineSpans(t *testing.T) { <-dbRestoreStarted srcDB.Exec(t, `BACKUP TO $1 WITH revision_history`, clusterBackupLoc) - // All the restore to finish. This will issue AddSSTable requests at a + var tsMidRestore string + srcDB.QueryRow(t, "SELECT cluster_logical_timestamp()").Scan(&tsMidRestore) + + // Allow the restore to finish. This will issue AddSSTable requests at a // timestamp that is before the last incremental we just took. close(blockDBRestore) @@ -885,16 +885,43 @@ func TestReintroduceOfflineSpans(t *testing.T) { srcDB.Exec(t, `BACKUP TO $1 WITH revision_history`, clusterBackupLoc) - // Restore the incremental backup chain that has missing writes. - destDB.Exec(t, `RESTORE FROM $1 AS OF SYSTEM TIME `+tsBefore, clusterBackupLoc) + t.Run("spans-reintroduced", func(t *testing.T) { + _, _, destDB, cleanupDst := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) + defer cleanupDst() + + // Restore the incremental backup chain that has missing writes. + destDB.Exec(t, `RESTORE FROM $1 AS OF SYSTEM TIME `+tsBefore, clusterBackupLoc) + + // Assert that the restored database has the same number of rows in both the + // source and destination cluster. + checkQuery := `SELECT count(*) FROM restoredb.bank AS OF SYSTEM TIME ` + tsBefore + expectedCount := srcDB.QueryStr(t, checkQuery) + destDB.CheckQueryResults(t, `SELECT count(*) FROM restoredb.bank`, expectedCount) + + checkQuery = `SELECT count(*) FROM restoredb.bank@new_idx AS OF SYSTEM TIME ` + tsBefore + expectedCount = srcDB.QueryStr(t, checkQuery) + destDB.CheckQueryResults(t, `SELECT count(*) FROM restoredb.bank@new_idx`, expectedCount) + }) + + t.Run("restore-canceled", func(t *testing.T) { + defer jobs.TestingSetAdoptAndCancelIntervals(100*time.Millisecond, 100*time.Millisecond)() + _, _, destDB, cleanupDst := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) + defer cleanupDst() - // Assert that the restored database has the same number - // of rows in both the source and destination cluster. - checkQuery := `SELECT count(*) FROM restoredb.bank AS OF SYSTEM TIME ` + tsBefore - expectedCount := srcDB.QueryStr(t, checkQuery) - destDB.CheckQueryResults(t, `SELECT count(*) FROM restoredb.bank`, expectedCount) + destDB.Exec(t, `RESTORE FROM $1 AS OF SYSTEM TIME `+tsMidRestore, clusterBackupLoc) - checkQuery = `SELECT count(*) FROM restoredb.bank@new_idx AS OF SYSTEM TIME ` + tsBefore - expectedCount = srcDB.QueryStr(t, checkQuery) - destDB.CheckQueryResults(t, `SELECT count(*) FROM restoredb.bank@new_idx`, expectedCount) + // Wait for the cluster restore job to finish, as well as the restored RESTORE TABLE + // job to cancel. + destDB.CheckQueryResultsRetry(t, ` + SELECT description, status FROM [SHOW JOBS] + WHERE job_type = 'RESTORE' AND status NOT IN ('succeeded', 'canceled')`, + [][]string{}, + ) + // The cluster restore should succeed, but the table restore should have failed. + destDB.CheckQueryResults(t, + `SELECT status, count(*) FROM [SHOW JOBS] WHERE job_type = 'RESTORE' GROUP BY status ORDER BY status`, + [][]string{{"canceled", "1"}, {"succeeded", "1"}}) + + destDB.ExpectErr(t, `relation "restoredb.bank" does not exist`, `SELECT count(*) FROM restoredb.bank`) + }) } diff --git a/pkg/ccl/backupccl/restoration_data.go b/pkg/ccl/backupccl/restoration_data.go index ae82a13b70ec..3b0ecd150308 100644 --- a/pkg/ccl/backupccl/restoration_data.go +++ b/pkg/ccl/backupccl/restoration_data.go @@ -9,6 +9,7 @@ package backupccl import ( + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -107,3 +108,20 @@ func (b *restorationDataBase) isEmpty() bool { // isMainBundle implements restorationData. func (restorationDataBase) isMainBundle() bool { return false } + +// checkForMigratedData checks to see if any of the system tables in the set of +// data that is to be restored has already been restored. If this is the case, +// it is not safe to try and restore the data again since the migration may have +// written to the temporary system table. +func checkForMigratedData(details jobspb.RestoreDetails, dataToRestore restorationData) bool { + for _, systemTable := range dataToRestore.getSystemTables() { + // We only need to check if _any_ of the system tables in this batch of + // data have been migrated. This is because the migration can only + // happen after all of the data in the batch has been restored. + if _, ok := details.SystemTablesMigrated[systemTable.GetName()]; ok { + return true + } + } + + return false +} diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 94a05db965f6..8e98951fd568 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -604,6 +604,14 @@ func restore( return emptyRowCount, nil } + // If we've already migrated some of the system tables we're about to + // restore, this implies that a previous attempt restored all of this data. + // We want to avoid restoring again since we'll be shadowing migrated keys. + details := job.Details().(jobspb.RestoreDetails) + if alreadyMigrated := checkForMigratedData(details, dataToRestore); alreadyMigrated { + return emptyRowCount, nil + } + mu := struct { syncutil.Mutex highWaterMark int @@ -1670,8 +1678,7 @@ func (r *restoreResumer) Resume(ctx context.Context, execCtx interface{}) error } // Reload the details as we may have updated the job. details = r.job.Details().(jobspb.RestoreDetails) - } - if details.DescriptorCoverage == tree.AllDescriptors { + if err := r.cleanupTempSystemTables(ctx); err != nil { return err } @@ -2425,6 +2432,10 @@ func (r *restoreResumer) restoreSystemTables( tables []catalog.TableDescriptor, ) error { tempSystemDBID := getTempSystemDBID(restoreDetails) + details := r.job.Details().(jobspb.RestoreDetails) + if details.SystemTablesMigrated == nil { + details.SystemTablesMigrated = make(map[string]bool) + } // Iterate through all the tables that we're restoring, and if it was restored // to the temporary system DB then copy it's data over to the real system @@ -2434,14 +2445,36 @@ func (r *restoreResumer) restoreSystemTables( continue } systemTableName := table.GetName() + stagingTableName := restoreTempSystemDB + "." + systemTableName + + config, ok := systemTableBackupConfiguration[systemTableName] + if !ok { + log.Warningf(ctx, "no configuration specified for table %s... skipping restoration", + systemTableName) + } + + if config.migrationFunc != nil { + if details.SystemTablesMigrated[systemTableName] { + continue + } + + if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := config.migrationFunc(ctx, r.execCfg, txn, stagingTableName); err != nil { + return err + } + + // Keep track of which system tables we've migrated so that future job + // restarts don't try to import data over our migrated data. This would + // fail since the restored data would shadow the migrated keys. + details.SystemTablesMigrated[systemTableName] = true + return r.job.SetDetails(ctx, txn, details) + }); err != nil { + return err + } + } if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { txn.SetDebugName("system-restore-txn") - config, ok := systemTableBackupConfiguration[systemTableName] - if !ok { - log.Warningf(ctx, "no configuration specified for table %s... skipping restoration", - systemTableName) - } restoreFunc := defaultSystemTableRestoreFunc if config.customRestoreFunc != nil { @@ -2450,7 +2483,7 @@ func (r *restoreResumer) restoreSystemTables( } log.Eventf(ctx, "restoring system table %s", systemTableName) - err := restoreFunc(ctx, r.execCfg, txn, systemTableName, restoreTempSystemDB+"."+systemTableName) + err := restoreFunc(ctx, r.execCfg, txn, systemTableName, stagingTableName) if err != nil { return errors.Wrapf(err, "restoring system table %s", systemTableName) } diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go index 4bc3b3311d21..b0b2ea31c100 100644 --- a/pkg/ccl/backupccl/system_schema.go +++ b/pkg/ccl/backupccl/system_schema.go @@ -11,10 +11,16 @@ package backupccl import ( "context" fmt "fmt" + "strings" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -55,6 +61,10 @@ type systemBackupConfiguration struct { // to support the restore (e.g. users that can run the restore, cluster settings // that control how the restore runs, etc...). restoreBeforeData bool + // migrationFunc performs the necessary migrations on the system table data in + // the crdb_temp staging table before it is loaded into the actual system + // table. + migrationFunc func(ctx context.Context, execCtx *sql.ExecutorConfig, txn *kv.Txn, tempTableName string) error // customRestoreFunc is responsible for restoring the data from a table that // holds the restore system table data into the given system table. If none // is provided then `defaultRestoreFunc` is used. @@ -93,6 +103,77 @@ func defaultSystemTableRestoreFunc( // Custom restore functions for different system tables. +// jobsMigrationFunc resets the progress on schema change jobs, and marks all +// other jobs as reverting. +func jobsMigrationFunc( + ctx context.Context, execCfg *sql.ExecutorConfig, txn *kv.Txn, tempTableName string, +) (err error) { + executor := execCfg.InternalExecutor + + const statesToRevert = `('` + string(jobs.StatusRunning) + `', ` + + `'` + string(jobs.StatusPauseRequested) + `', ` + + `'` + string(jobs.StatusPaused) + `')` + + jobsToRevert := make([]int64, 0) + query := `SELECT id, payload FROM ` + tempTableName + ` WHERE status IN ` + statesToRevert + it, err := executor.QueryIteratorEx( + ctx, "restore-fetching-job-payloads", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + query) + if err != nil { + return errors.Wrap(err, "fetching job payloads") + } + defer func() { + closeErr := it.Close() + if err == nil { + err = closeErr + } + }() + for { + ok, err := it.Next(ctx) + if !ok { + if err != nil { + return err + } + break + } + + r := it.Cur() + id, payloadBytes := r[0], r[1] + rawJobID, ok := id.(*tree.DInt) + if !ok { + return errors.Errorf("job: failed to read job id as DInt (was %T)", id) + } + jobID := int64(*rawJobID) + + payload, err := jobs.UnmarshalPayload(payloadBytes) + if err != nil { + return errors.Wrap(err, "failed to unmarshal job to restore") + } + if payload.Type() == jobspb.TypeImport || payload.Type() == jobspb.TypeRestore { + jobsToRevert = append(jobsToRevert, jobID) + } + } + + // Update the status for other jobs. + var updateStatusQuery strings.Builder + fmt.Fprintf(&updateStatusQuery, "UPDATE %s SET status = $1 WHERE id IN ", tempTableName) + fmt.Fprint(&updateStatusQuery, "(") + for i, job := range jobsToRevert { + if i > 0 { + fmt.Fprint(&updateStatusQuery, ", ") + } + fmt.Fprintf(&updateStatusQuery, "'%d'", job) + } + fmt.Fprint(&updateStatusQuery, ")") + + if _, err := executor.Exec(ctx, "updating-job-status", txn, updateStatusQuery.String(), jobs.StatusCancelRequested); err != nil { + return errors.Wrap(err, "updating restored jobs as reverting") + } + + return nil +} + // When restoring the jobs table we don't want to remove existing jobs, since // that includes the restore that we're running. func jobsRestoreFunc( @@ -178,6 +259,7 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{ }, systemschema.JobsTable.GetName(): { shouldIncludeInClusterBackup: optInToClusterBackup, + migrationFunc: jobsMigrationFunc, customRestoreFunc: jobsRestoreFunc, }, systemschema.ScheduledJobsTable.GetName(): { diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 41ae1a40ac98..d2415d39bec2 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -587,12 +587,13 @@ type RestoreDetails struct { // stages have completed via these flags. PrepareCompleted bool `protobuf:"varint,8,opt,name=prepare_completed,json=prepareCompleted,proto3" json:"prepare_completed,omitempty"` StatsInserted bool `protobuf:"varint,9,opt,name=stats_inserted,json=statsInserted,proto3" json:"stats_inserted,omitempty"` - // SystemTablesRestored keeps track of dynamic states that need to happen only - // once during the lifetime of a job. Note, that this state may be shared - // between job versions, so updates to this map must be considered carefully. - // It maps system table names to whether or not they have already been - // restored. - SystemTablesRestored map[string]bool `protobuf:"bytes,17,rep,name=system_tables_restored,json=systemTablesRestored,proto3" json:"system_tables_restored,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + // SystemTablesMigrated keeps track of which system tables data have been + // migrated. We need to keep track of this because if we've modified the + // restored data via a migration, we can't restore back into that span as the + // migrated keys will shadow the ones that will be restored. + // Note, that this state may be shared between job versions, so updates to + // this map must be considered carefully. + SystemTablesMigrated map[string]bool `protobuf:"bytes,17,rep,name=system_tables_migrated,json=systemTablesMigrated,proto3" json:"system_tables_migrated,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` // DescriptorsPublished indicates whether or not the descriptors written in // the job have been transactionally updated after the data was restored. DescriptorsPublished bool `protobuf:"varint,10,opt,name=descriptors_published,json=descriptorsPublished,proto3" json:"descriptors_published,omitempty"` @@ -2574,7 +2575,7 @@ func init() { proto.RegisterType((*BackupProgress)(nil), "cockroach.sql.jobs.jobspb.BackupProgress") proto.RegisterType((*RestoreDetails)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails") proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID]*RestoreDetails_DescriptorRewrite)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.DescriptorRewritesEntry") - proto.RegisterMapType((map[string]bool)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.SystemTablesRestoredEntry") + proto.RegisterMapType((map[string]bool)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.SystemTablesMigratedEntry") proto.RegisterType((*RestoreDetails_DescriptorRewrite)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.DescriptorRewrite") proto.RegisterType((*RestoreDetails_BackupLocalityInfo)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.BackupLocalityInfo") proto.RegisterMapType((map[string]string)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.BackupLocalityInfo.UrisByOriginalLocalityKvEntry") @@ -2624,7 +2625,7 @@ func init() { func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_6c315f3a2536c4ef) } var fileDescriptor_6c315f3a2536c4ef = []byte{ - // 5055 bytes of a gzipped FileDescriptorProto + // 5057 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5b, 0x4b, 0x70, 0x23, 0xc7, 0x79, 0xe6, 0x80, 0x20, 0x1e, 0x3f, 0x1e, 0x1c, 0x34, 0xb9, 0xbb, 0x10, 0x2c, 0x2d, 0x68, 0x58, 0x2b, 0xed, 0xae, 0x24, 0x50, 0xa6, 0x6c, 0x59, 0x5a, 0x4b, 0x2b, 0xe1, 0x45, 0x12, 0xe0, 0xf2, @@ -2710,237 +2711,238 @@ var fileDescriptor_6c315f3a2536c4ef = []byte{ 0xd7, 0xc1, 0x5d, 0xcd, 0xc1, 0xaa, 0x6e, 0x77, 0xba, 0x6d, 0xec, 0x62, 0x83, 0x6d, 0xe8, 0x98, 0x22, 0x8b, 0x82, 0x8a, 0x27, 0xe7, 0xbe, 0x5b, 0x73, 0x69, 0x58, 0x48, 0xb0, 0x43, 0x35, 0xe3, 0x4c, 0x33, 0xc5, 0xa4, 0x75, 0x21, 0x44, 0x87, 0x70, 0x99, 0x1c, 0x12, 0x17, 0x77, 0x54, 0x36, - 0xdd, 0x44, 0x75, 0xb8, 0x05, 0x19, 0xd9, 0x0c, 0x1b, 0x56, 0x65, 0x72, 0x63, 0x6b, 0x32, 0x1c, - 0xb6, 0x8c, 0x44, 0x14, 0x19, 0x3c, 0xf0, 0x99, 0x27, 0xc7, 0x14, 0xa1, 0xb7, 0xe0, 0xd2, 0x60, - 0x67, 0x10, 0xb5, 0xdb, 0x6b, 0xb5, 0x4d, 0xb2, 0x8f, 0x8d, 0x2c, 0xb0, 0x8e, 0xce, 0x07, 0x0a, - 0xb7, 0xbc, 0x32, 0x74, 0x38, 0xb4, 0xd9, 0x75, 0x3a, 0x3b, 0xda, 0x1e, 0xce, 0x26, 0x16, 0xa4, - 0xeb, 0x33, 0xe5, 0xd5, 0xa7, 0xfd, 0x7c, 0x75, 0xe2, 0x9d, 0x4a, 0x70, 0x67, 0xd1, 0x75, 0x30, - 0x0e, 0x6c, 0xfc, 0x8a, 0xc0, 0x0b, 0xee, 0x59, 0x4f, 0x86, 0x14, 0x80, 0x01, 0xe5, 0x67, 0x93, - 0xcf, 0xec, 0x8f, 0x02, 0x28, 0xc8, 0x02, 0xe4, 0xe0, 0x07, 0x5a, 0xdb, 0x34, 0x34, 0x17, 0xab, - 0xa6, 0x65, 0xe0, 0x47, 0x98, 0x64, 0x11, 0x9b, 0xfa, 0x77, 0x27, 0x9f, 0x7a, 0xc5, 0xc7, 0xa8, - 0x53, 0x08, 0x61, 0x69, 0x19, 0x67, 0x58, 0x8c, 0x49, 0xee, 0x7f, 0x25, 0xc8, 0x8c, 0xf1, 0x1c, - 0xda, 0x86, 0x90, 0x7f, 0x40, 0xa2, 0xae, 0x26, 0xc4, 0x0e, 0x47, 0x17, 0xe1, 0xbc, 0x90, 0x69, - 0xa0, 0x3d, 0x88, 0x53, 0x93, 0xb4, 0x5c, 0x7a, 0xfa, 0x0a, 0x31, 0xf0, 0xc6, 0x51, 0x3f, 0x1f, - 0xdb, 0x62, 0xc2, 0x0b, 0x37, 0x11, 0xe3, 0xe0, 0x75, 0x03, 0xe5, 0x21, 0xe1, 0xda, 0x2a, 0x7e, - 0x64, 0x12, 0xd7, 0xb4, 0xf6, 0x58, 0x5c, 0x14, 0x53, 0xc0, 0xb5, 0x6b, 0x42, 0x92, 0xfb, 0xcb, - 0x10, 0xa0, 0x71, 0x2a, 0x44, 0xff, 0x20, 0xc1, 0x8b, 0x5e, 0xb0, 0x64, 0x3b, 0xe6, 0x9e, 0x69, - 0x69, 0xed, 0xa1, 0xa8, 0x49, 0x62, 0xeb, 0xf0, 0xe9, 0x45, 0xf8, 0x56, 0x44, 0x52, 0x9b, 0x02, - 0x7e, 0x34, 0xa2, 0x7a, 0x91, 0x3a, 0x76, 0x1e, 0x51, 0x8d, 0xa9, 0xdc, 0x53, 0xb2, 0xbd, 0x13, - 0x2a, 0xe7, 0xd6, 0xe0, 0xa5, 0x53, 0x81, 0xcf, 0xe3, 0xa6, 0x73, 0x3f, 0x94, 0xe0, 0xca, 0x09, - 0xce, 0x2f, 0x88, 0x93, 0xe2, 0x38, 0x77, 0x83, 0x38, 0x89, 0xa5, 0xef, 0x5e, 0xc0, 0xc1, 0x06, - 0x3b, 0xb1, 0x02, 0x2f, 0x9c, 0x48, 0x20, 0x67, 0x8d, 0x26, 0x16, 0x04, 0xfa, 0x37, 0x09, 0x66, - 0x47, 0xf6, 0x03, 0xfa, 0x24, 0x60, 0xe0, 0x75, 0x7a, 0xbe, 0x65, 0x8d, 0x3c, 0x17, 0x2b, 0x3f, - 0x18, 0xb7, 0xf2, 0x0d, 0xda, 0x02, 0x6b, 0x98, 0xb5, 0xf0, 0xc1, 0x33, 0xb7, 0xc0, 0x21, 0x06, - 0x96, 0xde, 0x08, 0xc7, 0x24, 0x39, 0x54, 0x78, 0x93, 0x0e, 0x90, 0x4d, 0x8f, 0x7f, 0xe0, 0x79, - 0x09, 0x60, 0xdf, 0xdc, 0xdb, 0x57, 0x1f, 0x6a, 0x2e, 0x76, 0x44, 0x6a, 0x21, 0x4e, 0x25, 0x1f, - 0x51, 0x41, 0xe1, 0xcf, 0x01, 0x52, 0xf5, 0x4e, 0xd7, 0x76, 0x5c, 0x2f, 0xea, 0xba, 0x03, 0x11, - 0x4e, 0xf8, 0xc2, 0xc8, 0x8b, 0xa7, 0x2c, 0xe3, 0x50, 0x4d, 0xee, 0xa7, 0x05, 0xc3, 0x08, 0x0c, - 0xb4, 0x09, 0x51, 0xee, 0x1c, 0x49, 0xf6, 0x0a, 0x83, 0x5b, 0x9c, 0x18, 0x8e, 0xbb, 0x59, 0xcf, - 0x37, 0x0a, 0x14, 0x3f, 0x3e, 0x0a, 0x1d, 0x1b, 0x1f, 0xbd, 0x0f, 0x11, 0x9e, 0x98, 0x12, 0x27, - 0xc7, 0xfc, 0x31, 0x47, 0xce, 0xfa, 0xe6, 0xb2, 0xd9, 0xc6, 0xcb, 0x4c, 0xcd, 0xeb, 0x2d, 0xaf, - 0x84, 0x5e, 0x81, 0x18, 0x21, 0xae, 0x4a, 0xcc, 0x1f, 0xf0, 0x88, 0x73, 0x9a, 0xe7, 0x3c, 0x9a, - 0xcd, 0xed, 0xa6, 0xf9, 0x03, 0xac, 0x44, 0x09, 0x71, 0xe9, 0x0f, 0x74, 0x15, 0x98, 0xf7, 0x25, - 0x1a, 0xf5, 0xa9, 0xcc, 0x7d, 0x4e, 0x2b, 0x01, 0x09, 0xc3, 0x39, 0x30, 0xbb, 0xea, 0xee, 0x01, - 0xe1, 0x3e, 0x4b, 0xe0, 0x1c, 0x98, 0xdd, 0xe5, 0x35, 0xa2, 0x44, 0x69, 0xe1, 0xf2, 0x01, 0x41, - 0x39, 0x88, 0x3d, 0xd4, 0xda, 0x6d, 0x16, 0xe1, 0xce, 0x30, 0x14, 0xff, 0x7b, 0x98, 0x24, 0x23, - 0x5f, 0x2d, 0x49, 0x8a, 0x98, 0xb2, 0xab, 0xb9, 0xfb, 0xec, 0x38, 0x12, 0x57, 0x80, 0x8b, 0xb6, - 0x34, 0x77, 0x1f, 0x65, 0x21, 0xca, 0xc7, 0x45, 0xb2, 0xb1, 0x85, 0xe9, 0xeb, 0x49, 0xc5, 0xfb, - 0x44, 0xaf, 0xc2, 0xac, 0xc9, 0x8e, 0xd8, 0xaa, 0x61, 0x3a, 0x58, 0x77, 0xdb, 0x87, 0xcc, 0xdf, - 0xc6, 0x94, 0x34, 0x17, 0x57, 0x85, 0x14, 0xdd, 0x00, 0x79, 0x34, 0x40, 0x61, 0x7e, 0x32, 0xa6, - 0xcc, 0x8e, 0xc4, 0x27, 0x34, 0x96, 0x11, 0x6b, 0x1d, 0x70, 0xfc, 0x59, 0x1e, 0xcb, 0x88, 0x82, - 0x81, 0xd3, 0xbf, 0x01, 0xb2, 0x88, 0x4e, 0x06, 0xba, 0x29, 0x8e, 0xcb, 0xe5, 0x03, 0xd5, 0x22, - 0xcc, 0x75, 0x35, 0x87, 0x60, 0xb5, 0xd5, 0xb3, 0x8c, 0x36, 0x56, 0x39, 0x56, 0x36, 0xcd, 0xb4, - 0x33, 0xac, 0xa8, 0xcc, 0x4a, 0xb8, 0xdd, 0x9d, 0x75, 0x6a, 0xbb, 0xfc, 0xff, 0x70, 0x6a, 0xcb, - 0xfd, 0x3c, 0x04, 0x33, 0x6c, 0x97, 0xa1, 0x5b, 0x10, 0xa6, 0x6b, 0x29, 0xf2, 0x0c, 0x93, 0x46, - 0xce, 0xac, 0x0e, 0x42, 0x10, 0xb6, 0xb4, 0x0e, 0xce, 0x22, 0xb6, 0xd2, 0xec, 0x37, 0xba, 0x02, - 0x51, 0x82, 0xef, 0xab, 0x0f, 0xb4, 0x76, 0x76, 0x8e, 0x19, 0x62, 0x84, 0xe0, 0xfb, 0xf7, 0xb4, - 0x36, 0xba, 0x04, 0x11, 0x93, 0xa8, 0x16, 0x7e, 0x98, 0x9d, 0xe7, 0x7c, 0x6a, 0x92, 0x0d, 0xfc, - 0x10, 0x7d, 0x0d, 0xe2, 0x0f, 0x35, 0xa2, 0xe2, 0x4e, 0xd7, 0x3d, 0x64, 0x73, 0x11, 0xa3, 0xa6, - 0x4b, 0x6a, 0xf4, 0x9b, 0xb9, 0x5d, 0xcd, 0xd9, 0xc3, 0xae, 0xaa, 0xdb, 0x6d, 0x92, 0xbd, 0x44, - 0xb7, 0x2a, 0x8d, 0xda, 0xa9, 0xa8, 0x62, 0xb7, 0x49, 0x23, 0x1c, 0x0b, 0xc9, 0xd3, 0x8d, 0x70, - 0x6c, 0x5a, 0x0e, 0x37, 0xc2, 0xb1, 0xb0, 0x3c, 0xd3, 0x08, 0xc7, 0x66, 0xe4, 0x48, 0x23, 0x1c, - 0x8b, 0xc8, 0xd1, 0x46, 0x38, 0x16, 0x95, 0x63, 0x8d, 0x70, 0x2c, 0x26, 0xc7, 0x1b, 0xe1, 0x58, - 0x5c, 0x86, 0x46, 0x38, 0x06, 0x72, 0xa2, 0x11, 0x8e, 0x25, 0xe4, 0x64, 0x23, 0x1c, 0x4b, 0xca, - 0xa9, 0x46, 0x38, 0x96, 0x92, 0xd3, 0x8d, 0x70, 0x2c, 0x2d, 0xcf, 0x36, 0xc2, 0xb1, 0x59, 0x59, - 0x6e, 0x84, 0x63, 0xb2, 0x9c, 0x69, 0x84, 0x63, 0x19, 0x19, 0xe5, 0x6a, 0x22, 0xb1, 0xa5, 0xa1, - 0xef, 0x0e, 0xcd, 0xd3, 0xc4, 0x11, 0x3d, 0xab, 0x54, 0xf8, 0x85, 0x04, 0x72, 0x13, 0xdf, 0xef, - 0x61, 0x4b, 0xc7, 0xf7, 0xb4, 0x76, 0x65, 0xbf, 0x67, 0x1d, 0xa0, 0x57, 0x60, 0x56, 0xa7, 0x3f, - 0x54, 0x9e, 0xed, 0xa1, 0x33, 0x26, 0xb1, 0x19, 0x4b, 0x31, 0x71, 0x93, 0x4a, 0xe9, 0xc4, 0xbd, - 0x04, 0x20, 0xf4, 0x28, 0x9b, 0xf0, 0x4c, 0x70, 0x9c, 0xab, 0x50, 0x0a, 0x19, 0x81, 0x71, 0xec, - 0x87, 0x8c, 0xb2, 0x86, 0x60, 0x14, 0xfb, 0x21, 0x5a, 0x84, 0x79, 0x0b, 0x3f, 0x72, 0xd5, 0x51, - 0x65, 0x46, 0x4f, 0x4a, 0x86, 0x96, 0x55, 0x82, 0x15, 0x0a, 0xff, 0x1a, 0x82, 0x59, 0xaf, 0xd3, - 0x1e, 0xa7, 0xef, 0x82, 0x4c, 0x57, 0xd7, 0x34, 0x54, 0xd7, 0xe6, 0x48, 0x1e, 0xbb, 0xbf, 0x7f, - 0x0a, 0x1d, 0x8f, 0xa0, 0xd0, 0xef, 0xba, 0xb1, 0x6d, 0xb3, 0xe6, 0x78, 0x10, 0xa0, 0xa4, 0x48, - 0x50, 0x96, 0xdb, 0x81, 0xb4, 0x57, 0x89, 0x4b, 0x50, 0x05, 0x22, 0x43, 0xed, 0xbd, 0x36, 0x41, - 0x7b, 0xde, 0x54, 0x2b, 0xa2, 0x6a, 0xee, 0xf7, 0x01, 0x8d, 0xb7, 0x1d, 0x74, 0xfd, 0x33, 0xdc, - 0xf5, 0x6f, 0x0e, 0x07, 0x20, 0xef, 0x9e, 0x6f, 0x6c, 0x81, 0x6e, 0x07, 0x53, 0x15, 0xff, 0x1c, - 0x82, 0x34, 0x77, 0x4c, 0xbe, 0x4f, 0xa5, 0x14, 0x45, 0x19, 0xd0, 0xb4, 0xf6, 0xd4, 0xae, 0x10, - 0xb2, 0xf1, 0x85, 0x14, 0xd9, 0x2b, 0xf0, 0x95, 0xbf, 0x01, 0x29, 0x07, 0x6b, 0xc6, 0x40, 0x31, - 0xc4, 0x14, 0x93, 0x54, 0xe8, 0x2b, 0x5d, 0x83, 0x34, 0x8b, 0x7b, 0x06, 0x5a, 0xd3, 0x4c, 0x2b, - 0xc5, 0xa4, 0xbe, 0x5a, 0x19, 0x52, 0xa4, 0xab, 0x59, 0x03, 0xad, 0x30, 0x9b, 0xd4, 0x33, 0x12, - 0xab, 0x49, 0x5a, 0x27, 0x18, 0x10, 0x38, 0x98, 0xf4, 0x3a, 0x58, 0xed, 0xda, 0xfc, 0xe4, 0x3d, - 0xad, 0xc4, 0xb9, 0x64, 0xcb, 0x26, 0x68, 0x87, 0x99, 0x0a, 0x9b, 0x0b, 0xd5, 0xe0, 0x93, 0x93, - 0x8d, 0xb0, 0x56, 0x6e, 0x4e, 0x3e, 0x9d, 0xca, 0x2c, 0x19, 0x16, 0x14, 0xfe, 0x56, 0x82, 0x2b, - 0xf4, 0xfc, 0xcd, 0x77, 0x5a, 0x85, 0xdd, 0xe1, 0x78, 0xd6, 0xa9, 0x41, 0x94, 0x9d, 0xe1, 0xfd, - 0x40, 0x6c, 0xf5, 0xa8, 0x9f, 0x8f, 0x50, 0xed, 0x0b, 0x7b, 0xb9, 0x08, 0x05, 0xae, 0xb3, 0x13, - 0xa5, 0xeb, 0x68, 0x16, 0x31, 0xe9, 0xd9, 0x8a, 0x2e, 0x5b, 0x07, 0x77, 0x5a, 0xd8, 0xe1, 0x8b, - 0x91, 0x54, 0xe6, 0x87, 0x0a, 0xd7, 0x79, 0x59, 0x21, 0x07, 0xd9, 0xd1, 0x2e, 0xfb, 0xe9, 0xaa, - 0xdf, 0x81, 0xcb, 0x1b, 0xf8, 0xe1, 0x71, 0xa3, 0x29, 0x43, 0x94, 0x33, 0x9d, 0x67, 0xf2, 0xd7, - 0x47, 0x49, 0x27, 0x78, 0x8d, 0x55, 0x64, 0x3d, 0xdd, 0x66, 0x15, 0x14, 0xaf, 0x62, 0xe1, 0x53, - 0xb8, 0x32, 0x82, 0xee, 0x2f, 0xdf, 0x07, 0x10, 0xa1, 0xe7, 0x74, 0x11, 0x9e, 0xa5, 0xc7, 0x29, - 0x6d, 0x1c, 0xbd, 0x49, 0xf5, 0x15, 0x51, 0xad, 0xa0, 0xb0, 0x3c, 0x5b, 0xaf, 0x83, 0xa9, 0x85, - 0xdc, 0x31, 0x89, 0x8b, 0x3e, 0x84, 0xa4, 0xb0, 0x08, 0x6a, 0x28, 0x5e, 0xb7, 0xcf, 0x30, 0xaa, - 0x84, 0xe3, 0x83, 0x90, 0xc2, 0xdf, 0x49, 0x30, 0x57, 0x75, 0xec, 0x6e, 0x17, 0x1b, 0xc2, 0xe5, - 0xf0, 0xb9, 0xf0, 0x3c, 0x8d, 0x14, 0xf0, 0x34, 0x1b, 0x10, 0xaa, 0x57, 0x45, 0x3c, 0x7c, 0xfb, - 0xa2, 0x61, 0x76, 0xbd, 0x8a, 0xde, 0xe5, 0x13, 0xd2, 0x23, 0x8c, 0x3f, 0xd3, 0x63, 0xe9, 0x96, - 0x21, 0x33, 0x65, 0x8a, 0x8a, 0xa8, 0x50, 0xf8, 0x59, 0x14, 0x2e, 0x05, 0x27, 0x79, 0xa5, 0xe2, - 0x75, 0xfc, 0x33, 0x88, 0x7a, 0x47, 0xee, 0x09, 0x78, 0xf2, 0x38, 0x88, 0xa2, 0x98, 0x8f, 0xe0, - 0xb1, 0xdb, 0xc3, 0x44, 0x4d, 0xc8, 0x98, 0x96, 0x8b, 0x9d, 0x36, 0xd6, 0x1e, 0xd0, 0xe0, 0x82, - 0xce, 0x99, 0x48, 0x71, 0x4e, 0xea, 0xca, 0xe5, 0x00, 0x00, 0x0f, 0x09, 0x3e, 0x83, 0xb9, 0x20, - 0xa8, 0xd7, 0xff, 0xd3, 0x73, 0x6b, 0xac, 0x7b, 0x03, 0x58, 0x2f, 0x09, 0x18, 0x00, 0x12, 0x09, - 0x02, 0xf4, 0xb1, 0x7f, 0x2e, 0xe0, 0xf9, 0xd3, 0x5b, 0xcf, 0x3c, 0x23, 0xd5, 0x91, 0x33, 0xc2, - 0x50, 0xa4, 0xcb, 0x9c, 0xe0, 0x57, 0x14, 0xe9, 0xde, 0x83, 0x08, 0x4f, 0xb1, 0x89, 0x3b, 0x83, - 0xdb, 0xcf, 0x3a, 0x04, 0x9e, 0xba, 0x53, 0x04, 0x5a, 0xee, 0xcf, 0x24, 0x48, 0x06, 0x97, 0x1b, - 0x99, 0x10, 0x63, 0xd3, 0xef, 0x51, 0xda, 0xf4, 0x73, 0x3f, 0xf9, 0x71, 0x53, 0xaa, 0x1b, 0x34, - 0x10, 0x33, 0x1c, 0xbb, 0x3b, 0xb8, 0x33, 0x9a, 0x56, 0x62, 0x54, 0x40, 0x43, 0xc7, 0xdc, 0x1f, - 0x40, 0xdc, 0x9f, 0xf4, 0x40, 0x2e, 0x67, 0xfa, 0x39, 0xe6, 0x72, 0x4e, 0x6d, 0xbf, 0x0a, 0xa9, - 0xa1, 0x19, 0x43, 0x97, 0xfd, 0x3e, 0x84, 0xcb, 0x11, 0xde, 0x87, 0x33, 0x51, 0x0a, 0xbf, 0x8e, - 0xc0, 0xdc, 0x71, 0x4c, 0xfb, 0x09, 0xc8, 0x01, 0xde, 0x52, 0xdb, 0x26, 0x71, 0x85, 0x6d, 0xde, - 0x38, 0x3d, 0xf5, 0x10, 0x20, 0x3f, 0x61, 0x8a, 0x69, 0x67, 0x98, 0x12, 0xbf, 0x07, 0x69, 0x83, - 0x77, 0x5c, 0x64, 0x3f, 0xc5, 0xcd, 0xef, 0x69, 0x87, 0xe1, 0x63, 0x08, 0x50, 0xa0, 0xa7, 0x8c, - 0x40, 0x11, 0x41, 0x3a, 0xa4, 0x7c, 0xf0, 0xc3, 0xae, 0x38, 0x55, 0x5d, 0x9c, 0x0c, 0x93, 0x5e, - 0x2b, 0x14, 0x13, 0xed, 0xc1, 0xac, 0xd7, 0x88, 0x77, 0x00, 0x8f, 0x3f, 0x97, 0x66, 0xbc, 0x89, - 0x69, 0x8a, 0x03, 0xf9, 0x8f, 0x24, 0x98, 0xf3, 0x5a, 0xf2, 0x6f, 0x09, 0x4c, 0x83, 0xd1, 0x59, - 0xaa, 0xdc, 0x3c, 0xea, 0xe7, 0x33, 0x62, 0x66, 0xbc, 0x6b, 0x81, 0x0b, 0xdb, 0x5d, 0xc6, 0x18, - 0x01, 0x34, 0x68, 0x0c, 0x41, 0xcb, 0x69, 0xc3, 0x33, 0x83, 0x18, 0x82, 0x12, 0xdb, 0xc5, 0x63, - 0x08, 0xfa, 0xb3, 0x6e, 0xa0, 0x3f, 0x96, 0x20, 0xc3, 0x2f, 0x2d, 0x3a, 0x3d, 0x57, 0xe3, 0xf7, - 0x83, 0xde, 0xc9, 0xfc, 0x93, 0xa3, 0x7e, 0x7e, 0x96, 0x2d, 0xef, 0xba, 0x28, 0x63, 0xcd, 0x96, - 0x9f, 0xb5, 0xd9, 0x01, 0x8a, 0x38, 0xc8, 0xfa, 0x02, 0x03, 0xad, 0x41, 0x9a, 0xa7, 0x2b, 0xbc, - 0xa7, 0x2d, 0xec, 0xc8, 0x9e, 0x2a, 0xbf, 0xfc, 0xb4, 0x9f, 0x5f, 0x38, 0x66, 0x9f, 0xf0, 0x4c, - 0xc7, 0x3d, 0xae, 0xab, 0xa4, 0x76, 0x83, 0x9f, 0x22, 0x6f, 0x74, 0x19, 0xe6, 0x8f, 0x8d, 0x72, - 0xbe, 0x88, 0xc0, 0xe5, 0x61, 0x22, 0xf4, 0xe3, 0x10, 0x75, 0xd4, 0x43, 0x7e, 0x30, 0x31, 0x99, - 0x7a, 0x18, 0x9c, 0xcc, 0xbc, 0xaf, 0x51, 0x1f, 0xf9, 0xd9, 0x88, 0xbf, 0x79, 0x06, 0x7c, 0xb6, - 0x20, 0x23, 0xf8, 0x9e, 0xd3, 0xf9, 0xd8, 0xf7, 0x05, 0x3c, 0x53, 0xf4, 0xe1, 0x33, 0xc0, 0xb3, - 0xfa, 0xde, 0xa7, 0xef, 0x0d, 0xfe, 0x45, 0x82, 0xd4, 0xd0, 0xc8, 0x7e, 0x9b, 0xee, 0x60, 0xcb, - 0x8f, 0x86, 0xf8, 0x93, 0xa1, 0x77, 0xce, 0x3f, 0xac, 0xe1, 0x20, 0x29, 0xf7, 0xf7, 0x12, 0xa4, - 0x86, 0x26, 0xf2, 0x2b, 0x72, 0x24, 0xcf, 0xbf, 0xe7, 0x2d, 0x48, 0x0f, 0x2f, 0x51, 0xa0, 0x0d, - 0xe9, 0xf9, 0xb4, 0x51, 0xf8, 0x0e, 0x44, 0xb8, 0x04, 0x21, 0x48, 0x7f, 0x54, 0xaa, 0x6f, 0xd7, - 0x37, 0x56, 0xd4, 0xe5, 0x4d, 0x45, 0x5d, 0xa9, 0xc8, 0x53, 0x28, 0x09, 0xb1, 0x6a, 0xed, 0x4e, - 0x8d, 0x0a, 0x65, 0x09, 0x25, 0x20, 0xca, 0xbe, 0x6a, 0x55, 0x39, 0x54, 0x28, 0x83, 0xcc, 0xb1, - 0x77, 0x31, 0x75, 0x0c, 0x34, 0xee, 0x47, 0x45, 0x98, 0x63, 0x41, 0x7a, 0x87, 0xc6, 0x42, 0xd4, - 0x15, 0xaa, 0x81, 0xe8, 0x39, 0xe3, 0x17, 0x51, 0xa7, 0xb8, 0xa1, 0x75, 0x70, 0xe1, 0x97, 0x61, - 0xc8, 0x0c, 0x40, 0x3c, 0xb7, 0xf8, 0x37, 0xd2, 0xe0, 0x04, 0x12, 0x39, 0xf3, 0xbe, 0x68, 0xac, - 0xbe, 0x38, 0x8c, 0x88, 0x2b, 0xee, 0x8f, 0xe8, 0xa6, 0x79, 0xda, 0xcf, 0x67, 0x46, 0x3b, 0x4b, - 0x2e, 0x78, 0xf7, 0xed, 0x75, 0x91, 0xe5, 0x4a, 0x4d, 0xeb, 0x40, 0x1d, 0x3c, 0x5c, 0xe1, 0xb9, - 0x52, 0xd3, 0x3a, 0xd8, 0x51, 0xea, 0x4a, 0x94, 0x16, 0xee, 0x38, 0x26, 0x6a, 0x40, 0xd8, 0xee, - 0xba, 0xde, 0x91, 0xf7, 0xed, 0x73, 0x0d, 0x69, 0xb3, 0x2b, 0xc6, 0xa3, 0x30, 0x0c, 0xd4, 0xe0, - 0x57, 0xa0, 0x83, 0x89, 0x66, 0x14, 0x3a, 0xe1, 0xfb, 0x82, 0xd4, 0xd0, 0x42, 0xe4, 0xf6, 0x20, - 0x19, 0x9c, 0xb1, 0x63, 0xee, 0x45, 0x4a, 0xc3, 0x69, 0x89, 0xd7, 0x26, 0xea, 0xba, 0x38, 0x12, - 0x06, 0xae, 0x2f, 0xbe, 0x03, 0x71, 0x7f, 0x1c, 0xe7, 0xb9, 0xc5, 0xe1, 0x1c, 0xef, 0xe7, 0xdc, - 0x66, 0xe4, 0x48, 0xa1, 0x1f, 0x82, 0xa4, 0x82, 0x89, 0xdd, 0x7e, 0x80, 0x0d, 0x1a, 0xf3, 0xf8, - 0x0f, 0xb5, 0xa4, 0xc9, 0x1f, 0x6a, 0x95, 0x20, 0xee, 0x27, 0x45, 0xcf, 0xf3, 0x58, 0x69, 0x50, - 0x0b, 0xdd, 0x86, 0xaf, 0x05, 0x5f, 0x03, 0xd9, 0x3d, 0xcb, 0xd0, 0x9c, 0x43, 0xd5, 0xc1, 0x9a, - 0xbe, 0x8f, 0x0d, 0x71, 0x5d, 0xf7, 0x42, 0xe0, 0x39, 0x90, 0xd0, 0x50, 0xb8, 0x02, 0xfa, 0x04, - 0x52, 0x7e, 0x25, 0x1a, 0x49, 0xb1, 0x98, 0x23, 0xbd, 0xf4, 0xad, 0xd3, 0xa3, 0x3f, 0x7f, 0xd4, - 0x45, 0x0f, 0x8f, 0x46, 0x4c, 0x4a, 0xb2, 0x15, 0xf8, 0x2a, 0xbc, 0x0f, 0xc9, 0x60, 0x29, 0x8a, - 0x41, 0x78, 0x63, 0x73, 0xa3, 0xc6, 0xf7, 0x74, 0xb9, 0x54, 0x59, 0x5b, 0xae, 0xdf, 0xb9, 0x23, - 0x4b, 0x54, 0x5e, 0xfb, 0xb8, 0xbe, 0x2d, 0x87, 0xe8, 0xee, 0x56, 0x6a, 0xcd, 0xed, 0x92, 0xb2, - 0x2d, 0x4f, 0x17, 0x30, 0xa4, 0x82, 0x2d, 0x51, 0xce, 0xa4, 0x21, 0x26, 0x13, 0x0c, 0x9d, 0xb2, - 0x5f, 0x9d, 0xb0, 0xaf, 0x9e, 0xed, 0x39, 0x41, 0xd4, 0xc2, 0x8f, 0x43, 0x80, 0x06, 0x26, 0x13, - 0x20, 0xe8, 0xd1, 0xc6, 0x42, 0x17, 0x6f, 0x0c, 0xfd, 0xe4, 0xf4, 0x84, 0xf8, 0x34, 0x4b, 0x88, - 0x33, 0xbe, 0xf8, 0xad, 0x26, 0xc5, 0x45, 0x08, 0xf3, 0x3f, 0x61, 0x40, 0x15, 0x07, 0x6b, 0x2e, - 0xa6, 0x7c, 0x4c, 0x4e, 0xcb, 0x40, 0x94, 0x61, 0x86, 0x9f, 0xb8, 0x43, 0xe7, 0x39, 0x71, 0x8b, - 0x49, 0xe1, 0x55, 0xd1, 0xf7, 0x21, 0xa9, 0xdb, 0xed, 0x5e, 0xc7, 0x52, 0xd9, 0xab, 0x09, 0x71, - 0x3c, 0xf8, 0xf6, 0x69, 0x5b, 0x7b, 0xac, 0x73, 0xc5, 0x8a, 0xdd, 0xa6, 0xdf, 0xfe, 0xfb, 0x42, - 0x06, 0xc8, 0x34, 0xd0, 0x8b, 0x10, 0xf7, 0x69, 0x86, 0x99, 0x75, 0x5c, 0x19, 0x08, 0xd0, 0x12, - 0xcc, 0x68, 0x44, 0xb5, 0x77, 0x59, 0xac, 0x7b, 0xd6, 0xbe, 0x53, 0xc2, 0x1a, 0xd9, 0xdc, 0x45, - 0x37, 0x21, 0xd3, 0xd1, 0x1e, 0xa9, 0xbb, 0x8e, 0xa6, 0x8b, 0xd8, 0xb5, 0xcd, 0x69, 0x4f, 0x52, - 0x66, 0x3b, 0xda, 0xa3, 0x65, 0x21, 0xaf, 0x1b, 0x6d, 0x8c, 0xde, 0x82, 0xd4, 0xee, 0x7d, 0x7e, - 0xf2, 0xe1, 0x2e, 0x88, 0x3f, 0x41, 0x99, 0x3d, 0xea, 0xe7, 0x13, 0xcb, 0x77, 0xd9, 0xc4, 0x50, - 0x07, 0xa4, 0x24, 0x76, 0xef, 0xfb, 0x1f, 0xb9, 0xff, 0x96, 0x20, 0x2a, 0x46, 0x84, 0xba, 0x00, - 0x62, 0x7a, 0x4c, 0x83, 0x9b, 0x7a, 0xaa, 0x7c, 0xf7, 0xa8, 0x9f, 0x8f, 0x57, 0x98, 0xb4, 0x5e, - 0x25, 0x4f, 0xfb, 0xf9, 0x0f, 0x9f, 0xd5, 0x7d, 0x78, 0x20, 0x4a, 0x9c, 0x37, 0x52, 0x37, 0x58, - 0x9a, 0x75, 0x5f, 0x23, 0xea, 0xbe, 0x49, 0x5c, 0x7b, 0xcf, 0xd1, 0x3a, 0xe2, 0xfa, 0x37, 0xb9, - 0xaf, 0x91, 0x55, 0x4f, 0x86, 0x72, 0x34, 0x10, 0x7b, 0xc0, 0x1f, 0xbd, 0x70, 0x76, 0xf1, 0xbf, - 0xd1, 0x12, 0x5c, 0xf2, 0x2b, 0xab, 0x74, 0xa6, 0x5a, 0x3d, 0xfd, 0x00, 0x33, 0x87, 0x43, 0x99, - 0x7c, 0xce, 0x2f, 0x5c, 0xd7, 0x1e, 0x95, 0x79, 0x51, 0xe1, 0x12, 0xcc, 0x05, 0x96, 0xd5, 0x0f, - 0x9b, 0x31, 0xc8, 0xeb, 0xe6, 0x9e, 0xa3, 0x05, 0x5f, 0xcc, 0xde, 0x85, 0xd9, 0x91, 0x17, 0xe9, - 0x82, 0x6c, 0x83, 0xe9, 0xc1, 0xe1, 0x27, 0xec, 0xc5, 0x0a, 0xff, 0xf4, 0x42, 0xf7, 0xb4, 0x3e, - 0xf4, 0x5d, 0x98, 0x83, 0x8c, 0xdf, 0x8c, 0xdf, 0xf6, 0x6f, 0x92, 0x10, 0xdd, 0xd2, 0x0e, 0xdb, - 0xb6, 0x66, 0xa0, 0x05, 0x48, 0x78, 0xaf, 0x55, 0xbc, 0xf6, 0xe2, 0x4a, 0x50, 0x34, 0x6c, 0x66, - 0xf2, 0xa8, 0x99, 0x99, 0x90, 0xee, 0x11, 0xec, 0x50, 0x0b, 0x50, 0xd9, 0xf3, 0x79, 0xee, 0x5b, - 0xca, 0xe5, 0xa7, 0xfd, 0xfc, 0xed, 0xc9, 0x16, 0x0f, 0xeb, 0x3d, 0xc7, 0x74, 0x0f, 0x8b, 0xcd, - 0xbb, 0x77, 0x76, 0x04, 0x14, 0xdd, 0xe2, 0xb6, 0x92, 0xea, 0x05, 0x3f, 0xc5, 0xa3, 0x24, 0xba, - 0x10, 0x6a, 0xc7, 0xd4, 0x1d, 0x9b, 0x78, 0xb7, 0x21, 0x42, 0xba, 0xce, 0x84, 0xe8, 0x55, 0x98, - 0xdd, 0x35, 0x2d, 0x76, 0xa1, 0xe7, 0xe9, 0xf1, 0x8b, 0x90, 0xb4, 0x27, 0x16, 0x8a, 0x0f, 0x20, - 0x1d, 0x78, 0x0d, 0x44, 0x8d, 0x30, 0xc2, 0x8c, 0x70, 0xf3, 0xa8, 0x9f, 0x4f, 0x0d, 0x36, 0x35, - 0x37, 0xc4, 0x8b, 0xc4, 0x31, 0xa9, 0x41, 0x33, 0xd4, 0x0c, 0xe7, 0x61, 0x86, 0xfd, 0x71, 0x05, - 0x7f, 0x4e, 0xa9, 0xf0, 0x0f, 0x54, 0x83, 0x94, 0xc8, 0x54, 0xf0, 0xbf, 0xbc, 0x10, 0x4f, 0xa8, - 0x16, 0x02, 0x4b, 0xef, 0xfd, 0x6d, 0x46, 0xb1, 0x66, 0xe9, 0xb6, 0x81, 0x8d, 0x1a, 0xfd, 0x56, - 0x44, 0x62, 0x96, 0x7d, 0x10, 0xb4, 0x02, 0x69, 0xbd, 0x8d, 0x35, 0xab, 0xd7, 0xf5, 0x70, 0xd0, - 0x84, 0x38, 0x29, 0x51, 0x4f, 0x00, 0x6d, 0x00, 0xda, 0x65, 0x6f, 0x5a, 0x82, 0xbd, 0x62, 0x17, - 0x7f, 0x93, 0x80, 0xc9, 0xac, 0xae, 0x32, 0xe8, 0x19, 0x7a, 0x1b, 0x66, 0xda, 0x58, 0x23, 0x58, - 0xbc, 0x95, 0x5d, 0x38, 0x85, 0x06, 0xd9, 0x5f, 0x5f, 0x28, 0x5c, 0x1d, 0xbd, 0x0c, 0x29, 0xcb, - 0xb6, 0x74, 0xcd, 0xd2, 0x71, 0x9b, 0x31, 0x32, 0xbf, 0x63, 0x1c, 0x16, 0xa2, 0x32, 0x44, 0xf8, - 0x6d, 0x34, 0xbb, 0x4b, 0x1f, 0x4f, 0xa8, 0x9f, 0xf8, 0x58, 0x79, 0x75, 0x4a, 0x11, 0x35, 0x51, - 0x0d, 0xa2, 0xe2, 0xfd, 0x1a, 0xbb, 0xa1, 0x3e, 0x33, 0x45, 0x14, 0x78, 0x9d, 0xb2, 0x3a, 0xa5, - 0x78, 0x75, 0xd1, 0xb6, 0xf7, 0x50, 0x90, 0x7b, 0x5d, 0xf1, 0xd6, 0xab, 0x38, 0xe1, 0x31, 0x62, - 0x00, 0x38, 0x84, 0x42, 0x07, 0x68, 0xb2, 0x1b, 0x26, 0x76, 0x77, 0x7d, 0xfa, 0x00, 0x87, 0xde, - 0x48, 0xd0, 0x01, 0xf2, 0x9a, 0x68, 0x03, 0x40, 0xf7, 0x23, 0x01, 0x76, 0xab, 0x9d, 0x58, 0x7a, - 0xfd, 0x3c, 0x41, 0xf2, 0xea, 0x94, 0x12, 0x40, 0x40, 0x77, 0x21, 0xa1, 0x0f, 0xa8, 0x2d, 0x3b, - 0xcb, 0x00, 0xdf, 0x38, 0x97, 0x7f, 0x5b, 0xa5, 0x3e, 0x6d, 0x20, 0x45, 0x9f, 0x42, 0x9a, 0x0c, - 0x1d, 0xaa, 0xb2, 0x97, 0x18, 0xea, 0x9b, 0xe7, 0x4d, 0xc3, 0xae, 0x4e, 0x29, 0x23, 0x48, 0xe8, - 0x77, 0x41, 0x76, 0x47, 0xee, 0x6a, 0xd8, 0xb5, 0xf4, 0xe9, 0x0f, 0xf1, 0x4e, 0xb8, 0x91, 0x5a, - 0x9d, 0x52, 0xc6, 0xd0, 0xd0, 0x67, 0x30, 0x4b, 0x86, 0xff, 0xa8, 0x20, 0x7b, 0x85, 0x35, 0xf0, - 0xcd, 0x53, 0x2f, 0x1c, 0x8e, 0xfb, 0xf3, 0x89, 0xd5, 0x29, 0x65, 0x14, 0x8b, 0xc2, 0x5b, 0xc3, - 0x57, 0x3e, 0xec, 0xd1, 0xc3, 0xe9, 0xf0, 0xc7, 0x5f, 0x41, 0x51, 0xf8, 0x11, 0x2c, 0xb4, 0x06, - 0xf1, 0x8e, 0xe7, 0x2b, 0xb2, 0x2f, 0x9c, 0x79, 0x0e, 0x19, 0x75, 0x5f, 0xab, 0x53, 0xca, 0xa0, - 0x7e, 0x39, 0x0e, 0x51, 0x71, 0x35, 0xe8, 0xdf, 0xdb, 0x47, 0xe5, 0x58, 0xe1, 0x37, 0x31, 0x88, - 0xf9, 0x31, 0xe8, 0x22, 0x20, 0x3f, 0xd4, 0x18, 0xbc, 0x49, 0xa5, 0x2e, 0x28, 0xb4, 0x3a, 0xa5, - 0x64, 0xbc, 0xb2, 0xc1, 0xb3, 0xd4, 0xdb, 0x43, 0x0f, 0x95, 0x26, 0xf9, 0xc3, 0x0f, 0xda, 0x29, - 0xff, 0x25, 0x13, 0x75, 0x0d, 0x1d, 0xdb, 0x30, 0x77, 0xcd, 0x81, 0x6b, 0xe0, 0x89, 0xe4, 0xb4, - 0x27, 0x16, 0xae, 0xe1, 0x1a, 0xa4, 0x9d, 0x9e, 0xc5, 0x6e, 0x01, 0x45, 0x32, 0x80, 0xc7, 0x57, - 0x29, 0x21, 0x15, 0xe7, 0xf9, 0xca, 0x08, 0xeb, 0xdc, 0x38, 0x93, 0x75, 0xbc, 0xb1, 0xaf, 0x4a, - 0x3e, 0xed, 0x2c, 0x8f, 0xd2, 0xce, 0xcd, 0xb3, 0x69, 0x27, 0x00, 0xe3, 0xf3, 0xce, 0xce, 0xb1, - 0xbc, 0xb3, 0x38, 0xe1, 0xc6, 0x09, 0x20, 0x0e, 0x13, 0x4f, 0x65, 0x84, 0x78, 0x6e, 0x9c, 0x49, - 0x3c, 0xc1, 0x31, 0x0a, 0xe6, 0xd9, 0x3c, 0x86, 0x79, 0xde, 0x98, 0x88, 0x79, 0x02, 0x60, 0x41, - 0xea, 0x51, 0x8e, 0xa3, 0x9e, 0xe2, 0x64, 0xd4, 0x13, 0x80, 0x1c, 0xe2, 0x9e, 0xef, 0x8d, 0x71, - 0x8f, 0x7c, 0xf6, 0xe6, 0x3d, 0x36, 0x03, 0xb4, 0x2a, 0x8d, 0x91, 0x8f, 0x76, 0x0c, 0xf9, 0x64, - 0x18, 0xfc, 0x5b, 0xe7, 0x20, 0x9f, 0x40, 0x03, 0xe3, 0xec, 0xf3, 0x31, 0x24, 0x83, 0x8c, 0xc1, - 0xde, 0xee, 0x9c, 0xce, 0x6d, 0x27, 0xfc, 0x05, 0x14, 0xb3, 0x81, 0x40, 0x11, 0xfa, 0xfe, 0x38, - 0xf1, 0xcc, 0x9d, 0x09, 0x7e, 0xc2, 0xed, 0xf4, 0xaa, 0x34, 0xce, 0x3c, 0x77, 0x82, 0xcc, 0x33, - 0x7f, 0xa6, 0x5f, 0x1a, 0x8b, 0x68, 0x57, 0xa5, 0x20, 0xf5, 0x00, 0xc4, 0xbc, 0xc7, 0x0f, 0x01, - 0x1a, 0x2a, 0xfc, 0x54, 0x82, 0xe9, 0x86, 0xdd, 0x42, 0x2f, 0x05, 0x52, 0x93, 0x29, 0x71, 0x14, - 0x9d, 0x69, 0xd8, 0x2d, 0x91, 0x63, 0xfc, 0x60, 0x50, 0x5b, 0x1c, 0xfe, 0xbe, 0x71, 0x4a, 0x57, - 0xfc, 0xcc, 0xae, 0x5f, 0x09, 0xbd, 0x07, 0xd1, 0x2e, 0x0f, 0xae, 0x05, 0x43, 0x15, 0x4e, 0xab, - 0xcf, 0x35, 0x15, 0xaf, 0xca, 0xcd, 0x1b, 0xc1, 0x3f, 0x69, 0x5c, 0xb7, 0x0d, 0x8c, 0xd2, 0x00, - 0x5b, 0x1a, 0x21, 0xdd, 0x7d, 0x47, 0x23, 0x58, 0x9e, 0x42, 0x51, 0x98, 0x5e, 0x5b, 0x6f, 0xca, - 0xd2, 0xcd, 0x8f, 0x83, 0x79, 0xc5, 0xaa, 0x52, 0xaa, 0x6f, 0xd4, 0x37, 0x56, 0xd4, 0x8d, 0xd2, - 0x7a, 0xad, 0x29, 0x4f, 0xa1, 0x2c, 0xcc, 0x7f, 0x54, 0xaa, 0x6f, 0x8b, 0x44, 0xa3, 0x5a, 0xdf, - 0xd8, 0xae, 0x29, 0xf7, 0x4a, 0x77, 0x64, 0x09, 0x5d, 0x06, 0xa4, 0x6c, 0x56, 0xd6, 0x9a, 0xd5, - 0xb2, 0x5a, 0xd9, 0x5c, 0xdf, 0x2a, 0x55, 0xb6, 0xeb, 0x9b, 0x1b, 0x72, 0x08, 0xc5, 0x20, 0x5c, - 0xdd, 0xdc, 0xa8, 0xc9, 0x70, 0xf3, 0x27, 0x61, 0x08, 0xb3, 0x94, 0xc6, 0xcb, 0x90, 0xd8, 0xd9, - 0x68, 0x6e, 0xd5, 0x2a, 0xf5, 0xe5, 0x7a, 0xad, 0x2a, 0x4f, 0xe5, 0xe6, 0x1e, 0x3f, 0x59, 0x98, - 0xa5, 0x45, 0x3b, 0x16, 0xe9, 0x62, 0x9d, 0x71, 0x26, 0xca, 0x41, 0xa4, 0x5c, 0xaa, 0xac, 0xed, - 0x6c, 0xc9, 0x52, 0x2e, 0xfd, 0xf8, 0xc9, 0x02, 0x50, 0x05, 0xce, 0x77, 0xe8, 0x45, 0x9e, 0xf2, - 0xd8, 0x54, 0x6a, 0x72, 0x28, 0x37, 0xfb, 0xf8, 0xc9, 0x42, 0x82, 0x65, 0x52, 0x04, 0x67, 0xbd, - 0x0a, 0xa9, 0x66, 0x65, 0xb5, 0xb6, 0x5e, 0x52, 0x2b, 0xab, 0xa5, 0x8d, 0x95, 0x9a, 0x3c, 0x9d, - 0x9b, 0x7f, 0xfc, 0x64, 0x41, 0x1e, 0xb5, 0x7b, 0xda, 0x44, 0x7d, 0x7d, 0x6b, 0x53, 0xd9, 0x96, - 0xc3, 0x83, 0x26, 0x38, 0xdd, 0xa0, 0x02, 0x00, 0xaf, 0xbd, 0x5c, 0xab, 0x55, 0xe5, 0x99, 0x1c, - 0x7a, 0xfc, 0x64, 0x21, 0x4d, 0xcb, 0x07, 0x2c, 0x82, 0xae, 0x41, 0xb2, 0xa2, 0xd4, 0x4a, 0xdb, - 0x35, 0xb5, 0xb9, 0x5d, 0xda, 0x6e, 0xca, 0x91, 0xc1, 0x48, 0x02, 0xcc, 0x80, 0x8a, 0x90, 0x29, - 0xed, 0x6c, 0x6f, 0xaa, 0x43, 0xba, 0xd1, 0xdc, 0x95, 0xc7, 0x4f, 0x16, 0xe6, 0xa8, 0x6e, 0xa9, - 0xe7, 0xda, 0x41, 0xfd, 0xd7, 0x41, 0x1e, 0xea, 0xbf, 0xba, 0x52, 0x91, 0x63, 0xb9, 0xcb, 0x8f, - 0x9f, 0x2c, 0xa0, 0xd1, 0x21, 0xac, 0x54, 0xd0, 0xb7, 0xe0, 0xf2, 0xf6, 0x27, 0x5b, 0xb5, 0x6a, - 0xad, 0x59, 0x51, 0x87, 0x87, 0x1d, 0xcf, 0x65, 0x1f, 0x3f, 0x59, 0x98, 0xa7, 0x75, 0xc6, 0x86, - 0xfe, 0x06, 0xc8, 0xcd, 0x6d, 0xa5, 0x56, 0x5a, 0x57, 0xeb, 0x1b, 0x2b, 0xb5, 0x26, 0x5b, 0x2c, - 0x18, 0x74, 0x69, 0x64, 0x0f, 0xd3, 0x21, 0x6c, 0xd4, 0x3e, 0x1a, 0xc1, 0x4f, 0x0c, 0xf4, 0x47, - 0xb6, 0x25, 0x5a, 0x80, 0xf8, 0x7a, 0x7d, 0x45, 0x29, 0x31, 0xdc, 0x64, 0x2e, 0xf3, 0xf8, 0xc9, - 0x42, 0x8a, 0xea, 0xf9, 0x9b, 0x2c, 0x17, 0xfb, 0xf1, 0x4f, 0xaf, 0x4e, 0xfd, 0xf5, 0xcf, 0xae, - 0x4e, 0x95, 0xaf, 0x7f, 0xfe, 0x9f, 0x57, 0xa7, 0x3e, 0x3f, 0xba, 0x2a, 0xfd, 0xea, 0xe8, 0xaa, - 0xf4, 0xc5, 0xd1, 0x55, 0xe9, 0x3f, 0x8e, 0xae, 0x4a, 0x7f, 0xf2, 0xe5, 0xd5, 0xa9, 0x5f, 0x7d, - 0x79, 0x75, 0xea, 0x8b, 0x2f, 0xaf, 0x4e, 0x7d, 0x1a, 0xe1, 0x76, 0xdd, 0x8a, 0xb0, 0xb3, 0xe0, - 0x5b, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0xb8, 0x4c, 0x87, 0x4d, 0x83, 0x3e, 0x00, 0x00, + 0xdd, 0x44, 0xed, 0x98, 0x7b, 0x0e, 0x25, 0xe5, 0x6c, 0x86, 0x0d, 0xab, 0x32, 0xb9, 0xb1, 0x35, + 0x19, 0x0e, 0x5b, 0x46, 0xb2, 0x2e, 0x50, 0x78, 0xe0, 0x33, 0x4f, 0x8e, 0x29, 0x42, 0x6f, 0xc1, + 0xa5, 0xc1, 0xce, 0x20, 0x6a, 0xb7, 0xd7, 0x6a, 0x9b, 0x64, 0x1f, 0x1b, 0x59, 0x60, 0x1d, 0x9d, + 0x0f, 0x14, 0x6e, 0x79, 0x65, 0xe8, 0x70, 0x68, 0xb3, 0xeb, 0x74, 0x76, 0xb4, 0x3d, 0x9c, 0x4d, + 0x2c, 0x48, 0xd7, 0x67, 0xca, 0xab, 0x4f, 0xfb, 0xf9, 0xea, 0xc4, 0x3b, 0x95, 0xe0, 0xce, 0xa2, + 0xeb, 0x60, 0x1c, 0xd8, 0xf8, 0x15, 0x81, 0x17, 0xdc, 0xb3, 0x9e, 0x0c, 0x29, 0x00, 0x03, 0xca, + 0xcf, 0x26, 0x9f, 0xd9, 0x1f, 0x05, 0x50, 0x90, 0x05, 0xc8, 0xc1, 0x0f, 0xb4, 0xb6, 0x69, 0x68, + 0x2e, 0x56, 0x4d, 0xcb, 0xc0, 0x8f, 0x30, 0xc9, 0x22, 0x36, 0xf5, 0xef, 0x4e, 0x3e, 0xf5, 0x8a, + 0x8f, 0x51, 0xa7, 0x10, 0xc2, 0xd2, 0x32, 0xce, 0xb0, 0x18, 0x93, 0xdc, 0xff, 0x4a, 0x90, 0x19, + 0xe3, 0x39, 0xb4, 0x0d, 0x21, 0xff, 0x80, 0x44, 0x5d, 0x4d, 0x88, 0x1d, 0x8e, 0x2e, 0xc2, 0x79, + 0x21, 0xd3, 0x40, 0x7b, 0x10, 0xa7, 0x26, 0x69, 0xb9, 0xf4, 0xf4, 0x15, 0x62, 0xe0, 0x8d, 0xa3, + 0x7e, 0x3e, 0xb6, 0xc5, 0x84, 0x17, 0x6e, 0x22, 0xc6, 0xc1, 0xeb, 0x06, 0xca, 0x43, 0xc2, 0xb5, + 0x55, 0xfc, 0xc8, 0x24, 0xae, 0x69, 0xed, 0xb1, 0xb8, 0x28, 0xa6, 0x80, 0x6b, 0xd7, 0x84, 0x24, + 0xf7, 0x97, 0x21, 0x40, 0xe3, 0x54, 0x88, 0xfe, 0x41, 0x82, 0x17, 0xbd, 0x60, 0xc9, 0x76, 0xcc, + 0x3d, 0xd3, 0xd2, 0xda, 0x43, 0x51, 0x93, 0xc4, 0xd6, 0xe1, 0xd3, 0x8b, 0xf0, 0xad, 0x88, 0xa4, + 0x36, 0x05, 0xfc, 0x68, 0x44, 0xf5, 0x22, 0x75, 0xec, 0x3c, 0xa2, 0x1a, 0x53, 0xb9, 0xa7, 0x64, + 0x7b, 0x27, 0x54, 0xce, 0xad, 0xc1, 0x4b, 0xa7, 0x02, 0x9f, 0xc7, 0x4d, 0xe7, 0x7e, 0x28, 0xc1, + 0x95, 0x13, 0x9c, 0x5f, 0x10, 0x27, 0xc5, 0x71, 0xee, 0x06, 0x71, 0x12, 0x4b, 0xdf, 0xbd, 0x80, + 0x83, 0x0d, 0x76, 0x62, 0x05, 0x5e, 0x38, 0x91, 0x40, 0xce, 0x1a, 0x4d, 0x2c, 0x08, 0xf4, 0x6f, + 0x12, 0xcc, 0x8e, 0xec, 0x07, 0xf4, 0x49, 0xc0, 0xc0, 0xeb, 0xf4, 0x7c, 0xcb, 0x1a, 0x79, 0x2e, + 0x56, 0x7e, 0x30, 0x6e, 0xe5, 0x1b, 0xb4, 0x05, 0xd6, 0x30, 0x6b, 0xe1, 0x83, 0x67, 0x6e, 0x81, + 0x43, 0x0c, 0x2c, 0xbd, 0x11, 0x8e, 0x49, 0x72, 0xa8, 0xf0, 0x26, 0x1d, 0x20, 0x9b, 0x59, 0xff, + 0xc0, 0xf3, 0x12, 0xc0, 0xbe, 0xb9, 0xb7, 0xaf, 0x3e, 0xd4, 0x5c, 0xec, 0x88, 0xd4, 0x42, 0x9c, + 0x4a, 0x3e, 0xa2, 0x82, 0xc2, 0x9f, 0x03, 0xa4, 0xea, 0x9d, 0xae, 0xed, 0xb8, 0x5e, 0xd4, 0x75, + 0x07, 0x22, 0x9c, 0xf0, 0x85, 0x91, 0x17, 0x4f, 0x59, 0xc6, 0xa1, 0x9a, 0xdc, 0x4f, 0x0b, 0x86, + 0x11, 0x18, 0x68, 0x13, 0xa2, 0xdc, 0x39, 0x92, 0xec, 0x15, 0x06, 0xb7, 0x38, 0x31, 0x1c, 0x77, + 0xb3, 0x9e, 0x6f, 0x14, 0x28, 0x7e, 0x7c, 0x14, 0x3a, 0x36, 0x3e, 0x7a, 0x1f, 0x22, 0x3c, 0x31, + 0x25, 0x4e, 0x8e, 0xf9, 0x63, 0x8e, 0x9c, 0xf5, 0xcd, 0x65, 0xb3, 0x8d, 0x97, 0x99, 0x9a, 0xd7, + 0x5b, 0x5e, 0x09, 0xbd, 0x02, 0x31, 0x42, 0x5c, 0x95, 0x98, 0x3f, 0xe0, 0x11, 0xe7, 0x34, 0xcf, + 0x79, 0x34, 0x9b, 0xdb, 0x4d, 0xf3, 0x07, 0x58, 0x89, 0x12, 0xe2, 0xd2, 0x1f, 0xe8, 0x2a, 0x30, + 0xef, 0x4b, 0x34, 0xea, 0x53, 0x99, 0xfb, 0x9c, 0x56, 0x02, 0x12, 0x86, 0x73, 0x60, 0x76, 0xd5, + 0xdd, 0x03, 0xc2, 0x7d, 0x96, 0xc0, 0x39, 0x30, 0xbb, 0xcb, 0x6b, 0x44, 0x89, 0xd2, 0xc2, 0xe5, + 0x03, 0x82, 0x72, 0x10, 0x7b, 0xa8, 0xb5, 0xdb, 0x2c, 0xc2, 0x9d, 0x61, 0x28, 0xfe, 0xf7, 0x30, + 0x49, 0x46, 0xbe, 0x5a, 0x92, 0x14, 0x31, 0x65, 0x57, 0x73, 0xf7, 0xd9, 0x71, 0x24, 0xae, 0x00, + 0x17, 0x6d, 0x69, 0xee, 0x3e, 0xca, 0x42, 0x94, 0x8f, 0x8b, 0x64, 0x63, 0x0b, 0xd3, 0xd7, 0x93, + 0x8a, 0xf7, 0x89, 0x5e, 0x85, 0x59, 0x93, 0x1d, 0xb1, 0x55, 0xc3, 0x74, 0xb0, 0xee, 0xb6, 0x0f, + 0x99, 0xbf, 0x8d, 0x29, 0x69, 0x2e, 0xae, 0x0a, 0x29, 0xba, 0x01, 0xf2, 0x68, 0x80, 0xc2, 0xfc, + 0x64, 0x4c, 0x99, 0x1d, 0x89, 0x4f, 0x68, 0x2c, 0x23, 0xd6, 0x3a, 0xe0, 0xf8, 0xb3, 0x3c, 0x96, + 0x11, 0x05, 0x03, 0xa7, 0x7f, 0x03, 0x64, 0x11, 0x9d, 0x0c, 0x74, 0x53, 0x1c, 0x97, 0xcb, 0x07, + 0xaa, 0x45, 0x98, 0xeb, 0x6a, 0x0e, 0xc1, 0x6a, 0xab, 0x67, 0x19, 0x6d, 0xac, 0x72, 0xac, 0x6c, + 0x9a, 0x69, 0x67, 0x58, 0x51, 0x99, 0x95, 0x70, 0xbb, 0x3b, 0xeb, 0xd4, 0x76, 0xf9, 0xff, 0xe1, + 0xd4, 0x96, 0xfb, 0x79, 0x08, 0x66, 0xd8, 0x2e, 0x43, 0xb7, 0x20, 0x4c, 0xd7, 0x52, 0xe4, 0x19, + 0x26, 0x8d, 0x9c, 0x59, 0x1d, 0x84, 0x20, 0x6c, 0x69, 0x1d, 0x9c, 0x45, 0x6c, 0xa5, 0xd9, 0x6f, + 0x74, 0x05, 0xa2, 0x04, 0xdf, 0x57, 0x1f, 0x68, 0xed, 0xec, 0x1c, 0x33, 0xc4, 0x08, 0xc1, 0xf7, + 0xef, 0x69, 0x6d, 0x74, 0x09, 0x22, 0x26, 0x51, 0x2d, 0xfc, 0x30, 0x3b, 0xcf, 0xf9, 0xd4, 0x24, + 0x1b, 0xf8, 0x21, 0xfa, 0x1a, 0xc4, 0x1f, 0x6a, 0x44, 0xc5, 0x9d, 0xae, 0x7b, 0xc8, 0xe6, 0x22, + 0x46, 0x4d, 0x97, 0xd4, 0xe8, 0x37, 0x73, 0xbb, 0x9a, 0xb3, 0x87, 0x5d, 0x55, 0xb7, 0xdb, 0x24, + 0x7b, 0x89, 0x6e, 0x55, 0x1a, 0xb5, 0x53, 0x51, 0xc5, 0x6e, 0x93, 0x46, 0x38, 0x16, 0x92, 0xa7, + 0x1b, 0xe1, 0xd8, 0xb4, 0x1c, 0x6e, 0x84, 0x63, 0x61, 0x79, 0xa6, 0x11, 0x8e, 0xcd, 0xc8, 0x91, + 0x46, 0x38, 0x16, 0x91, 0xa3, 0x8d, 0x70, 0x2c, 0x2a, 0xc7, 0x1a, 0xe1, 0x58, 0x4c, 0x8e, 0x37, + 0xc2, 0xb1, 0xb8, 0x0c, 0x8d, 0x70, 0x0c, 0xe4, 0x44, 0x23, 0x1c, 0x4b, 0xc8, 0xc9, 0x46, 0x38, + 0x96, 0x94, 0x53, 0x8d, 0x70, 0x2c, 0x25, 0xa7, 0x1b, 0xe1, 0x58, 0x5a, 0x9e, 0x6d, 0x84, 0x63, + 0xb3, 0xb2, 0xdc, 0x08, 0xc7, 0x64, 0x39, 0xd3, 0x08, 0xc7, 0x32, 0x32, 0xca, 0xd5, 0x44, 0x62, + 0x4b, 0x43, 0xdf, 0x1d, 0x9a, 0xa7, 0x89, 0x23, 0x7a, 0x56, 0xa9, 0xf0, 0x0b, 0x09, 0xe4, 0x26, + 0xbe, 0xdf, 0xc3, 0x96, 0x8e, 0xef, 0x69, 0xed, 0xca, 0x7e, 0xcf, 0x3a, 0x40, 0xaf, 0xc0, 0xac, + 0x4e, 0x7f, 0xa8, 0x3c, 0xdb, 0x43, 0x67, 0x4c, 0x62, 0x33, 0x96, 0x62, 0xe2, 0x26, 0x95, 0xd2, + 0x89, 0x7b, 0x09, 0x40, 0xe8, 0x51, 0x36, 0xe1, 0x99, 0xe0, 0x38, 0x57, 0xa1, 0x14, 0x32, 0x02, + 0xe3, 0xd8, 0x0f, 0x19, 0x65, 0x0d, 0xc1, 0x28, 0xf6, 0x43, 0xb4, 0x08, 0xf3, 0x16, 0x7e, 0xe4, + 0xaa, 0xa3, 0xca, 0x8c, 0x9e, 0x94, 0x0c, 0x2d, 0xab, 0x04, 0x2b, 0x14, 0xfe, 0x35, 0x04, 0xb3, + 0x5e, 0xa7, 0x3d, 0x4e, 0xdf, 0x05, 0x99, 0xae, 0xae, 0x69, 0xa8, 0xae, 0xcd, 0x91, 0x3c, 0x76, + 0x7f, 0xff, 0x14, 0x3a, 0x1e, 0x41, 0xa1, 0xdf, 0x75, 0x63, 0xdb, 0x66, 0xcd, 0xf1, 0x20, 0x40, + 0x49, 0x91, 0xa0, 0x2c, 0xb7, 0x03, 0x69, 0xaf, 0x12, 0x97, 0xa0, 0x0a, 0x44, 0x86, 0xda, 0x7b, + 0x6d, 0x82, 0xf6, 0xbc, 0xa9, 0x56, 0x44, 0xd5, 0xdc, 0xef, 0x03, 0x1a, 0x6f, 0x3b, 0xe8, 0xfa, + 0x67, 0xb8, 0xeb, 0xdf, 0x1c, 0x0e, 0x40, 0xde, 0x3d, 0xdf, 0xd8, 0x02, 0xdd, 0x0e, 0xa6, 0x2a, + 0xfe, 0x39, 0x04, 0x69, 0xee, 0x98, 0x7c, 0x9f, 0x4a, 0x29, 0x8a, 0x32, 0xa0, 0x69, 0xed, 0xa9, + 0x5d, 0x21, 0x64, 0xe3, 0x0b, 0x29, 0xb2, 0x57, 0xe0, 0x2b, 0x7f, 0x03, 0x52, 0x0e, 0xd6, 0x8c, + 0x81, 0x62, 0x88, 0x29, 0x26, 0xa9, 0xd0, 0x57, 0xba, 0x06, 0x69, 0x16, 0xf7, 0x0c, 0xb4, 0xa6, + 0x99, 0x56, 0x8a, 0x49, 0x7d, 0xb5, 0x32, 0xa4, 0x48, 0x57, 0xb3, 0x06, 0x5a, 0x61, 0x36, 0xa9, + 0x67, 0x24, 0x56, 0x93, 0xb4, 0x4e, 0x30, 0x20, 0x70, 0x30, 0xe9, 0x75, 0xb0, 0xda, 0xb5, 0xf9, + 0xc9, 0x7b, 0x5a, 0x89, 0x73, 0xc9, 0x96, 0x4d, 0xd0, 0x0e, 0x33, 0x15, 0x36, 0x17, 0xaa, 0xc1, + 0x27, 0x27, 0x1b, 0x61, 0xad, 0xdc, 0x9c, 0x7c, 0x3a, 0x95, 0x59, 0x32, 0x2c, 0x28, 0xfc, 0xad, + 0x04, 0x57, 0xe8, 0xf9, 0x9b, 0xef, 0xb4, 0x0a, 0xbb, 0xc3, 0xf1, 0xac, 0x53, 0x83, 0x28, 0x3b, + 0xc3, 0xfb, 0x81, 0xd8, 0xea, 0x51, 0x3f, 0x1f, 0xa1, 0xda, 0x17, 0xf6, 0x72, 0x11, 0x0a, 0x5c, + 0x67, 0x27, 0x4a, 0xd7, 0xd1, 0x2c, 0x62, 0xd2, 0xb3, 0x15, 0x5d, 0xb6, 0x0e, 0xee, 0xb4, 0xb0, + 0xc3, 0x17, 0x23, 0xa9, 0xcc, 0x0f, 0x15, 0xae, 0xf3, 0xb2, 0x42, 0x0e, 0xb2, 0xa3, 0x5d, 0xf6, + 0xd3, 0x55, 0xbf, 0x03, 0x97, 0x37, 0xf0, 0xc3, 0xe3, 0x46, 0x53, 0x86, 0x28, 0x67, 0x3a, 0xcf, + 0xe4, 0xaf, 0x8f, 0x92, 0x4e, 0xf0, 0x1a, 0xab, 0xc8, 0x7a, 0xba, 0xcd, 0x2a, 0x28, 0x5e, 0xc5, + 0xc2, 0xa7, 0x70, 0x65, 0x04, 0xdd, 0x5f, 0xbe, 0x0f, 0x20, 0x42, 0xcf, 0xe9, 0x22, 0x3c, 0x4b, + 0x8f, 0x53, 0xda, 0x38, 0x7a, 0x93, 0xea, 0x2b, 0xa2, 0x5a, 0x41, 0x61, 0x79, 0xb6, 0x5e, 0x07, + 0x53, 0x0b, 0xb9, 0x63, 0x12, 0x17, 0x7d, 0x08, 0x49, 0x61, 0x11, 0xd4, 0x50, 0xbc, 0x6e, 0x9f, + 0x61, 0x54, 0x09, 0xc7, 0x07, 0x21, 0x85, 0xbf, 0x93, 0x60, 0xae, 0xea, 0xd8, 0xdd, 0x2e, 0x36, + 0x84, 0xcb, 0xe1, 0x73, 0xe1, 0x79, 0x1a, 0x29, 0xe0, 0x69, 0x36, 0x20, 0x54, 0xaf, 0x8a, 0x78, + 0xf8, 0xf6, 0x45, 0xc3, 0xec, 0x7a, 0x15, 0xbd, 0xcb, 0x27, 0xa4, 0x47, 0x18, 0x7f, 0xa6, 0xc7, + 0xd2, 0x2d, 0x43, 0x66, 0xca, 0x14, 0x15, 0x51, 0xa1, 0xf0, 0xb3, 0x28, 0x5c, 0x0a, 0x4e, 0xf2, + 0x4a, 0xc5, 0xeb, 0xf8, 0x67, 0x10, 0xf5, 0x8e, 0xdc, 0x13, 0xf0, 0xe4, 0x71, 0x10, 0x45, 0x31, + 0x1f, 0xc1, 0x63, 0xb7, 0x87, 0x89, 0x9a, 0x90, 0x31, 0x2d, 0x17, 0x3b, 0x6d, 0xac, 0x3d, 0xa0, + 0xc1, 0x05, 0x9d, 0x33, 0x91, 0xe2, 0x9c, 0xd4, 0x95, 0xcb, 0x01, 0x00, 0x1e, 0x12, 0x7c, 0x06, + 0x73, 0x41, 0x50, 0xaf, 0xff, 0xa7, 0xe7, 0xd6, 0x58, 0xf7, 0x06, 0xb0, 0x5e, 0x12, 0x30, 0x00, + 0x24, 0x12, 0x04, 0xe8, 0x63, 0xff, 0x5c, 0xc0, 0xf3, 0xa7, 0xb7, 0x9e, 0x79, 0x46, 0xaa, 0x23, + 0x67, 0x84, 0xa1, 0x48, 0x97, 0x39, 0xc1, 0xaf, 0x28, 0xd2, 0xbd, 0x07, 0x11, 0x9e, 0x62, 0x13, + 0x77, 0x06, 0xb7, 0x9f, 0x75, 0x08, 0x3c, 0x75, 0xa7, 0x08, 0xb4, 0xdc, 0x9f, 0x49, 0x90, 0x0c, + 0x2e, 0x37, 0x32, 0x21, 0xc6, 0xa6, 0xdf, 0xa3, 0xb4, 0xe9, 0xe7, 0x7e, 0xf2, 0xe3, 0xa6, 0x54, + 0x37, 0x68, 0x20, 0x66, 0x38, 0x76, 0x77, 0x70, 0x67, 0x34, 0xad, 0xc4, 0xa8, 0x80, 0x86, 0x8e, + 0xb9, 0x3f, 0x80, 0xb8, 0x3f, 0xe9, 0x81, 0x5c, 0xce, 0xf4, 0x73, 0xcc, 0xe5, 0x9c, 0xda, 0x7e, + 0x15, 0x52, 0x43, 0x33, 0x86, 0x2e, 0xfb, 0x7d, 0x08, 0x97, 0x23, 0xbc, 0x0f, 0x67, 0xa2, 0x14, + 0x7e, 0x1d, 0x81, 0xb9, 0xe3, 0x98, 0xf6, 0x13, 0x90, 0x03, 0xbc, 0xa5, 0xb6, 0x4d, 0xe2, 0x0a, + 0xdb, 0xbc, 0x71, 0x7a, 0xea, 0x21, 0x40, 0x7e, 0xc2, 0x14, 0xd3, 0xce, 0x30, 0x25, 0x7e, 0x0f, + 0xd2, 0x06, 0xef, 0xb8, 0xc8, 0x7e, 0x8a, 0x9b, 0xdf, 0xd3, 0x0e, 0xc3, 0xc7, 0x10, 0xa0, 0x40, + 0x4f, 0x19, 0x81, 0x22, 0x82, 0x74, 0x48, 0xf9, 0xe0, 0x87, 0x5d, 0x71, 0xaa, 0xba, 0x38, 0x19, + 0x26, 0xbd, 0x56, 0x28, 0x26, 0xda, 0x83, 0x59, 0xaf, 0x11, 0xef, 0x00, 0x1e, 0x7f, 0x2e, 0xcd, + 0x78, 0x13, 0xd3, 0x14, 0x07, 0xf2, 0x1f, 0x49, 0x30, 0xe7, 0xb5, 0xe4, 0xdf, 0x12, 0x98, 0x06, + 0xa3, 0xb3, 0x54, 0xb9, 0x79, 0xd4, 0xcf, 0x67, 0xc4, 0xcc, 0x78, 0xd7, 0x02, 0x17, 0xb6, 0xbb, + 0x8c, 0x31, 0x02, 0x68, 0xd0, 0x18, 0x82, 0x96, 0xd3, 0x86, 0x67, 0x06, 0x31, 0x04, 0x25, 0xb6, + 0x8b, 0xc7, 0x10, 0xf4, 0x67, 0xdd, 0x40, 0x7f, 0x2c, 0x41, 0x86, 0x5f, 0x5a, 0x74, 0x7a, 0xae, + 0xc6, 0xef, 0x07, 0xbd, 0x93, 0xf9, 0x27, 0x47, 0xfd, 0xfc, 0x2c, 0x5b, 0xde, 0x75, 0x51, 0xc6, + 0x9a, 0x2d, 0x3f, 0x6b, 0xb3, 0x03, 0x14, 0x71, 0x90, 0xf5, 0x05, 0x06, 0x5a, 0x83, 0x34, 0x4f, + 0x57, 0x78, 0x4f, 0x5b, 0xd8, 0x91, 0x3d, 0x55, 0x7e, 0xf9, 0x69, 0x3f, 0xbf, 0x70, 0xcc, 0x3e, + 0xe1, 0x99, 0x8e, 0x7b, 0x5c, 0x57, 0x49, 0xed, 0x06, 0x3f, 0x45, 0xde, 0xe8, 0x32, 0xcc, 0x1f, + 0x1b, 0xe5, 0x7c, 0x11, 0x81, 0xcb, 0xc3, 0x44, 0xe8, 0xc7, 0x21, 0xea, 0xa8, 0x87, 0xfc, 0x60, + 0x62, 0x32, 0xf5, 0x30, 0x38, 0x99, 0x79, 0x5f, 0xa3, 0x3e, 0xf2, 0xb3, 0x11, 0x7f, 0xf3, 0x0c, + 0xf8, 0x6c, 0x41, 0x46, 0xf0, 0x3d, 0xa7, 0xf3, 0xb1, 0xef, 0x0b, 0x78, 0xa6, 0xe8, 0xc3, 0x67, + 0x80, 0x67, 0xf5, 0xbd, 0x4f, 0xdf, 0x1b, 0xfc, 0x8b, 0x04, 0xa9, 0xa1, 0x91, 0xfd, 0x36, 0xdd, + 0xc1, 0x96, 0x1f, 0x0d, 0xf1, 0x27, 0x43, 0xef, 0x9c, 0x7f, 0x58, 0xc3, 0x41, 0x52, 0xee, 0xef, + 0x25, 0x48, 0x0d, 0x4d, 0xe4, 0x57, 0xe4, 0x48, 0x9e, 0x7f, 0xcf, 0x5b, 0x90, 0x1e, 0x5e, 0xa2, + 0x40, 0x1b, 0xd2, 0xf3, 0x69, 0xa3, 0xf0, 0x1d, 0x88, 0x70, 0x09, 0x42, 0x90, 0xfe, 0xa8, 0x54, + 0xdf, 0xae, 0x6f, 0xac, 0xa8, 0xcb, 0x9b, 0x8a, 0xba, 0x52, 0x91, 0xa7, 0x50, 0x12, 0x62, 0xd5, + 0xda, 0x9d, 0x1a, 0x15, 0xca, 0x12, 0x4a, 0x40, 0x94, 0x7d, 0xd5, 0xaa, 0x72, 0xa8, 0x50, 0x06, + 0x99, 0x63, 0xef, 0x62, 0xea, 0x18, 0x68, 0xdc, 0x8f, 0x8a, 0x30, 0xc7, 0x82, 0xf4, 0x0e, 0x8d, + 0x85, 0xa8, 0x2b, 0x54, 0x03, 0xd1, 0x73, 0xc6, 0x2f, 0xa2, 0x4e, 0x71, 0x43, 0xeb, 0xe0, 0xc2, + 0x2f, 0xc3, 0x90, 0x19, 0x80, 0x78, 0x6e, 0xf1, 0x6f, 0xa4, 0xc1, 0x09, 0x24, 0x72, 0xe6, 0x7d, + 0xd1, 0x58, 0x7d, 0x71, 0x18, 0x11, 0x57, 0xdc, 0x1f, 0xd1, 0x4d, 0xf3, 0xb4, 0x9f, 0xcf, 0x8c, + 0x76, 0x96, 0x5c, 0xf0, 0xee, 0xdb, 0xeb, 0x22, 0xcb, 0x95, 0x9a, 0xd6, 0x81, 0x3a, 0x78, 0xb8, + 0xc2, 0x73, 0xa5, 0xa6, 0x75, 0xb0, 0xa3, 0xd4, 0x95, 0x28, 0x2d, 0xdc, 0x71, 0x4c, 0xd4, 0x80, + 0xb0, 0xdd, 0x75, 0xbd, 0x23, 0xef, 0xdb, 0xe7, 0x1a, 0xd2, 0x66, 0x57, 0x8c, 0x47, 0x61, 0x18, + 0xa8, 0xc1, 0xaf, 0x40, 0x07, 0x13, 0xcd, 0x28, 0x74, 0xc2, 0xf7, 0x05, 0xa9, 0xa1, 0x85, 0xc8, + 0xed, 0x41, 0x32, 0x38, 0x63, 0xc7, 0xdc, 0x8b, 0x94, 0x86, 0xd3, 0x12, 0xaf, 0x4d, 0xd4, 0x75, + 0x71, 0x24, 0x0c, 0x5c, 0x5f, 0x7c, 0x07, 0xe2, 0xfe, 0x38, 0xce, 0x73, 0x8b, 0xc3, 0x39, 0xde, + 0xcf, 0xb9, 0xcd, 0xc8, 0x91, 0x42, 0x3f, 0x04, 0x49, 0x05, 0x13, 0xbb, 0xfd, 0x00, 0x1b, 0x34, + 0xe6, 0xf1, 0x1f, 0x6a, 0x49, 0x93, 0x3f, 0xd4, 0x2a, 0x41, 0xdc, 0x4f, 0x8a, 0x9e, 0xe7, 0xb1, + 0xd2, 0xa0, 0x16, 0xba, 0x0d, 0x5f, 0x0b, 0xbe, 0x06, 0xb2, 0x7b, 0x96, 0xa1, 0x39, 0x87, 0xaa, + 0x83, 0x35, 0x7d, 0x1f, 0x1b, 0xe2, 0xba, 0xee, 0x85, 0xc0, 0x73, 0x20, 0xa1, 0xa1, 0x70, 0x05, + 0xf4, 0x09, 0xa4, 0xfc, 0x4a, 0x34, 0x92, 0x62, 0x31, 0x47, 0x7a, 0xe9, 0x5b, 0xa7, 0x47, 0x7f, + 0xfe, 0xa8, 0x8b, 0x1e, 0x1e, 0x8d, 0x98, 0x94, 0x64, 0x2b, 0xf0, 0x55, 0x78, 0x1f, 0x92, 0xc1, + 0x52, 0x14, 0x83, 0xf0, 0xc6, 0xe6, 0x46, 0x8d, 0xef, 0xe9, 0x72, 0xa9, 0xb2, 0xb6, 0x5c, 0xbf, + 0x73, 0x47, 0x96, 0xa8, 0xbc, 0xf6, 0x71, 0x7d, 0x5b, 0x0e, 0xd1, 0xdd, 0xad, 0xd4, 0x9a, 0xdb, + 0x25, 0x65, 0x5b, 0x9e, 0x2e, 0x60, 0x48, 0x05, 0x5b, 0xa2, 0x9c, 0x49, 0x43, 0x4c, 0x26, 0x18, + 0x3a, 0x65, 0xbf, 0x3a, 0x61, 0x5f, 0x3d, 0xdb, 0x73, 0x82, 0xa8, 0x85, 0x1f, 0x87, 0x00, 0x0d, + 0x4c, 0x26, 0x40, 0xd0, 0xa3, 0x8d, 0x85, 0x2e, 0xde, 0x18, 0xfa, 0xc9, 0xe9, 0x09, 0xf1, 0x69, + 0x96, 0x10, 0x67, 0x7c, 0xf1, 0x5b, 0x4d, 0x8a, 0x8b, 0x10, 0xe6, 0x7f, 0xc2, 0x80, 0x2a, 0x0e, + 0xd6, 0x5c, 0x4c, 0xf9, 0x98, 0x9c, 0x96, 0x81, 0x28, 0xc3, 0x0c, 0x3f, 0x71, 0x87, 0xce, 0x73, + 0xe2, 0x16, 0x93, 0xc2, 0xab, 0xa2, 0xef, 0x43, 0x52, 0xb7, 0xdb, 0xbd, 0x8e, 0xa5, 0xb2, 0x57, + 0x13, 0xe2, 0x78, 0xf0, 0xed, 0xd3, 0xb6, 0xf6, 0x58, 0xe7, 0x8a, 0x15, 0xbb, 0x4d, 0xbf, 0xfd, + 0xf7, 0x85, 0x0c, 0x90, 0x69, 0xa0, 0x17, 0x21, 0xee, 0xd3, 0x0c, 0x33, 0xeb, 0xb8, 0x32, 0x10, + 0xa0, 0x25, 0x98, 0xd1, 0x88, 0x6a, 0xef, 0xb2, 0x58, 0xf7, 0xac, 0x7d, 0xa7, 0x84, 0x35, 0xb2, + 0xb9, 0x8b, 0x6e, 0x42, 0xa6, 0xa3, 0x3d, 0x52, 0x77, 0x1d, 0x4d, 0x17, 0xb1, 0x6b, 0x9b, 0xd3, + 0x9e, 0xa4, 0xcc, 0x76, 0xb4, 0x47, 0xcb, 0x42, 0x5e, 0x37, 0xda, 0x18, 0xbd, 0x05, 0xa9, 0xdd, + 0xfb, 0xfc, 0xe4, 0xc3, 0x5d, 0x10, 0x7f, 0x82, 0x32, 0x7b, 0xd4, 0xcf, 0x27, 0x96, 0xef, 0xb2, + 0x89, 0xa1, 0x0e, 0x48, 0x49, 0xec, 0xde, 0xf7, 0x3f, 0x72, 0xff, 0x2d, 0x41, 0x54, 0x8c, 0x08, + 0x75, 0x01, 0xc4, 0xf4, 0x98, 0x06, 0x37, 0xf5, 0x54, 0xf9, 0xee, 0x51, 0x3f, 0x1f, 0xaf, 0x30, + 0x69, 0xbd, 0x4a, 0x9e, 0xf6, 0xf3, 0x1f, 0x3e, 0xab, 0xfb, 0xf0, 0x40, 0x94, 0x38, 0x6f, 0xa4, + 0x6e, 0xb0, 0x34, 0xeb, 0xbe, 0x46, 0xd4, 0x7d, 0x93, 0xb8, 0xf6, 0x9e, 0xa3, 0x75, 0xc4, 0xf5, + 0x6f, 0x72, 0x5f, 0x23, 0xab, 0x9e, 0x0c, 0xe5, 0x68, 0x20, 0xf6, 0x80, 0x3f, 0x7a, 0xe1, 0xec, + 0xe2, 0x7f, 0xa3, 0x25, 0xb8, 0xe4, 0x57, 0x56, 0xe9, 0x4c, 0xb5, 0x7a, 0xfa, 0x01, 0x66, 0x0e, + 0x87, 0x32, 0xf9, 0x9c, 0x5f, 0xb8, 0xae, 0x3d, 0x2a, 0xf3, 0xa2, 0xc2, 0x25, 0x98, 0x0b, 0x2c, + 0xab, 0x1f, 0x36, 0x63, 0x90, 0xf9, 0x2d, 0x75, 0xe0, 0xc5, 0xec, 0x5d, 0x98, 0x1d, 0x79, 0x91, + 0x2e, 0xc8, 0x36, 0x98, 0x1e, 0x1c, 0x7e, 0xc2, 0x5e, 0xac, 0xf0, 0x4f, 0x2f, 0x74, 0x4f, 0xeb, + 0x43, 0xdf, 0x85, 0x39, 0xc8, 0xf8, 0xcd, 0xf8, 0x6d, 0xff, 0x26, 0x09, 0xd1, 0x2d, 0xed, 0xb0, + 0x6d, 0x6b, 0x06, 0x5a, 0x80, 0x84, 0xf7, 0x5a, 0xc5, 0x6b, 0x2f, 0xae, 0x04, 0x45, 0xc3, 0x66, + 0x26, 0x8f, 0x9a, 0x99, 0x09, 0xe9, 0x1e, 0xc1, 0x0e, 0xb5, 0x00, 0x95, 0x3d, 0x9f, 0xe7, 0xbe, + 0xa5, 0x5c, 0x7e, 0xda, 0xcf, 0xdf, 0x9e, 0x6c, 0xf1, 0xb0, 0xde, 0x73, 0x4c, 0xf7, 0xb0, 0xd8, + 0xbc, 0x7b, 0x67, 0x47, 0x40, 0xd1, 0x2d, 0x6e, 0x2b, 0xa9, 0x5e, 0xf0, 0x53, 0x3c, 0x4a, 0xa2, + 0x0b, 0xa1, 0x76, 0x4c, 0xdd, 0xb1, 0x89, 0x77, 0x1b, 0x22, 0xa4, 0xeb, 0x4c, 0x88, 0x5e, 0x85, + 0xd9, 0x5d, 0xd3, 0x62, 0x17, 0x7a, 0x9e, 0x1e, 0xbf, 0x08, 0x49, 0x7b, 0x62, 0xa1, 0xf8, 0x00, + 0xd2, 0x81, 0xd7, 0x40, 0xd4, 0x08, 0x23, 0xcc, 0x08, 0x37, 0x8f, 0xfa, 0xf9, 0xd4, 0x60, 0x53, + 0x73, 0x43, 0xbc, 0x48, 0x1c, 0x93, 0x1a, 0x34, 0x43, 0xcd, 0x70, 0x1e, 0x66, 0xd8, 0x1f, 0x57, + 0xf0, 0xe7, 0x94, 0x0a, 0xff, 0x40, 0x35, 0x48, 0x89, 0x4c, 0x05, 0xff, 0xcb, 0x0b, 0xf1, 0x84, + 0x6a, 0x21, 0xb0, 0xf4, 0xde, 0xdf, 0x66, 0x14, 0x6b, 0x96, 0x6e, 0x1b, 0xd8, 0xa8, 0xd1, 0x6f, + 0x45, 0x24, 0x66, 0xd9, 0x07, 0x41, 0x2b, 0x90, 0xd6, 0xdb, 0x58, 0xb3, 0x7a, 0x5d, 0x0f, 0x07, + 0x4d, 0x88, 0x93, 0x12, 0xf5, 0x04, 0xd0, 0x06, 0xa0, 0x5d, 0xf6, 0xa6, 0x25, 0xd8, 0x2b, 0x76, + 0xf1, 0x37, 0x09, 0x98, 0xcc, 0xea, 0x2a, 0x83, 0x9e, 0xa1, 0xb7, 0x61, 0xa6, 0x8d, 0x35, 0x82, + 0xc5, 0x5b, 0xd9, 0x85, 0x53, 0x68, 0x90, 0xfd, 0xf5, 0x85, 0xc2, 0xd5, 0xd1, 0xcb, 0x90, 0xb2, + 0x6c, 0x4b, 0xd7, 0x2c, 0x1d, 0xb7, 0x19, 0x23, 0xf3, 0x3b, 0xc6, 0x61, 0x21, 0x2a, 0x43, 0x84, + 0xdf, 0x46, 0xb3, 0xbb, 0xf4, 0xf1, 0x84, 0xfa, 0x89, 0x8f, 0x95, 0x57, 0xa7, 0x14, 0x51, 0x13, + 0xd5, 0x20, 0xea, 0xf0, 0x97, 0x11, 0xec, 0x86, 0xfa, 0xcc, 0x14, 0x51, 0xe0, 0x75, 0xca, 0xea, + 0x94, 0xe2, 0xd5, 0x45, 0xdb, 0xde, 0x43, 0x41, 0xee, 0x75, 0xc5, 0x5b, 0xaf, 0xe2, 0x84, 0xc7, + 0x88, 0x01, 0xe0, 0x10, 0x0a, 0x1d, 0xa0, 0xc9, 0x6e, 0x98, 0xd8, 0xdd, 0xf5, 0xe9, 0x03, 0x1c, + 0x7a, 0x23, 0x41, 0x07, 0xc8, 0x6b, 0xa2, 0x0d, 0x00, 0xdd, 0x8f, 0x04, 0xd8, 0xad, 0x76, 0x62, + 0xe9, 0xf5, 0xf3, 0x04, 0xc9, 0xab, 0x53, 0x4a, 0x00, 0x01, 0xdd, 0x85, 0x84, 0x3e, 0xa0, 0xb6, + 0xec, 0x2c, 0x03, 0x7c, 0xe3, 0x5c, 0xfe, 0x6d, 0x95, 0xfa, 0xb4, 0x81, 0x14, 0x7d, 0x0a, 0x69, + 0x32, 0x74, 0xa8, 0xca, 0x5e, 0x62, 0xa8, 0x6f, 0x9e, 0x37, 0x0d, 0xbb, 0x3a, 0xa5, 0x8c, 0x20, + 0xa1, 0xdf, 0x05, 0xd9, 0x1d, 0xb9, 0xab, 0x61, 0xd7, 0xd2, 0xa7, 0x3f, 0xc4, 0x3b, 0xe1, 0x46, + 0x6a, 0x75, 0x4a, 0x19, 0x43, 0x43, 0x9f, 0xc1, 0x2c, 0x19, 0xfe, 0xa3, 0x82, 0xec, 0x15, 0xd6, + 0xc0, 0x37, 0x4f, 0xbd, 0x70, 0x38, 0xee, 0xcf, 0x27, 0x56, 0xa7, 0x94, 0x51, 0x2c, 0x0a, 0x6f, + 0x0d, 0x5f, 0xf9, 0xb0, 0x47, 0x0f, 0xa7, 0xc3, 0x1f, 0x7f, 0x05, 0x45, 0xe1, 0x47, 0xb0, 0xd0, + 0x1a, 0xc4, 0x3b, 0x9e, 0xaf, 0xc8, 0xbe, 0x70, 0xe6, 0x39, 0x64, 0xd4, 0x7d, 0xad, 0x4e, 0x29, + 0x83, 0xfa, 0xe5, 0x38, 0x44, 0xc5, 0xd5, 0xa0, 0x7f, 0x6f, 0x1f, 0x95, 0x63, 0x85, 0xdf, 0xc4, + 0x20, 0xe6, 0xc7, 0xa0, 0x8b, 0x80, 0xfc, 0x50, 0x63, 0xf0, 0x26, 0x95, 0xba, 0xa0, 0xd0, 0xea, + 0x94, 0x92, 0xf1, 0xca, 0x06, 0xcf, 0x52, 0x6f, 0x0f, 0x3d, 0x54, 0x9a, 0xe4, 0x0f, 0x3f, 0x68, + 0xa7, 0xfc, 0x97, 0x4c, 0xd4, 0x35, 0x74, 0x6c, 0xc3, 0xdc, 0x35, 0x07, 0xae, 0x81, 0x27, 0x92, + 0xd3, 0x9e, 0x58, 0xb8, 0x86, 0x6b, 0x90, 0x76, 0x7a, 0x16, 0xbb, 0x05, 0x14, 0xc9, 0x00, 0x1e, + 0x5f, 0xa5, 0x84, 0x54, 0x9c, 0xe7, 0x2b, 0x23, 0xac, 0x73, 0xe3, 0x4c, 0xd6, 0xf1, 0xc6, 0xbe, + 0x2a, 0xf9, 0xb4, 0xb3, 0x3c, 0x4a, 0x3b, 0x37, 0xcf, 0xa6, 0x9d, 0x00, 0x8c, 0xcf, 0x3b, 0x3b, + 0xc7, 0xf2, 0xce, 0xe2, 0x84, 0x1b, 0x27, 0x80, 0x38, 0x4c, 0x3c, 0x95, 0x11, 0xe2, 0xb9, 0x71, + 0x26, 0xf1, 0x04, 0xc7, 0x28, 0x98, 0x67, 0xf3, 0x18, 0xe6, 0x79, 0x63, 0x22, 0xe6, 0x09, 0x80, + 0x05, 0xa9, 0x47, 0x39, 0x8e, 0x7a, 0x8a, 0x93, 0x51, 0x4f, 0x00, 0x72, 0x88, 0x7b, 0xbe, 0x37, + 0xc6, 0x3d, 0xf2, 0xd9, 0x9b, 0xf7, 0xd8, 0x0c, 0xd0, 0xaa, 0x34, 0x46, 0x3e, 0xda, 0x31, 0xe4, + 0x93, 0x61, 0xf0, 0x6f, 0x9d, 0x83, 0x7c, 0x02, 0x0d, 0x8c, 0xb3, 0xcf, 0xc7, 0x90, 0x0c, 0x32, + 0x06, 0x7b, 0xbb, 0x73, 0x3a, 0xb7, 0x9d, 0xf0, 0x17, 0x50, 0xcc, 0x06, 0x02, 0x45, 0xe8, 0xfb, + 0xe3, 0xc4, 0x33, 0x77, 0x26, 0xf8, 0x09, 0xb7, 0xd3, 0xab, 0xd2, 0x38, 0xf3, 0xdc, 0x09, 0x32, + 0xcf, 0xfc, 0x99, 0x7e, 0x69, 0x2c, 0xa2, 0x5d, 0x95, 0x82, 0xd4, 0x03, 0x10, 0xf3, 0x1e, 0x3f, + 0x04, 0x68, 0xa8, 0xf0, 0x53, 0x09, 0xa6, 0x1b, 0x76, 0x0b, 0xbd, 0x14, 0x48, 0x4d, 0xa6, 0xc4, + 0x51, 0x74, 0xa6, 0x61, 0xb7, 0x44, 0x8e, 0xf1, 0x83, 0x41, 0x6d, 0x71, 0xf8, 0xfb, 0xc6, 0x29, + 0x5d, 0xf1, 0x33, 0xbb, 0x7e, 0x25, 0xf4, 0x1e, 0x44, 0xbb, 0x3c, 0xb8, 0x16, 0x0c, 0x55, 0x38, + 0xad, 0x3e, 0xd7, 0x54, 0xbc, 0x2a, 0x37, 0x6f, 0x04, 0xff, 0xa4, 0x71, 0xdd, 0x36, 0x30, 0x4a, + 0x03, 0x6c, 0x69, 0x84, 0x74, 0xf7, 0x1d, 0x8d, 0x60, 0x79, 0x0a, 0x45, 0x61, 0x7a, 0x6d, 0xbd, + 0x29, 0x4b, 0x37, 0x3f, 0x0e, 0xe6, 0x15, 0xab, 0x4a, 0xa9, 0xbe, 0x51, 0xdf, 0x58, 0x51, 0x37, + 0x4a, 0xeb, 0xb5, 0xa6, 0x3c, 0x85, 0xb2, 0x30, 0xff, 0x51, 0xa9, 0xbe, 0x2d, 0x12, 0x8d, 0x6a, + 0x7d, 0x63, 0xbb, 0xa6, 0xdc, 0x2b, 0xdd, 0x91, 0x25, 0x74, 0x19, 0x90, 0xb2, 0x59, 0x59, 0x6b, + 0x56, 0xcb, 0x6a, 0x65, 0x73, 0x7d, 0xab, 0x54, 0xd9, 0xae, 0x6f, 0x6e, 0xc8, 0x21, 0x14, 0x83, + 0x70, 0x75, 0x73, 0xa3, 0x26, 0xc3, 0xcd, 0x9f, 0x84, 0x21, 0xcc, 0x52, 0x1a, 0x2f, 0x43, 0x62, + 0x67, 0xa3, 0xb9, 0x55, 0xab, 0xd4, 0x97, 0xeb, 0xb5, 0xaa, 0x3c, 0x95, 0x9b, 0x7b, 0xfc, 0x64, + 0x61, 0x96, 0x16, 0xed, 0x58, 0xa4, 0x8b, 0x75, 0xc6, 0x99, 0x28, 0x07, 0x91, 0x72, 0xa9, 0xb2, + 0xb6, 0xb3, 0x25, 0x4b, 0xb9, 0xf4, 0xe3, 0x27, 0x0b, 0x40, 0x15, 0x38, 0xdf, 0xa1, 0x17, 0x79, + 0xca, 0x63, 0x53, 0xa9, 0xc9, 0xa1, 0xdc, 0xec, 0xe3, 0x27, 0x0b, 0x09, 0x96, 0x49, 0x11, 0x9c, + 0xf5, 0x2a, 0xa4, 0x9a, 0x95, 0xd5, 0xda, 0x7a, 0x49, 0xad, 0xac, 0x96, 0x36, 0x56, 0x6a, 0xf2, + 0x74, 0x6e, 0xfe, 0xf1, 0x93, 0x05, 0x79, 0xd4, 0xee, 0x69, 0x13, 0xf5, 0xf5, 0xad, 0x4d, 0x65, + 0x5b, 0x0e, 0x0f, 0x9a, 0xe0, 0x74, 0x83, 0x0a, 0x00, 0xbc, 0xf6, 0x72, 0xad, 0x56, 0x95, 0x67, + 0x72, 0xe8, 0xf1, 0x93, 0x85, 0x34, 0x2d, 0x1f, 0xb0, 0x08, 0xba, 0x06, 0xc9, 0x8a, 0x52, 0x2b, + 0x6d, 0xd7, 0xd4, 0xe6, 0x76, 0x69, 0xbb, 0x29, 0x47, 0x06, 0x23, 0x09, 0x30, 0x03, 0x2a, 0x42, + 0xa6, 0xb4, 0xb3, 0xbd, 0xa9, 0x0e, 0xe9, 0x46, 0x73, 0x57, 0x1e, 0x3f, 0x59, 0x98, 0xa3, 0xba, + 0xa5, 0x9e, 0x6b, 0x07, 0xf5, 0x5f, 0x07, 0x79, 0xa8, 0xff, 0xea, 0x4a, 0x45, 0x8e, 0xe5, 0x2e, + 0x3f, 0x7e, 0xb2, 0x80, 0x46, 0x87, 0xb0, 0x52, 0x41, 0xdf, 0x82, 0xcb, 0xdb, 0x9f, 0x6c, 0xd5, + 0xaa, 0xb5, 0x66, 0x45, 0x1d, 0x1e, 0x76, 0x3c, 0x97, 0x7d, 0xfc, 0x64, 0x61, 0x9e, 0xd6, 0x19, + 0x1b, 0xfa, 0x1b, 0x20, 0x37, 0xb7, 0x95, 0x5a, 0x69, 0x5d, 0xad, 0x6f, 0xac, 0xd4, 0x9a, 0x6c, + 0xb1, 0x60, 0xd0, 0xa5, 0x91, 0x3d, 0x4c, 0x87, 0xb0, 0x51, 0xfb, 0x68, 0x04, 0x3f, 0x31, 0xd0, + 0x1f, 0xd9, 0x96, 0x68, 0x01, 0xe2, 0xeb, 0xf5, 0x15, 0xa5, 0xc4, 0x70, 0x93, 0xb9, 0xcc, 0xe3, + 0x27, 0x0b, 0x29, 0xaa, 0xe7, 0x6f, 0xb2, 0x5c, 0xec, 0xc7, 0x3f, 0xbd, 0x3a, 0xf5, 0xd7, 0x3f, + 0xbb, 0x3a, 0x55, 0xbe, 0xfe, 0xf9, 0x7f, 0x5e, 0x9d, 0xfa, 0xfc, 0xe8, 0xaa, 0xf4, 0xab, 0xa3, + 0xab, 0xd2, 0x17, 0x47, 0x57, 0xa5, 0xff, 0x38, 0xba, 0x2a, 0xfd, 0xc9, 0x97, 0x57, 0xa7, 0x7e, + 0xf5, 0xe5, 0xd5, 0xa9, 0x2f, 0xbe, 0xbc, 0x3a, 0xf5, 0x69, 0x84, 0xdb, 0x75, 0x2b, 0xc2, 0xce, + 0x82, 0x6f, 0xfd, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x06, 0x36, 0x30, 0x9c, 0x83, 0x3e, 0x00, + 0x00, } func (this *Lease) Equal(that interface{}) bool { @@ -3506,14 +3508,14 @@ func (m *RestoreDetails) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x92 } } - if len(m.SystemTablesRestored) > 0 { - keysForSystemTablesRestored := make([]string, 0, len(m.SystemTablesRestored)) - for k := range m.SystemTablesRestored { - keysForSystemTablesRestored = append(keysForSystemTablesRestored, string(k)) + if len(m.SystemTablesMigrated) > 0 { + keysForSystemTablesMigrated := make([]string, 0, len(m.SystemTablesMigrated)) + for k := range m.SystemTablesMigrated { + keysForSystemTablesMigrated = append(keysForSystemTablesMigrated, string(k)) } - github_com_gogo_protobuf_sortkeys.Strings(keysForSystemTablesRestored) - for iNdEx := len(keysForSystemTablesRestored) - 1; iNdEx >= 0; iNdEx-- { - v := m.SystemTablesRestored[string(keysForSystemTablesRestored[iNdEx])] + github_com_gogo_protobuf_sortkeys.Strings(keysForSystemTablesMigrated) + for iNdEx := len(keysForSystemTablesMigrated) - 1; iNdEx >= 0; iNdEx-- { + v := m.SystemTablesMigrated[string(keysForSystemTablesMigrated[iNdEx])] baseI := i i-- if v { @@ -3523,9 +3525,9 @@ func (m *RestoreDetails) MarshalToSizedBuffer(dAtA []byte) (int, error) { } i-- dAtA[i] = 0x10 - i -= len(keysForSystemTablesRestored[iNdEx]) - copy(dAtA[i:], keysForSystemTablesRestored[iNdEx]) - i = encodeVarintJobs(dAtA, i, uint64(len(keysForSystemTablesRestored[iNdEx]))) + i -= len(keysForSystemTablesMigrated[iNdEx]) + copy(dAtA[i:], keysForSystemTablesMigrated[iNdEx]) + i = encodeVarintJobs(dAtA, i, uint64(len(keysForSystemTablesMigrated[iNdEx]))) i-- dAtA[i] = 0xa i = encodeVarintJobs(dAtA, i, uint64(baseI-i)) @@ -6644,8 +6646,8 @@ func (m *RestoreDetails) Size() (n int) { n += 2 + l + sovJobs(uint64(l)) } } - if len(m.SystemTablesRestored) > 0 { - for k, v := range m.SystemTablesRestored { + if len(m.SystemTablesMigrated) > 0 { + for k, v := range m.SystemTablesMigrated { _ = k _ = v mapEntrySize := 1 + len(k) + sovJobs(uint64(len(k))) + 1 + 1 @@ -9685,7 +9687,7 @@ func (m *RestoreDetails) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 17: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SystemTablesRestored", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SystemTablesMigrated", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -9712,8 +9714,8 @@ func (m *RestoreDetails) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.SystemTablesRestored == nil { - m.SystemTablesRestored = make(map[string]bool) + if m.SystemTablesMigrated == nil { + m.SystemTablesMigrated = make(map[string]bool) } var mapkey string var mapvalue bool @@ -9796,7 +9798,7 @@ func (m *RestoreDetails) Unmarshal(dAtA []byte) error { iNdEx += skippy } } - m.SystemTablesRestored[mapkey] = mapvalue + m.SystemTablesMigrated[mapkey] = mapvalue iNdEx = postIndex case 18: if wireType != 2 { diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index e46fed79b07b..459c913e6668 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -184,12 +184,13 @@ message RestoreDetails { // stages have completed via these flags. bool prepare_completed = 8; bool stats_inserted = 9; - // SystemTablesRestored keeps track of dynamic states that need to happen only - // once during the lifetime of a job. Note, that this state may be shared - // between job versions, so updates to this map must be considered carefully. - // It maps system table names to whether or not they have already been - // restored. - map<string, bool> system_tables_restored = 17; + // SystemTablesMigrated keeps track of which system tables data have been + // migrated. We need to keep track of this because if we've modified the + // restored data via a migration, we can't restore back into that span as the + // migrated keys will shadow the ones that will be restored. + // Note, that this state may be shared between job versions, so updates to + // this map must be considered carefully. + map<string, bool> system_tables_migrated = 17; // DescriptorsPublished indicates whether or not the descriptors written in // the job have been transactionally updated after the data was restored. bool descriptors_published = 10; From 0b7f5c4113ac1c4a1ee5b777ef3f0419c8ff9ca6 Mon Sep 17 00:00:00 2001 From: arulajmani <arulajmani@gmail.com> Date: Fri, 16 Apr 2021 17:59:49 -0400 Subject: [PATCH 20/37] sql: add more testing for concurrent region/rbr interactions Informs #63462 This patch adds TestRegionAddDropEnclosingRegionalByRowOps, which contains subtests with the following sketch: - Client 1 performs an ALTER ADD / DROP REGION. Let the user txn commit. - Block in the type schema changer. - Client 2 performs an operation on a REGIONAL BY ROW table, such as creating an index, unique constraint, altering the primary key etc. - Test both a rollback in the type schema changer and a successful execution. - Ensure the partitions on the REGIONAL BY ROW table are sane. Release note: None --- pkg/ccl/multiregionccl/region_test.go | 195 ++++++++++++++++++++++++++ 1 file changed, 195 insertions(+) diff --git a/pkg/ccl/multiregionccl/region_test.go b/pkg/ccl/multiregionccl/region_test.go index c1199cde5a77..fdb75d237f77 100644 --- a/pkg/ccl/multiregionccl/region_test.go +++ b/pkg/ccl/multiregionccl/region_test.go @@ -348,6 +348,201 @@ CREATE DATABASE db WITH PRIMARY REGION "us-east1" REGIONS "us-east2", "us-east3" } } +// TestRegionAddDropEnclosingRegionalByRowOps tests adding/dropping regions +// (which may or may not succeed) with a concurrent operation on a regional by +// row table. The sketch of the test is as follows: +// - Client 1 performs an ALTER ADD / DROP REGION. Let the user txn commit. +// - Block in the type schema changer. +// - Client 2 performs an operation on a REGIONAL BY ROW table, such as creating +// an index, unique constraint, altering the primary key etc. +// - Test both a rollback in the type schema changer and a successful execution. +// - Ensure the partitions on the REGIONAL BY ROW table are sane. +func TestRegionAddDropEnclosingRegionalByRowOps(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + skip.UnderRace(t, "times out under race") + + // Decrease the adopt loop interval so that retries happen quickly. + defer sqltestutils.SetTestJobsAdoptInterval()() + + regionAlterCmds := []struct { + name string + cmd string + shouldSucceed bool + expectedPartitions []string + }{ + { + name: "drop-region-fail", + cmd: `ALTER DATABASE db DROP REGION "us-east3"`, + shouldSucceed: false, + expectedPartitions: []string{"us-east1", "us-east2", "us-east3"}, + }, + { + name: "drop-region-succeed", + cmd: `ALTER DATABASE db DROP REGION "us-east3"`, + shouldSucceed: true, + expectedPartitions: []string{"us-east1", "us-east2"}, + }, + { + name: "add-region", + cmd: `ALTER DATABASE db ADD REGION "us-east4"`, + shouldSucceed: false, + expectedPartitions: []string{"us-east1", "us-east2", "us-east3"}, + }, + { + name: "add-region", + cmd: `ALTER DATABASE db ADD REGION "us-east4"`, + shouldSucceed: true, + expectedPartitions: []string{"us-east1", "us-east2", "us-east3", "us-east4"}, + }, + } + + testCases := []struct { + name string + op string + expectedIndexes []string + }{ + { + name: "create-rbr-table", + op: `DROP TABLE db.rbr; CREATE TABLE db.rbr() LOCALITY REGIONAL BY ROW`, + expectedIndexes: []string{"rbr@primary"}, + }, + { + name: "create-index", + op: `CREATE INDEX idx ON db.rbr(v)`, + expectedIndexes: []string{"rbr@primary", "rbr@idx"}, + }, + { + name: "add-column", + op: `ALTER TABLE db.rbr ADD COLUMN v2 INT`, + expectedIndexes: []string{"rbr@primary"}, + }, + { + name: "alter-pk", + op: `ALTER TABLE db.rbr ALTER PRIMARY KEY USING COLUMNS (k, v)`, + expectedIndexes: []string{"rbr@primary"}, + }, + { + name: "drop-column", + op: `ALTER TABLE db.rbr DROP COLUMN v`, + expectedIndexes: []string{"rbr@primary"}, + }, + { + name: "unique-index", + op: `CREATE UNIQUE INDEX uniq ON db.rbr(v)`, + expectedIndexes: []string{"rbr@primary", "rbr@uniq"}, + }, + } + + for _, tc := range testCases { + for _, regionAlterCmd := range regionAlterCmds { + t.Run(regionAlterCmd.name+"-"+tc.name, func(t *testing.T) { + var mu syncutil.Mutex + typeChangeStarted := make(chan struct{}) + typeChangeFinished := make(chan struct{}) + rbrOpFinished := make(chan struct{}) + + knobs := base.TestingKnobs{ + SQLTypeSchemaChanger: &sql.TypeSchemaChangerTestingKnobs{ + RunBeforeEnumMemberPromotion: func() error { + mu.Lock() + defer mu.Unlock() + close(typeChangeStarted) + <-rbrOpFinished + if !regionAlterCmd.shouldSucceed { + // Trigger a roll-back. + return errors.New("boom") + } + // Trod on. + return nil + }, + }, + } + + _, sqlDB, cleanup := multiregionccltestutils.TestingCreateMultiRegionCluster( + t, 4 /* numServers */, knobs, nil, /* baseDir */ + ) + defer cleanup() + + _, err := sqlDB.Exec(` +DROP DATABASE IF EXISTS db; +CREATE DATABASE db WITH PRIMARY REGION "us-east1" REGIONS "us-east2", "us-east3"; +CREATE TABLE db.rbr(k INT PRIMARY KEY, v INT NOT NULL) LOCALITY REGIONAL BY ROW; +`) + require.NoError(t, err) + + go func() { + defer func() { + close(typeChangeFinished) + }() + _, err := sqlDB.Exec(regionAlterCmd.cmd) + if regionAlterCmd.shouldSucceed { + if err != nil { + t.Errorf("expected success, got %v", err) + } + } else { + if !testutils.IsError(err, "boom") { + t.Errorf("expected error boom, found %v", err) + } + } + }() + + <-typeChangeStarted + + _, err = sqlDB.Exec(tc.op) + close(rbrOpFinished) + require.NoError(t, err) + + testutils.SucceedsSoon(t, func() error { + rows, err := sqlDB.Query("SELECT index_name, partition_name FROM [SHOW PARTITIONS FROM TABLE db.rbr] ORDER BY partition_name") + if err != nil { + return err + } + defer rows.Close() + + indexPartitions := make(map[string][]string) + for rows.Next() { + var indexName string + var partitionName string + if err := rows.Scan(&indexName, &partitionName); err != nil { + return err + } + + indexPartitions[indexName] = append(indexPartitions[indexName], partitionName) + } + + for _, expectedIndex := range tc.expectedIndexes { + partitions, found := indexPartitions[expectedIndex] + if !found { + return errors.AssertionFailedf("did not find index %s", expectedIndex) + } + + if len(partitions) != len(regionAlterCmd.expectedPartitions) { + return errors.AssertionFailedf( + "unexpected number of partitions; expected %d, found %d", + len(partitions), + len(regionAlterCmd.expectedPartitions), + ) + } + for i, expectedPartition := range regionAlterCmd.expectedPartitions { + if expectedPartition != partitions[i] { + return errors.AssertionFailedf( + "unexpected partitions; expected %v, found %v", + regionAlterCmd.expectedPartitions, + partitions, + ) + } + } + } + return nil + }) + <-typeChangeFinished + }) + } + } +} + func TestSettingPrimaryRegionAmidstDrop(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) From c2d84bf0053d8d5608e03eb849e5fbbea0c37cb9 Mon Sep 17 00:00:00 2001 From: Marius Posta <marius@cockroachlabs.com> Date: Thu, 15 Apr 2021 20:00:57 -0400 Subject: [PATCH 21/37] sql: use catalog.Mutation where possible This commit is a refactor which introduces catalog.Mutation wherever possible, in an effort to avoid using the descpb.DescriptorMutation type directly. Release note: None --- pkg/sql/backfill.go | 388 +++++++++---------- pkg/sql/catalog/table_elements.go | 88 +++++ pkg/sql/catalog/tabledesc/BUILD.bazel | 1 + pkg/sql/catalog/tabledesc/helpers_test.go | 11 + pkg/sql/catalog/tabledesc/mutation.go | 148 +++++++ pkg/sql/catalog/tabledesc/structured.go | 38 +- pkg/sql/catalog/tabledesc/structured_test.go | 4 +- pkg/sql/region_util.go | 9 +- pkg/sql/schema_changer.go | 274 ++++++------- pkg/sql/truncate.go | 13 +- 10 files changed, 569 insertions(+), 405 deletions(-) diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 43a4217908e2..e7e068a985b7 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -192,11 +192,11 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { var addedIndexSpans []roachpb.Span var addedIndexes []descpb.IndexID - var constraintsToDrop []descpb.ConstraintToUpdate - var constraintsToAddBeforeValidation []descpb.ConstraintToUpdate - var constraintsToValidate []descpb.ConstraintToUpdate + var constraintsToDrop []catalog.ConstraintToUpdate + var constraintsToAddBeforeValidation []catalog.ConstraintToUpdate + var constraintsToValidate []catalog.ConstraintToUpdate - var viewToRefresh *descpb.MaterializedViewRefresh + var viewToRefresh catalog.MaterializedViewRefresh // Note that this descriptor is intentionally not leased. If the schema change // held the lease, certain non-mutation related schema changes would not be @@ -229,76 +229,59 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { log.Infof(ctx, "running backfill for %q, v=%d", tableDesc.Name, tableDesc.Version) needColumnBackfill := false - for mutationIdx, m := range tableDesc.Mutations { - if m.MutationID != sc.mutationID { + for _, m := range tableDesc.AllMutations() { + if m.MutationID() != sc.mutationID { break } // If the current mutation is discarded, then // skip over processing. - if discarded, _ := isCurrentMutationDiscarded(tableDesc, m, mutationIdx+1); discarded { + if discarded, _ := isCurrentMutationDiscarded(tableDesc, m, m.MutationOrdinal()+1); discarded { continue } - switch m.Direction { - case descpb.DescriptorMutation_ADD: - switch t := m.Descriptor_.(type) { - case *descpb.DescriptorMutation_Column: - if tabledesc.ColumnNeedsBackfill(m.Direction, m.GetColumn()) { - needColumnBackfill = true - } - case *descpb.DescriptorMutation_Index: - addedIndexSpans = append(addedIndexSpans, tableDesc.IndexSpan(sc.execCfg.Codec, t.Index.ID)) - addedIndexes = append(addedIndexes, t.Index.ID) - case *descpb.DescriptorMutation_Constraint: - switch t.Constraint.ConstraintType { - case descpb.ConstraintToUpdate_CHECK: - if t.Constraint.Check.Validity == descpb.ConstraintValidity_Validating { - constraintsToAddBeforeValidation = append(constraintsToAddBeforeValidation, *t.Constraint) - constraintsToValidate = append(constraintsToValidate, *t.Constraint) - } - case descpb.ConstraintToUpdate_FOREIGN_KEY: - if t.Constraint.ForeignKey.Validity == descpb.ConstraintValidity_Validating { - constraintsToAddBeforeValidation = append(constraintsToAddBeforeValidation, *t.Constraint) - constraintsToValidate = append(constraintsToValidate, *t.Constraint) - } - case descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX: - if t.Constraint.UniqueWithoutIndexConstraint.Validity == descpb.ConstraintValidity_Validating { - constraintsToAddBeforeValidation = append(constraintsToAddBeforeValidation, *t.Constraint) - constraintsToValidate = append(constraintsToValidate, *t.Constraint) - } - case descpb.ConstraintToUpdate_NOT_NULL: + if m.Adding() { + if col := m.AsColumn(); col != nil { + needColumnBackfill = catalog.ColumnNeedsBackfill(col) + } else if idx := m.AsIndex(); idx != nil { + addedIndexSpans = append(addedIndexSpans, tableDesc.IndexSpan(sc.execCfg.Codec, idx.GetID())) + addedIndexes = append(addedIndexes, idx.GetID()) + } else if c := m.AsConstraint(); c != nil { + isValidating := false + if c.IsCheck() { + isValidating = c.Check().Validity == descpb.ConstraintValidity_Validating + } else if c.IsForeignKey() { + isValidating = c.ForeignKey().Validity == descpb.ConstraintValidity_Validating + } else if c.IsUniqueWithoutIndex() { + isValidating = c.UniqueWithoutIndex().Validity == descpb.ConstraintValidity_Validating + } else if c.IsNotNull() { // NOT NULL constraints are always validated before they can be added - constraintsToAddBeforeValidation = append(constraintsToAddBeforeValidation, *t.Constraint) - constraintsToValidate = append(constraintsToValidate, *t.Constraint) + isValidating = true + } + if isValidating { + constraintsToAddBeforeValidation = append(constraintsToAddBeforeValidation, c) + constraintsToValidate = append(constraintsToValidate, c) } - case *descpb.DescriptorMutation_PrimaryKeySwap, *descpb.DescriptorMutation_ComputedColumnSwap: + } else if mvRefresh := m.AsMaterializedViewRefresh(); mvRefresh != nil { + viewToRefresh = mvRefresh + } else if m.AsPrimaryKeySwap() != nil || m.AsComputedColumnSwap() != nil { // The backfiller doesn't need to do anything here. - case *descpb.DescriptorMutation_MaterializedViewRefresh: - viewToRefresh = t.MaterializedViewRefresh - default: - return errors.AssertionFailedf( - "unsupported mutation: %+v", m) + } else { + return errors.AssertionFailedf("unsupported mutation: %+v", m) } - case descpb.DescriptorMutation_DROP: - switch t := m.Descriptor_.(type) { - case *descpb.DescriptorMutation_Column: - if tabledesc.ColumnNeedsBackfill(m.Direction, m.GetColumn()) { - needColumnBackfill = true - } - case *descpb.DescriptorMutation_Index: - if !canClearRangeForDrop(t.Index) { - droppedIndexDescs = append(droppedIndexDescs, *t.Index) + } else if m.Dropped() { + if col := m.AsColumn(); col != nil { + needColumnBackfill = catalog.ColumnNeedsBackfill(col) + } else if idx := m.AsIndex(); idx != nil { + if !canClearRangeForDrop(idx.IndexDesc()) { + droppedIndexDescs = append(droppedIndexDescs, *idx.IndexDesc()) } - case *descpb.DescriptorMutation_Constraint: - constraintsToDrop = append(constraintsToDrop, *t.Constraint) - case *descpb.DescriptorMutation_PrimaryKeySwap, - *descpb.DescriptorMutation_ComputedColumnSwap, - *descpb.DescriptorMutation_MaterializedViewRefresh: + } else if c := m.AsConstraint(); c != nil { + constraintsToDrop = append(constraintsToDrop, c) + } else if m.AsPrimaryKeySwap() != nil || m.AsComputedColumnSwap() != nil || m.AsMaterializedViewRefresh() != nil { // The backfiller doesn't need to do anything here. - default: - return errors.AssertionFailedf( - "unsupported mutation: %+v", m) + } else { + return errors.AssertionFailedf("unsupported mutation: %+v", m) } } } @@ -381,16 +364,17 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { // the given constraint removed from it, and waits until the entire cluster is // on the new version of the table descriptor. It returns the new table descs. func (sc *SchemaChanger) dropConstraints( - ctx context.Context, constraints []descpb.ConstraintToUpdate, + ctx context.Context, constraints []catalog.ConstraintToUpdate, ) (map[descpb.ID]catalog.TableDescriptor, error) { log.Infof(ctx, "dropping %d constraints", len(constraints)) - fksByBackrefTable := make(map[descpb.ID][]*descpb.ConstraintToUpdate) - for i := range constraints { - c := &constraints[i] - if c.ConstraintType == descpb.ConstraintToUpdate_FOREIGN_KEY && - c.ForeignKey.ReferencedTableID != sc.descID { - fksByBackrefTable[c.ForeignKey.ReferencedTableID] = append(fksByBackrefTable[c.ForeignKey.ReferencedTableID], c) + fksByBackrefTable := make(map[descpb.ID][]catalog.ConstraintToUpdate) + for _, c := range constraints { + if c.IsForeignKey() { + id := c.ForeignKey().ReferencedTableID + if id != sc.descID { + fksByBackrefTable[id] = append(fksByBackrefTable[id], c) + } } } @@ -401,13 +385,11 @@ func (sc *SchemaChanger) dropConstraints( return err } b := txn.NewBatch() - for i := range constraints { - constraint := &constraints[i] - switch constraint.ConstraintType { - case descpb.ConstraintToUpdate_CHECK, descpb.ConstraintToUpdate_NOT_NULL: + for _, constraint := range constraints { + if constraint.IsCheck() || constraint.IsNotNull() { found := false for j, c := range scTable.Checks { - if c.Name == constraint.Name { + if c.Name == constraint.GetName() { scTable.Checks = append(scTable.Checks[:j], scTable.Checks[j+1:]...) found = true break @@ -418,18 +400,18 @@ func (sc *SchemaChanger) dropConstraints( ctx, 2, "backfiller tried to drop constraint %+v but it was not found, "+ "presumably due to a retry or rollback", - constraint, + constraint.ConstraintToUpdateDesc(), ) } - case descpb.ConstraintToUpdate_FOREIGN_KEY: + } else if constraint.IsForeignKey() { var foundExisting bool for j := range scTable.OutboundFKs { def := &scTable.OutboundFKs[j] - if def.Name != constraint.Name { + if def.Name != constraint.GetName() { continue } backrefTable, err := descsCol.GetMutableTableVersionByID(ctx, - constraint.ForeignKey.ReferencedTableID, txn) + constraint.ForeignKey().ReferencedTableID, txn) if err != nil { return err } @@ -452,13 +434,13 @@ func (sc *SchemaChanger) dropConstraints( ctx, 2, "backfiller tried to drop constraint %+v but it was not found, "+ "presumably due to a retry or rollback", - constraint, + constraint.ConstraintToUpdateDesc(), ) } - case descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX: + } else if constraint.IsUniqueWithoutIndex() { found := false for j, c := range scTable.UniqueWithoutIndexConstraints { - if c.Name == constraint.Name { + if c.Name == constraint.GetName() { scTable.UniqueWithoutIndexConstraints = append( scTable.UniqueWithoutIndexConstraints[:j], scTable.UniqueWithoutIndexConstraints[j+1:]..., @@ -472,7 +454,7 @@ func (sc *SchemaChanger) dropConstraints( ctx, 2, "backfiller tried to drop constraint %+v but it was not found, "+ "presumably due to a retry or rollback", - constraint, + constraint.ConstraintToUpdateDesc(), ) } } @@ -524,16 +506,17 @@ func (sc *SchemaChanger) dropConstraints( // given constraint added to it, and waits until the entire cluster is on // the new version of the table descriptor. func (sc *SchemaChanger) addConstraints( - ctx context.Context, constraints []descpb.ConstraintToUpdate, + ctx context.Context, constraints []catalog.ConstraintToUpdate, ) error { log.Infof(ctx, "adding %d constraints", len(constraints)) - fksByBackrefTable := make(map[descpb.ID][]*descpb.ConstraintToUpdate) - for i := range constraints { - c := &constraints[i] - if c.ConstraintType == descpb.ConstraintToUpdate_FOREIGN_KEY && - c.ForeignKey.ReferencedTableID != sc.descID { - fksByBackrefTable[c.ForeignKey.ReferencedTableID] = append(fksByBackrefTable[c.ForeignKey.ReferencedTableID], c) + fksByBackrefTable := make(map[descpb.ID][]catalog.ConstraintToUpdate) + for _, c := range constraints { + if c.IsForeignKey() { + id := c.ForeignKey().ReferencedTableID + if id != sc.descID { + fksByBackrefTable[id] = append(fksByBackrefTable[id], c) + } } } @@ -547,13 +530,11 @@ func (sc *SchemaChanger) addConstraints( } b := txn.NewBatch() - for i := range constraints { - constraint := &constraints[i] - switch constraint.ConstraintType { - case descpb.ConstraintToUpdate_CHECK, descpb.ConstraintToUpdate_NOT_NULL: + for _, constraint := range constraints { + if constraint.IsCheck() || constraint.IsNotNull() { found := false for _, c := range scTable.Checks { - if c.Name == constraint.Name { + if c.Name == constraint.GetName() { log.VEventf( ctx, 2, "backfiller tried to add constraint %+v but found existing constraint %+v, "+ @@ -568,19 +549,19 @@ func (sc *SchemaChanger) addConstraints( } } if !found { - scTable.Checks = append(scTable.Checks, &constraints[i].Check) + scTable.Checks = append(scTable.Checks, &constraint.ConstraintToUpdateDesc().Check) } - case descpb.ConstraintToUpdate_FOREIGN_KEY: + } else if constraint.IsForeignKey() { var foundExisting bool for j := range scTable.OutboundFKs { def := &scTable.OutboundFKs[j] - if def.Name == constraint.Name { + if def.Name == constraint.GetName() { if log.V(2) { log.VEventf( ctx, 2, "backfiller tried to add constraint %+v but found existing constraint %+v, "+ "presumably due to a retry or rollback", - constraint, def, + constraint.ConstraintToUpdateDesc(), def, ) } // Ensure the constraint on the descriptor is set to Validating, in @@ -591,8 +572,8 @@ func (sc *SchemaChanger) addConstraints( } } if !foundExisting { - scTable.OutboundFKs = append(scTable.OutboundFKs, constraint.ForeignKey) - backrefTable, err := descsCol.GetMutableTableVersionByID(ctx, constraint.ForeignKey.ReferencedTableID, txn) + scTable.OutboundFKs = append(scTable.OutboundFKs, constraint.ForeignKey()) + backrefTable, err := descsCol.GetMutableTableVersionByID(ctx, constraint.ForeignKey().ReferencedTableID, txn) if err != nil { return err } @@ -600,11 +581,11 @@ func (sc *SchemaChanger) addConstraints( // referenced table. It's possible for the unique index found during // planning to have been dropped in the meantime, since only the // presence of the backreference prevents it. - _, err = tabledesc.FindFKReferencedUniqueConstraint(backrefTable, constraint.ForeignKey.ReferencedColumnIDs) + _, err = tabledesc.FindFKReferencedUniqueConstraint(backrefTable, constraint.ForeignKey().ReferencedColumnIDs) if err != nil { return err } - backrefTable.InboundFKs = append(backrefTable.InboundFKs, constraint.ForeignKey) + backrefTable.InboundFKs = append(backrefTable.InboundFKs, constraint.ForeignKey()) // Note that this code may add the same descriptor to the batch // multiple times if it is referenced multiple times. That's fine as @@ -618,15 +599,15 @@ func (sc *SchemaChanger) addConstraints( } } } - case descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX: + } else if constraint.IsUniqueWithoutIndex() { found := false for _, c := range scTable.UniqueWithoutIndexConstraints { - if c.Name == constraint.Name { + if c.Name == constraint.GetName() { log.VEventf( ctx, 2, "backfiller tried to add constraint %+v but found existing constraint %+v, "+ "presumably due to a retry or rollback", - constraint, c, + constraint.ConstraintToUpdateDesc(), c, ) // Ensure the constraint on the descriptor is set to Validating, in // case we're in the middle of rolling back DROP CONSTRAINT @@ -636,9 +617,8 @@ func (sc *SchemaChanger) addConstraints( } } if !found { - scTable.UniqueWithoutIndexConstraints = append( - scTable.UniqueWithoutIndexConstraints, constraints[i].UniqueWithoutIndexConstraint, - ) + scTable.UniqueWithoutIndexConstraints = append(scTable.UniqueWithoutIndexConstraints, + constraint.UniqueWithoutIndex()) } } } @@ -670,7 +650,7 @@ func (sc *SchemaChanger) addConstraints( // This operates over multiple goroutines concurrently and is thus not // able to reuse the original kv.Txn safely, so it makes its own. func (sc *SchemaChanger) validateConstraints( - ctx context.Context, constraints []descpb.ConstraintToUpdate, + ctx context.Context, constraints []catalog.ConstraintToUpdate, ) error { if lease.TestingTableLeasesAreDisabled() { return nil @@ -731,28 +711,27 @@ func (sc *SchemaChanger) validateConstraints( // TODO (rohany): When to release this? As of now this is only going to get released // after the check is validated. defer func() { collection.ReleaseAll(ctx) }() - switch c.ConstraintType { - case descpb.ConstraintToUpdate_CHECK: - if err := validateCheckInTxn(ctx, sc.leaseMgr, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check.Expr); err != nil { + if c.IsCheck() { + if err := validateCheckInTxn(ctx, sc.leaseMgr, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check().Expr); err != nil { return err } - case descpb.ConstraintToUpdate_FOREIGN_KEY: - if err := validateFkInTxn(ctx, sc.leaseMgr, &evalCtx.EvalContext, desc, txn, c.Name); err != nil { + } else if c.IsForeignKey() { + if err := validateFkInTxn(ctx, sc.leaseMgr, &evalCtx.EvalContext, desc, txn, c.GetName()); err != nil { return err } - case descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX: - if err := validateUniqueWithoutIndexConstraintInTxn(ctx, &evalCtx.EvalContext, desc, txn, c.Name); err != nil { + } else if c.IsUniqueWithoutIndex() { + if err := validateUniqueWithoutIndexConstraintInTxn(ctx, &evalCtx.EvalContext, desc, txn, c.GetName()); err != nil { return err } - case descpb.ConstraintToUpdate_NOT_NULL: - if err := validateCheckInTxn(ctx, sc.leaseMgr, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check.Expr); err != nil { + } else if c.IsNotNull() { + if err := validateCheckInTxn(ctx, sc.leaseMgr, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check().Expr); err != nil { // TODO (lucy): This should distinguish between constraint // validation errors and other types of unexpected errors, and // return a different error code in the former case return errors.Wrap(err, "validation of NOT NULL constraint failed") } - default: - return errors.Errorf("unsupported constraint type: %d", c.ConstraintType) + } else { + return errors.Errorf("unsupported constraint type: %d", c.ConstraintToUpdateDesc().ConstraintType) } return nil }) @@ -1975,85 +1954,72 @@ func runSchemaChangesInTxn( // in the world of transactional schema changes. // Collect constraint mutations to process later. - var constraintAdditionMutations []descpb.DescriptorMutation + var constraintAdditionMutations []catalog.ConstraintToUpdate // We use a range loop here as the processing of some mutations // such as the primary key swap mutations result in queueing more // mutations that need to be processed. - for i := 0; i < len(tableDesc.Mutations); i++ { - m := tableDesc.Mutations[i] + for _, m := range tableDesc.AllMutations() { // Skip mutations that get canceled by later operations - if discarded, _ := isCurrentMutationDiscarded(tableDesc, m, i+1); discarded { + if discarded, _ := isCurrentMutationDiscarded(tableDesc, m, m.MutationOrdinal()+1); discarded { continue } immutDesc := tabledesc.NewBuilder(tableDesc.TableDesc()).BuildImmutableTable() - switch m.Direction { - case descpb.DescriptorMutation_ADD: - switch m.Descriptor_.(type) { - case *descpb.DescriptorMutation_PrimaryKeySwap: + + if m.Adding() { + if m.AsPrimaryKeySwap() != nil { // Don't need to do anything here, as the call to MakeMutationComplete // will perform the steps for this operation. - case *descpb.DescriptorMutation_ComputedColumnSwap: + } else if m.AsComputedColumnSwap() != nil { return AlterColTypeInTxnNotSupportedErr - case *descpb.DescriptorMutation_Column: - if doneColumnBackfill || !tabledesc.ColumnNeedsBackfill(m.Direction, m.GetColumn()) { - break - } - if err := columnBackfillInTxn(ctx, planner.Txn(), planner.EvalContext(), planner.SemaCtx(), immutDesc, traceKV); err != nil { - return err + } else if col := m.AsColumn(); col != nil { + if !doneColumnBackfill && catalog.ColumnNeedsBackfill(col) { + if err := columnBackfillInTxn(ctx, planner.Txn(), planner.EvalContext(), planner.SemaCtx(), immutDesc, traceKV); err != nil { + return err + } + doneColumnBackfill = true } - doneColumnBackfill = true - - case *descpb.DescriptorMutation_Index: + } else if idx := m.AsIndex(); idx != nil { if err := indexBackfillInTxn(ctx, planner.Txn(), planner.EvalContext(), planner.SemaCtx(), immutDesc, traceKV); err != nil { return err } - - case *descpb.DescriptorMutation_Constraint: + } else if c := m.AsConstraint(); c != nil { // This is processed later. Do not proceed to MakeMutationComplete. - constraintAdditionMutations = append(constraintAdditionMutations, m) + constraintAdditionMutations = append(constraintAdditionMutations, c) continue - - default: - return errors.AssertionFailedf( - "unsupported mutation: %+v", m) + } else { + return errors.AssertionFailedf("unsupported mutation: %+v", m) } - - case descpb.DescriptorMutation_DROP: + } else if m.Dropped() { // Drop the name and drop the associated data later. - switch t := m.Descriptor_.(type) { - case *descpb.DescriptorMutation_Column: - if doneColumnBackfill || !tabledesc.ColumnNeedsBackfill(m.Direction, m.GetColumn()) { - break - } - if err := columnBackfillInTxn( - ctx, planner.Txn(), planner.EvalContext(), planner.SemaCtx(), immutDesc, traceKV, - ); err != nil { - return err + if col := m.AsColumn(); col != nil { + if !doneColumnBackfill && catalog.ColumnNeedsBackfill(col) { + if err := columnBackfillInTxn( + ctx, planner.Txn(), planner.EvalContext(), planner.SemaCtx(), immutDesc, traceKV, + ); err != nil { + return err + } + doneColumnBackfill = true } - doneColumnBackfill = true - - case *descpb.DescriptorMutation_Index: + } else if idx := m.AsIndex(); idx != nil { if err := indexTruncateInTxn( - ctx, planner.Txn(), planner.ExecCfg(), planner.EvalContext(), immutDesc, t.Index, traceKV, + ctx, planner.Txn(), planner.ExecCfg(), planner.EvalContext(), immutDesc, idx.IndexDesc(), traceKV, ); err != nil { return err } - - case *descpb.DescriptorMutation_Constraint: - switch t.Constraint.ConstraintType { - case descpb.ConstraintToUpdate_CHECK, descpb.ConstraintToUpdate_NOT_NULL: + } else if c := m.AsConstraint(); c != nil { + if c.IsCheck() || c.IsNotNull() { for i := range tableDesc.Checks { - if tableDesc.Checks[i].Name == t.Constraint.Name { + if tableDesc.Checks[i].Name == c.GetName() { tableDesc.Checks = append(tableDesc.Checks[:i], tableDesc.Checks[i+1:]...) break } } - case descpb.ConstraintToUpdate_FOREIGN_KEY: + } else if c.IsForeignKey() { for i := range tableDesc.OutboundFKs { fk := &tableDesc.OutboundFKs[i] - if fk.Name == t.Constraint.Name { + if fk.Name == c.GetName() { if err := planner.removeFKBackReference(ctx, tableDesc, fk); err != nil { return err } @@ -2061,9 +2027,9 @@ func runSchemaChangesInTxn( break } } - case descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX: + } else if c.IsUniqueWithoutIndex() { for i := range tableDesc.UniqueWithoutIndexConstraints { - if tableDesc.UniqueWithoutIndexConstraints[i].Name == t.Constraint.Name { + if tableDesc.UniqueWithoutIndexConstraints[i].Name == c.GetName() { tableDesc.UniqueWithoutIndexConstraints = append( tableDesc.UniqueWithoutIndexConstraints[:i], tableDesc.UniqueWithoutIndexConstraints[i+1:]..., @@ -2071,17 +2037,13 @@ func runSchemaChangesInTxn( break } } - default: - return errors.AssertionFailedf( - "unsupported constraint type: %d", errors.Safe(t.Constraint.ConstraintType)) + } else { + return errors.AssertionFailedf("unsupported constraint type: %d", c.ConstraintToUpdateDesc().ConstraintType) } - - default: - return errors.AssertionFailedf("unsupported mutation: %+v", m) } - } - if err := tableDesc.MakeMutationComplete(m); err != nil { + + if err := tableDesc.MakeMutationComplete(tableDesc.Mutations[m.MutationOrdinal()]); err != nil { return err } @@ -2089,15 +2051,14 @@ func runSchemaChangesInTxn( // extra work that needs to be done. Note that we don't need to create // a job to clean up the dropped indexes because those mutations can // get processed in this txn on the new table. - if pkSwap := m.GetPrimaryKeySwap(); pkSwap != nil { + if pkSwap := m.AsPrimaryKeySwap(); pkSwap != nil { // If any old index had an interleaved parent, remove the // backreference from the parent. // N.B. This logic needs to be kept up to date with the // corresponding piece in (*SchemaChanger).done. It is slightly // different because of how it access tables and how it needs to // write the modified table descriptors explicitly. - for _, idxID := range append( - []descpb.IndexID{pkSwap.OldPrimaryIndexId}, pkSwap.OldIndexes...) { + err := pkSwap.ForEachOldIndexIDs(func(idxID descpb.IndexID) error { oldIndex, err := tableDesc.FindIndexWithID(idxID) if err != nil { return err @@ -2134,27 +2095,42 @@ func runSchemaChangesInTxn( } } } + return nil + }) + if err != nil { + return err } } } // Clear all the mutations except for adding constraints. - tableDesc.Mutations = constraintAdditionMutations + tableDesc.Mutations = make([]descpb.DescriptorMutation, len(constraintAdditionMutations)) + for i, c := range constraintAdditionMutations { + tableDesc.Mutations[i] = descpb.DescriptorMutation{ + Descriptor_: &descpb.DescriptorMutation_Constraint{Constraint: c.ConstraintToUpdateDesc()}, + Direction: descpb.DescriptorMutation_ADD, + MutationID: c.MutationID(), + } + if c.DeleteOnly() { + tableDesc.Mutations[i].State = descpb.DescriptorMutation_DELETE_ONLY + } else if c.WriteAndDeleteOnly() { + tableDesc.Mutations[i].State = descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY + } + } // Now that the table descriptor is in a valid state with all column and index // mutations applied, it can be used for validating check/FK constraints. - for _, m := range constraintAdditionMutations { - constraint := m.GetConstraint() - switch constraint.ConstraintType { - case descpb.ConstraintToUpdate_CHECK, descpb.ConstraintToUpdate_NOT_NULL: - if constraint.Check.Validity == descpb.ConstraintValidity_Validating { + for _, c := range constraintAdditionMutations { + if c.IsCheck() || c.IsNotNull() { + check := &c.ConstraintToUpdateDesc().Check + if check.Validity == descpb.ConstraintValidity_Validating { if err := validateCheckInTxn( - ctx, planner.Descriptors().LeaseManager(), &planner.semaCtx, planner.EvalContext(), tableDesc, planner.txn, constraint.Check.Expr, + ctx, planner.Descriptors().LeaseManager(), &planner.semaCtx, planner.EvalContext(), tableDesc, planner.txn, check.Expr, ); err != nil { return err } - constraint.Check.Validity = descpb.ConstraintValidity_Validated + check.Validity = descpb.ConstraintValidity_Validated } - case descpb.ConstraintToUpdate_FOREIGN_KEY: + } else if c.IsForeignKey() { // We can't support adding a validated foreign key constraint in the same // transaction as the CREATE TABLE statement. This would require adding // the backreference to the other table and then validating the constraint @@ -2167,32 +2143,31 @@ func runSchemaChangesInTxn( // schema change framework eventually. // // For now, just always add the FK as unvalidated. - constraint.ForeignKey.Validity = descpb.ConstraintValidity_Unvalidated - case descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX: - if constraint.UniqueWithoutIndexConstraint.Validity == descpb.ConstraintValidity_Validating { + c.ConstraintToUpdateDesc().ForeignKey.Validity = descpb.ConstraintValidity_Unvalidated + } else if c.IsUniqueWithoutIndex() { + uwi := &c.ConstraintToUpdateDesc().UniqueWithoutIndexConstraint + if uwi.Validity == descpb.ConstraintValidity_Validating { if err := validateUniqueWithoutIndexConstraintInTxn( - ctx, planner.EvalContext(), tableDesc, planner.txn, constraint.Name, + ctx, planner.EvalContext(), tableDesc, planner.txn, c.GetName(), ); err != nil { return err } - constraint.UniqueWithoutIndexConstraint.Validity = descpb.ConstraintValidity_Validated + uwi.Validity = descpb.ConstraintValidity_Validated } - default: - return errors.AssertionFailedf( - "unsupported constraint type: %d", errors.Safe(constraint.ConstraintType)) + } else { + return errors.AssertionFailedf("unsupported constraint type: %d", c.ConstraintToUpdateDesc().ConstraintType) } + } // Finally, add the constraints. We bypass MakeMutationsComplete (which makes // certain assumptions about the state in the usual schema changer) and just // update the table descriptor directly. - for _, m := range constraintAdditionMutations { - constraint := m.GetConstraint() - switch constraint.ConstraintType { - case descpb.ConstraintToUpdate_CHECK, descpb.ConstraintToUpdate_NOT_NULL: - tableDesc.Checks = append(tableDesc.Checks, &constraint.Check) - case descpb.ConstraintToUpdate_FOREIGN_KEY: - fk := constraint.ForeignKey + for _, c := range constraintAdditionMutations { + if c.IsCheck() || c.IsNotNull() { + tableDesc.Checks = append(tableDesc.Checks, &c.ConstraintToUpdateDesc().Check) + } else if c.IsForeignKey() { + fk := c.ConstraintToUpdateDesc().ForeignKey var referencedTableDesc *tabledesc.Mutable // We don't want to lookup/edit a second copy of the same table. selfReference := tableDesc.ID == fk.ReferencedTableID @@ -2219,13 +2194,12 @@ func runSchemaChangesInTxn( return err } } - case descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX: + } else if c.IsUniqueWithoutIndex() { tableDesc.UniqueWithoutIndexConstraints = append( - tableDesc.UniqueWithoutIndexConstraints, constraint.UniqueWithoutIndexConstraint, + tableDesc.UniqueWithoutIndexConstraints, c.ConstraintToUpdateDesc().UniqueWithoutIndexConstraint, ) - default: - return errors.AssertionFailedf( - "unsupported constraint type: %d", errors.Safe(constraint.ConstraintType)) + } else { + return errors.AssertionFailedf("unsupported constraint type: %d", c.ConstraintToUpdateDesc().ConstraintType) } } tableDesc.Mutations = nil diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go index 9f9f24ea9fb6..4553d49f6874 100644 --- a/pkg/sql/catalog/table_elements.go +++ b/pkg/sql/catalog/table_elements.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" ) @@ -272,6 +273,35 @@ type ConstraintToUpdate interface { // ConstraintToUpdateDesc returns the underlying protobuf descriptor. ConstraintToUpdateDesc() *descpb.ConstraintToUpdate + + // GetName returns the name of this constraint update mutation. + GetName() string + + // IsCheck returns true iff this is an update for a check constraint. + IsCheck() bool + + // IsForeignKey returns true iff this is an update for a fk constraint. + IsForeignKey() bool + + // IsNotNull returns true iff this is an update for a not-null constraint. + IsNotNull() bool + + // IsUniqueWithoutIndex returns true iff this is an update for a unique + // without index constraint. + IsUniqueWithoutIndex() bool + + // Check returns the underlying check constraint, if there is one. + Check() descpb.TableDescriptor_CheckConstraint + + // ForeignKey returns the underlying fk constraint, if there is one. + ForeignKey() descpb.ForeignKeyConstraint + + // NotNullColumnID returns the underlying not-null column ID, if there is one. + NotNullColumnID() descpb.ColumnID + + // UniqueWithoutIndex returns the underlying unique without index constraint, if + // there is one. + UniqueWithoutIndex() descpb.UniqueWithoutIndexConstraint } // PrimaryKeySwap is an interface around a primary key swap mutation. @@ -280,6 +310,26 @@ type PrimaryKeySwap interface { // PrimaryKeySwapDesc returns the underlying protobuf descriptor. PrimaryKeySwapDesc() *descpb.PrimaryKeySwap + + // NumOldIndexes returns the number of old active indexes to swap out. + NumOldIndexes() int + + // ForEachOldIndexIDs iterates through each of the old index IDs. + // iterutil.Done is supported. + ForEachOldIndexIDs(fn func(id descpb.IndexID) error) error + + // NumNewIndexes returns the number of new active indexes to swap in. + NumNewIndexes() int + + // ForEachNewIndexIDs iterates through each of the new index IDs. + // iterutil.Done is supported. + ForEachNewIndexIDs(fn func(id descpb.IndexID) error) error + + // HasLocalityConfig returns true iff the locality config is swapped also. + HasLocalityConfig() bool + + // LocalityConfigSwap returns the locality config swap, if there is one. + LocalityConfigSwap() descpb.PrimaryKeySwap_LocalityConfigSwap } // ComputedColumnSwap is an interface around a computed column swap mutation. @@ -297,6 +347,21 @@ type MaterializedViewRefresh interface { // MaterializedViewRefreshDesc returns the underlying protobuf descriptor. MaterializedViewRefreshDesc() *descpb.MaterializedViewRefresh + + // ShouldBackfill returns true iff the query should be backfilled into the + // indexes. + ShouldBackfill() bool + + // AsOf returns the timestamp at which the query should be run. + AsOf() hlc.Timestamp + + // ForEachIndexID iterates through each of the index IDs. + // iterutil.Done is supported. + ForEachIndexID(func(id descpb.IndexID) error) error + + // TableWithNewIndexes returns a new TableDescriptor based on the old one + // but with the refreshed indexes put in. + TableWithNewIndexes(tbl TableDescriptor) TableDescriptor } func isIndexInSearchSet(desc TableDescriptor, opts IndexOpts, idx Index) bool { @@ -490,3 +555,26 @@ func ColumnTypesWithVirtualCol(columns []Column, virtualCol Column) []*types.T { } return t } + +// ColumnNeedsBackfill returns true if adding or dropping (according to +// the direction) the given column requires backfill. +func ColumnNeedsBackfill(col Column) bool { + if col.IsVirtual() { + // Virtual columns are not stored in the primary index, so they do not need + // backfill. + return false + } + if col.Dropped() { + // In all other cases, DROP requires backfill. + return true + } + // ADD requires backfill for: + // - columns with non-NULL default value + // - computed columns + // - non-nullable columns (note: if a non-nullable column doesn't have a + // default value, the backfill will fail unless the table is empty). + if col.ColumnDesc().HasNullDefault() { + return false + } + return col.HasDefault() || !col.IsNullable() || col.IsComputed() +} diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index a0123ece4c6c..1040537c211f 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -37,6 +37,7 @@ go_library( "//pkg/util/errorutil/unimplemented", "//pkg/util/hlc", "//pkg/util/interval", + "//pkg/util/iterutil", "//pkg/util/log", "//pkg/util/protoutil", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/catalog/tabledesc/helpers_test.go b/pkg/sql/catalog/tabledesc/helpers_test.go index 9b1eac6fb624..cb4d075d0c4a 100644 --- a/pkg/sql/catalog/tabledesc/helpers_test.go +++ b/pkg/sql/catalog/tabledesc/helpers_test.go @@ -12,6 +12,7 @@ package tabledesc import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/errors" ) @@ -34,3 +35,13 @@ func GetPostDeserializationChanges( } var FitColumnToFamily = fitColumnToFamily + +func TestingMakeColumn( + direction descpb.DescriptorMutation_Direction, desc *descpb.ColumnDescriptor, +) catalog.Column { + return &column{ + maybeMutation: maybeMutation{mutationDirection: direction}, + desc: desc, + ordinal: 0, + } +} diff --git a/pkg/sql/catalog/tabledesc/mutation.go b/pkg/sql/catalog/tabledesc/mutation.go index f472c372a816..b0ba13c09000 100644 --- a/pkg/sql/catalog/tabledesc/mutation.go +++ b/pkg/sql/catalog/tabledesc/mutation.go @@ -13,6 +13,8 @@ package tabledesc import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/iterutil" ) var _ catalog.TableElementMaybeMutation = maybeMutation{} @@ -71,6 +73,7 @@ func (mm maybeMutation) Dropped() bool { } // constraintToUpdate implements the catalog.ConstraintToUpdate interface. +// It also type constraintToUpdate struct { maybeMutation desc *descpb.ConstraintToUpdate @@ -81,6 +84,53 @@ func (c constraintToUpdate) ConstraintToUpdateDesc() *descpb.ConstraintToUpdate return c.desc } +// GetName returns the name of this constraint update mutation. +func (c constraintToUpdate) GetName() string { + return c.desc.Name +} + +// IsCheck returns true iff this is an update for a check constraint. +func (c constraintToUpdate) IsCheck() bool { + return c.desc.ConstraintType == descpb.ConstraintToUpdate_CHECK +} + +// Check returns the underlying check constraint, if there is one. +func (c constraintToUpdate) Check() descpb.TableDescriptor_CheckConstraint { + return c.desc.Check +} + +// IsForeignKey returns true iff this is an update for a fk constraint. +func (c constraintToUpdate) IsForeignKey() bool { + return c.desc.ConstraintType == descpb.ConstraintToUpdate_FOREIGN_KEY +} + +// ForeignKey returns the underlying fk constraint, if there is one. +func (c constraintToUpdate) ForeignKey() descpb.ForeignKeyConstraint { + return c.desc.ForeignKey +} + +// IsNotNull returns true iff this is an update for a not-null constraint. +func (c constraintToUpdate) IsNotNull() bool { + return c.desc.ConstraintType == descpb.ConstraintToUpdate_NOT_NULL +} + +// NotNullColumnID returns the underlying not-null column ID, if there is one. +func (c constraintToUpdate) NotNullColumnID() descpb.ColumnID { + return c.desc.NotNullColumn +} + +// IsUniqueWithoutIndex returns true iff this is an update for a unique without +// index constraint. +func (c constraintToUpdate) IsUniqueWithoutIndex() bool { + return c.desc.ConstraintType == descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX +} + +// UniqueWithoutIndex returns the underlying unique without index constraint, if +// there is one. +func (c constraintToUpdate) UniqueWithoutIndex() descpb.UniqueWithoutIndexConstraint { + return c.desc.UniqueWithoutIndexConstraint +} + // primaryKeySwap implements the catalog.PrimaryKeySwap interface. type primaryKeySwap struct { maybeMutation @@ -92,6 +142,60 @@ func (c primaryKeySwap) PrimaryKeySwapDesc() *descpb.PrimaryKeySwap { return c.desc } +// NumOldIndexes returns the number of old active indexes to swap out. +func (c primaryKeySwap) NumOldIndexes() int { + return 1 + len(c.desc.OldIndexes) +} + +// ForEachOldIndexIDs iterates through each of the old index IDs. +// iterutil.Done is supported. +func (c primaryKeySwap) ForEachOldIndexIDs(fn func(id descpb.IndexID) error) error { + return c.forEachIndexIDs(c.desc.OldPrimaryIndexId, c.desc.OldIndexes, fn) +} + +// NumNewIndexes returns the number of new active indexes to swap in. +func (c primaryKeySwap) NumNewIndexes() int { + return 1 + len(c.desc.NewIndexes) +} + +// ForEachNewIndexIDs iterates through each of the new index IDs. +// iterutil.Done is supported. +func (c primaryKeySwap) ForEachNewIndexIDs(fn func(id descpb.IndexID) error) error { + return c.forEachIndexIDs(c.desc.NewPrimaryIndexId, c.desc.NewIndexes, fn) +} + +func (c primaryKeySwap) forEachIndexIDs( + pkID descpb.IndexID, secIDs []descpb.IndexID, fn func(id descpb.IndexID) error, +) error { + err := fn(pkID) + if err != nil { + if iterutil.Done(err) { + return nil + } + return err + } + for _, id := range secIDs { + err = fn(id) + if err != nil { + if iterutil.Done(err) { + return nil + } + return err + } + } + return nil +} + +// HasLocalityConfig returns true iff the locality config is swapped also. +func (c primaryKeySwap) HasLocalityConfig() bool { + return c.desc.LocalityConfigSwap != nil +} + +// LocalityConfigSwap returns the locality config swap, if there is one. +func (c primaryKeySwap) LocalityConfigSwap() descpb.PrimaryKeySwap_LocalityConfigSwap { + return *c.desc.LocalityConfigSwap +} + // computedColumnSwap implements the catalog.ComputedColumnSwap interface. type computedColumnSwap struct { maybeMutation @@ -114,6 +218,50 @@ func (c materializedViewRefresh) MaterializedViewRefreshDesc() *descpb.Materiali return c.desc } +// ShouldBackfill returns true iff the query should be backfilled into the +// indexes. +func (c materializedViewRefresh) ShouldBackfill() bool { + return c.desc.ShouldBackfill +} + +// AsOf returns the timestamp at which the query should be run. +func (c materializedViewRefresh) AsOf() hlc.Timestamp { + return c.desc.AsOf +} + +// ForEachIndexID iterates through each of the index IDs. +// iterutil.Done is supported. +func (c materializedViewRefresh) ForEachIndexID(fn func(id descpb.IndexID) error) error { + err := fn(c.desc.NewPrimaryIndex.ID) + if err != nil { + if iterutil.Done(err) { + return nil + } + return err + } + for i := range c.desc.NewIndexes { + err = fn(c.desc.NewIndexes[i].ID) + if err != nil { + if iterutil.Done(err) { + return nil + } + return err + } + } + return nil +} + +// TableWithNewIndexes returns a new TableDescriptor based on the old one +// but with the refreshed indexes put in. +func (c materializedViewRefresh) TableWithNewIndexes( + tbl catalog.TableDescriptor, +) catalog.TableDescriptor { + deepCopy := NewBuilder(tbl.TableDesc()).BuildCreatedMutableTable().TableDesc() + deepCopy.PrimaryIndex = c.desc.NewPrimaryIndex + deepCopy.Indexes = c.desc.NewIndexes + return NewBuilder(deepCopy).BuildImmutableTable() +} + // mutation implements the type mutation struct { maybeMutation diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 493df5ad36e6..00da3a382732 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -2053,31 +2053,6 @@ func (desc *wrapper) MakePublic() catalog.TableDescriptor { return table } -// ColumnNeedsBackfill returns true if adding or dropping (according to -// the direction) the given column requires backfill. -func ColumnNeedsBackfill( - direction descpb.DescriptorMutation_Direction, desc *descpb.ColumnDescriptor, -) bool { - if desc.Virtual { - // Virtual columns are not stored in the primary index, so they do not need - // backfill. - return false - } - if direction == descpb.DescriptorMutation_DROP { - // In all other cases, DROP requires backfill. - return true - } - // ADD requires backfill for: - // - columns with non-NULL default value - // - computed columns - // - non-nullable columns (note: if a non-nullable column doesn't have a - // default value, the backfill will fail unless the table is empty). - if desc.HasNullDefault() { - return false - } - return desc.HasDefault() || !desc.Nullable || desc.IsComputed() -} - // HasPrimaryKey returns true if the table has a primary key. func (desc *wrapper) HasPrimaryKey() bool { return !desc.PrimaryIndex.Disabled @@ -2086,14 +2061,11 @@ func (desc *wrapper) HasPrimaryKey() bool { // HasColumnBackfillMutation returns whether the table has any queued column // mutations that require a backfill. func (desc *wrapper) HasColumnBackfillMutation() bool { - for _, m := range desc.Mutations { - col := m.GetColumn() - if col == nil { - // Index backfills don't affect changefeeds. - continue - } - if ColumnNeedsBackfill(m.Direction, col) { - return true + for _, m := range desc.AllMutations() { + if col := m.AsColumn(); col != nil { + if catalog.ColumnNeedsBackfill(col) { + return true + } } } return false diff --git a/pkg/sql/catalog/tabledesc/structured_test.go b/pkg/sql/catalog/tabledesc/structured_test.go index b05fb47652b7..74de8f53f541 100644 --- a/pkg/sql/catalog/tabledesc/structured_test.go +++ b/pkg/sql/catalog/tabledesc/structured_test.go @@ -446,10 +446,10 @@ func TestColumnNeedsBackfill(t *testing.T) { } for _, tc := range testCases { - if ColumnNeedsBackfill(descpb.DescriptorMutation_ADD, &tc.desc) != tc.add { + if catalog.ColumnNeedsBackfill(TestingMakeColumn(descpb.DescriptorMutation_ADD, &tc.desc)) != tc.add { t.Errorf("expected ColumnNeedsBackfill to be %v for adding %s", tc.add, tc.info) } - if ColumnNeedsBackfill(descpb.DescriptorMutation_DROP, &tc.desc) != tc.drop { + if catalog.ColumnNeedsBackfill(TestingMakeColumn(descpb.DescriptorMutation_DROP, &tc.desc)) != tc.drop { t.Errorf("expected ColumnNeedsBackfill to be %v for dropping %s", tc.drop, tc.info) } } diff --git a/pkg/sql/region_util.go b/pkg/sql/region_util.go index 6ef51bb4563c..9c6cd1a84842 100644 --- a/pkg/sql/region_util.go +++ b/pkg/sql/region_util.go @@ -1481,12 +1481,11 @@ func (p *planner) validateZoneConfigForMultiRegionTable( regionalByRowNewIndexes := make(map[uint32]struct{}) for _, mut := range desc.AllMutations() { if pkSwap := mut.AsPrimaryKeySwap(); pkSwap != nil { - swapDesc := pkSwap.PrimaryKeySwapDesc() - if swapDesc.LocalityConfigSwap != nil { - for _, id := range swapDesc.NewIndexes { + if pkSwap.HasLocalityConfig() { + _ = pkSwap.ForEachNewIndexIDs(func(id descpb.IndexID) error { regionalByRowNewIndexes[uint32(id)] = struct{}{} - } - regionalByRowNewIndexes[uint32(swapDesc.NewPrimaryIndexId)] = struct{}{} + return nil + }) } // There can only be one pkSwap at a time, so break now. break diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index e8dfa1235ef8..4b604eaa4eb0 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -220,10 +220,10 @@ func (e errTableVersionMismatch) Error() string { // refreshMaterializedView updates the physical data for a materialized view. func (sc *SchemaChanger) refreshMaterializedView( - ctx context.Context, table *tabledesc.Mutable, refresh *descpb.MaterializedViewRefresh, + ctx context.Context, table *tabledesc.Mutable, refresh catalog.MaterializedViewRefresh, ) error { // If we aren't requested to backfill any data, then return immediately. - if !refresh.ShouldBackfill { + if !refresh.ShouldBackfill() { return nil } // The data for the materialized view is stored under the current set of @@ -233,14 +233,12 @@ func (sc *SchemaChanger) refreshMaterializedView( // set of indexes. We then backfill into this modified table, which writes // data only to the new desired indexes. In SchemaChanger.done(), we'll swap // the indexes from the old versions into the new ones. - tableToRefresh := protoutil.Clone(table.TableDesc()).(*descpb.TableDescriptor) - tableToRefresh.PrimaryIndex = refresh.NewPrimaryIndex - tableToRefresh.Indexes = refresh.NewIndexes - return sc.backfillQueryIntoTable(ctx, tableToRefresh, table.ViewQuery, refresh.AsOf, "refreshView") + tableToRefresh := refresh.TableWithNewIndexes(table) + return sc.backfillQueryIntoTable(ctx, tableToRefresh, table.ViewQuery, refresh.AsOf(), "refreshView") } func (sc *SchemaChanger) backfillQueryIntoTable( - ctx context.Context, table *descpb.TableDescriptor, query string, ts hlc.Timestamp, desc string, + ctx context.Context, table catalog.TableDescriptor, query string, ts hlc.Timestamp, desc string, ) error { if fn := sc.testingKnobs.RunBeforeQueryBackfill; fn != nil { if err := fn(); err != nil { @@ -329,7 +327,7 @@ func (sc *SchemaChanger) backfillQueryIntoTable( localPlanner.curPlan.main, ).WillDistribute() out := execinfrapb.ProcessorCoreUnion{BulkRowWriter: &execinfrapb.BulkRowWriterSpec{ - Table: *table, + Table: *table.TableDesc(), }} PlanAndRunCTAS(ctx, sc.distSQLPlanner, localPlanner, @@ -358,7 +356,7 @@ func (sc *SchemaChanger) maybeBackfillCreateTableAs( } log.Infof(ctx, "starting backfill for CREATE TABLE AS with query %q", table.GetCreateQuery()) - return sc.backfillQueryIntoTable(ctx, table.TableDesc(), table.GetCreateQuery(), table.GetCreateAsOfTime(), "ctasBackfill") + return sc.backfillQueryIntoTable(ctx, table, table.GetCreateQuery(), table.GetCreateAsOfTime(), "ctasBackfill") } func (sc *SchemaChanger) maybeBackfillMaterializedView( @@ -369,7 +367,7 @@ func (sc *SchemaChanger) maybeBackfillMaterializedView( } log.Infof(ctx, "starting backfill for CREATE MATERIALIZED VIEW with query %q", table.GetViewQuery()) - return sc.backfillQueryIntoTable(ctx, table.TableDesc(), table.GetViewQuery(), table.GetCreateAsOfTime(), "materializedViewBackfill") + return sc.backfillQueryIntoTable(ctx, table, table.GetViewQuery(), table.GetCreateAsOfTime(), "materializedViewBackfill") } // maybe make a table PUBLIC if it's in the ADD state. @@ -905,38 +903,30 @@ func (sc *SchemaChanger) RunStateMachineBeforeBackfill(ctx context.Context) erro } runStatus = "" // Apply mutations belonging to the same version. - for i, mutation := range tbl.Mutations { - if mutation.MutationID != sc.mutationID { + for _, m := range tbl.AllMutations() { + if m.MutationID() != sc.mutationID { // Mutations are applied in a FIFO order. Only apply the first set of // mutations if they have the mutation ID we're looking for. break } - switch mutation.Direction { - case descpb.DescriptorMutation_ADD: - switch mutation.State { - case descpb.DescriptorMutation_DELETE_ONLY: + if m.Adding() { + if m.DeleteOnly() { // TODO(vivek): while moving up the state is appropriate, // it will be better to run the backfill of a unique index // twice: once in the DELETE_ONLY state to confirm that // the index can indeed be created, and subsequently in the // DELETE_AND_WRITE_ONLY state to fill in the missing elements of the // index (INSERT and UPDATE that happened in the interim). - tbl.Mutations[i].State = descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY + tbl.Mutations[m.MutationOrdinal()].State = descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY runStatus = RunningStatusDeleteAndWriteOnly - - case descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY: - // The state change has already moved forward. } - - case descpb.DescriptorMutation_DROP: - switch mutation.State { - case descpb.DescriptorMutation_DELETE_ONLY: - // The state change has already moved forward. - - case descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY: - tbl.Mutations[i].State = descpb.DescriptorMutation_DELETE_ONLY + // else if DELETE_AND_WRITE_ONLY, then the state change has already moved forward. + } else if m.Dropped() { + if m.WriteAndDeleteOnly() { + tbl.Mutations[m.MutationOrdinal()].State = descpb.DescriptorMutation_DELETE_ONLY runStatus = RunningStatusDeleteOnly } + // else if DELETE_ONLY, then the state change has already moved forward. } // We might have to update some zone configs for indexes that are // being rewritten. It is important that this is done _before_ the @@ -948,7 +938,7 @@ func (sc *SchemaChanger) RunStateMachineBeforeBackfill(ctx context.Context) erro txn, dbDesc, tbl, - mutation, + m, false, // isDone descsCol, ); err != nil { @@ -984,13 +974,13 @@ func (sc *SchemaChanger) RunStateMachineBeforeBackfill(ctx context.Context) erro } func (sc *SchemaChanger) createIndexGCJob( - ctx context.Context, index *descpb.IndexDescriptor, txn *kv.Txn, jobDesc string, + ctx context.Context, indexID descpb.IndexID, txn *kv.Txn, jobDesc string, ) error { dropTime := timeutil.Now().UnixNano() indexGCDetails := jobspb.SchemaChangeGCDetails{ Indexes: []jobspb.SchemaChangeGCDetails_DroppedIndex{ { - IndexID: index.ID, + IndexID: indexID, DropTime: dropTime, }, }, @@ -1081,23 +1071,22 @@ func (sc *SchemaChanger) done(ctx context.Context) error { var i int // set to determine whether there is a mutation var isRollback bool // set based on the mutation - for _, mutation := range scTable.Mutations { - if mutation.MutationID != sc.mutationID { + for _, m := range scTable.AllMutations() { + if m.MutationID() != sc.mutationID { // Mutations are applied in a FIFO order. Only apply the first set of // mutations if they have the mutation ID we're looking for. break } - isRollback = mutation.Rollback - if indexDesc := mutation.GetIndex(); mutation.Direction == descpb.DescriptorMutation_DROP && - indexDesc != nil { - if canClearRangeForDrop(indexDesc) { + isRollback = m.IsRollback() + if idx := m.AsIndex(); m.Dropped() && idx != nil { + if canClearRangeForDrop(idx.IndexDesc()) { // how we keep track of dropped index names (for, e.g., zone config // lookups), even though in the absence of a GC job there's nothing to // clean them up. scTable.GCMutations = append( scTable.GCMutations, descpb.TableDescriptor_GCDescriptorMutation{ - IndexID: indexDesc.ID, + IndexID: idx.GetID(), }) description := sc.job.Payload().Description @@ -1105,24 +1094,22 @@ func (sc *SchemaChanger) done(ctx context.Context) error { description = "ROLLBACK of " + description } - if err := sc.createIndexGCJob(ctx, indexDesc, txn, description); err != nil { + if err := sc.createIndexGCJob(ctx, idx.GetID(), txn, description); err != nil { return err } } } - if constraint := mutation.GetConstraint(); constraint != nil && - constraint.ConstraintType == descpb.ConstraintToUpdate_FOREIGN_KEY && - mutation.Direction == descpb.DescriptorMutation_ADD && - constraint.ForeignKey.Validity == descpb.ConstraintValidity_Unvalidated { - // Add backreference on the referenced table (which could be the same table) - backrefTable, err := descsCol.GetMutableTableVersionByID(ctx, - constraint.ForeignKey.ReferencedTableID, txn) - if err != nil { - return err - } - backrefTable.InboundFKs = append(backrefTable.InboundFKs, constraint.ForeignKey) - if err := descsCol.WriteDescToBatch(ctx, kvTrace, backrefTable, b); err != nil { - return err + if constraint := m.AsConstraint(); constraint != nil && constraint.Adding() { + if constraint.IsForeignKey() && constraint.ForeignKey().Validity == descpb.ConstraintValidity_Unvalidated { + // Add backreference on the referenced table (which could be the same table) + backrefTable, err := descsCol.GetMutableTableVersionByID(ctx, constraint.ForeignKey().ReferencedTableID, txn) + if err != nil { + return err + } + backrefTable.InboundFKs = append(backrefTable.InboundFKs, constraint.ForeignKey()) + if err := descsCol.WriteDescToBatch(ctx, kvTrace, backrefTable, b); err != nil { + return err + } } } @@ -1134,7 +1121,7 @@ func (sc *SchemaChanger) done(ctx context.Context) error { txn, dbDesc, scTable, - mutation, + m, true, // isDone descsCol, ); err != nil { @@ -1145,7 +1132,7 @@ func (sc *SchemaChanger) done(ctx context.Context) error { // of the existing indexes in the view. We do this before the call to // MakeMutationComplete, which swaps out the existing indexes for the // backfilled ones. - if refresh := mutation.GetMaterializedViewRefresh(); refresh != nil { + if refresh := m.AsMaterializedViewRefresh(); refresh != nil { if fn := sc.testingKnobs.RunBeforeMaterializedViewRefreshCommit; fn != nil { if err := fn(); err != nil { return err @@ -1153,44 +1140,40 @@ func (sc *SchemaChanger) done(ctx context.Context) error { } // If we are mutation is in the ADD state, then start GC jobs for the // existing indexes on the table. - if mutation.Direction == descpb.DescriptorMutation_ADD { + if m.Adding() { desc := fmt.Sprintf("REFRESH MATERIALIZED VIEW %q cleanup", scTable.Name) - if err := sc.createIndexGCJob(ctx, scTable.GetPrimaryIndex().IndexDesc(), txn, desc); err != nil { - return err - } - for _, idx := range scTable.PublicNonPrimaryIndexes() { - if err := sc.createIndexGCJob(ctx, idx.IndexDesc(), txn, desc); err != nil { + for _, idx := range scTable.ActiveIndexes() { + if err := sc.createIndexGCJob(ctx, idx.GetID(), txn, desc); err != nil { return err } } - } else if mutation.Direction == descpb.DescriptorMutation_DROP { + } else if m.Dropped() { // Otherwise, the refresh job ran into an error and is being rolled // back. So, we need to GC all of the indexes that were going to be // created, in case any data was written to them. desc := fmt.Sprintf("ROLLBACK OF REFRESH MATERIALIZED VIEW %q", scTable.Name) - if err := sc.createIndexGCJob(ctx, &refresh.NewPrimaryIndex, txn, desc); err != nil { + err = refresh.ForEachIndexID(func(id descpb.IndexID) error { + return sc.createIndexGCJob(ctx, id, txn, desc) + }) + if err != nil { return err } - for i := range refresh.NewIndexes { - if err := sc.createIndexGCJob(ctx, &refresh.NewIndexes[i], txn, desc); err != nil { - return err - } - } } } - if err := scTable.MakeMutationComplete(mutation); err != nil { + if err := scTable.MakeMutationComplete(scTable.Mutations[m.MutationOrdinal()]); err != nil { return err } - if pkSwap := mutation.GetPrimaryKeySwap(); pkSwap != nil { + if pkSwap := m.AsPrimaryKeySwap(); pkSwap != nil { if fn := sc.testingKnobs.RunBeforePrimaryKeySwap; fn != nil { fn() } // For locality swaps, ensure the table descriptor fields are correctly filled. - if lcSwap := pkSwap.LocalityConfigSwap; lcSwap != nil { + if pkSwap.HasLocalityConfig() { + lcSwap := pkSwap.LocalityConfigSwap() localityConfigToSwapTo := lcSwap.NewLocalityConfig - if mutation.Direction == descpb.DescriptorMutation_ADD { + if m.Adding() { // Sanity check that locality has not been changed during backfill. if !scTable.LocalityConfig.Equal(lcSwap.OldLocalityConfig) { return errors.AssertionFailedf( @@ -1235,8 +1218,7 @@ func (sc *SchemaChanger) done(ctx context.Context) error { // backreference from the parent. // N.B. This logic needs to be kept up to date with the // corresponding piece in runSchemaChangesInTxn. - for _, idxID := range append( - []descpb.IndexID{pkSwap.OldPrimaryIndexId}, pkSwap.OldIndexes...) { + err := pkSwap.ForEachOldIndexIDs(func(idxID descpb.IndexID) error { oldIndex, err := scTable.FindIndexWithID(idxID) if err != nil { return err @@ -1269,6 +1251,10 @@ func (sc *SchemaChanger) done(ctx context.Context) error { } } } + return nil + }) + if err != nil { + return err } // If we performed MakeMutationComplete on a PrimaryKeySwap mutation, then we need to start // a job for the index deletion mutations that the primary key swap mutation added, if any. @@ -1277,7 +1263,7 @@ func (sc *SchemaChanger) done(ctx context.Context) error { } } - if computedColumnSwap := mutation.GetComputedColumnSwap(); computedColumnSwap != nil { + if m.AsComputedColumnSwap() != nil { if fn := sc.testingKnobs.RunBeforeComputedColumnSwap; fn != nil { fn() } @@ -1296,7 +1282,7 @@ func (sc *SchemaChanger) done(ctx context.Context) error { // The table descriptor is unchanged, return without writing anything. return nil } - committedMutations := scTable.Mutations[:i] + committedMutations := scTable.AllMutations()[:i] // Trim the executed mutations from the descriptor. scTable.Mutations = scTable.Mutations[i:] @@ -1573,54 +1559,52 @@ func (sc *SchemaChanger) maybeReverseMutations(ctx context.Context, causingError columns := make(map[string]struct{}) droppedMutations = nil b := txn.NewBatch() - for i, mutation := range scTable.Mutations { - if mutation.MutationID != sc.mutationID { + for _, m := range scTable.AllMutations() { + if m.MutationID() != sc.mutationID { break } - if mutation.Rollback { + if m.IsRollback() { // Can actually never happen. Since we should have checked for this case // above. - return errors.AssertionFailedf("mutation already rolled back: %v", mutation) + return errors.AssertionFailedf("mutation already rolled back: %v", scTable.Mutations[m.MutationOrdinal()]) } // Ignore mutations that would be skipped, nothing // to reverse here. - if discarded, _ := isCurrentMutationDiscarded(scTable, mutation, i+1); discarded { + if discarded, _ := isCurrentMutationDiscarded(scTable, m, m.MutationOrdinal()+1); discarded { continue } - log.Warningf(ctx, "reverse schema change mutation: %+v", mutation) - scTable.Mutations[i], columns = sc.reverseMutation(mutation, false /*notStarted*/, columns) + log.Warningf(ctx, "reverse schema change mutation: %+v", scTable.Mutations[m.MutationOrdinal()]) + scTable.Mutations[m.MutationOrdinal()], columns = sc.reverseMutation(scTable.Mutations[m.MutationOrdinal()], false /*notStarted*/, columns) // If the mutation is for validating a constraint that is being added, // drop the constraint because validation has failed. - if constraint := mutation.GetConstraint(); constraint != nil && - mutation.Direction == descpb.DescriptorMutation_ADD { - log.Warningf(ctx, "dropping constraint %+v", constraint) + if constraint := m.AsConstraint(); constraint != nil && constraint.Adding() { + log.Warningf(ctx, "dropping constraint %+v", constraint.ConstraintToUpdateDesc()) if err := sc.maybeDropValidatingConstraint(ctx, scTable, constraint); err != nil { return err } // Get the foreign key backreferences to remove. - if constraint.ConstraintType == descpb.ConstraintToUpdate_FOREIGN_KEY { - fk := &constraint.ForeignKey - backrefTable, err := descsCol.GetMutableTableVersionByID(ctx, fk.ReferencedTableID, txn) + if constraint.IsForeignKey() { + backrefTable, err := descsCol.GetMutableTableVersionByID(ctx, constraint.ForeignKey().ReferencedTableID, txn) if err != nil { return err } - if err := removeFKBackReferenceFromTable(backrefTable, fk.Name, scTable); err != nil { + if err := removeFKBackReferenceFromTable(backrefTable, constraint.GetName(), scTable); err != nil { // The function being called will return an assertion error if the // backreference was not found, but it may not have been installed // during the incomplete schema change, so we swallow the error. log.Infof(ctx, - "error attempting to remove backreference %s during rollback: %s", fk.Name, err) + "error attempting to remove backreference %s during rollback: %s", constraint.GetName(), err) } if err := descsCol.WriteDescToBatch(ctx, kvTrace, backrefTable, b); err != nil { return err } } } - scTable.Mutations[i].Rollback = true + scTable.Mutations[m.MutationOrdinal()].Rollback = true } // Delete all mutations that reference any of the reversed columns @@ -1725,15 +1709,14 @@ func (sc *SchemaChanger) updateJobForRollback( } func (sc *SchemaChanger) maybeDropValidatingConstraint( - ctx context.Context, desc *tabledesc.Mutable, constraint *descpb.ConstraintToUpdate, + ctx context.Context, desc *tabledesc.Mutable, constraint catalog.ConstraintToUpdate, ) error { - switch constraint.ConstraintType { - case descpb.ConstraintToUpdate_CHECK, descpb.ConstraintToUpdate_NOT_NULL: - if constraint.Check.Validity == descpb.ConstraintValidity_Unvalidated { + if constraint.IsCheck() || constraint.IsNotNull() { + if constraint.Check().Validity == descpb.ConstraintValidity_Unvalidated { return nil } for j, c := range desc.Checks { - if c.Name == constraint.Check.Name { + if c.Name == constraint.Check().Name { desc.Checks = append(desc.Checks[:j], desc.Checks[j+1:]...) return nil } @@ -1741,11 +1724,11 @@ func (sc *SchemaChanger) maybeDropValidatingConstraint( log.Infof( ctx, "attempted to drop constraint %s, but it hadn't been added to the table descriptor yet", - constraint.Check.Name, + constraint.Check().Name, ) - case descpb.ConstraintToUpdate_FOREIGN_KEY: + } else if constraint.IsForeignKey() { for i, fk := range desc.OutboundFKs { - if fk.Name == constraint.ForeignKey.Name { + if fk.Name == constraint.ForeignKey().Name { desc.OutboundFKs = append(desc.OutboundFKs[:i], desc.OutboundFKs[i+1:]...) return nil } @@ -1753,14 +1736,14 @@ func (sc *SchemaChanger) maybeDropValidatingConstraint( log.Infof( ctx, "attempted to drop constraint %s, but it hadn't been added to the table descriptor yet", - constraint.ForeignKey.Name, + constraint.ForeignKey().Name, ) - case descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX: - if constraint.UniqueWithoutIndexConstraint.Validity == descpb.ConstraintValidity_Unvalidated { + } else if constraint.IsUniqueWithoutIndex() { + if constraint.UniqueWithoutIndex().Validity == descpb.ConstraintValidity_Unvalidated { return nil } for j, c := range desc.UniqueWithoutIndexConstraints { - if c.Name == constraint.UniqueWithoutIndexConstraint.Name { + if c.Name == constraint.UniqueWithoutIndex().Name { desc.UniqueWithoutIndexConstraints = append( desc.UniqueWithoutIndexConstraints[:j], desc.UniqueWithoutIndexConstraints[j+1:]..., ) @@ -1770,10 +1753,10 @@ func (sc *SchemaChanger) maybeDropValidatingConstraint( log.Infof( ctx, "attempted to drop constraint %s, but it hadn't been added to the table descriptor yet", - constraint.UniqueWithoutIndexConstraint.Name, + constraint.UniqueWithoutIndex().Name, ) - default: - return errors.AssertionFailedf("unsupported constraint type: %d", errors.Safe(constraint.ConstraintType)) + } else { + return errors.AssertionFailedf("unsupported constraint type: %d", constraint.ConstraintToUpdateDesc().ConstraintType) } return nil } @@ -1818,8 +1801,9 @@ func (sc *SchemaChanger) deleteIndexMutationsWithReversedColumns( } // Drop mutations. newMutations := make([]descpb.DescriptorMutation, 0, len(desc.Mutations)) - for _, mutation := range desc.Mutations { - if _, ok := dropMutations[mutation.MutationID]; ok { + for _, m := range desc.AllMutations() { + mutation := desc.Mutations[m.MutationOrdinal()] + if _, ok := dropMutations[m.MutationID()]; ok { // Reverse mutation. Update columns to reflect additional // columns that have been purged. This mutation doesn't need // a rollback because it was not started. @@ -2438,21 +2422,22 @@ func (sc *SchemaChanger) applyZoneConfigChangeForMutation( txn *kv.Txn, dbDesc catalog.DatabaseDescriptor, tableDesc *tabledesc.Mutable, - mutation descpb.DescriptorMutation, + mutation catalog.Mutation, isDone bool, descsCol *descs.Collection, ) error { - if pkSwap := mutation.GetPrimaryKeySwap(); pkSwap != nil { - if lcSwap := pkSwap.LocalityConfigSwap; lcSwap != nil { + if pkSwap := mutation.AsPrimaryKeySwap(); pkSwap != nil { + if pkSwap.HasLocalityConfig() { // We will add up to three options - one for the table itself, // one for dropping any zone configs for old indexes and one // for all the new indexes associated with the table. opts := make([]applyZoneConfigForMultiRegionTableOption, 0, 3) + lcSwap := pkSwap.LocalityConfigSwap() // For locality configs, we need to update the zone configs to match // the new multi-region locality configuration, instead of // copying the old zone configs over. - if mutation.Direction == descpb.DescriptorMutation_ADD { + if mutation.Adding() { // Only apply the zone configuration on the table when the mutation // is complete. if isDone { @@ -2460,15 +2445,12 @@ func (sc *SchemaChanger) applyZoneConfigChangeForMutation( // us here, but if we're coming from REGIONAL BY ROW, it's also // necessary to drop the zone configurations on the index partitions. if lcSwap.OldLocalityConfig.GetRegionalByRow() != nil { - opts = append( - opts, - dropZoneConfigsForMultiRegionIndexes( - append( - []descpb.IndexID{pkSwap.OldPrimaryIndexId}, - pkSwap.OldIndexes..., - )..., - ), - ) + oldIndexIDs := make([]descpb.IndexID, 0, pkSwap.NumOldIndexes()) + _ = pkSwap.ForEachOldIndexIDs(func(id descpb.IndexID) error { + oldIndexIDs = append(oldIndexIDs, id) + return nil + }) + opts = append(opts, dropZoneConfigsForMultiRegionIndexes(oldIndexIDs...)) } opts = append( @@ -2483,15 +2465,12 @@ func (sc *SchemaChanger) applyZoneConfigChangeForMutation( *descpb.TableDescriptor_LocalityConfig_RegionalByTable_: case *descpb.TableDescriptor_LocalityConfig_RegionalByRow_: // Apply new zone configurations for all newly partitioned indexes. - opts = append( - opts, - applyZoneConfigForMultiRegionTableOptionNewIndexes( - append( - []descpb.IndexID{pkSwap.NewPrimaryIndexId}, - pkSwap.NewIndexes..., - )..., - ), - ) + newIndexIDs := make([]descpb.IndexID, 0, pkSwap.NumNewIndexes()) + _ = pkSwap.ForEachNewIndexIDs(func(id descpb.IndexID) error { + newIndexIDs = append(newIndexIDs, id) + return nil + }) + opts = append(opts, applyZoneConfigForMultiRegionTableOptionNewIndexes(newIndexIDs...)) default: return errors.AssertionFailedf( "unknown locality on PK swap: %T", @@ -2526,7 +2505,7 @@ func (sc *SchemaChanger) applyZoneConfigChangeForMutation( // for any new indexes. // Note this is done even for isDone = true, though not strictly necessary. return maybeUpdateZoneConfigsForPKChange( - ctx, txn, sc.execCfg, tableDesc, pkSwap, + ctx, txn, sc.execCfg, tableDesc, pkSwap.PrimaryKeySwapDesc(), ) } return nil @@ -2560,7 +2539,7 @@ func DeleteTableDescAndZoneConfig( // A will have a unique index, but it later gets dropped. Then for this mutation // to be successful the drop column job has to be successful too. func (sc *SchemaChanger) getDependentMutationsJobs( - ctx context.Context, tableDesc *tabledesc.Mutable, mutations []descpb.DescriptorMutation, + ctx context.Context, tableDesc *tabledesc.Mutable, mutations []catalog.Mutation, ) ([]jobspb.JobID, error) { dependentJobs := make([]jobspb.JobID, 0, len(tableDesc.MutationJobs)) for _, m := range mutations { @@ -2581,38 +2560,35 @@ func (sc *SchemaChanger) getDependentMutationsJobs( // by a later operation. The nextMutationIdx provides the index at which to check for // later mutation. func isCurrentMutationDiscarded( - tableDesc *tabledesc.Mutable, currentMutation descpb.DescriptorMutation, nextMutationIdx int, + tableDesc *tabledesc.Mutable, currentMutation catalog.Mutation, nextMutationIdx int, ) (bool, descpb.MutationID) { if nextMutationIdx+1 > len(tableDesc.Mutations) { return false, descpb.InvalidMutationID } - // Drops will never get canceled out, since we need - // clean up. - if currentMutation.Direction == descpb.DescriptorMutation_DROP { + // Drops will never get canceled out, since we need clean up. + if currentMutation.Dropped() { return false, descpb.InvalidMutationID } colToCheck := make([]descpb.ColumnID, 0, 1) // Both NOT NULL related updates and check constraint updates // involving this column will get canceled out by a drop column. - if constraint := currentMutation.GetConstraint(); constraint != nil { - if constraint.ConstraintType == descpb.ConstraintToUpdate_NOT_NULL { - colToCheck = append(colToCheck, constraint.NotNullColumn) - } else if constraint.ConstraintType == descpb.ConstraintToUpdate_CHECK { - colToCheck = constraint.Check.ColumnIDs + if constraint := currentMutation.AsConstraint(); constraint != nil { + if constraint.IsNotNull() { + colToCheck = append(colToCheck, constraint.NotNullColumnID()) + } else if constraint.IsCheck() { + colToCheck = constraint.Check().ColumnIDs } } - for _, m := range tableDesc.Mutations[nextMutationIdx:] { - colDesc := m.GetColumn() - if m.Direction == descpb.DescriptorMutation_DROP && - colDesc != nil && - !m.Rollback { + for _, m := range tableDesc.AllMutations()[nextMutationIdx:] { + col := m.AsColumn() + if col != nil && col.Dropped() && !col.IsRollback() { // Column was dropped later on, so this operation // should be a no-op. - for _, col := range colToCheck { - if colDesc.ID == col { - return true, m.MutationID + for _, id := range colToCheck { + if col.GetID() == id { + return true, m.MutationID() } } } diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 7fa6e02774b0..9701bce4dfc1 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -360,19 +360,14 @@ func checkTableForDisallowedMutationsWithTruncate(desc *tabledesc.Mutable) error "dropped which depends on another object", desc.GetName(), col.GetName()) } } else if c := m.AsConstraint(); c != nil { - switch ct := c.ConstraintToUpdateDesc().ConstraintType; ct { - case descpb.ConstraintToUpdate_CHECK, - descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX, - descpb.ConstraintToUpdate_NOT_NULL, - descpb.ConstraintToUpdate_FOREIGN_KEY: + if c.IsCheck() || c.IsNotNull() || c.IsForeignKey() || c.IsUniqueWithoutIndex() { return unimplemented.Newf( "TRUNCATE concurrent with ongoing schema change", "cannot perform TRUNCATE on %q which has an ongoing %s "+ - "constraint change", desc.GetName(), ct) - default: - return errors.AssertionFailedf("cannot perform TRUNCATE due to "+ - "unknown constraint type %v on mutation %d in %v", ct, i, desc) + "constraint change", desc.GetName(), c.ConstraintToUpdateDesc().ConstraintType) } + return errors.AssertionFailedf("cannot perform TRUNCATE due to "+ + "unknown constraint type %v on mutation %d in %v", c.ConstraintToUpdateDesc().ConstraintType, i, desc) } else if s := m.AsPrimaryKeySwap(); s != nil { return unimplemented.Newf( "TRUNCATE concurrent with ongoing schema change", From c9e116e586f24c5f3a831ac653f14fd03f588b93 Mon Sep 17 00:00:00 2001 From: Marius Posta <marius@cockroachlabs.com> Date: Thu, 15 Apr 2021 22:18:15 -0400 Subject: [PATCH 22/37] sql: use catalog.Column and catalog.Index where possible This commit is a refactor which introduces the catalog.Column and catalog.Index interfaces wherever possible. These had recently been added in an effort to avoid using the descpb.ColumnDescriptor and descpb.IndexDescriptor types directly. This commit generalizes their usage throughout the sql packages. Fixes #63755. Release note: None --- pkg/ccl/backupccl/restore_job.go | 10 +- pkg/ccl/changefeedccl/avro.go | 39 ++-- pkg/ccl/changefeedccl/avro_test.go | 6 +- pkg/ccl/changefeedccl/encoder.go | 2 +- pkg/ccl/changefeedccl/rowfetcher_cache.go | 7 +- pkg/ccl/cliccl/load.go | 6 +- pkg/ccl/importccl/read_import_avro.go | 4 +- pkg/ccl/importccl/read_import_csv.go | 2 +- pkg/ccl/importccl/read_import_mysql_test.go | 4 +- pkg/ccl/importccl/read_import_mysqlout.go | 2 +- pkg/ccl/importccl/read_import_pgcopy.go | 2 +- pkg/ccl/importccl/read_import_pgdump.go | 2 +- pkg/ccl/partitionccl/partition.go | 25 ++- pkg/ccl/partitionccl/partition_test.go | 2 +- pkg/ccl/storageccl/key_rewriter.go | 2 +- .../streamingccl/streamingtest/encoding.go | 4 +- pkg/server/settingswatcher/row_decoder.go | 4 +- pkg/sql/BUILD.bazel | 1 + pkg/sql/alter_column_type.go | 59 +++--- pkg/sql/alter_index.go | 32 ++-- pkg/sql/alter_primary_key.go | 56 +++--- pkg/sql/alter_table.go | 54 +++--- pkg/sql/backfill.go | 103 ++++++----- pkg/sql/backfill/backfill.go | 97 ++++------ pkg/sql/catalog/catalogkeys/BUILD.bazel | 1 + pkg/sql/catalog/catalogkeys/keys.go | 16 +- pkg/sql/catalog/catformat/index.go | 12 ++ pkg/sql/catalog/catformat/index_test.go | 2 +- pkg/sql/catalog/colinfo/col_type_info.go | 16 +- .../catalog/colinfo/column_type_properties.go | 8 +- pkg/sql/catalog/colinfo/result_columns.go | 23 +-- pkg/sql/catalog/descpb/index.go | 32 ---- .../catalog/schemaexpr/check_constraint.go | 4 +- pkg/sql/catalog/schemaexpr/column.go | 39 ++-- pkg/sql/catalog/schemaexpr/column_test.go | 5 +- pkg/sql/catalog/schemaexpr/computed_column.go | 26 ++- pkg/sql/catalog/schemaexpr/computed_exprs.go | 5 +- pkg/sql/catalog/schemaexpr/default_exprs.go | 33 ++-- pkg/sql/catalog/schemaexpr/expr.go | 22 +-- pkg/sql/catalog/schemaexpr/partial_index.go | 10 +- pkg/sql/catalog/table_elements.go | 26 +++ pkg/sql/catalog/tabledesc/structured.go | 56 ++---- pkg/sql/catalog/tabledesc/validate.go | 25 ++- pkg/sql/colencoding/key_encoding.go | 11 +- pkg/sql/colfetcher/cfetcher.go | 68 +++---- pkg/sql/colfetcher/colbatch_scan.go | 8 +- pkg/sql/comment_on_index.go | 11 +- pkg/sql/crdb_internal.go | 46 ++--- pkg/sql/create_index.go | 30 +-- pkg/sql/create_table.go | 12 +- pkg/sql/database_region_change_finalizer.go | 2 +- pkg/sql/delete_range.go | 4 +- pkg/sql/distsql_physical_planner.go | 9 +- pkg/sql/distsql_spec_exec_factory.go | 8 +- pkg/sql/drop_index.go | 63 ++++--- pkg/sql/drop_sequence.go | 23 +-- pkg/sql/drop_table.go | 20 +- pkg/sql/exec_factory_util.go | 4 +- pkg/sql/execinfrapb/flow_diagram.go | 2 +- pkg/sql/explain_plan.go | 6 +- pkg/sql/gcjob/gc_job.go | 6 +- pkg/sql/gcjob/index_garbage_collection.go | 19 +- pkg/sql/index_join.go | 4 +- pkg/sql/indexbackfiller_test.go | 4 +- pkg/sql/information_schema.go | 12 +- pkg/sql/insert.go | 5 +- pkg/sql/insert_fast_path.go | 24 ++- pkg/sql/opt/invertedidx/BUILD.bazel | 1 + .../opt/invertedidx/inverted_index_expr.go | 9 +- pkg/sql/opt_catalog.go | 149 ++++++++------- pkg/sql/opt_exec_factory.go | 161 ++++++++--------- pkg/sql/partition.go | 18 +- pkg/sql/partition_utils.go | 12 +- pkg/sql/pg_catalog.go | 63 ++++--- pkg/sql/pgwire/testdata/pgtest/notice | 2 +- .../physicalplan/fake_span_resolver_test.go | 2 +- pkg/sql/randgen/mutator.go | 10 +- pkg/sql/randgen/schema.go | 2 +- pkg/sql/relocate.go | 3 +- pkg/sql/rename_column.go | 8 +- pkg/sql/rename_index.go | 13 +- pkg/sql/resolver.go | 4 +- pkg/sql/row/deleter.go | 20 +- pkg/sql/row/errors.go | 18 +- pkg/sql/row/fetcher.go | 99 +++++----- pkg/sql/row/fetcher_mvcc_test.go | 6 +- pkg/sql/row/fetcher_test.go | 23 +-- pkg/sql/row/helper.go | 14 +- pkg/sql/row/inserter.go | 15 +- pkg/sql/row/row_converter.go | 74 ++++---- pkg/sql/row/updater.go | 42 ++--- pkg/sql/row/writer.go | 22 +-- pkg/sql/rowenc/client_index_encoding_test.go | 4 +- pkg/sql/rowenc/column_type_encoding.go | 22 ++- pkg/sql/rowenc/column_type_encoding_test.go | 5 +- pkg/sql/rowenc/index_encoding.go | 171 ++++++++++-------- pkg/sql/rowenc/index_encoding_test.go | 67 ++++--- pkg/sql/rowenc/partition.go | 14 +- pkg/sql/rowexec/inverted_joiner.go | 10 +- pkg/sql/rowexec/joinreader.go | 17 +- pkg/sql/rowexec/rowfetcher.go | 9 +- pkg/sql/rowexec/sampler.go | 2 +- pkg/sql/rowexec/stats.go | 3 +- pkg/sql/rowexec/zigzagjoiner.go | 32 ++-- pkg/sql/scan.go | 10 +- pkg/sql/scatter.go | 11 +- pkg/sql/schema_changer.go | 2 +- pkg/sql/schemachanger/scbuild/builder.go | 6 +- pkg/sql/scrub.go | 10 +- pkg/sql/scrub_constraint.go | 9 +- pkg/sql/scrub_fk.go | 9 - pkg/sql/scrub_index.go | 41 ++--- pkg/sql/scrub_physical.go | 27 +-- pkg/sql/scrub_test.go | 22 +-- pkg/sql/sem/builtins/builtins.go | 29 +-- pkg/sql/sequence.go | 39 ++-- pkg/sql/set_zone_config.go | 24 +-- pkg/sql/show_create.go | 12 +- pkg/sql/show_create_clauses.go | 24 +-- pkg/sql/show_fingerprints.go | 44 +++-- pkg/sql/span/span_builder.go | 33 ++-- pkg/sql/span/span_builder_test.go | 2 +- pkg/sql/span_builder_test.go | 2 +- pkg/sql/split.go | 16 +- pkg/sql/stats/stats_cache.go | 2 +- pkg/sql/tablewriter_delete.go | 22 +-- pkg/sql/tablewriter_upsert_opt.go | 10 +- pkg/sql/tests/hash_sharded_test.go | 3 +- pkg/sql/truncate.go | 2 +- pkg/sql/unsplit.go | 9 +- pkg/sql/update.go | 25 ++- pkg/sql/upsert.go | 4 +- pkg/sql/virtual_schema.go | 13 +- pkg/sql/virtual_table.go | 2 +- pkg/sql/zone_config.go | 21 ++- pkg/sql/zone_config_test.go | 23 ++- 136 files changed, 1410 insertions(+), 1531 deletions(-) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 39f477939aae..7d820efecea0 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -1755,14 +1755,14 @@ func revalidateIndexes( } tableDesc := tabledesc.NewBuilder(tbl).BuildExistingMutableTable() - var forward, inverted []*descpb.IndexDescriptor + var forward, inverted []catalog.Index for _, idx := range tableDesc.AllIndexes() { if _, ok := indexes[idx.GetID()]; ok { switch idx.GetType() { case descpb.IndexDescriptor_FORWARD: - forward = append(forward, idx.IndexDesc()) + forward = append(forward, idx) case descpb.IndexDescriptor_INVERTED: - inverted = append(inverted, idx.IndexDesc()) + inverted = append(inverted, idx) } } } @@ -1901,9 +1901,9 @@ func (r *restoreResumer) publishDescriptors( if err != nil { return err } - newIdx := protoutil.Clone(found.IndexDesc()).(*descpb.IndexDescriptor) + newIdx := found.IndexDescDeepCopy() mutTable.RemovePublicNonPrimaryIndex(found.Ordinal()) - if err := mutTable.AddIndexMutation(newIdx, descpb.DescriptorMutation_ADD); err != nil { + if err := mutTable.AddIndexMutation(&newIdx, descpb.DescriptorMutation_ADD); err != nil { return err } } diff --git a/pkg/ccl/changefeedccl/avro.go b/pkg/ccl/changefeedccl/avro.go index 6d2894a57143..68ef1fd8eb65 100644 --- a/pkg/ccl/changefeedccl/avro.go +++ b/pkg/ccl/changefeedccl/avro.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/geo" "github.com/cockroachdb/cockroach/pkg/geo/geopb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -145,18 +144,18 @@ type avroEnvelopeRecord struct { before, after *avroDataRecord } -// columnDescToAvroSchema converts a column descriptor into its corresponding +// columnToAvroSchema converts a column descriptor into its corresponding // avro field schema. -func columnDescToAvroSchema(colDesc *descpb.ColumnDescriptor) (*avroSchemaField, error) { +func columnToAvroSchema(col catalog.Column) (*avroSchemaField, error) { schema := &avroSchemaField{ - Name: SQLNameToAvroName(colDesc.Name), - Metadata: colDesc.SQLStringNotHumanReadable(), + Name: SQLNameToAvroName(col.GetName()), + Metadata: col.ColumnDesc().SQLStringNotHumanReadable(), Default: nil, - typ: colDesc.Type, + typ: col.GetType(), } var avroType avroSchemaType - switch colDesc.Type.Family() { + switch col.GetType().Family() { case types.IntFamily: avroType = avroSchemaLong schema.encodeFn = func(d tree.Datum) (interface{}, error) { @@ -242,7 +241,7 @@ func columnDescToAvroSchema(colDesc *descpb.ColumnDescriptor) (*avroSchemaField, date := *d.(*tree.DDate) if !date.IsFinite() { return nil, errors.Errorf( - `column %s: infinite date not yet supported with avro`, colDesc.Name) + `column %s: infinite date not yet supported with avro`, col.GetName()) } // The avro library requires us to return this as a time.Time. return date.ToTime() @@ -300,12 +299,12 @@ func columnDescToAvroSchema(colDesc *descpb.ColumnDescriptor) (*avroSchemaField, return tree.MakeDTimestampTZ(x.(time.Time), time.Microsecond) } case types.DecimalFamily: - if colDesc.Type.Precision() == 0 { + if col.GetType().Precision() == 0 { return nil, errors.Errorf( - `column %s: decimal with no precision not yet supported with avro`, colDesc.Name) + `column %s: decimal with no precision not yet supported with avro`, col.GetName()) } - width := int(colDesc.Type.Width()) - prec := int(colDesc.Type.Precision()) + width := int(col.GetType().Width()) + prec := int(col.GetType().Precision()) avroType = avroLogicalType{ SchemaType: avroSchemaBytes, LogicalType: `decimal`, @@ -317,7 +316,7 @@ func columnDescToAvroSchema(colDesc *descpb.ColumnDescriptor) (*avroSchemaField, // If the decimal happens to fit a smaller width than the // column allows, add trailing zeroes so the scale is constant - if colDesc.Type.Width() > -dec.Exponent { + if col.GetType().Width() > -dec.Exponent { _, err := tree.DecimalCtx.WithPrecision(uint32(prec)).Quantize(&dec, &dec, -int32(width)) if err != nil { // This should always be possible without rounding since we're using the column def, @@ -368,7 +367,7 @@ func columnDescToAvroSchema(colDesc *descpb.ColumnDescriptor) (*avroSchemaField, } default: return nil, errors.Errorf(`column %s: type %s not yet supported with avro`, - colDesc.Name, colDesc.Type.SQLString()) + col.GetName(), col.GetType().SQLString()) } schema.SchemaType = avroType @@ -409,10 +408,7 @@ func columnDescToAvroSchema(colDesc *descpb.ColumnDescriptor) (*avroSchemaField, // record schema. The fields are kept in the same order as columns in the index. // sqlName can be any string but should uniquely identify a schema. func indexToAvroSchema( - tableDesc catalog.TableDescriptor, - indexDesc *descpb.IndexDescriptor, - sqlName string, - namespace string, + tableDesc catalog.TableDescriptor, index catalog.Index, sqlName string, namespace string, ) (*avroDataRecord, error) { schema := &avroDataRecord{ avroRecord: avroRecord{ @@ -424,13 +420,14 @@ func indexToAvroSchema( colIdxByFieldIdx: make(map[int]int), } colIdxByID := catalog.ColumnIDToOrdinalMap(tableDesc.PublicColumns()) - for _, colID := range indexDesc.ColumnIDs { + for i := 0; i < index.NumColumns(); i++ { + colID := index.GetColumnID(i) colIdx, ok := colIdxByID.Get(colID) if !ok { return nil, errors.Errorf(`unknown column id: %d`, colID) } col := tableDesc.PublicColumns()[colIdx] - field, err := columnDescToAvroSchema(col.ColumnDesc()) + field, err := columnToAvroSchema(col) if err != nil { return nil, err } @@ -476,7 +473,7 @@ func tableToAvroSchema( colIdxByFieldIdx: make(map[int]int), } for _, col := range tableDesc.PublicColumns() { - field, err := columnDescToAvroSchema(col.ColumnDesc()) + field, err := columnToAvroSchema(col) if err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/avro_test.go b/pkg/ccl/changefeedccl/avro_test.go index 67bb6552739d..34777240a6fb 100644 --- a/pkg/ccl/changefeedccl/avro_test.go +++ b/pkg/ccl/changefeedccl/avro_test.go @@ -116,7 +116,7 @@ func parseAvroSchema(j string) (*avroDataRecord, error) { for _, f := range s.Fields { // s.Fields[idx] has `Name` and `SchemaType` set but nothing else. // They're needed for serialization/deserialization, so fake out a - // column descriptor so that we can reuse columnDescToAvroSchema to get + // column descriptor so that we can reuse columnToAvroSchema to get // all the various fields of avroSchemaField populated for free. colDesc, err := avroFieldMetadataToColDesc(f.Metadata) if err != nil { @@ -286,7 +286,7 @@ func TestAvroSchema(t *testing.T) { `{"type":["null","long"],"name":"_u0001f366_","default":null,`+ `"__crdb__":"🍦 INT8 NOT NULL"}]}`, tableSchema.codec.Schema()) - indexSchema, err := indexToAvroSchema(tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), tableDesc.GetName(), "") + indexSchema, err := indexToAvroSchema(tableDesc, tableDesc.GetPrimaryIndex(), tableDesc.GetName(), "") require.NoError(t, err) require.Equal(t, `{"type":"record","name":"_u2603_","fields":[`+ @@ -329,7 +329,7 @@ func TestAvroSchema(t *testing.T) { colType := typ.SQLString() tableDesc, err := parseTableDesc(`CREATE TABLE foo (pk INT PRIMARY KEY, a ` + colType + `)`) require.NoError(t, err) - field, err := columnDescToAvroSchema(tableDesc.PublicColumns()[1].ColumnDesc()) + field, err := columnToAvroSchema(tableDesc.PublicColumns()[1]) require.NoError(t, err) schema, err := json.Marshal(field.SchemaType) require.NoError(t, err) diff --git a/pkg/ccl/changefeedccl/encoder.go b/pkg/ccl/changefeedccl/encoder.go index 2e94b6f81142..47ff5badcd41 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -360,7 +360,7 @@ func (e *confluentAvroEncoder) EncodeKey(ctx context.Context, row encodeRow) ([] if !ok { var err error tableName := e.rawTableName(row.tableDesc) - registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.GetPrimaryIndex().IndexDesc(), tableName, e.schemaPrefix) + registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.GetPrimaryIndex(), tableName, e.schemaPrefix) if err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/rowfetcher_cache.go b/pkg/ccl/changefeedccl/rowfetcher_cache.go index bf074613aae6..987fe80ebc66 100644 --- a/pkg/ccl/changefeedccl/rowfetcher_cache.go +++ b/pkg/ccl/changefeedccl/rowfetcher_cache.go @@ -163,15 +163,12 @@ func (c *rowFetcherCache) RowFetcherForTableDesc( rfArgs := row.FetcherTableArgs{ Spans: tableDesc.AllIndexSpans(c.codec), Desc: tableDesc, - Index: tableDesc.GetPrimaryIndex().IndexDesc(), + Index: tableDesc.GetPrimaryIndex(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, - Cols: make([]descpb.ColumnDescriptor, len(tableDesc.PublicColumns())), + Cols: tableDesc.PublicColumns(), ValNeededForCol: valNeededForCol, } - for i, col := range tableDesc.PublicColumns() { - rfArgs.Cols[i] = *col.ColumnDesc() - } if err := rf.Init( context.TODO(), c.codec, diff --git a/pkg/ccl/cliccl/load.go b/pkg/ccl/cliccl/load.go index 8ac13f60062f..480e35981e13 100644 --- a/pkg/ccl/cliccl/load.go +++ b/pkg/ccl/cliccl/load.go @@ -483,19 +483,17 @@ func makeRowFetcher( ) (row.Fetcher, error) { var colIdxMap catalog.TableColMap var valNeededForCol util.FastIntSet - colDescs := make([]descpb.ColumnDescriptor, len(entry.Desc.PublicColumns())) for i, col := range entry.Desc.PublicColumns() { colIdxMap.Set(col.GetID(), i) valNeededForCol.Add(i) - colDescs[i] = *col.ColumnDesc() } table := row.FetcherTableArgs{ Spans: []roachpb.Span{entry.Span}, Desc: entry.Desc, - Index: entry.Desc.GetPrimaryIndex().IndexDesc(), + Index: entry.Desc.GetPrimaryIndex(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, - Cols: colDescs, + Cols: entry.Desc.PublicColumns(), ValNeededForCol: valNeededForCol, } diff --git a/pkg/ccl/importccl/read_import_avro.go b/pkg/ccl/importccl/read_import_avro.go index 7547d4c7a736..35b9f1953e3c 100644 --- a/pkg/ccl/importccl/read_import_avro.go +++ b/pkg/ccl/importccl/read_import_avro.go @@ -187,7 +187,7 @@ func (a *avroConsumer) convertNative(x interface{}, conv *row.DatumRowConverter) typ := conv.VisibleColTypes[idx] avroT, ok := familyToAvroT[typ.Family()] if !ok { - return fmt.Errorf("cannot convert avro value %v to col %s", v, conv.VisibleCols[idx].Type.Name()) + return fmt.Errorf("cannot convert avro value %v to col %s", v, conv.VisibleCols[idx].GetType().Name()) } datum, err := nativeToDatum(v, typ, avroT, conv.EvalCtx) @@ -212,7 +212,7 @@ func (a *avroConsumer) FillDatums( for i := range conv.Datums { if conv.TargetColOrds.Contains(i) && conv.Datums[i] == nil { if a.strict { - return fmt.Errorf("field %s was not set in the avro import", conv.VisibleCols[i].Name) + return fmt.Errorf("field %s was not set in the avro import", conv.VisibleCols[i].GetName()) } conv.Datums[i] = tree.DNull } diff --git a/pkg/ccl/importccl/read_import_csv.go b/pkg/ccl/importccl/read_import_csv.go index b087ddc39f25..0850cc4b7775 100644 --- a/pkg/ccl/importccl/read_import_csv.go +++ b/pkg/ccl/importccl/read_import_csv.go @@ -196,7 +196,7 @@ func (c *csvRowConsumer) FillDatums( if err != nil { col := conv.VisibleCols[i] return newImportRowError( - errors.Wrapf(err, "parse %q as %s", col.Name, col.Type.SQLString()), + errors.Wrapf(err, "parse %q as %s", col.GetName(), col.GetType().SQLString()), strRecord(record, c.opts.Comma), rowNum) } diff --git a/pkg/ccl/importccl/read_import_mysql_test.go b/pkg/ccl/importccl/read_import_mysql_test.go index c4ca5e209404..4e848e474246 100644 --- a/pkg/ccl/importccl/read_import_mysql_test.go +++ b/pkg/ccl/importccl/read_import_mysql_test.go @@ -224,13 +224,13 @@ func compareTables(t *testing.T, expected, got *descpb.TableDescriptor) { expectedDesc := tabledesc.NewBuilder(expected).BuildImmutableTable() gotDesc := tabledesc.NewBuilder(got).BuildImmutableTable() e, err := catformat.IndexForDisplay( - ctx, expectedDesc, tableName, &expected.Indexes[i], "" /* partition */, "" /* interleave */, &semaCtx, + ctx, expectedDesc, tableName, expectedDesc.PublicNonPrimaryIndexes()[i], "" /* partition */, "" /* interleave */, &semaCtx, ) if err != nil { t.Fatalf("unexpected error: %s", err) } g, err := catformat.IndexForDisplay( - ctx, gotDesc, tableName, &got.Indexes[i], "" /* partition */, "" /* interleave */, &semaCtx, + ctx, gotDesc, tableName, gotDesc.PublicNonPrimaryIndexes()[i], "" /* partition */, "" /* interleave */, &semaCtx, ) if err != nil { t.Fatalf("unexpected error: %s", err) diff --git a/pkg/ccl/importccl/read_import_mysqlout.go b/pkg/ccl/importccl/read_import_mysqlout.go index 43eb38506f2f..5912d1b98fdf 100644 --- a/pkg/ccl/importccl/read_import_mysqlout.go +++ b/pkg/ccl/importccl/read_import_mysqlout.go @@ -252,7 +252,7 @@ func (d *delimitedConsumer) FillDatums( if err != nil { col := conv.VisibleCols[datumIdx] return newImportRowError( - fmt.Errorf("error %s while parse %q as %s", err, col.Name, col.Type.SQLString()), + fmt.Errorf("error %s while parse %q as %s", err, col.GetName(), col.GetType().SQLString()), string(data), rowNum) } } diff --git a/pkg/ccl/importccl/read_import_pgcopy.go b/pkg/ccl/importccl/read_import_pgcopy.go index 225de1d14e67..4010119bf066 100644 --- a/pkg/ccl/importccl/read_import_pgcopy.go +++ b/pkg/ccl/importccl/read_import_pgcopy.go @@ -333,7 +333,7 @@ func (p *pgCopyConsumer) FillDatums( col := conv.VisibleCols[i] return newImportRowError(fmt.Errorf( "encountered error %s when attempting to parse %q as %s", - err.Error(), col.Name, col.Type.SQLString()), data.String(), rowNum) + err.Error(), col.GetName(), col.GetType().SQLString()), data.String(), rowNum) } } } diff --git a/pkg/ccl/importccl/read_import_pgdump.go b/pkg/ccl/importccl/read_import_pgdump.go index caa90bcd79f1..5f75918b0639 100644 --- a/pkg/ccl/importccl/read_import_pgdump.go +++ b/pkg/ccl/importccl/read_import_pgdump.go @@ -1149,7 +1149,7 @@ func (m *pgDumpReader) readFile( if err != nil { col := conv.VisibleCols[idx] return wrapRowErr(err, "", count, pgcode.Syntax, - "parse %q as %s", col.Name, col.Type.SQLString()) + "parse %q as %s", col.GetName(), col.GetType().SQLString()) } } } diff --git a/pkg/ccl/partitionccl/partition.go b/pkg/ccl/partitionccl/partition.go index 8e3856a61aa0..6c1899917fb7 100644 --- a/pkg/ccl/partitionccl/partition.go +++ b/pkg/ccl/partitionccl/partition.go @@ -39,10 +39,7 @@ import ( // TODO(dan): The typechecking here should be run during plan construction, so // we can support placeholders. func valueEncodePartitionTuple( - typ tree.PartitionByType, - evalCtx *tree.EvalContext, - maybeTuple tree.Expr, - cols []descpb.ColumnDescriptor, + typ tree.PartitionByType, evalCtx *tree.EvalContext, maybeTuple tree.Expr, cols []catalog.Column, ) ([]byte, error) { // Replace any occurrences of the MINVALUE/MAXVALUE pseudo-names // into MinVal and MaxVal, to be recognized below. @@ -98,7 +95,7 @@ func valueEncodePartitionTuple( } var semaCtx tree.SemaContext - typedExpr, err := schemaexpr.SanitizeVarFreeExpr(evalCtx.Context, expr, cols[i].Type, "partition", + typedExpr, err := schemaexpr.SanitizeVarFreeExpr(evalCtx.Context, expr, cols[i].GetType(), "partition", &semaCtx, tree.VolatilityImmutable, ) @@ -113,7 +110,7 @@ func valueEncodePartitionTuple( if err != nil { return nil, errors.Wrapf(err, "evaluating %s", typedExpr) } - if err := colinfo.CheckDatumTypeFitsColumnType(&cols[i], datum.ResolvedType()); err != nil { + if err := colinfo.CheckDatumTypeFitsColumnType(cols[i], datum.ResolvedType()); err != nil { return nil, err } value, err = rowenc.EncodeTableValue( @@ -176,7 +173,7 @@ func createPartitioningImpl( return strings.Join(partCols, ", ") } - var cols []descpb.ColumnDescriptor + var cols []catalog.Column for i := 0; i < len(partBy.Fields); i++ { if colOffset+i >= len(indexDesc.ColumnNames) { return partDesc, pgerror.Newf(pgcode.Syntax, @@ -193,7 +190,7 @@ func createPartitioningImpl( if err != nil { return partDesc, err } - cols = append(cols, *col.ColumnDesc()) + cols = append(cols, col) if string(partBy.Fields[i]) != col.GetName() { // This used to print the first `colOffset + len(partBy.Fields)` fields // but there might not be this many columns in the index. See #37682. @@ -437,7 +434,7 @@ func selectPartitionExprs( AddMutations: true, }, func(idx catalog.Index) error { return selectPartitionExprsByName( - a, evalCtx, tableDesc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, prefixDatums, exprsByPartName, true /* genExpr */) + a, evalCtx, tableDesc, idx, &idx.IndexDesc().Partitioning, prefixDatums, exprsByPartName, true /* genExpr */) }); err != nil { return nil, err } @@ -490,7 +487,7 @@ func selectPartitionExprsByName( a *rowenc.DatumAlloc, evalCtx *tree.EvalContext, tableDesc catalog.TableDescriptor, - idxDesc *descpb.IndexDescriptor, + idx catalog.Index, partDesc *descpb.PartitioningDescriptor, prefixDatums tree.Datums, exprsByPartName map[string]tree.TypedExpr, @@ -507,7 +504,7 @@ func selectPartitionExprsByName( exprsByPartName[l.Name] = tree.DBoolFalse var fakeDatums tree.Datums if err := selectPartitionExprsByName( - a, evalCtx, tableDesc, idxDesc, &l.Subpartitioning, fakeDatums, exprsByPartName, genExpr, + a, evalCtx, tableDesc, idx, &l.Subpartitioning, fakeDatums, exprsByPartName, genExpr, ); err != nil { return err } @@ -524,7 +521,7 @@ func selectPartitionExprsByName( // the column ordinal references, so reconstruct them here. colVars = make(tree.Exprs, len(prefixDatums)+int(partDesc.NumColumns)) for i := range colVars { - col, err := tabledesc.FindPublicColumnWithID(tableDesc, idxDesc.ColumnIDs[i]) + col, err := tabledesc.FindPublicColumnWithID(tableDesc, idx.GetColumnID(i)) if err != nil { return err } @@ -547,7 +544,7 @@ func selectPartitionExprsByName( for _, l := range partDesc.List { for _, valueEncBuf := range l.Values { t, _, err := rowenc.DecodePartitionTuple( - a, evalCtx.Codec, tableDesc, idxDesc, partDesc, valueEncBuf, prefixDatums) + a, evalCtx.Codec, tableDesc, idx, partDesc, valueEncBuf, prefixDatums) if err != nil { return err } @@ -581,7 +578,7 @@ func selectPartitionExprsByName( genExpr = false } if err := selectPartitionExprsByName( - a, evalCtx, tableDesc, idxDesc, &l.Subpartitioning, allDatums, exprsByPartName, genExpr, + a, evalCtx, tableDesc, idx, &l.Subpartitioning, allDatums, exprsByPartName, genExpr, ); err != nil { return err } diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index 0a27af4fee91..66745c7d1da9 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -1346,7 +1346,7 @@ func TestRepartitioning(t *testing.T) { repartition.WriteString(`PARTITION BY NOTHING`) } else { if err := sql.ShowCreatePartitioning( - &rowenc.DatumAlloc{}, keys.SystemSQLCodec, test.new.parsed.tableDesc, testIndex.IndexDesc(), + &rowenc.DatumAlloc{}, keys.SystemSQLCodec, test.new.parsed.tableDesc, testIndex, &testIndex.IndexDesc().Partitioning, &repartition, 0 /* indent */, 0, /* colOffset */ ); err != nil { t.Fatalf("%+v", err) diff --git a/pkg/ccl/storageccl/key_rewriter.go b/pkg/ccl/storageccl/key_rewriter.go index 935b40c98a36..3ac967f91e27 100644 --- a/pkg/ccl/storageccl/key_rewriter.go +++ b/pkg/ccl/storageccl/key_rewriter.go @@ -241,7 +241,7 @@ func (kr *KeyRewriter) rewriteTableKey(key []byte, isFromSpan bool) ([]byte, boo if !idx.Primary() { return nil, false, errors.New("restoring interleaved secondary indexes not supported") } - colIDs, _ := idx.IndexDesc().FullColumnIDs() + colIDs, _ := catalog.FullIndexColumnIDs(idx) var skipCols int for i := 0; i < idx.NumInterleaveAncestors(); i++ { skipCols += int(idx.GetInterleaveAncestor(i).SharedPrefixLen) diff --git a/pkg/ccl/streamingccl/streamingtest/encoding.go b/pkg/ccl/streamingccl/streamingtest/encoding.go index c3120b72e737..fec3eb3d0be2 100644 --- a/pkg/ccl/streamingccl/streamingtest/encoding.go +++ b/pkg/ccl/streamingccl/streamingtest/encoding.go @@ -26,8 +26,8 @@ func EncodeKV( t *testing.T, codec keys.SQLCodec, descr catalog.TableDescriptor, pkeyVals ...interface{}, ) roachpb.KeyValue { require.Equal(t, 1, descr.NumFamilies(), "there can be only one") - primary := descr.GetPrimaryIndex().IndexDesc() - require.LessOrEqual(t, len(primary.ColumnIDs), len(pkeyVals)) + primary := descr.GetPrimaryIndex() + require.LessOrEqual(t, primary.NumColumns(), len(pkeyVals)) var datums tree.Datums var colMap catalog.TableColMap diff --git a/pkg/server/settingswatcher/row_decoder.go b/pkg/server/settingswatcher/row_decoder.go index 0a80e238c062..52c8c740d31a 100644 --- a/pkg/server/settingswatcher/row_decoder.go +++ b/pkg/server/settingswatcher/row_decoder.go @@ -36,7 +36,7 @@ func MakeRowDecoder(codec keys.SQLCodec) RowDecoder { return RowDecoder{ codec: codec, colIdxMap: row.ColIDtoRowIndexFromCols( - systemschema.SettingsTable.TableDesc().Columns, + systemschema.SettingsTable.PublicColumns(), ), } } @@ -52,7 +52,7 @@ func (d *RowDecoder) DecodeRow( { types := []*types.T{tbl.PublicColumns()[0].GetType()} nameRow := make([]rowenc.EncDatum, 1) - _, matches, _, err := rowenc.DecodeIndexKey(d.codec, tbl, tbl.GetPrimaryIndex().IndexDesc(), types, nameRow, nil, kv.Key) + _, matches, _, err := rowenc.DecodeIndexKey(d.codec, tbl, tbl.GetPrimaryIndex(), types, nameRow, nil, kv.Key) if err != nil { return "", "", "", false, errors.Wrap(err, "failed to decode key") } diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 75bf21782293..e13fbe548df8 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -532,6 +532,7 @@ go_test( "//pkg/sql/catalog/descs", "//pkg/sql/catalog/lease", "//pkg/sql/catalog/multiregion", + "//pkg/sql/catalog/systemschema", "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", "//pkg/sql/distsql", diff --git a/pkg/sql/alter_column_type.go b/pkg/sql/alter_column_type.go index 05d08adf70d1..70d86c70e48e 100644 --- a/pkg/sql/alter_column_type.go +++ b/pkg/sql/alter_column_type.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" @@ -57,7 +58,7 @@ var alterColTypeInCombinationNotSupportedErr = unimplemented.NewWithIssuef( func AlterColumnType( ctx context.Context, tableDesc *tabledesc.Mutable, - col *descpb.ColumnDescriptor, + col catalog.Column, t *tree.AlterTableAlterColumnType, params runParams, cmds tree.AlterTableCmds, @@ -66,13 +67,13 @@ func AlterColumnType( for _, tableRef := range tableDesc.DependedOnBy { found := false for _, colID := range tableRef.ColumnIDs { - if colID == col.ID { + if colID == col.GetID() { found = true } } if found { return params.p.dependentViewError( - ctx, "column", col.Name, tableDesc.ParentID, tableRef.ID, "alter type of", + ctx, "column", col.GetName(), tableDesc.ParentID, tableRef.ID, "alter type of", ) } } @@ -114,7 +115,7 @@ func AlterColumnType( // using the expression. kind = schemachange.ColumnConversionGeneral } else { - kind, err = schemachange.ClassifyConversion(ctx, col.Type, typ) + kind, err = schemachange.ClassifyConversion(ctx, col.GetType(), typ) if err != nil { return err } @@ -127,9 +128,9 @@ func AlterColumnType( // what they're going for. return pgerror.Newf(pgcode.CannotCoerce, "the requested type conversion (%s -> %s) requires an explicit USING expression", - col.Type.SQLString(), typ.SQLString()) + col.GetType().SQLString(), typ.SQLString()) case schemachange.ColumnConversionTrivial: - col.Type = typ + col.ColumnDesc().Type = typ case schemachange.ColumnConversionGeneral, schemachange.ColumnConversionValidate: if err := alterColumnTypeGeneral(ctx, tableDesc, col, typ, t.Using, params, cmds, tn); err != nil { return err @@ -142,7 +143,7 @@ func AlterColumnType( "some writes to the altered column may be rejected until the schema change is finalized")) default: return errors.AssertionFailedf("unknown conversion for %s -> %s", - col.Type.SQLString(), typ.SQLString()) + col.GetType().SQLString(), typ.SQLString()) } return nil @@ -151,7 +152,7 @@ func AlterColumnType( func alterColumnTypeGeneral( ctx context.Context, tableDesc *tabledesc.Mutable, - col *descpb.ColumnDescriptor, + col catalog.Column, toType *types.T, using tree.Expr, params runParams, @@ -174,7 +175,7 @@ func alterColumnTypeGeneral( errors.WithIssueLink( errors.Newf("ALTER COLUMN TYPE from %v to %v is only "+ "supported experimentally", - col.Type, toType), + col.GetType(), toType), errors.IssueLink{IssueURL: build.MakeIssueURL(49329)}), "you can enable alter column type general support by running "+ "`SET enable_experimental_alter_column_type_general = true`"), @@ -182,14 +183,14 @@ func alterColumnTypeGeneral( } // Disallow ALTER COLUMN TYPE general for columns that own sequences. - if len(col.OwnsSequenceIds) != 0 { + if col.NumOwnsSequences() != 0 { return colOwnsSequenceNotSupportedErr } // Disallow ALTER COLUMN TYPE general for columns that have a check // constraint. for i := range tableDesc.Checks { - uses, err := tableDesc.CheckConstraintUsesColumn(tableDesc.Checks[i], col.ID) + uses, err := tableDesc.CheckConstraintUsesColumn(tableDesc.Checks[i], col.GetID()) if err != nil { return err } @@ -202,7 +203,7 @@ func alterColumnTypeGeneral( // UNIQUE WITHOUT INDEX constraint. for _, uc := range tableDesc.AllActiveAndInactiveUniqueWithoutIndexConstraints() { for _, id := range uc.ColumnIDs { - if col.ID == id { + if col.GetID() == id { return colWithConstraintNotSupportedErr } } @@ -212,7 +213,7 @@ func alterColumnTypeGeneral( // constraint. for _, fk := range tableDesc.AllActiveAndInactiveForeignKeys() { for _, id := range append(fk.OriginColumnIDs, fk.ReferencedColumnIDs...) { - if col.ID == id { + if col.GetID() == id { return colWithConstraintNotSupportedErr } } @@ -222,12 +223,12 @@ func alterColumnTypeGeneral( // part of indexes. for _, idx := range tableDesc.NonDropIndexes() { for i := 0; i < idx.NumColumns(); i++ { - if idx.GetColumnID(i) == col.ID { + if idx.GetColumnID(i) == col.GetID() { return colInIndexNotSupportedErr } } for i := 0; i < idx.NumExtraColumns(); i++ { - if idx.GetExtraColumnID(i) == col.ID { + if idx.GetExtraColumnID(i) == col.GetID() { return colInIndexNotSupportedErr } } @@ -258,7 +259,7 @@ func alterColumnTypeGeneral( return err == nil } - shadowColName := tabledesc.GenerateUniqueConstraintName(col.Name, nameExists) + shadowColName := tabledesc.GenerateUniqueConstraintName(col.GetName(), nameExists) var newColComputeExpr *string // oldCol still needs to have values written to it in case nodes read it from @@ -289,7 +290,7 @@ func alterColumnTypeGeneral( newColComputeExpr = &expr insertedValToString := tree.CastExpr{ - Expr: &tree.ColumnItem{ColumnName: tree.Name(col.Name)}, + Expr: &tree.ColumnItem{ColumnName: tree.Name(col.GetName())}, Type: types.String, SyntaxMode: tree.CastShort, } @@ -307,9 +308,9 @@ func alterColumnTypeGeneral( "'column %s is undergoing the ALTER COLUMN TYPE USING EXPRESSION "+ "schema change, inserts are not supported until the schema change is "+ "finalized, '", - col.Name) + col.GetName()) failedInsertMsg := fmt.Sprintf( - "'tried to insert ', %s, ' into %s'", insertedVal, col.Name, + "'tried to insert ', %s, ' into %s'", insertedVal, col.GetName(), ) inverseExpr = fmt.Sprintf( "crdb_internal.force_error('%s', concat(%s, %s))", @@ -317,7 +318,7 @@ func alterColumnTypeGeneral( } else { // The default computed expression is casting the column to the new type. newComputedExpr := tree.CastExpr{ - Expr: &tree.ColumnItem{ColumnName: tree.Name(col.Name)}, + Expr: &tree.ColumnItem{ColumnName: tree.Name(col.GetName())}, Type: toType, SyntaxMode: tree.CastShort, } @@ -325,8 +326,8 @@ func alterColumnTypeGeneral( newColComputeExpr = &s oldColComputeExpr := tree.CastExpr{ - Expr: &tree.ColumnItem{ColumnName: tree.Name(col.Name)}, - Type: col.Type, + Expr: &tree.ColumnItem{ColumnName: tree.Name(col.GetName())}, + Type: col.GetType(), SyntaxMode: tree.CastShort, } inverseExpr = tree.Serialize(&oldColComputeExpr) @@ -336,13 +337,13 @@ func alterColumnTypeGeneral( hasDefault := col.HasDefault() var newColDefaultExpr *string if hasDefault { - if col.HasNullDefault() { + if col.ColumnDesc().HasNullDefault() { s := tree.Serialize(tree.DNull) newColDefaultExpr = &s } else { // The default expression for the new column is applying the // computed expression to the previous default expression. - expr, err := parser.ParseExpr(*col.DefaultExpr) + expr, err := parser.ParseExpr(col.GetDefaultExpr()) if err != nil { return err } @@ -363,16 +364,16 @@ func alterColumnTypeGeneral( newCol := descpb.ColumnDescriptor{ Name: shadowColName, Type: toType, - Nullable: col.Nullable, + Nullable: col.IsNullable(), DefaultExpr: newColDefaultExpr, - UsesSequenceIds: col.UsesSequenceIds, - OwnsSequenceIds: col.OwnsSequenceIds, + UsesSequenceIds: col.ColumnDesc().UsesSequenceIds, + OwnsSequenceIds: col.ColumnDesc().OwnsSequenceIds, ComputeExpr: newColComputeExpr, } // Ensure new column is created in the same column family as the original // so backfiller writes to the same column family. - family, err := tableDesc.GetFamilyOfColumn(col.ID) + family, err := tableDesc.GetFamilyOfColumn(col.GetID()) if err != nil { return err } @@ -389,7 +390,7 @@ func alterColumnTypeGeneral( } swapArgs := &descpb.ComputedColumnSwap{ - OldColumnId: col.ID, + OldColumnId: col.GetID(), NewColumnId: newCol.ID, InverseExpr: inverseExpr, } diff --git a/pkg/sql/alter_index.go b/pkg/sql/alter_index.go index 51d6343ad94e..869358cd1633 100644 --- a/pkg/sql/alter_index.go +++ b/pkg/sql/alter_index.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -29,7 +30,7 @@ import ( type alterIndexNode struct { n *tree.AlterIndex tableDesc *tabledesc.Mutable - indexDesc *descpb.IndexDescriptor + index catalog.Index } // AlterIndex applies a schema change on an index. @@ -43,19 +44,11 @@ func (p *planner) AlterIndex(ctx context.Context, n *tree.AlterIndex) (planNode, return nil, err } - tableDesc, indexDesc, err := p.getTableAndIndex(ctx, &n.Index, privilege.CREATE) + tableDesc, index, err := p.getTableAndIndex(ctx, &n.Index, privilege.CREATE) if err != nil { return nil, err } - // As an artifact of finding the index by name, we get a pointer to a - // different copy than the one in the tableDesc. To make it easier for the - // code below, get a pointer to the index descriptor that's actually in - // tableDesc. - index, err := tableDesc.FindIndexWithID(indexDesc.ID) - if err != nil { - return nil, err - } - return &alterIndexNode{n: n, tableDesc: tableDesc, indexDesc: index.IndexDesc()}, nil + return &alterIndexNode{n: n, tableDesc: tableDesc, index: index}, nil } // ReadingOwnWrites implements the planNodeReadingOwnWrites interface. @@ -86,7 +79,8 @@ func (n *alterIndexNode) startExec(params runParams) error { "cannot change the partitioning of an index if the table has PARTITION ALL BY defined", ) } - if n.indexDesc.Partitioning.NumImplicitColumns > 0 { + existingIndexDesc := n.index.IndexDescDeepCopy() + if existingIndexDesc.Partitioning.NumImplicitColumns > 0 { return unimplemented.New( "ALTER INDEX PARTITION BY", "cannot ALTER INDEX PARTITION BY on an index which already has implicit column partitioning", @@ -99,7 +93,7 @@ func (n *alterIndexNode) startExec(params runParams) error { params.extendedEvalCtx.Settings, params.EvalContext(), n.tableDesc, - *n.indexDesc, + existingIndexDesc, t.PartitionBy, nil, /* allowedNewColumnNames */ allowImplicitPartitioning, @@ -113,19 +107,21 @@ func (n *alterIndexNode) startExec(params runParams) error { "cannot ALTER INDEX and change the partitioning to contain implicit columns", ) } - descriptorChanged = !n.indexDesc.Equal(&newIndexDesc) + descriptorChanged = !existingIndexDesc.Equal(&newIndexDesc) if err = deleteRemovedPartitionZoneConfigs( params.ctx, params.p.txn, n.tableDesc, - n.indexDesc, - &n.indexDesc.Partitioning, + n.index.GetID(), + &existingIndexDesc.Partitioning, &newIndexDesc.Partitioning, params.extendedEvalCtx.ExecCfg, ); err != nil { return err } - *n.indexDesc = newIndexDesc + // Update value to make sure it doesn't become stale. + n.index = n.tableDesc.AllIndexes()[n.index.Ordinal()] + *n.index.IndexDesc() = newIndexDesc default: return errors.AssertionFailedf( "unsupported alter command: %T", cmd) @@ -158,7 +154,7 @@ func (n *alterIndexNode) startExec(params runParams) error { n.tableDesc.ID, &eventpb.AlterIndex{ TableName: n.n.Index.Table.FQString(), - IndexName: n.indexDesc.Name, + IndexName: n.index.GetName(), MutationID: uint32(mutationID), }) } diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index 59b11381ab8a..cdb011647ffb 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -15,6 +15,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -24,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" ) @@ -383,7 +383,7 @@ func (p *planner) AlterPrimaryKey( // * depend on uniqueness from the old primary key (inverted, non-unique, or unique with nulls). // * don't store or index all columns in the new primary key. // * is affected by a locality config swap. - shouldRewriteIndex := func(idx *descpb.IndexDescriptor) (bool, error) { + shouldRewriteIndex := func(idx catalog.Index) (bool, error) { if alterPrimaryKeyLocalitySwap != nil { return true, nil } @@ -392,8 +392,9 @@ func (p *planner) AlterPrimaryKey( return true, nil } } - if idx.Unique { - for _, colID := range idx.ColumnIDs { + if idx.IsUnique() { + for i := 0; i < idx.NumColumns(); i++ { + colID := idx.GetColumnID(i) col, err := tableDesc.FindColumnWithID(colID) if err != nil { return false, err @@ -403,26 +404,24 @@ func (p *planner) AlterPrimaryKey( } } } - return !idx.Unique || idx.Type == descpb.IndexDescriptor_INVERTED, nil + return !idx.IsUnique() || idx.GetType() == descpb.IndexDescriptor_INVERTED, nil } - var indexesToRewrite []*descpb.IndexDescriptor + var indexesToRewrite []catalog.Index for _, idx := range tableDesc.PublicNonPrimaryIndexes() { - shouldRewrite, err := shouldRewriteIndex(idx.IndexDesc()) + shouldRewrite, err := shouldRewriteIndex(idx) if err != nil { return err } if idx.GetID() != newPrimaryIndexDesc.ID && shouldRewrite { - indexesToRewrite = append(indexesToRewrite, idx.IndexDesc()) + indexesToRewrite = append(indexesToRewrite, idx) } } // TODO (rohany): this loop will be unused until #45510 is resolved. - for i := range tableDesc.Mutations { - mut := &tableDesc.Mutations[i] + for _, mut := range tableDesc.AllMutations() { // If there is an index that is getting built right now that started in a previous txn, we // need to potentially rebuild that index as well. - if idx := mut.GetIndex(); mut.MutationID < currentMutationID && idx != nil && - mut.Direction == descpb.DescriptorMutation_ADD { + if idx := mut.AsIndex(); mut.MutationID() < currentMutationID && idx != nil && mut.Adding() { shouldRewrite, err := shouldRewriteIndex(idx) if err != nil { return err @@ -439,7 +438,7 @@ func (p *planner) AlterPrimaryKey( var oldIndexIDs, newIndexIDs []descpb.IndexID for _, idx := range indexesToRewrite { // Clone the index that we want to rewrite. - newIndex := protoutil.Clone(idx).(*descpb.IndexDescriptor) + newIndex := idx.IndexDescDeepCopy() basename := newIndex.Name + "_rewrite_for_primary_key_change" // Drop any PARTITION ALL BY clause. @@ -451,7 +450,7 @@ func (p *planner) AlterPrimaryKey( } newIndex.Name = tabledesc.GenerateUniqueConstraintName(basename, nameExists) - if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, newIndex, newPrimaryIndexDesc); err != nil { + if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, &newIndex, newPrimaryIndexDesc); err != nil { return err } // If the index that we are rewriting is interleaved, we need to setup the rewritten @@ -459,18 +458,18 @@ func (p *planner) AlterPrimaryKey( // on the new index is already set up. So, we just need to add the backreference from the // parent to this new index. if len(newIndex.Interleave.Ancestors) != 0 { - if err := p.finalizeInterleave(ctx, tableDesc, newIndex); err != nil { + if err := p.finalizeInterleave(ctx, tableDesc, &newIndex); err != nil { return err } } // Create partitioning if we are newly adding a PARTITION BY ALL statement. if isNewPartitionAllBy { - if *newIndex, err = CreatePartitioning( + if newIndex, err = CreatePartitioning( ctx, p.ExecCfg().Settings, p.EvalContext(), tableDesc, - *newIndex, + newIndex, partitionAllBy, allowedNewColumnNames, allowImplicitPartitioning, @@ -478,7 +477,7 @@ func (p *planner) AlterPrimaryKey( return err } } - oldIndexIDs = append(oldIndexIDs, idx.ID) + oldIndexIDs = append(oldIndexIDs, idx.GetID()) newIndexIDs = append(newIndexIDs, newIndex.ID) } @@ -537,7 +536,7 @@ func (p *planner) shouldCreateIndexes( oldPK := desc.GetPrimaryIndex() // Validate if basic properties between the two match. - if len(oldPK.IndexDesc().ColumnIDs) != len(alterPKNode.Columns) || + if oldPK.NumColumns() != len(alterPKNode.Columns) || oldPK.IsSharded() != (alterPKNode.Sharded != nil) || oldPK.IsInterleaved() != (alterPKNode.Interleave != nil) { return true, nil @@ -549,7 +548,7 @@ func (p *planner) shouldCreateIndexes( if err != nil { return true, err } - if oldPK.IndexDesc().Sharded.ShardBuckets != shardBuckets { + if oldPK.GetSharded().ShardBuckets != shardBuckets { return true, nil } } @@ -565,15 +564,14 @@ func (p *planner) shouldCreateIndexes( return true, err } - ancestors := oldPK.IndexDesc().Interleave.Ancestors - if len(ancestors) == 0 { + if oldPK.NumInterleaveAncestors() == 0 { return true, nil } - if ancestors[len(ancestors)-1].TableID != + if oldPK.GetInterleaveAncestor(oldPK.NumInterleaveAncestors()-1).TableID != parentTable.GetID() { return true, nil } - if ancestors[len(ancestors)-1].IndexID != + if oldPK.GetInterleaveAncestor(oldPK.NumInterleaveAncestors()-1).IndexID != parentTable.GetPrimaryIndexID() { return true, nil } @@ -581,7 +579,7 @@ func (p *planner) shouldCreateIndexes( // If the old primary key is dropped, then recreation // is required. - if oldPK.IndexDesc().Disabled { + if oldPK.IsDisabled() { return true, nil } @@ -592,13 +590,13 @@ func (p *planner) shouldCreateIndexes( return true, err } - if col.GetID() != oldPK.IndexDesc().ColumnIDs[idx] { + if col.GetID() != oldPK.GetColumnID(idx) { return true, nil } if (elem.Direction == tree.Ascending && - oldPK.IndexDesc().ColumnDirections[idx] != descpb.IndexDescriptor_ASC) || + oldPK.GetColumnDirection(idx) != descpb.IndexDescriptor_ASC) || (elem.Direction == tree.Descending && - oldPK.IndexDesc().ColumnDirections[idx] != descpb.IndexDescriptor_DESC) { + oldPK.GetColumnDirection(idx) != descpb.IndexDescriptor_DESC) { return true, nil } } @@ -612,7 +610,7 @@ func (p *planner) shouldCreateIndexes( return true, nil } if localitySwapConfig.NewRegionalByRowColumnID != nil && - *localitySwapConfig.NewRegionalByRowColumnID != oldPK.IndexDesc().ColumnIDs[0] { + *localitySwapConfig.NewRegionalByRowColumnID != oldPK.GetColumnID(0) { return true, nil } } diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 457634fa92c7..21c617f944ec 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -427,18 +427,18 @@ func (n *alterTableNode) startExec(params runParams) error { // If the dropped column uses a sequence, remove references to it from that sequence. if colToDrop.NumUsesSequences() > 0 { - if err := params.p.removeSequenceDependencies(params.ctx, n.tableDesc, colToDrop.ColumnDesc()); err != nil { + if err := params.p.removeSequenceDependencies(params.ctx, n.tableDesc, colToDrop); err != nil { return err } } // You can't remove a column that owns a sequence that is depended on // by another column - if err := params.p.canRemoveAllColumnOwnedSequences(params.ctx, n.tableDesc, colToDrop.ColumnDesc(), t.DropBehavior); err != nil { + if err := params.p.canRemoveAllColumnOwnedSequences(params.ctx, n.tableDesc, colToDrop, t.DropBehavior); err != nil { return err } - if err := params.p.dropSequencesOwnedByCol(params.ctx, colToDrop.ColumnDesc(), true /* queueJob */, t.DropBehavior); err != nil { + if err := params.p.dropSequencesOwnedByCol(params.ctx, colToDrop, true /* queueJob */, t.DropBehavior); err != nil { return err } @@ -489,7 +489,7 @@ func (n *alterTableNode) startExec(params runParams) error { ¶ms.p.semaCtx, tn, ) - if err := computedColValidator.ValidateNoDependents(colToDrop.ColumnDesc()); err != nil { + if err := computedColValidator.ValidateNoDependents(colToDrop); err != nil { return err } @@ -810,7 +810,7 @@ func (n *alterTableNode) startExec(params runParams) error { "column %q in the middle of being dropped", t.GetColumn()) } // Apply mutations to copy of column descriptor. - if err := applyColumnMutation(params.ctx, n.tableDesc, col.ColumnDesc(), t, params, n.n.Cmds, tn); err != nil { + if err := applyColumnMutation(params.ctx, n.tableDesc, col, t, params, n.n.Cmds, tn); err != nil { return err } descriptorChanged = true @@ -859,7 +859,7 @@ func (n *alterTableNode) startExec(params runParams) error { params.ctx, params.p.txn, n.tableDesc, - n.tableDesc.GetPrimaryIndex().IndexDesc(), + n.tableDesc.GetPrimaryIndexID(), &oldPartitioning, &newPrimaryIndex.Partitioning, params.extendedEvalCtx.ExecCfg, @@ -1052,7 +1052,7 @@ func addIndexMutationWithSpecificPrimaryKey( func applyColumnMutation( ctx context.Context, tableDesc *tabledesc.Mutable, - col *descpb.ColumnDescriptor, + col catalog.Column, mut tree.ColumnMutationCmd, params runParams, cmds tree.AlterTableCmds, @@ -1063,15 +1063,15 @@ func applyColumnMutation( return AlterColumnType(ctx, tableDesc, col, t, params, cmds, tn) case *tree.AlterTableSetDefault: - if len(col.UsesSequenceIds) > 0 { + if col.NumUsesSequences() > 0 { if err := params.p.removeSequenceDependencies(params.ctx, tableDesc, col); err != nil { return err } } if t.Default == nil { - col.DefaultExpr = nil + col.ColumnDesc().DefaultExpr = nil } else { - colDatumType := col.Type + colDatumType := col.GetType() expr, err := schemaexpr.SanitizeVarFreeExpr( params.ctx, t.Default, colDatumType, "DEFAULT", ¶ms.p.semaCtx, tree.VolatilityVolatile, ) @@ -1079,11 +1079,11 @@ func applyColumnMutation( return pgerror.WithCandidateCode(err, pgcode.DatatypeMismatch) } s := tree.Serialize(expr) - col.DefaultExpr = &s + col.ColumnDesc().DefaultExpr = &s // Add references to the sequence descriptors this column is now using. changedSeqDescs, err := maybeAddSequenceDependencies( - params.ctx, params.p.ExecCfg().Settings, params.p, tableDesc, col, expr, nil, /* backrefs */ + params.ctx, params.p.ExecCfg().Settings, params.p, tableDesc, col.ColumnDesc(), expr, nil, /* backrefs */ ) if err != nil { return err @@ -1100,21 +1100,21 @@ func applyColumnMutation( } case *tree.AlterTableSetVisible: - col, err := tableDesc.FindActiveOrNewColumnByName(col.ColName()) + column, err := tableDesc.FindActiveOrNewColumnByName(col.ColName()) if err != nil { return err } - col.ColumnDesc().Hidden = !t.Visible + column.ColumnDesc().Hidden = !t.Visible case *tree.AlterTableSetNotNull: - if !col.Nullable { + if !col.IsNullable() { return nil } // See if there's already a mutation to add a not null constraint for i := range tableDesc.Mutations { if constraint := tableDesc.Mutations[i].GetConstraint(); constraint != nil && constraint.ConstraintType == descpb.ConstraintToUpdate_NOT_NULL && - constraint.NotNullColumn == col.ID { + constraint.NotNullColumn == col.GetID() { if tableDesc.Mutations[i].Direction == descpb.DescriptorMutation_ADD { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "constraint in the middle of being added") @@ -1132,25 +1132,25 @@ func applyColumnMutation( for k := range info { inuseNames[k] = struct{}{} } - check := tabledesc.MakeNotNullCheckConstraint(col.Name, col.ID, inuseNames, descpb.ConstraintValidity_Validating) + check := tabledesc.MakeNotNullCheckConstraint(col.GetName(), col.GetID(), inuseNames, descpb.ConstraintValidity_Validating) tableDesc.AddNotNullMutation(check, descpb.DescriptorMutation_ADD) case *tree.AlterTableDropNotNull: - if col.Nullable { + if col.IsNullable() { return nil } // Prevent a column in a primary key from becoming non-null. - if tableDesc.GetPrimaryIndex().ContainsColumnID(col.ID) { + if tableDesc.GetPrimaryIndex().ContainsColumnID(col.GetID()) { return pgerror.Newf(pgcode.InvalidTableDefinition, - `column "%s" is in a primary index`, col.Name) + `column "%s" is in a primary index`, col.GetName()) } // See if there's already a mutation to add/drop a not null constraint. for i := range tableDesc.Mutations { if constraint := tableDesc.Mutations[i].GetConstraint(); constraint != nil && constraint.ConstraintType == descpb.ConstraintToUpdate_NOT_NULL && - constraint.NotNullColumn == col.ID { + constraint.NotNullColumn == col.GetID() { if tableDesc.Mutations[i].Direction == descpb.DescriptorMutation_ADD { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "constraint in the middle of being added, try again later") @@ -1167,24 +1167,24 @@ func applyColumnMutation( for k := range info { inuseNames[k] = struct{}{} } - col.Nullable = true + col.ColumnDesc().Nullable = true // Add a check constraint equivalent to the non-null constraint and drop // it in the schema changer. - check := tabledesc.MakeNotNullCheckConstraint(col.Name, col.ID, inuseNames, descpb.ConstraintValidity_Dropping) + check := tabledesc.MakeNotNullCheckConstraint(col.GetName(), col.GetID(), inuseNames, descpb.ConstraintValidity_Dropping) tableDesc.Checks = append(tableDesc.Checks, check) tableDesc.AddNotNullMutation(check, descpb.DescriptorMutation_DROP) case *tree.AlterTableDropStored: if !col.IsComputed() { return pgerror.Newf(pgcode.InvalidColumnDefinition, - "column %q is not a computed column", col.Name) + "column %q is not a computed column", col.GetName()) } - if col.Virtual { + if col.IsVirtual() { return pgerror.Newf(pgcode.InvalidColumnDefinition, - "column %q is not a stored computed column", col.Name) + "column %q is not a stored computed column", col.GetName()) } - col.ComputeExpr = nil + col.ColumnDesc().ComputeExpr = nil } return nil } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index e7e068a985b7..98b2a7e5383e 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -188,7 +188,7 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { // Mutations are applied in a FIFO order. Only apply the first set of // mutations. Collect the elements that are part of the mutation. - var droppedIndexDescs []descpb.IndexDescriptor + var droppedIndexes []catalog.Index var addedIndexSpans []roachpb.Span var addedIndexes []descpb.IndexID @@ -268,13 +268,12 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { } else { return errors.AssertionFailedf("unsupported mutation: %+v", m) } - } else if m.Dropped() { if col := m.AsColumn(); col != nil { needColumnBackfill = catalog.ColumnNeedsBackfill(col) } else if idx := m.AsIndex(); idx != nil { - if !canClearRangeForDrop(idx.IndexDesc()) { - droppedIndexDescs = append(droppedIndexDescs, *idx.IndexDesc()) + if !canClearRangeForDrop(idx) { + droppedIndexes = append(droppedIndexes, idx) } } else if c := m.AsConstraint(); c != nil { constraintsToDrop = append(constraintsToDrop, c) @@ -306,8 +305,8 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { } // Drop indexes not to be removed by `ClearRange`. - if len(droppedIndexDescs) > 0 { - if err := sc.truncateIndexes(ctx, version, droppedIndexDescs); err != nil { + if len(droppedIndexes) > 0 { + if err := sc.truncateIndexes(ctx, version, droppedIndexes); err != nil { return err } } @@ -774,13 +773,21 @@ func TruncateInterleavedIndexes( ctx context.Context, execCfg *ExecutorConfig, table catalog.TableDescriptor, - indexes []descpb.IndexDescriptor, + indexIDs []descpb.IndexID, ) error { - log.Infof(ctx, "truncating %d interleaved indexes", len(indexes)) + log.Infof(ctx, "truncating %d interleaved indexes", len(indexIDs)) chunkSize := int64(indexTruncateChunkSize) alloc := &rowenc.DatumAlloc{} codec, db := execCfg.Codec, execCfg.DB - for _, desc := range indexes { + zoneConfigIndexIDList := make([]uint32, len(indexIDs)) + for i, id := range indexIDs { + zoneConfigIndexIDList[i] = uint32(id) + } + for _, id := range indexIDs { + idx, err := table.FindIndexWithID(id) + if err != nil { + return err + } var resume roachpb.Span for rowIdx, done := int64(0), false; !done; rowIdx += chunkSize { log.VEventf(ctx, 2, "truncate interleaved index (%d) at row: %d, span: %s", table.GetID(), rowIdx, resume) @@ -794,7 +801,7 @@ func TruncateInterleavedIndexes( } resume, err := td.deleteIndex( ctx, - &desc, + idx, resumeAt, chunkSize, false, /* traceKV */ @@ -812,7 +819,7 @@ func TruncateInterleavedIndexes( if err != nil { return err } - return RemoveIndexZoneConfigs(ctx, txn, execCfg, freshTableDesc, indexes) + return RemoveIndexZoneConfigs(ctx, txn, execCfg, freshTableDesc, zoneConfigIndexIDList) }); err != nil { return err } @@ -826,7 +833,7 @@ func TruncateInterleavedIndexes( // The indexes are dropped chunk by chunk, each chunk being deleted in // its own txn. func (sc *SchemaChanger) truncateIndexes( - ctx context.Context, version descpb.DescriptorVersion, dropped []descpb.IndexDescriptor, + ctx context.Context, version descpb.DescriptorVersion, dropped []catalog.Index, ) error { log.Infof(ctx, "clearing data for %d indexes", len(dropped)) @@ -835,7 +842,11 @@ func (sc *SchemaChanger) truncateIndexes( chunkSize = sc.testingKnobs.BackfillChunkSize } alloc := &rowenc.DatumAlloc{} - for _, desc := range dropped { + droppedIndexIDs := make([]uint32, len(dropped)) + for i, idx := range dropped { + droppedIndexIDs[i] = uint32(idx.GetID()) + } + for _, idx := range dropped { var resume roachpb.Span for rowIdx, done := int64(0), false; !done; rowIdx += chunkSize { resumeAt := resume @@ -867,10 +878,10 @@ func (sc *SchemaChanger) truncateIndexes( if err := td.init(ctx, txn, nil /* *tree.EvalContext */); err != nil { return err } - if !canClearRangeForDrop(&desc) { + if !canClearRangeForDrop(idx) { resume, err = td.deleteIndex( ctx, - &desc, + idx, resumeAt, chunkSize, false, /* traceKV */ @@ -879,7 +890,7 @@ func (sc *SchemaChanger) truncateIndexes( return err } done = true - return td.clearIndex(ctx, &desc) + return td.clearIndex(ctx, idx) }); err != nil { return err } @@ -892,7 +903,7 @@ func (sc *SchemaChanger) truncateIndexes( if err != nil { return err } - return RemoveIndexZoneConfigs(ctx, txn, sc.execCfg, table, dropped) + return RemoveIndexZoneConfigs(ctx, txn, sc.execCfg, table, droppedIndexIDs) }); err != nil { return err } @@ -1433,8 +1444,7 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error { return err } - var forwardIndexes []*descpb.IndexDescriptor - var invertedIndexes []*descpb.IndexDescriptor + var forwardIndexes, invertedIndexes []catalog.Index for _, m := range tableDesc.AllMutations() { if sc.mutationID != m.MutationID() { @@ -1446,9 +1456,9 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error { } switch idx.GetType() { case descpb.IndexDescriptor_FORWARD: - forwardIndexes = append(forwardIndexes, idx.IndexDesc()) + forwardIndexes = append(forwardIndexes, idx) case descpb.IndexDescriptor_INVERTED: - invertedIndexes = append(invertedIndexes, idx.IndexDesc()) + invertedIndexes = append(invertedIndexes, idx) } } if len(forwardIndexes) == 0 && len(invertedIndexes) == 0 { @@ -1495,7 +1505,7 @@ func ValidateInvertedIndexes( ctx context.Context, codec keys.SQLCodec, tableDesc catalog.TableDescriptor, - indexes []*descpb.IndexDescriptor, + indexes []catalog.Index, runHistoricalTxn HistoricalInternalExecTxnRunner, gatherAllInvalid bool, ) error { @@ -1518,7 +1528,7 @@ func ValidateInvertedIndexes( // distributed execution and avoid bypassing the SQL decoding start := timeutil.Now() var idxLen int64 - span := tableDesc.IndexSpan(codec, idx.ID) + span := tableDesc.IndexSpan(codec, idx.GetID()) key := span.Key endKey := span.EndKey if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, _ *InternalExecutor) error { @@ -1538,12 +1548,12 @@ func ValidateInvertedIndexes( return err } log.Infof(ctx, "inverted index %s/%s count = %d, took %s", - tableDesc.GetName(), idx.Name, idxLen, timeutil.Since(start)) + tableDesc.GetName(), idx.GetName(), idxLen, timeutil.Since(start)) select { case <-countReady[i]: if idxLen != expectedCount[i] { if gatherAllInvalid { - invalid <- idx.ID + invalid <- idx.GetID() return nil } // JSON columns cannot have unique indexes, so if the expected and @@ -1551,7 +1561,7 @@ func ValidateInvertedIndexes( // uniqueness violation. return errors.AssertionFailedf( "validation of index %s failed: expected %d rows, found %d", - idx.Name, errors.Safe(expectedCount[i]), errors.Safe(idxLen)) + idx.GetName(), errors.Safe(expectedCount[i]), errors.Safe(idxLen)) } case <-ctx.Done(): return ctx.Err() @@ -1567,21 +1577,22 @@ func ValidateInvertedIndexes( if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, ie *InternalExecutor) error { var stmt string - if geoindex.IsEmptyConfig(&idx.GeoConfig) { + geoConfig := idx.GetGeoConfig() + if geoindex.IsEmptyConfig(&geoConfig) { stmt = fmt.Sprintf( `SELECT coalesce(sum_int(crdb_internal.num_inverted_index_entries(%q, %d)), 0) FROM [%d AS t]`, - col, idx.Version, tableDesc.GetID(), + col, idx.GetVersion(), tableDesc.GetID(), ) } else { stmt = fmt.Sprintf( `SELECT coalesce(sum_int(crdb_internal.num_geo_inverted_index_entries(%d, %d, %q)), 0) FROM [%d AS t]`, - tableDesc.GetID(), idx.ID, col, tableDesc.GetID(), + tableDesc.GetID(), idx.GetID(), col, tableDesc.GetID(), ) } // If the index is a partial index the predicate must be added // as a filter to the query. if idx.IsPartial() { - stmt = fmt.Sprintf(`%s WHERE %s`, stmt, idx.Predicate) + stmt = fmt.Sprintf(`%s WHERE %s`, stmt, idx.GetPredicate()) } return ie.WithSyntheticDescriptors([]catalog.Descriptor{tableDesc}, func() error { row, err := ie.QueryRowEx(ctx, "verify-inverted-idx-count", txn, sessiondata.InternalExecutorOverride{}, stmt) @@ -1632,7 +1643,7 @@ func ValidateInvertedIndexes( func ValidateForwardIndexes( ctx context.Context, tableDesc catalog.TableDescriptor, - indexes []*descpb.IndexDescriptor, + indexes []catalog.Index, runHistoricalTxn HistoricalInternalExecTxnRunner, withFirstMutationPublic bool, gatherAllInvalid bool, @@ -1698,11 +1709,11 @@ func ValidateForwardIndexes( // Retrieve the row count in the index. var idxLen int64 if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, ie *InternalExecutor) error { - query := fmt.Sprintf(`SELECT count(1) FROM [%d AS t]@[%d]`, desc.GetID(), idx.ID) + query := fmt.Sprintf(`SELECT count(1) FROM [%d AS t]@[%d]`, desc.GetID(), idx.GetID()) // If the index is a partial index the predicate must be added // as a filter to the query to force scanning the index. if idx.IsPartial() { - query = fmt.Sprintf(`%s WHERE %s`, query, idx.Predicate) + query = fmt.Sprintf(`%s WHERE %s`, query, idx.GetPredicate()) } return ie.WithSyntheticDescriptors([]catalog.Descriptor{desc}, func() error { @@ -1717,13 +1728,13 @@ func ValidateForwardIndexes( // For implicitly partitioned unique indexes, we need to independently // validate that the non-implicitly partitioned columns are unique. - if idx.Unique && idx.Partitioning.NumImplicitColumns > 0 && !skipUniquenessChecks { + if idx.IsUnique() && idx.GetPartitioning().NumImplicitColumns > 0 && !skipUniquenessChecks { if err := validateUniqueConstraint( ctx, tableDesc, idx.GetName(), - idx.ColumnIDs[idx.Partitioning.NumImplicitColumns:], - idx.Predicate, + idx.IndexDesc().ColumnIDs[idx.GetPartitioning().NumImplicitColumns:], + idx.GetPredicate(), ie, txn, ); err != nil { @@ -1737,7 +1748,7 @@ func ValidateForwardIndexes( } log.Infof(ctx, "validation: index %s/%s row count = %d, time so far %s", - tableDesc.GetName(), idx.Name, idxLen, timeutil.Since(start)) + tableDesc.GetName(), idx.GetName(), idxLen, timeutil.Since(start)) // Now compare with the row count in the table. select { @@ -1746,19 +1757,19 @@ func ValidateForwardIndexes( // If the index is a partial index, the expected number of rows // is different than the total number of rows in the table. if idx.IsPartial() { - expectedCount = partialIndexExpectedCounts[idx.ID] + expectedCount = partialIndexExpectedCounts[idx.GetID()] } if idxLen != expectedCount { if gatherAllInvalid { - invalid <- idx.ID + invalid <- idx.GetID() return nil } // TODO(vivek): find the offending row and include it in the error. return pgerror.WithConstraintName(pgerror.Newf(pgcode.UniqueViolation, "duplicate key value violates unique constraint %q", - idx.Name), - idx.Name) + idx.GetName()), + idx.GetName()) } @@ -1795,7 +1806,7 @@ func ValidateForwardIndexes( // For partial indexes, count the number of rows in the table // for which the predicate expression evaluates to true. if idx.IsPartial() { - s.WriteString(fmt.Sprintf(`, count(1) FILTER (WHERE %s)`, idx.Predicate)) + s.WriteString(fmt.Sprintf(`, count(1) FILTER (WHERE %s)`, idx.GetPredicate())) } } partialIndexCounts := s.String() @@ -1817,7 +1828,7 @@ func ValidateForwardIndexes( cntIdx := 1 for _, idx := range indexes { if idx.IsPartial() { - partialIndexExpectedCounts[idx.ID] = int64(tree.MustBeDInt(cnt[cntIdx])) + partialIndexExpectedCounts[idx.GetID()] = int64(tree.MustBeDInt(cnt[cntIdx])) cntIdx++ } } @@ -2004,7 +2015,7 @@ func runSchemaChangesInTxn( } } else if idx := m.AsIndex(); idx != nil { if err := indexTruncateInTxn( - ctx, planner.Txn(), planner.ExecCfg(), planner.EvalContext(), immutDesc, idx.IndexDesc(), traceKV, + ctx, planner.Txn(), planner.ExecCfg(), planner.EvalContext(), immutDesc, idx, traceKV, ); err != nil { return err } @@ -2410,7 +2421,7 @@ func indexTruncateInTxn( execCfg *ExecutorConfig, evalCtx *tree.EvalContext, tableDesc catalog.TableDescriptor, - idx *descpb.IndexDescriptor, + idx catalog.Index, traceKV bool, ) error { alloc := &rowenc.DatumAlloc{} @@ -2430,5 +2441,5 @@ func indexTruncateInTxn( } } // Remove index zone configs. - return RemoveIndexZoneConfigs(ctx, txn, execCfg, tableDesc, []descpb.IndexDescriptor{*idx}) + return RemoveIndexZoneConfigs(ctx, txn, execCfg, tableDesc, []uint32{uint32(idx.GetID())}) } diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index cb77e41ad7ac..00f4362f0312 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -53,11 +53,11 @@ func IndexMutationFilter(m catalog.Mutation) bool { // ColumnBackfiller is capable of running a column backfill for all // updateCols. type ColumnBackfiller struct { - added []descpb.ColumnDescriptor - dropped []descpb.ColumnDescriptor + added []catalog.Column + dropped []catalog.Column // updateCols is a slice of all column descriptors that are being modified. - updateCols []descpb.ColumnDescriptor + updateCols []catalog.Column updateExprs []tree.TypedExpr evalCtx *tree.EvalContext @@ -72,7 +72,7 @@ type ColumnBackfiller struct { func (cb *ColumnBackfiller) initCols(desc catalog.TableDescriptor) { for _, m := range desc.AllMutations() { if ColumnMutationFilter(m) { - col := *m.AsColumn().ColumnDesc() + col := m.AsColumn() if m.Adding() { cb.added = append(cb.added, col) } else if m.Dropped() { @@ -95,8 +95,7 @@ func (cb *ColumnBackfiller) init( cb.updateCols = append(cb.added, cb.dropped...) // Populate default or computed values. cb.updateExprs = make([]tree.TypedExpr, len(cb.updateCols)) - for j := range cb.added { - col := &cb.added[j] + for j, col := range cb.added { if col.IsComputed() { cb.updateExprs[j] = computedExprs[j] } else if defaultExprs == nil || defaultExprs[j] == nil { @@ -115,14 +114,11 @@ func (cb *ColumnBackfiller) init( tableArgs := row.FetcherTableArgs{ Desc: desc, - Index: desc.GetPrimaryIndex().IndexDesc(), + Index: desc.GetPrimaryIndex(), ColIdxMap: catalog.ColumnIDToOrdinalMap(desc.PublicColumns()), - Cols: make([]descpb.ColumnDescriptor, len(desc.PublicColumns())), + Cols: desc.PublicColumns(), ValNeededForCol: valNeededForCol, } - for i, col := range desc.PublicColumns() { - tableArgs.Cols[i] = *col.ColumnDesc() - } // Create a bound account associated with the column backfiller. if mon == nil { @@ -160,14 +156,10 @@ func (cb *ColumnBackfiller) InitForLocalUse( if err != nil { return err } - colDescs := make([]descpb.ColumnDescriptor, len(desc.PublicColumns())) - for i, col := range desc.PublicColumns() { - colDescs[i] = *col.ColumnDesc() - } computedExprs, _, err := schemaexpr.MakeComputedExprs( ctx, cb.added, - colDescs, + desc.PublicColumns(), desc, tree.NewUnqualifiedTableName(tree.Name(desc.GetName())), evalCtx, @@ -192,10 +184,6 @@ func (cb *ColumnBackfiller) InitForDistributedUse( ) error { cb.initCols(desc) evalCtx := flowCtx.NewEvalCtx() - cols := make([]descpb.ColumnDescriptor, len(desc.PublicColumns())) - for i, col := range desc.PublicColumns() { - cols[i] = *col.ColumnDesc() - } var defaultExprs, computedExprs []tree.TypedExpr // Install type metadata in the target descriptors, as well as resolve any // user defined types in the column expressions. @@ -218,7 +206,7 @@ func (cb *ColumnBackfiller) InitForDistributedUse( computedExprs, _, err = schemaexpr.MakeComputedExprs( ctx, cb.added, - cols, + desc.PublicColumns(), desc, tree.NewUnqualifiedTableName(tree.Name(desc.GetName())), evalCtx, @@ -260,10 +248,8 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( ) (roachpb.Key, error) { // TODO(dan): Tighten up the bound on the requestedCols parameter to // makeRowUpdater. - requestedCols := make([]descpb.ColumnDescriptor, 0, len(tableDesc.PublicColumns())+len(cb.added)+len(cb.dropped)) - for _, col := range tableDesc.PublicColumns() { - requestedCols = append(requestedCols, *col.ColumnDesc()) - } + requestedCols := make([]catalog.Column, 0, len(tableDesc.PublicColumns())+len(cb.added)+len(cb.dropped)) + requestedCols = append(requestedCols, tableDesc.PublicColumns()...) requestedCols = append(requestedCols, cb.added...) requestedCols = append(requestedCols, cb.dropped...) ru, err := row.MakeUpdater( @@ -308,12 +294,10 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( b := txn.NewBatch() rowLength := 0 iv := &schemaexpr.RowIndexedVarContainer{ - Cols: make([]descpb.ColumnDescriptor, 0, len(tableDesc.PublicColumns())+len(cb.added)), + Cols: make([]catalog.Column, 0, len(tableDesc.PublicColumns())+len(cb.added)), Mapping: ru.FetchColIDtoRowIndex, } - for _, col := range tableDesc.PublicColumns() { - iv.Cols = append(iv.Cols, *col.ColumnDesc()) - } + iv.Cols = append(iv.Cols, tableDesc.PublicColumns()...) iv.Cols = append(iv.Cols, cb.added...) cb.evalCtx.IVarContainer = iv for i := int64(0); i < chunkSize; i++ { @@ -333,8 +317,8 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( if err != nil { return roachpb.Key{}, sqlerrors.NewInvalidSchemaDefinitionError(err) } - if j < len(cb.added) && !cb.added[j].Nullable && val == tree.DNull { - return roachpb.Key{}, sqlerrors.NewNonNullViolationError(cb.added[j].Name) + if j < len(cb.added) && !cb.added[j].IsNullable() && val == tree.DNull { + return roachpb.Key{}, sqlerrors.NewNonNullViolationError(cb.added[j].GetName()) } // Added computed column values should be usable for the next @@ -401,7 +385,7 @@ type muBoundAccount struct { // IndexBackfiller is capable of backfilling all the added index. type IndexBackfiller struct { - added []*descpb.IndexDescriptor + added []catalog.Index // colIdxMap maps ColumnIDs to indices into desc.Columns and desc.Mutations. colIdxMap catalog.TableColMap @@ -411,16 +395,16 @@ type IndexBackfiller struct { // cols are all of the writable (PUBLIC and DELETE_AND_WRITE_ONLY) columns in // the descriptor. - cols []descpb.ColumnDescriptor + cols []catalog.Column // addedCols are the columns in DELETE_AND_WRITE_ONLY being added as part of // this index which are not computed. - addedCols []descpb.ColumnDescriptor + addedCols []catalog.Column // computedCols are the columns in this index which are computed and do // not have concrete values in the source index. This is virtual computed // columns and stored computed columns which are non-public. - computedCols []descpb.ColumnDescriptor + computedCols []catalog.Column // Map of columns which need to be evaluated to their expressions. colExprs map[descpb.ColumnID]tree.TypedExpr @@ -432,7 +416,7 @@ type IndexBackfiller struct { // indexesToEncode is a list of indexes to encode entries for a given row. // It is a field of IndexBackfiller to avoid allocating a slice for each row // backfilled. - indexesToEncode []*descpb.IndexDescriptor + indexesToEncode []catalog.Index valNeededForCol util.FastIntSet @@ -446,7 +430,7 @@ type IndexBackfiller struct { // ContainsInvertedIndex returns true if backfilling an inverted index. func (ib *IndexBackfiller) ContainsInvertedIndex() bool { for _, idx := range ib.added { - if idx.Type == descpb.IndexDescriptor_INVERTED { + if idx.GetType() == descpb.IndexDescriptor_INVERTED { return true } } @@ -496,8 +480,8 @@ func (ib *IndexBackfiller) InitForLocalUse( func constructExprs( ctx context.Context, desc catalog.TableDescriptor, - addedIndexes []*descpb.IndexDescriptor, - cols, addedCols, computedCols []descpb.ColumnDescriptor, + addedIndexes []catalog.Index, + cols, addedCols, computedCols []catalog.Column, evalCtx *tree.EvalContext, semaCtx *tree.SemaContext, ) ( @@ -546,12 +530,12 @@ func constructExprs( colExprs = make(map[descpb.ColumnID]tree.TypedExpr, numColExprs) var addedColSet catalog.TableColSet for i := range defaultExprs { - id := addedCols[i].ID + id := addedCols[i].GetID() colExprs[id] = defaultExprs[i] addedColSet.Add(id) } for i := range computedCols { - id := computedCols[i].ID + id := computedCols[i].GetID() colExprs[id] = computedExprs[i] } @@ -660,27 +644,26 @@ func (ib *IndexBackfiller) ShrinkBoundAccount(ctx context.Context, shrinkBy int6 // initCols is a helper to populate column metadata of an IndexBackfiller. It // populates the cols and colIdxMap fields. func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) { - ib.cols = make([]descpb.ColumnDescriptor, 0, len(desc.DeletableColumns())) + ib.cols = make([]catalog.Column, 0, len(desc.DeletableColumns())) for _, column := range desc.DeletableColumns() { - columnDesc := *column.ColumnDesc() if column.Public() { if column.IsComputed() && column.IsVirtual() { - ib.computedCols = append(ib.computedCols, columnDesc) + ib.computedCols = append(ib.computedCols, column) } } else if column.Adding() && column.WriteAndDeleteOnly() { // If there are ongoing mutations, add columns that are being added and in // the DELETE_AND_WRITE_ONLY state. if column.IsComputed() { - ib.computedCols = append(ib.computedCols, columnDesc) + ib.computedCols = append(ib.computedCols, column) } else { - ib.addedCols = append(ib.addedCols, columnDesc) + ib.addedCols = append(ib.addedCols, column) } } else { continue } // Create a map of each column's ID to its ordinal. ib.colIdxMap.Set(column.GetID(), len(ib.cols)) - ib.cols = append(ib.cols, columnDesc) + ib.cols = append(ib.cols, column) } } @@ -699,14 +682,14 @@ func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) util.FastIn break } if IndexMutationFilter(m) { - idx := m.AsIndex().IndexDesc() + idx := m.AsIndex() ib.added = append(ib.added, idx) for i := range ib.cols { - id := ib.cols[i].ID + id := ib.cols[i].GetID() idxContainsColumn := idx.ContainsColumnID(id) - isPrimaryIndex := idx.GetEncodingType(desc.GetPrimaryIndexID()) == descpb.PrimaryIndexEncoding + isPrimaryIndex := idx.GetEncodingType() == descpb.PrimaryIndexEncoding if (idxContainsColumn || isPrimaryIndex) && - !ib.cols[i].Virtual && + !ib.cols[i].IsVirtual() && i < len(desc.PublicColumns()) { valNeededForCol.Add(i) } @@ -734,12 +717,12 @@ func (ib *IndexBackfiller) init( // reset in BuildIndexEntriesChunk for every row added. ib.indexesToEncode = ib.added if len(ib.predicates) > 0 { - ib.indexesToEncode = make([]*descpb.IndexDescriptor, 0, len(ib.added)) + ib.indexesToEncode = make([]catalog.Index, 0, len(ib.added)) } ib.types = make([]*types.T, len(ib.cols)) for i := range ib.cols { - ib.types[i] = ib.cols[i].Type + ib.types[i] = ib.cols[i].GetType() } // Create a bound account associated with the index backfiller monitor. @@ -791,7 +774,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk( // read or used tableArgs := row.FetcherTableArgs{ Desc: tableDesc, - Index: tableDesc.GetPrimaryIndex().IndexDesc(), + Index: tableDesc.GetPrimaryIndex(), ColIdxMap: ib.colIdxMap, Cols: ib.cols, ValNeededForCol: ib.valNeededForCol, @@ -832,9 +815,9 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk( } memUsedPerChunk += indexEntriesPerRowInitialBufferSize buffer := make([]rowenc.IndexEntry, len(ib.added)) - evaluateExprs := func(cols []descpb.ColumnDescriptor) error { + evaluateExprs := func(cols []catalog.Column) error { for i := range cols { - colID := cols[i].ID + colID := cols[i].GetID() texpr, ok := ib.colExprs[colID] if !ok { continue @@ -896,7 +879,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk( // If the index is a partial index, only include it if the // predicate expression evaluates to true. - texpr := ib.predicates[idx.ID] + texpr := ib.predicates[idx.GetID()] val, err := texpr.Eval(ib.evalCtx) if err != nil { diff --git a/pkg/sql/catalog/catalogkeys/BUILD.bazel b/pkg/sql/catalog/catalogkeys/BUILD.bazel index 72e8b8c7d61e..3543abc98c83 100644 --- a/pkg/sql/catalog/catalogkeys/BUILD.bazel +++ b/pkg/sql/catalog/catalogkeys/BUILD.bazel @@ -8,6 +8,7 @@ go_library( deps = [ "//pkg/keys", "//pkg/roachpb", + "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/systemschema", "//pkg/sql/sem/tree", diff --git a/pkg/sql/catalog/catalogkeys/keys.go b/pkg/sql/catalog/catalogkeys/keys.go index 47d52ee734b9..3772e7eacb92 100644 --- a/pkg/sql/catalog/catalogkeys/keys.go +++ b/pkg/sql/catalog/catalogkeys/keys.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -64,19 +65,20 @@ func IsDefaultCreatedDescriptor(descID descpb.ID) bool { // /51/1/42/#/51/2/1337 // which would return the slice // {ASC, ASC, ASC, 0, ASC, ASC, DESC} -func IndexKeyValDirs(index *descpb.IndexDescriptor) []encoding.Direction { +func IndexKeyValDirs(index catalog.Index) []encoding.Direction { if index == nil { return nil } - dirs := make([]encoding.Direction, 0, (len(index.Interleave.Ancestors)+1)*2+len(index.ColumnDirections)) + dirs := make([]encoding.Direction, 0, (index.NumInterleaveAncestors()+1)*2+index.NumColumns()) colIdx := 0 - for _, ancs := range index.Interleave.Ancestors { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + ancs := index.GetInterleaveAncestor(i) // Table/Index IDs are always encoded ascending. dirs = append(dirs, encoding.Ascending, encoding.Ascending) for i := 0; i < int(ancs.SharedPrefixLen); i++ { - d, err := index.ColumnDirections[colIdx].ToEncodingDirection() + d, err := index.GetColumnDirection(colIdx).ToEncodingDirection() if err != nil { panic(err) } @@ -93,8 +95,8 @@ func IndexKeyValDirs(index *descpb.IndexDescriptor) []encoding.Direction { // The index's table/index ID. dirs = append(dirs, encoding.Ascending, encoding.Ascending) - for colIdx < len(index.ColumnDirections) { - d, err := index.ColumnDirections[colIdx].ToEncodingDirection() + for colIdx < index.NumColumns() { + d, err := index.GetColumnDirection(colIdx).ToEncodingDirection() if err != nil { panic(err) } @@ -140,7 +142,7 @@ func PrettySpan(valDirs []encoding.Direction, span roachpb.Span, skip int) strin // PrettySpans returns a human-readable description of the spans. // If index is nil, then pretty print subroutines will use their default // settings. -func PrettySpans(index *descpb.IndexDescriptor, spans []roachpb.Span, skip int) string { +func PrettySpans(index catalog.Index, spans []roachpb.Span, skip int) string { if len(spans) == 0 { return "" } diff --git a/pkg/sql/catalog/catformat/index.go b/pkg/sql/catalog/catformat/index.go index 716b85075d13..7b0ff61b9cb4 100644 --- a/pkg/sql/catalog/catformat/index.go +++ b/pkg/sql/catalog/catformat/index.go @@ -37,6 +37,18 @@ import ( // INDEX i ON t (a) WHERE b > 0 // func IndexForDisplay( + ctx context.Context, + table catalog.TableDescriptor, + tableName *tree.TableName, + index catalog.Index, + partition string, + interleave string, + semaCtx *tree.SemaContext, +) (string, error) { + return indexForDisplay(ctx, table, tableName, index.IndexDesc(), partition, interleave, semaCtx) +} + +func indexForDisplay( ctx context.Context, table catalog.TableDescriptor, tableName *tree.TableName, diff --git a/pkg/sql/catalog/catformat/index_test.go b/pkg/sql/catalog/catformat/index_test.go index efc25b04aced..7ca481b15ed2 100644 --- a/pkg/sql/catalog/catformat/index_test.go +++ b/pkg/sql/catalog/catformat/index_test.go @@ -94,7 +94,7 @@ func TestIndexForDisplay(t *testing.T) { for testIdx, tc := range testData { t.Run(strconv.Itoa(testIdx), func(t *testing.T) { - got, err := IndexForDisplay( + got, err := indexForDisplay( ctx, tableDesc, &tc.tableName, &tc.index, tc.partition, tc.interleave, &semaCtx, ) if err != nil { diff --git a/pkg/sql/catalog/colinfo/col_type_info.go b/pkg/sql/catalog/colinfo/col_type_info.go index 201c5785f16c..702f6d902977 100644 --- a/pkg/sql/catalog/colinfo/col_type_info.go +++ b/pkg/sql/catalog/colinfo/col_type_info.go @@ -41,11 +41,11 @@ func ColTypeInfoFromColTypes(colTypes []*types.T) ColTypeInfo { return ColTypeInfo{colTypes: colTypes} } -// ColTypeInfoFromColDescs creates a ColTypeInfo from []ColumnDescriptor. -func ColTypeInfoFromColDescs(colDescs []descpb.ColumnDescriptor) ColTypeInfo { - colTypes := make([]*types.T, len(colDescs)) - for i, colDesc := range colDescs { - colTypes[i] = colDesc.Type +// ColTypeInfoFromColumns creates a ColTypeInfo from []catalog.Column. +func ColTypeInfoFromColumns(columns []catalog.Column) ColTypeInfo { + colTypes := make([]*types.T, len(columns)) + for i, col := range columns { + colTypes[i] = col.GetType() } return ColTypeInfoFromColTypes(colTypes) } @@ -171,7 +171,7 @@ func GetColumnTypes( // IDs into the outTypes slice, returning it. You must use the returned slice, // as this function might allocate a new slice. func GetColumnTypesFromColDescs( - cols []descpb.ColumnDescriptor, columnIDs []descpb.ColumnID, outTypes []*types.T, + cols []catalog.Column, columnIDs []descpb.ColumnID, outTypes []*types.T, ) []*types.T { if cap(outTypes) < len(columnIDs) { outTypes = make([]*types.T, len(columnIDs)) @@ -180,8 +180,8 @@ func GetColumnTypesFromColDescs( } for i, id := range columnIDs { for j := range cols { - if id == cols[j].ID { - outTypes[i] = cols[j].Type + if id == cols[j].GetID() { + outTypes[i] = cols[j].GetType() break } } diff --git a/pkg/sql/catalog/colinfo/column_type_properties.go b/pkg/sql/catalog/colinfo/column_type_properties.go index 4896bb4028bd..4d803794be1f 100644 --- a/pkg/sql/catalog/colinfo/column_type_properties.go +++ b/pkg/sql/catalog/colinfo/column_type_properties.go @@ -11,7 +11,7 @@ package colinfo import ( - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -26,14 +26,14 @@ import ( // scalar type String). // // This is used by the UPDATE, INSERT and UPSERT code. -func CheckDatumTypeFitsColumnType(col *descpb.ColumnDescriptor, typ *types.T) error { +func CheckDatumTypeFitsColumnType(col catalog.Column, typ *types.T) error { if typ.Family() == types.UnknownFamily { return nil } - if !typ.Equivalent(col.Type) { + if !typ.Equivalent(col.GetType()) { return pgerror.Newf(pgcode.DatatypeMismatch, "value type %s doesn't match type %s of column %q", - typ.String(), col.Type.String(), tree.ErrNameString(col.Name)) + typ.String(), col.GetType().String(), tree.ErrNameString(col.GetName())) } return nil } diff --git a/pkg/sql/catalog/colinfo/result_columns.go b/pkg/sql/catalog/colinfo/result_columns.go index 74ca84db5724..0d2c1a17d187 100644 --- a/pkg/sql/catalog/colinfo/result_columns.go +++ b/pkg/sql/catalog/colinfo/result_columns.go @@ -37,32 +37,15 @@ type ResultColumn struct { // describe the column types of a table. type ResultColumns []ResultColumn -// ResultColumnsFromColDescs converts []descpb.ColumnDescriptor to []ResultColumn. -func ResultColumnsFromColDescs( - tableID descpb.ID, colDescs []descpb.ColumnDescriptor, -) ResultColumns { - return resultColumnsFromColDescs(tableID, len(colDescs), func(i int) *descpb.ColumnDescriptor { - return &colDescs[i] - }) -} - -// ResultColumnsFromColDescPtrs converts []*descpb.ColumnDescriptor to []ResultColumn. -func ResultColumnsFromColDescPtrs( - tableID descpb.ID, colDescs []*descpb.ColumnDescriptor, -) ResultColumns { - return resultColumnsFromColDescs(tableID, len(colDescs), func(i int) *descpb.ColumnDescriptor { - return colDescs[i] - }) -} - // ResultColumnsFromColumns converts []catalog.Column to []ResultColumn. func ResultColumnsFromColumns(tableID descpb.ID, columns []catalog.Column) ResultColumns { - return resultColumnsFromColDescs(tableID, len(columns), func(i int) *descpb.ColumnDescriptor { + return ResultColumnsFromColDescs(tableID, len(columns), func(i int) *descpb.ColumnDescriptor { return columns[i].ColumnDesc() }) } -func resultColumnsFromColDescs( +// ResultColumnsFromColDescs is used by ResultColumnsFromColumns and by tests. +func ResultColumnsFromColDescs( tableID descpb.ID, numCols int, getColDesc func(int) *descpb.ColumnDescriptor, ) ResultColumns { cols := make(ResultColumns, numCols) diff --git a/pkg/sql/catalog/descpb/index.go b/pkg/sql/catalog/descpb/index.go index 78a68feb86e0..adef92493e85 100644 --- a/pkg/sql/catalog/descpb/index.go +++ b/pkg/sql/catalog/descpb/index.go @@ -39,19 +39,6 @@ func (desc *IndexDescriptor) RunOverAllColumns(fn func(id ColumnID) error) error return nil } -// GetEncodingType returns the encoding type of this index. For backward -// compatibility reasons, this might not match what is stored in -// desc.EncodingType. The primary index's ID must be passed so we can check if -// this index is primary or secondary. -func (desc *IndexDescriptor) GetEncodingType(primaryIndexID IndexID) IndexDescriptorEncodingType { - if desc.ID == primaryIndexID { - // Primary indexes always use the PrimaryIndexEncoding, regardless of what - // desc.EncodingType indicates. - return PrimaryIndexEncoding - } - return desc.EncodingType -} - // IsInterleaved returns whether the index is interleaved or not. func (desc *IndexDescriptor) IsInterleaved() bool { return len(desc.Interleave.Ancestors) > 0 || len(desc.InterleavedBy) > 0 @@ -133,25 +120,6 @@ func (desc *IndexDescriptor) ContainsColumnID(colID ColumnID) bool { }) != nil } -// FullColumnIDs returns the index column IDs including any extra (implicit or -// stored (old STORING encoding)) column IDs for non-unique indexes. It also -// returns the direction with which each column was encoded. -func (desc *IndexDescriptor) FullColumnIDs() ([]ColumnID, []IndexDescriptor_Direction) { - if desc.Unique { - return desc.ColumnIDs, desc.ColumnDirections - } - // Non-unique indexes have some of the primary-key columns appended to - // their key. - columnIDs := append([]ColumnID(nil), desc.ColumnIDs...) - columnIDs = append(columnIDs, desc.ExtraColumnIDs...) - dirs := append([]IndexDescriptor_Direction(nil), desc.ColumnDirections...) - for range desc.ExtraColumnIDs { - // Extra columns are encoded in ascending order. - dirs = append(dirs, IndexDescriptor_ASC) - } - return columnIDs, dirs -} - // TODO (tyler): Issue #39771 This method needs more thorough testing, probably // in structured_test.go. Or possibly replace it with a format method taking // a format context as argument. diff --git a/pkg/sql/catalog/schemaexpr/check_constraint.go b/pkg/sql/catalog/schemaexpr/check_constraint.go index aed66b18f0a9..a4f4b6f0425a 100644 --- a/pkg/sql/catalog/schemaexpr/check_constraint.go +++ b/pkg/sql/catalog/schemaexpr/check_constraint.go @@ -161,9 +161,9 @@ func (b *CheckConstraintBuilder) DefaultName(expr tree.Expr) (string, error) { var nameBuf bytes.Buffer nameBuf.WriteString("check") - err := iterColDescriptors(b.desc, expr, func(c *descpb.ColumnDescriptor) error { + err := iterColDescriptors(b.desc, expr, func(c catalog.Column) error { nameBuf.WriteByte('_') - nameBuf.WriteString(c.Name) + nameBuf.WriteString(c.GetName()) return nil }) if err != nil { diff --git a/pkg/sql/catalog/schemaexpr/column.go b/pkg/sql/catalog/schemaexpr/column.go index f66645036e0d..65f70641b21c 100644 --- a/pkg/sql/catalog/schemaexpr/column.go +++ b/pkg/sql/catalog/schemaexpr/column.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -79,39 +78,37 @@ func dequalifyColumnRefs( // converts user defined types in default and computed expressions to a // human-readable form. func FormatColumnForDisplay( - ctx context.Context, - tbl catalog.TableDescriptor, - desc *descpb.ColumnDescriptor, - semaCtx *tree.SemaContext, + ctx context.Context, tbl catalog.TableDescriptor, col catalog.Column, semaCtx *tree.SemaContext, ) (string, error) { f := tree.NewFmtCtx(tree.FmtSimple) - f.FormatNameP(&desc.Name) + name := col.GetName() + f.FormatNameP(&name) f.WriteByte(' ') - f.WriteString(desc.Type.SQLString()) - if desc.Hidden { + f.WriteString(col.GetType().SQLString()) + if col.IsHidden() { f.WriteString(" NOT VISIBLE") } - if desc.Nullable { + if col.IsNullable() { f.WriteString(" NULL") } else { f.WriteString(" NOT NULL") } - if desc.DefaultExpr != nil { + if col.HasDefault() { f.WriteString(" DEFAULT ") - defExpr, err := FormatExprForDisplay(ctx, tbl, *desc.DefaultExpr, semaCtx, tree.FmtParsable) + defExpr, err := FormatExprForDisplay(ctx, tbl, col.GetDefaultExpr(), semaCtx, tree.FmtParsable) if err != nil { return "", err } f.WriteString(defExpr) } - if desc.IsComputed() { + if col.IsComputed() { f.WriteString(" AS (") - compExpr, err := FormatExprForDisplay(ctx, tbl, *desc.ComputeExpr, semaCtx, tree.FmtParsable) + compExpr, err := FormatExprForDisplay(ctx, tbl, col.GetComputeExpr(), semaCtx, tree.FmtParsable) if err != nil { return "", err } f.WriteString(compExpr) - if desc.Virtual { + if col.IsVirtual() { f.WriteString(") VIRTUAL") } else { f.WriteString(") STORED") @@ -158,7 +155,7 @@ func RenameColumn(expr string, from tree.Name, to tree.Name) (string, error) { // If the expression references a column that does not exist in the table // descriptor, iterColDescriptors errs with pgcode.UndefinedColumn. func iterColDescriptors( - desc catalog.TableDescriptor, rootExpr tree.Expr, f func(*descpb.ColumnDescriptor) error, + desc catalog.TableDescriptor, rootExpr tree.Expr, f func(column catalog.Column) error, ) error { _, err := tree.SimpleVisit(rootExpr, func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { vBase, ok := expr.(tree.VarName) @@ -183,7 +180,7 @@ func iterColDescriptors( "column %q does not exist, referenced in %q", c.ColumnName, rootExpr.String()) } - if err := f(col.ColumnDesc()); err != nil { + if err := f(col); err != nil { return false, nil, err } return false, expr, err @@ -276,16 +273,6 @@ func replaceColumnVars( return newExpr, colIDs, err } -// columnDescriptorsToPtrs returns a list of references to the input -// ColumnDescriptors. -func columnDescriptorsToPtrs(cols []descpb.ColumnDescriptor) []*descpb.ColumnDescriptor { - ptrs := make([]*descpb.ColumnDescriptor, len(cols)) - for i := range cols { - ptrs[i] = &cols[i] - } - return ptrs -} - // ReplaceIDsWithFQNames walks the given expr and replaces occurrences // of regclass IDs in the expr with the descriptor's fully qualified name. // For example, nextval(12345::REGCLASS) => nextval('foo.public.seq'). diff --git a/pkg/sql/catalog/schemaexpr/column_test.go b/pkg/sql/catalog/schemaexpr/column_test.go index e193f9980029..1e2d72ebb865 100644 --- a/pkg/sql/catalog/schemaexpr/column_test.go +++ b/pkg/sql/catalog/schemaexpr/column_test.go @@ -62,7 +62,10 @@ func TestDequalifyColumnRefs(t *testing.T) { source := colinfo.NewSourceInfoForSingleTable( tn, colinfo.ResultColumnsFromColDescs( descpb.ID(1), - cols, + len(cols), + func(i int) *descpb.ColumnDescriptor { + return &cols[i] + }, ), ) diff --git a/pkg/sql/catalog/schemaexpr/computed_column.go b/pkg/sql/catalog/schemaexpr/computed_column.go index 3500c3c1451b..efdd66759535 100644 --- a/pkg/sql/catalog/schemaexpr/computed_column.go +++ b/pkg/sql/catalog/schemaexpr/computed_column.go @@ -69,12 +69,12 @@ func (v *ComputedColumnValidator) Validate( var depColIDs catalog.TableColSet // First, check that no column in the expression is a computed column. - err := iterColDescriptors(v.desc, d.Computed.Expr, func(c *descpb.ColumnDescriptor) error { + err := iterColDescriptors(v.desc, d.Computed.Expr, func(c catalog.Column) error { if c.IsComputed() { return pgerror.New(pgcode.InvalidTableDefinition, "computed columns cannot reference other computed columns") } - depColIDs.Add(c.ID) + depColIDs.Add(c.GetID()) return nil }) @@ -171,7 +171,7 @@ func (v *ComputedColumnValidator) Validate( // computed column. The function errs if any existing computed columns or // computed columns being added reference the given column. // TODO(mgartner): Add unit tests for ValidateNoDependents. -func (v *ComputedColumnValidator) ValidateNoDependents(col *descpb.ColumnDescriptor) error { +func (v *ComputedColumnValidator) ValidateNoDependents(col catalog.Column) error { for _, c := range v.desc.NonDropColumns() { if !c.IsComputed() { continue @@ -183,12 +183,12 @@ func (v *ComputedColumnValidator) ValidateNoDependents(col *descpb.ColumnDescrip return errors.WithAssertionFailure(err) } - err = iterColDescriptors(v.desc, expr, func(colVar *descpb.ColumnDescriptor) error { - if colVar.ID == col.ID { + err = iterColDescriptors(v.desc, expr, func(colVar catalog.Column) error { + if colVar.GetID() == col.GetID() { return pgerror.Newf( pgcode.InvalidColumnReference, "column %q is referenced by computed column %q", - col.Name, + col.GetName(), c.GetName(), ) } @@ -214,7 +214,7 @@ func (v *ComputedColumnValidator) ValidateNoDependents(col *descpb.ColumnDescrip // columns that come after them in input. func MakeComputedExprs( ctx context.Context, - input, sourceColumns []descpb.ColumnDescriptor, + input, sourceColumns []catalog.Column, tableDesc catalog.TableDescriptor, tn *tree.TableName, evalCtx *tree.EvalContext, @@ -237,10 +237,9 @@ func MakeComputedExprs( // Build the computed expressions map from the parsed statement. computedExprs := make([]tree.TypedExpr, 0, len(input)) exprStrings := make([]string, 0, len(input)) - for i := range input { - col := &input[i] + for _, col := range input { if col.IsComputed() { - exprStrings = append(exprStrings, *col.ComputeExpr) + exprStrings = append(exprStrings, col.GetComputeExpr()) } } @@ -249,13 +248,12 @@ func MakeComputedExprs( return nil, catalog.TableColSet{}, err } - nr := newNameResolver(evalCtx, tableDesc.GetID(), tn, columnDescriptorsToPtrs(sourceColumns)) + nr := newNameResolver(evalCtx, tableDesc.GetID(), tn, sourceColumns) nr.addIVarContainerToSemaCtx(semaCtx) var txCtx transform.ExprTransformContext compExprIdx := 0 - for i := range input { - col := &input[i] + for _, col := range input { if !col.IsComputed() { computedExprs = append(computedExprs, tree.DNull) nr.addColumn(col) @@ -275,7 +273,7 @@ func MakeComputedExprs( return nil, catalog.TableColSet{}, err } - typedExpr, err := tree.TypeCheck(ctx, expr, semaCtx, col.Type) + typedExpr, err := tree.TypeCheck(ctx, expr, semaCtx, col.GetType()) if err != nil { return nil, catalog.TableColSet{}, err } diff --git a/pkg/sql/catalog/schemaexpr/computed_exprs.go b/pkg/sql/catalog/schemaexpr/computed_exprs.go index 980a43fc5f29..981105a21ed6 100644 --- a/pkg/sql/catalog/schemaexpr/computed_exprs.go +++ b/pkg/sql/catalog/schemaexpr/computed_exprs.go @@ -12,7 +12,6 @@ package schemaexpr import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -26,7 +25,7 @@ type RowIndexedVarContainer struct { // Because the rows we have might not be permuted in the same way as the // original table, we need to store a mapping between them. - Cols []descpb.ColumnDescriptor + Cols []catalog.Column Mapping catalog.TableColMap } @@ -36,7 +35,7 @@ var _ tree.IndexedVarContainer = &RowIndexedVarContainer{} func (r *RowIndexedVarContainer) IndexedVarEval( idx int, ctx *tree.EvalContext, ) (tree.Datum, error) { - rowIdx, ok := r.Mapping.Get(r.Cols[idx].ID) + rowIdx, ok := r.Mapping.Get(r.Cols[idx].GetID()) if !ok { return tree.DNull, nil } diff --git a/pkg/sql/catalog/schemaexpr/default_exprs.go b/pkg/sql/catalog/schemaexpr/default_exprs.go index 98dd54353a29..1b647ecbc01d 100644 --- a/pkg/sql/catalog/schemaexpr/default_exprs.go +++ b/pkg/sql/catalog/schemaexpr/default_exprs.go @@ -14,7 +14,6 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/transform" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -28,7 +27,7 @@ import ( // as default. func MakeDefaultExprs( ctx context.Context, - cols []descpb.ColumnDescriptor, + cols []catalog.Column, txCtx *transform.ExprTransformContext, evalCtx *tree.EvalContext, semaCtx *tree.SemaContext, @@ -37,8 +36,8 @@ func MakeDefaultExprs( // are no DEFAULT expressions, we don't bother with constructing the // defaults map as the defaults are all NULL. haveDefaults := false - for i := range cols { - if cols[i].DefaultExpr != nil { + for _, col := range cols { + if col.HasDefault() { haveDefaults = true break } @@ -50,10 +49,9 @@ func MakeDefaultExprs( // Build the default expressions map from the parsed SELECT statement. defaultExprs := make([]tree.TypedExpr, 0, len(cols)) exprStrings := make([]string, 0, len(cols)) - for i := range cols { - col := &cols[i] - if col.DefaultExpr != nil { - exprStrings = append(exprStrings, *col.DefaultExpr) + for _, col := range cols { + if col.HasDefault() { + exprStrings = append(exprStrings, col.GetDefaultExpr()) } } exprs, err := parser.ParseExprs(exprStrings) @@ -62,14 +60,13 @@ func MakeDefaultExprs( } defExprIdx := 0 - for i := range cols { - col := &cols[i] - if col.DefaultExpr == nil { + for _, col := range cols { + if !col.HasDefault() { defaultExprs = append(defaultExprs, tree.DNull) continue } expr := exprs[defExprIdx] - typedExpr, err := tree.TypeCheck(ctx, expr, semaCtx, col.Type) + typedExpr, err := tree.TypeCheck(ctx, expr, semaCtx, col.GetType()) if err != nil { return nil, err } @@ -85,22 +82,20 @@ func MakeDefaultExprs( // ProcessColumnSet returns columns in cols, and other writable // columns in tableDesc that fulfills a given criteria in inSet. func ProcessColumnSet( - cols []descpb.ColumnDescriptor, - tableDesc catalog.TableDescriptor, - inSet func(*descpb.ColumnDescriptor) bool, -) []descpb.ColumnDescriptor { + cols []catalog.Column, tableDesc catalog.TableDescriptor, inSet func(column catalog.Column) bool, +) []catalog.Column { var colIDSet catalog.TableColSet for i := range cols { - colIDSet.Add(cols[i].ID) + colIDSet.Add(cols[i].GetID()) } // Add all public or columns in DELETE_AND_WRITE_ONLY state // that satisfy the condition. for _, col := range tableDesc.WritableColumns() { - if inSet(col.ColumnDesc()) { + if inSet(col) { if !colIDSet.Contains(col.GetID()) { colIDSet.Add(col.GetID()) - cols = append(cols, *col.ColumnDesc()) + cols = append(cols, col) } } } diff --git a/pkg/sql/catalog/schemaexpr/expr.go b/pkg/sql/catalog/schemaexpr/expr.go index 3c2034fe0b12..c230ac42c6bc 100644 --- a/pkg/sql/catalog/schemaexpr/expr.go +++ b/pkg/sql/catalog/schemaexpr/expr.go @@ -46,16 +46,8 @@ func DequalifyAndValidateExpr( ) (string, catalog.TableColSet, error) { var colIDs catalog.TableColSet nonDropColumns := desc.NonDropColumns() - nonDropColumnDescs := make([]descpb.ColumnDescriptor, len(nonDropColumns)) - for i, col := range nonDropColumns { - nonDropColumnDescs[i] = *col.ColumnDesc() - } - sourceInfo := colinfo.NewSourceInfoForSingleTable( - *tn, colinfo.ResultColumnsFromColDescs( - desc.GetID(), - nonDropColumnDescs, - ), + *tn, colinfo.ResultColumnsFromColumns(desc.GetID(), nonDropColumns), ) expr, err := dequalifyColumnRefs(ctx, sourceInfo, expr) if err != nil { @@ -241,11 +233,11 @@ type nameResolver struct { // newNameResolver creates and returns a nameResolver. func newNameResolver( - evalCtx *tree.EvalContext, tableID descpb.ID, tn *tree.TableName, cols []*descpb.ColumnDescriptor, + evalCtx *tree.EvalContext, tableID descpb.ID, tn *tree.TableName, cols []catalog.Column, ) *nameResolver { source := colinfo.NewSourceInfoForSingleTable( *tn, - colinfo.ResultColumnsFromColDescPtrs(tableID, cols), + colinfo.ResultColumnsFromColumns(tableID, cols), ) nrc := &nameResolverIVarContainer{cols} ivarHelper := tree.MakeIndexedVarHelper(nrc, len(cols)) @@ -268,10 +260,10 @@ func (nr *nameResolver) resolveNames(expr tree.Expr) (tree.Expr, error) { // addColumn adds a new column to the nameResolver so that it can be resolved in // future calls to resolveNames. -func (nr *nameResolver) addColumn(col *descpb.ColumnDescriptor) { +func (nr *nameResolver) addColumn(col catalog.Column) { nr.ivarHelper.AppendSlot() nr.nrc.cols = append(nr.nrc.cols, col) - newCols := colinfo.ResultColumnsFromColDescs(nr.tableID, []descpb.ColumnDescriptor{*col}) + newCols := colinfo.ResultColumnsFromColumns(nr.tableID, []catalog.Column{col}) nr.source.SourceColumns = append(nr.source.SourceColumns, newCols...) } @@ -285,7 +277,7 @@ func (nr *nameResolver) addIVarContainerToSemaCtx(semaCtx *tree.SemaContext) { // tree.IndexedVarContainer. It is used to resolve and type check columns in // expressions. It does not support evaluation. type nameResolverIVarContainer struct { - cols []*descpb.ColumnDescriptor + cols []catalog.Column } // IndexedVarEval implements the tree.IndexedVarContainer interface. @@ -298,7 +290,7 @@ func (nrc *nameResolverIVarContainer) IndexedVarEval( // IndexedVarResolvedType implements the tree.IndexedVarContainer interface. func (nrc *nameResolverIVarContainer) IndexedVarResolvedType(idx int) *types.T { - return nrc.cols[idx].Type + return nrc.cols[idx].GetType() } // IndexVarNodeFormatter implements the tree.IndexedVarContainer interface. diff --git a/pkg/sql/catalog/schemaexpr/partial_index.go b/pkg/sql/catalog/schemaexpr/partial_index.go index 5ae1166de0dd..aad052305dba 100644 --- a/pkg/sql/catalog/schemaexpr/partial_index.go +++ b/pkg/sql/catalog/schemaexpr/partial_index.go @@ -87,8 +87,8 @@ func (v *IndexPredicateValidator) Validate(e tree.Expr) (string, error) { // that are added previously in the same transaction. func MakePartialIndexExprs( ctx context.Context, - indexes []*descpb.IndexDescriptor, - cols []descpb.ColumnDescriptor, + indexes []catalog.Index, + cols []catalog.Column, tableDesc catalog.TableDescriptor, evalCtx *tree.EvalContext, semaCtx *tree.SemaContext, @@ -107,13 +107,13 @@ func MakePartialIndexExprs( exprs := make(map[descpb.IndexID]tree.TypedExpr, partialIndexCount) tn := tree.NewUnqualifiedTableName(tree.Name(tableDesc.GetName())) - nr := newNameResolver(evalCtx, tableDesc.GetID(), tn, columnDescriptorsToPtrs(cols)) + nr := newNameResolver(evalCtx, tableDesc.GetID(), tn, cols) nr.addIVarContainerToSemaCtx(semaCtx) var txCtx transform.ExprTransformContext for _, idx := range indexes { if idx.IsPartial() { - expr, err := parser.ParseExpr(idx.Predicate) + expr, err := parser.ParseExpr(idx.GetPredicate()) if err != nil { return nil, refColIDs, err } @@ -140,7 +140,7 @@ func MakePartialIndexExprs( return nil, refColIDs, err } - exprs[idx.ID] = typedExpr + exprs[idx.GetID()] = typedExpr } } diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go index 4553d49f6874..7b78e80de649 100644 --- a/pkg/sql/catalog/table_elements.go +++ b/pkg/sql/catalog/table_elements.go @@ -512,6 +512,32 @@ func FindDeleteOnlyNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bo return findIndex(desc.DeleteOnlyNonPrimaryIndexes(), test) } +// FullIndexColumnIDs returns the index column IDs including any extra (implicit or +// stored (old STORING encoding)) column IDs for non-unique indexes. It also +// returns the direction with which each column was encoded. +func FullIndexColumnIDs(idx Index) ([]descpb.ColumnID, []descpb.IndexDescriptor_Direction) { + n := idx.NumColumns() + if !idx.IsUnique() { + n += idx.NumExtraColumns() + } + ids := make([]descpb.ColumnID, 0, n) + dirs := make([]descpb.IndexDescriptor_Direction, 0, n) + for i := 0; i < idx.NumColumns(); i++ { + ids = append(ids, idx.GetColumnID(i)) + dirs = append(dirs, idx.GetColumnDirection(i)) + } + // Non-unique indexes have some of the primary-key columns appended to + // their key. + if !idx.IsUnique() { + for i := 0; i < idx.NumExtraColumns(); i++ { + // Extra columns are encoded in ascending order. + ids = append(ids, idx.GetExtraColumnID(i)) + dirs = append(dirs, descpb.IndexDescriptor_ASC) + } + } + return ids, dirs +} + // UserDefinedTypeColsHaveSameVersion returns whether one table descriptor's // columns with user defined type metadata have the same versions of metadata // as in the other descriptor. Note that this function is only valid on two diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 00da3a382732..026b7a16e232 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -87,10 +87,8 @@ type PostDeserializationTableDescriptorChanges struct { // FindIndexPartitionByName searches this index descriptor for a partition whose name // is the input and returns it, or nil if no match is found. -func FindIndexPartitionByName( - desc *descpb.IndexDescriptor, name string, -) *descpb.PartitioningDescriptor { - return desc.Partitioning.FindPartitionByName(name) +func FindIndexPartitionByName(idx catalog.Index, name string) *descpb.PartitioningDescriptor { + return idx.IndexDesc().Partitioning.FindPartitionByName(name) } // DescriptorType returns the type of this descriptor. @@ -1140,9 +1138,9 @@ func (desc *Mutable) RemoveColumnFromFamily(colID descpb.ColumnID) { // RenameColumnDescriptor updates all references to a column name in // a table descriptor including indexes and families. -func (desc *Mutable) RenameColumnDescriptor(column *descpb.ColumnDescriptor, newColName string) { - colID := column.ID - column.Name = newColName +func (desc *Mutable) RenameColumnDescriptor(column catalog.Column, newColName string) { + colID := column.GetID() + column.ColumnDesc().Name = newColName for i := range desc.Families { for j := range desc.Families[i].ColumnIDs { @@ -1213,31 +1211,6 @@ func (desc *wrapper) NamesForColumnIDs(ids descpb.ColumnIDs) ([]string, error) { return names, nil } -// RenameIndexDescriptor renames an index descriptor. -func (desc *Mutable) RenameIndexDescriptor(index *descpb.IndexDescriptor, name string) error { - id := index.ID - if id == desc.PrimaryIndex.ID { - idx := desc.PrimaryIndex - idx.Name = name - desc.SetPrimaryIndex(idx) - return nil - } - for i, idx := range desc.Indexes { - if idx.ID == id { - idx.Name = name - desc.SetPublicNonPrimaryIndex(i+1, idx) - return nil - } - } - for _, m := range desc.Mutations { - if idx := m.GetIndex(); idx != nil && idx.ID == id { - idx.Name = name - return nil - } - } - return fmt.Errorf("index with id = %d does not exist", id) -} - // DropConstraint drops a constraint, either by removing it from the table // descriptor or by queuing a mutation for a schema change. func (desc *Mutable) DropConstraint( @@ -1393,7 +1366,12 @@ func (desc *Mutable) RenameConstraint( } return dependentViewRenameError("index", tableRef.ID) } - return desc.RenameIndexDescriptor(detail.Index, newName) + idx, err := desc.FindIndexWithID(detail.Index.ID) + if err != nil { + return err + } + idx.IndexDesc().Name = newName + return nil case descpb.ConstraintTypeUnique: if detail.Index != nil { @@ -1403,9 +1381,11 @@ func (desc *Mutable) RenameConstraint( } return dependentViewRenameError("index", tableRef.ID) } - if err := desc.RenameIndexDescriptor(detail.Index, newName); err != nil { + idx, err := desc.FindIndexWithID(detail.Index.ID) + if err != nil { return err } + idx.IndexDesc().Name = newName } else if detail.UniqueWithoutIndexConstraint != nil { if detail.UniqueWithoutIndexConstraint.Validity == descpb.ConstraintValidity_Validating { return unimplemented.NewWithIssueDetailf(42844, @@ -1745,8 +1725,8 @@ func (desc *Mutable) performComputedColumnSwap(swap *descpb.ComputedColumnSwap) oldColName := oldCol.GetName() // Rename old column to this new name, and rename newCol to oldCol's name. - desc.RenameColumnDescriptor(oldCol.ColumnDesc(), uniqueName) - desc.RenameColumnDescriptor(newCol.ColumnDesc(), oldColName) + desc.RenameColumnDescriptor(oldCol, uniqueName) + desc.RenameColumnDescriptor(newCol, oldColName) // Swap Column Family ordering for oldCol and newCol. // Both columns must be in the same family since the new column is @@ -2283,9 +2263,9 @@ func (desc *immutable) ActiveChecks() []descpb.TableDescriptor_CheckConstraint { // IsShardColumn returns true if col corresponds to a non-dropped hash sharded // index. This method assumes that col is currently a member of desc. -func (desc *Mutable) IsShardColumn(col *descpb.ColumnDescriptor) bool { +func (desc *Mutable) IsShardColumn(col catalog.Column) bool { return nil != catalog.FindNonDropIndex(desc, func(idx catalog.Index) bool { - return idx.IsSharded() && idx.GetShardColumnName() == col.Name + return idx.IsSharded() && idx.GetShardColumnName() == col.GetName() }) } diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 8572f2436497..94f2265577f8 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -1020,7 +1020,7 @@ func (desc *wrapper) ensureShardedIndexNotComputed(index *descpb.IndexDescriptor // indicates how many index columns to skip over. func (desc *wrapper) validatePartitioningDescriptor( a *rowenc.DatumAlloc, - idxDesc *descpb.IndexDescriptor, + idx catalog.Index, partDesc *descpb.PartitioningDescriptor, colOffset int, partitionNames map[string]string, @@ -1041,9 +1041,9 @@ func (desc *wrapper) validatePartitioningDescriptor( // InterleavedBy is fine, so using the root of the interleave hierarchy will // work. It is expected that this is sufficient for real-world use cases. // Revisit this restriction if that expectation is wrong. - if len(idxDesc.Interleave.Ancestors) > 0 { + if idx.NumInterleaveAncestors() > 0 { return errors.Errorf("cannot set a zone config for interleaved index %s; "+ - "set it on the root of the interleaved hierarchy instead", idxDesc.Name) + "set it on the root of the interleaved hierarchy instead", idx.GetName()) } // We don't need real prefixes in the DecodePartitionTuple calls because we're @@ -1071,10 +1071,10 @@ func (desc *wrapper) validatePartitioningDescriptor( // The partitioning descriptor may be invalid and refer to columns // not stored in the index. In that case, skip this check as the // validation will fail later. - if i >= len(idxDesc.ColumnIDs) { + if i >= idx.NumColumns() { continue } - col, err := desc.FindColumnWithID(idxDesc.ColumnIDs[i]) + col, err := desc.FindColumnWithID(idx.GetColumnID(i)) if err != nil { return err } @@ -1089,12 +1089,12 @@ func (desc *wrapper) validatePartitioningDescriptor( return fmt.Errorf("PARTITION name must be non-empty") } if indexName, exists := partitionNames[name]; exists { - if indexName == idxDesc.Name { + if indexName == idx.GetName() { return fmt.Errorf("PARTITION %s: name must be unique (used twice in index %q)", name, indexName) } } - partitionNames[name] = idxDesc.Name + partitionNames[name] = idx.GetName() return nil } @@ -1117,7 +1117,7 @@ func (desc *wrapper) validatePartitioningDescriptor( // to match the behavior of the value when indexed. for _, valueEncBuf := range p.Values { tuple, keyPrefix, err := rowenc.DecodePartitionTuple( - a, codec, desc, idxDesc, partDesc, valueEncBuf, fakePrefixDatums) + a, codec, desc, idx, partDesc, valueEncBuf, fakePrefixDatums) if err != nil { return fmt.Errorf("PARTITION %s: %v", p.Name, err) } @@ -1129,7 +1129,7 @@ func (desc *wrapper) validatePartitioningDescriptor( newColOffset := colOffset + int(partDesc.NumColumns) if err := desc.validatePartitioningDescriptor( - a, idxDesc, &p.Subpartitioning, newColOffset, partitionNames, + a, idx, &p.Subpartitioning, newColOffset, partitionNames, ); err != nil { return err } @@ -1146,12 +1146,12 @@ func (desc *wrapper) validatePartitioningDescriptor( // NB: key encoding is used to check uniqueness because it has to match // the behavior of the value when indexed. fromDatums, fromKey, err := rowenc.DecodePartitionTuple( - a, codec, desc, idxDesc, partDesc, p.FromInclusive, fakePrefixDatums) + a, codec, desc, idx, partDesc, p.FromInclusive, fakePrefixDatums) if err != nil { return fmt.Errorf("PARTITION %s: %v", p.Name, err) } toDatums, toKey, err := rowenc.DecodePartitionTuple( - a, codec, desc, idxDesc, partDesc, p.ToExclusive, fakePrefixDatums) + a, codec, desc, idx, partDesc, p.ToExclusive, fakePrefixDatums) if err != nil { return fmt.Errorf("PARTITION %s: %v", p.Name, err) } @@ -1198,9 +1198,8 @@ func (desc *wrapper) validatePartitioning() error { a := &rowenc.DatumAlloc{} return catalog.ForEachNonDropIndex(desc, func(idx catalog.Index) error { - idxDesc := idx.IndexDesc() return desc.validatePartitioningDescriptor( - a, idxDesc, &idxDesc.Partitioning, 0 /* colOffset */, partitionNames, + a, idx, &idx.IndexDesc().Partitioning, 0 /* colOffset */, partitionNames, ) }) } diff --git a/pkg/sql/colencoding/key_encoding.go b/pkg/sql/colencoding/key_encoding.go index a069d7756f1e..51c2388e78ef 100644 --- a/pkg/sql/colencoding/key_encoding.go +++ b/pkg/sql/colencoding/key_encoding.go @@ -40,7 +40,7 @@ func DecodeIndexKeyToCols( vecs []coldata.Vec, idx int, desc catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, indexColIdx []int, types []*types.T, colDirs []descpb.IndexDescriptor_Direction, @@ -53,8 +53,9 @@ func DecodeIndexKeyToCols( origKey := key - if len(index.Interleave.Ancestors) > 0 { - for i, ancestor := range index.Interleave.Ancestors { + if index.NumInterleaveAncestors() > 0 { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + ancestor := index.GetInterleaveAncestor(i) // Our input key had its first table id / index id chopped off, so // don't try to decode those for the first ancestor. if i != 0 { @@ -101,11 +102,11 @@ func DecodeIndexKeyToCols( if err != nil { return nil, false, false, err } - if decodedTableID != desc.GetID() || decodedIndexID != index.ID { + if decodedTableID != desc.GetID() || decodedIndexID != index.GetID() { // We don't match. Return a key with the table ID / index ID we're // searching for, so the caller knows what to seek to. curPos := len(origKey) - len(key) - key = rowenc.EncodePartialTableIDIndexID(origKey[:curPos], desc.GetID(), index.ID) + key = rowenc.EncodePartialTableIDIndexID(origKey[:curPos], desc.GetID(), index.GetID()) return key, false, false, nil } } diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index 16d79dd9acef..9d6089308b12 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -50,13 +50,13 @@ type cTableInfo struct { // want to scan. spans roachpb.Spans desc catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index isSecondaryIndex bool indexColumnDirs []descpb.IndexDescriptor_Direction // The table columns to use for fetching, possibly including ones currently in // schema changes. - cols []descpb.ColumnDescriptor + cols []catalog.Column // The ordered list of ColumnIDs that are required. neededColsList []int @@ -392,7 +392,7 @@ func (rf *cFetcher) Init( colDescriptors := tableArgs.Cols for i := range colDescriptors { //gcassert:bce - id := colDescriptors[i].ID + id := colDescriptors[i].GetID() table.colIdxMap.vals = append(table.colIdxMap.vals, id) table.colIdxMap.ords = append(table.colIdxMap.ords, tableArgs.ColIdxMap.GetDefault(id)) } @@ -422,7 +422,7 @@ func (rf *cFetcher) Init( _ = typs[len(colDescriptors)-1] for i := range colDescriptors { //gcassert:bce - typs[i] = colDescriptors[i].Type + typs[i] = colDescriptors[i].GetType() } var err error @@ -435,7 +435,7 @@ func (rf *cFetcher) Init( } for i := range colDescriptors { //gcassert:bce - col := colDescriptors[i].ID + col := colDescriptors[i].GetID() idx := tableArgs.ColIdxMap.GetDefault(col) if tableArgs.ValNeededForCol.Contains(idx) { // The idx-th column is required. @@ -453,13 +453,14 @@ func (rf *cFetcher) Init( } sort.Ints(table.neededColsList) - table.knownPrefixLength = len(rowenc.MakeIndexKeyPrefix(codec, table.desc, table.index.ID)) + table.knownPrefixLength = len(rowenc.MakeIndexKeyPrefix(codec, table.desc, table.index.GetID())) var indexColumnIDs []descpb.ColumnID - indexColumnIDs, table.indexColumnDirs = table.index.FullColumnIDs() + indexColumnIDs, table.indexColumnDirs = catalog.FullIndexColumnIDs(table.index) compositeColumnIDs := util.MakeFastIntSet() - for _, id := range table.index.CompositeColumnIDs { + for i := 0; i < table.index.NumCompositeColumns(); i++ { + id := table.index.GetCompositeColumnID(i) compositeColumnIDs.Add(int(id)) } @@ -516,7 +517,7 @@ func (rf *cFetcher) Init( } } table.invertedColOrdinal = -1 - if table.index.Type == descpb.IndexDescriptor_INVERTED { + if table.index.GetType() == descpb.IndexDescriptor_INVERTED { id := table.index.InvertedColumnID() colIdx, ok := tableArgs.ColIdxMap.Get(id) if ok && neededCols.Contains(int(id)) { @@ -528,8 +529,9 @@ func (rf *cFetcher) Init( // Unique secondary indexes contain the extra column IDs as part of // the value component. We process these separately, so we need to know // what extra columns are composite or not. - if table.isSecondaryIndex && table.index.Unique { - for _, id := range table.index.ExtraColumnIDs { + if table.isSecondaryIndex && table.index.IsUnique() { + for i := 0; i < table.index.NumExtraColumns(); i++ { + id := table.index.GetExtraColumnID(i) colIdx, ok := tableArgs.ColIdxMap.Get(id) if ok && neededCols.Contains(int(id)) { if compositeColumnIDs.Contains(int(id)) { @@ -546,16 +548,16 @@ func (rf *cFetcher) Init( // - If there are needed columns from the index key, we need to read it. // // Otherwise, we can completely avoid decoding the index key. - if neededIndexCols > 0 || len(table.index.InterleavedBy) > 0 || len(table.index.Interleave.Ancestors) > 0 { + if neededIndexCols > 0 || table.index.NumInterleavedBy() > 0 || table.index.NumInterleaveAncestors() > 0 { rf.mustDecodeIndexKey = true } if table.isSecondaryIndex { for i := range colDescriptors { //gcassert:bce - id := colDescriptors[i].ID + id := colDescriptors[i].GetID() if neededCols.Contains(int(id)) && !table.index.ContainsColumnID(id) { - return errors.Errorf("requested column %s not in index", colDescriptors[i].Name) + return errors.Errorf("requested column %s not in index", colDescriptors[i].GetName()) } } } @@ -564,17 +566,16 @@ func (rf *cFetcher) Init( table.keyValTypes = colinfo.GetColumnTypesFromColDescs( colDescriptors, indexColumnIDs, table.keyValTypes, ) - if len(table.index.ExtraColumnIDs) > 0 { + if table.index.NumExtraColumns() > 0 { // Unique secondary indexes have a value that is the // primary index key. // Primary indexes only contain ascendingly-encoded // values. If this ever changes, we'll probably have to // figure out the directions here too. - extraColumnIDs := table.index.ExtraColumnIDs table.extraTypes = colinfo.GetColumnTypesFromColDescs( - colDescriptors, extraColumnIDs, table.extraTypes, + colDescriptors, table.index.IndexDesc().ExtraColumnIDs, table.extraTypes, ) - nExtraColumns := len(extraColumnIDs) + nExtraColumns := table.index.NumExtraColumns() if cap(table.extraValColOrdinals) >= nExtraColumns { table.extraValColOrdinals = table.extraValColOrdinals[:nExtraColumns] } else { @@ -588,10 +589,11 @@ func (rf *cFetcher) Init( } extraValColOrdinals := table.extraValColOrdinals - _ = extraValColOrdinals[len(extraColumnIDs)-1] + _ = extraValColOrdinals[nExtraColumns-1] allExtraValColOrdinals := table.allExtraValColOrdinals - _ = allExtraValColOrdinals[len(extraColumnIDs)-1] - for i, id := range extraColumnIDs { + _ = allExtraValColOrdinals[nExtraColumns-1] + for i := 0; i < nExtraColumns; i++ { + id := table.index.GetExtraColumnID(i) idx := tableArgs.ColIdxMap.GetDefault(id) //gcassert:bce allExtraValColOrdinals[i] = idx @@ -607,7 +609,7 @@ func (rf *cFetcher) Init( // Keep track of the maximum keys per row to accommodate a // limitHint when StartScan is invoked. - keysPerRow, err := table.desc.KeysPerRow(table.index.ID) + keysPerRow, err := table.desc.KeysPerRow(table.index.GetID()) if err != nil { return err } @@ -912,7 +914,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { // them when processing the index. The difference with unique secondary indexes // is that the extra columns are not always there, and are used to unique-ify // the index key, rather than provide the primary key column values. - if foundNull && rf.table.isSecondaryIndex && rf.table.index.Unique && rf.table.desc.NumFamilies() != 1 { + if foundNull && rf.table.isSecondaryIndex && rf.table.index.IsUnique() && rf.table.desc.NumFamilies() != 1 { // We get the remaining bytes after the computed prefix, and then // slice off the extra encoded columns from those bytes. We calculate // how many bytes were sliced away, and then extend lastRowPrefix @@ -920,7 +922,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { prefixLen := len(rf.machine.lastRowPrefix) remainingBytes := rf.machine.nextKV.Key[prefixLen:] origRemainingBytesLen := len(remainingBytes) - for range rf.table.index.ExtraColumnIDs { + for i := 0; i < rf.table.index.NumExtraColumns(); i++ { var err error // Slice off an extra encoded column from remainingBytes. remainingBytes, err = rowenc.SkipTableKey(remainingBytes) @@ -1143,7 +1145,7 @@ func (rf *cFetcher) processValue( buf.WriteByte('/') buf.WriteString(rf.table.desc.GetName()) buf.WriteByte('/') - buf.WriteString(rf.table.index.Name) + buf.WriteString(rf.table.index.GetName()) for _, idx := range rf.table.allIndexColOrdinals { buf.WriteByte('/') if idx != -1 { @@ -1164,7 +1166,7 @@ func (rf *cFetcher) processValue( } val := rf.machine.nextKV.Value - if !table.isSecondaryIndex || table.index.EncodingType == descpb.PrimaryIndexEncoding { + if !table.isSecondaryIndex || table.index.GetEncodingType() == descpb.PrimaryIndexEncoding { // If familyID is 0, kv.Value contains values for composite key columns. // These columns already have a table.row value assigned above, but that value // (obtained from the key encoding) might not be correct (e.g. for decimals, @@ -1216,7 +1218,7 @@ func (rf *cFetcher) processValue( return "", "", scrub.WrapError(scrub.IndexValueDecodingError, err) } - if table.isSecondaryIndex && table.index.Unique { + if table.isSecondaryIndex && table.index.IsUnique() { // This is a unique secondary index; decode the extra // column values from the value. var err error @@ -1314,7 +1316,7 @@ func (rf *cFetcher) processValueSingle( if len(val.RawBytes) == 0 { return prettyKey, "", nil } - typ := table.cols[idx].Type + typ := table.cols[idx].GetType() err := colencoding.UnmarshalColumnValueToCol( &table.da, rf.machine.colvecs[idx], rf.machine.rowIdx, typ, val, ) @@ -1414,7 +1416,7 @@ func (rf *cFetcher) processValueBytes( vec := rf.machine.colvecs[idx] - valTyp := table.cols[idx].Type + valTyp := table.cols[idx].GetType() valueBytes, err = colencoding.DecodeTableValueToCol( &table.da, vec, rf.machine.rowIdx, typ, dataOffset, valTyp, valueBytes, ) @@ -1454,7 +1456,7 @@ func (rf *cFetcher) fillNulls() error { if table.compositeIndexColOrdinals.Contains(i) { continue } - if !table.cols[i].Nullable { + if !table.cols[i].IsNullable() { var indexColValues []string for _, idx := range table.indexColOrdinals { if idx != -1 { @@ -1464,8 +1466,8 @@ func (rf *cFetcher) fillNulls() error { } return scrub.WrapError(scrub.UnexpectedNullValueError, errors.Errorf( "non-nullable column \"%s:%s\" with no value! Index scanned was %q with the index key columns (%s) and the values (%s)", - table.desc.GetName(), table.cols[i].Name, table.index.Name, - strings.Join(table.index.ColumnNames, ","), strings.Join(indexColValues, ","))) + table.desc.GetName(), table.cols[i].GetName(), table.index.GetName(), + strings.Join(table.index.IndexDesc().ColumnNames, ","), strings.Join(indexColValues, ","))) } } rf.machine.colvecs[i].Nulls().SetNull(rf.machine.rowIdx) @@ -1509,7 +1511,7 @@ func (rf *cFetcher) KeyToDesc(key roachpb.Key) (catalog.TableDescriptor, bool) { if len(key) < rf.table.knownPrefixLength { return nil, false } - nIndexCols := len(rf.table.index.ColumnIDs) + len(rf.table.index.ExtraColumnIDs) + nIndexCols := rf.table.index.NumColumns() + rf.table.index.NumExtraColumns() tableKeyVals := make([]rowenc.EncDatum, nIndexCols) _, ok, _, err := rowenc.DecodeIndexKeyWithoutTableIDIndexIDPrefix( rf.table.desc, diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index cffeed53fe26..e39fbdbf4553 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" @@ -303,14 +302,13 @@ func initCRowFetcher( valNeededForCol util.FastIntSet, spec *execinfrapb.TableReaderSpec, withSystemColumns bool, -) (index *descpb.IndexDescriptor, isSecondaryIndex bool, err error) { +) (index catalog.Index, isSecondaryIndex bool, err error) { indexIdx := int(spec.IndexIdx) if indexIdx >= len(desc.ActiveIndexes()) { return nil, false, errors.Errorf("invalid indexIdx %d", indexIdx) } - indexI := desc.ActiveIndexes()[indexIdx] - index = indexI.IndexDesc() - isSecondaryIndex = !indexI.Primary() + index = desc.ActiveIndexes()[indexIdx] + isSecondaryIndex = !index.Primary() tableArgs := row.FetcherTableArgs{ Desc: desc, diff --git a/pkg/sql/comment_on_index.go b/pkg/sql/comment_on_index.go index ae1cbe76d19d..82d5521d5373 100644 --- a/pkg/sql/comment_on_index.go +++ b/pkg/sql/comment_on_index.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/privilege" @@ -26,7 +27,7 @@ import ( type commentOnIndexNode struct { n *tree.CommentOnIndex tableDesc *tabledesc.Mutable - indexDesc *descpb.IndexDescriptor + index catalog.Index } // CommentOnIndex adds a comment on an index. @@ -40,12 +41,12 @@ func (p *planner) CommentOnIndex(ctx context.Context, n *tree.CommentOnIndex) (p return nil, err } - tableDesc, indexDesc, err := p.getTableAndIndex(ctx, &n.Index, privilege.CREATE) + tableDesc, index, err := p.getTableAndIndex(ctx, &n.Index, privilege.CREATE) if err != nil { return nil, err } - return &commentOnIndexNode{n: n, tableDesc: tableDesc, indexDesc: indexDesc}, nil + return &commentOnIndexNode{n: n, tableDesc: tableDesc, index: index}, nil } func (n *commentOnIndexNode) startExec(params runParams) error { @@ -53,13 +54,13 @@ func (n *commentOnIndexNode) startExec(params runParams) error { err := params.p.upsertIndexComment( params.ctx, n.tableDesc.ID, - n.indexDesc.ID, + n.index.GetID(), *n.n.Comment) if err != nil { return err } } else { - err := params.p.removeIndexComment(params.ctx, n.tableDesc.ID, n.indexDesc.ID) + err := params.p.removeIndexComment(params.ctx, n.tableDesc.ID, n.index.GetID()) if err != nil { return err } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 70478822f419..04c0445a9ca2 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2174,7 +2174,7 @@ func showAlterStatementWithInterleave( sharedPrefixLen += int(idx.GetInterleaveAncestor(i).SharedPrefixLen) } // Write the CREATE INDEX statements. - if err := showCreateIndexWithInterleave(ctx, f, table, idx.IndexDesc(), tableName, parentName, sharedPrefixLen, semaCtx); err != nil { + if err := showCreateIndexWithInterleave(ctx, f, table, idx, tableName, parentName, sharedPrefixLen, semaCtx); err != nil { return err } if err := alterStmts.Append(tree.NewDString(f.CloseAndGetString())); err != nil { @@ -2189,7 +2189,7 @@ func showCreateIndexWithInterleave( ctx context.Context, f *tree.FmtCtx, table catalog.TableDescriptor, - idx *descpb.IndexDescriptor, + idx catalog.Index, tableName tree.TableName, parentName tree.TableName, sharedPrefixLen int, @@ -2208,7 +2208,8 @@ func showCreateIndexWithInterleave( f.WriteString(" (") // Get all of the columns and write them. comma := "" - for _, name := range idx.ColumnNames[:sharedPrefixLen] { + for i := 0; i < sharedPrefixLen; i++ { + name := idx.GetColumnName(i) f.WriteString(comma) f.FormatNameP(&name) comma = ", " @@ -2361,32 +2362,31 @@ CREATE TABLE crdb_internal.index_columns ( parentName := parent.GetName() tableName := tree.NewDString(table.GetName()) - reportIndex := func(idxI catalog.Index) error { - idx := idxI.IndexDesc() - idxID := tree.NewDInt(tree.DInt(idx.ID)) - idxName := tree.NewDString(idx.Name) + reportIndex := func(idx catalog.Index) error { + idxID := tree.NewDInt(tree.DInt(idx.GetID())) + idxName := tree.NewDString(idx.GetName()) // Report the main (key) columns. - for i, c := range idx.ColumnIDs { + for i, c := range idx.IndexDesc().ColumnIDs { colName := tree.DNull colDir := tree.DNull - if i >= len(idx.ColumnNames) { + if i >= len(idx.IndexDesc().ColumnNames) { // We log an error here, instead of reporting an error // to the user, because we really want to see the // erroneous data in the virtual table. log.Errorf(ctx, "index descriptor for [%d@%d] (%s.%s@%s) has more key column IDs (%d) than names (%d) (corrupted schema?)", - table.GetID(), idx.ID, parentName, table.GetName(), idx.Name, - len(idx.ColumnIDs), len(idx.ColumnNames)) + table.GetID(), idx.GetID(), parentName, table.GetName(), idx.GetName(), + len(idx.IndexDesc().ColumnIDs), len(idx.IndexDesc().ColumnNames)) } else { - colName = tree.NewDString(idx.ColumnNames[i]) + colName = tree.NewDString(idx.GetColumnName(i)) } - if i >= len(idx.ColumnDirections) { + if i >= len(idx.IndexDesc().ColumnDirections) { // See comment above. log.Errorf(ctx, "index descriptor for [%d@%d] (%s.%s@%s) has more key column IDs (%d) than directions (%d) (corrupted schema?)", - table.GetID(), idx.ID, parentName, table.GetName(), idx.Name, - len(idx.ColumnIDs), len(idx.ColumnDirections)) + table.GetID(), idx.GetID(), parentName, table.GetName(), idx.GetName(), + len(idx.IndexDesc().ColumnIDs), len(idx.IndexDesc().ColumnDirections)) } else { - colDir = idxDirMap[idx.ColumnDirections[i]] + colDir = idxDirMap[idx.GetColumnDirection(i)] } if err := addRow( @@ -2401,7 +2401,7 @@ CREATE TABLE crdb_internal.index_columns ( notImplicit := tree.DBoolFalse // Report the stored columns. - for _, c := range idx.StoreColumnIDs { + for _, c := range idx.IndexDesc().StoreColumnIDs { if err := addRow( tableID, tableName, idxID, idxName, storing, tree.NewDInt(tree.DInt(c)), tree.DNull, tree.DNull, @@ -2412,7 +2412,7 @@ CREATE TABLE crdb_internal.index_columns ( } // Report the extra columns. - for _, c := range idx.ExtraColumnIDs { + for _, c := range idx.IndexDesc().ExtraColumnIDs { if err := addRow( tableID, tableName, idxID, idxName, extra, tree.NewDInt(tree.DInt(c)), tree.DNull, tree.DNull, @@ -2423,7 +2423,7 @@ CREATE TABLE crdb_internal.index_columns ( } // Report the composite columns - for _, c := range idx.CompositeColumnIDs { + for _, c := range idx.IndexDesc().CompositeColumnIDs { if err := addRow( tableID, tableName, idxID, idxName, composite, tree.NewDInt(tree.DInt(c)), tree.DNull, tree.DNull, @@ -3570,7 +3570,7 @@ func addPartitioningRows( p *planner, database string, table catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, partitioning *descpb.PartitioningDescriptor, parentName tree.Datum, colOffset int, @@ -3585,7 +3585,7 @@ func addPartitioningRows( } tableID := tree.NewDInt(tree.DInt(table.GetID())) - indexID := tree.NewDInt(tree.DInt(index.ID)) + indexID := tree.NewDInt(tree.DInt(index.GetID())) numColumns := tree.NewDInt(tree.DInt(partitioning.NumColumns)) var buf bytes.Buffer @@ -3593,7 +3593,7 @@ func addPartitioningRows( if i != uint32(colOffset) { buf.WriteString(`, `) } - buf.WriteString(index.ColumnNames[i]) + buf.WriteString(index.GetColumnName(int(i))) } colNames := tree.NewDString(buf.String()) @@ -3746,7 +3746,7 @@ CREATE TABLE crdb_internal.partitions ( return catalog.ForEachIndex(table, catalog.IndexOpts{ AddMutations: true, }, func(index catalog.Index) error { - return addPartitioningRows(ctx, p, dbName, table, index.IndexDesc(), &index.IndexDesc().Partitioning, + return addPartitioningRows(ctx, p, dbName, table, index, &index.IndexDesc().Partitioning, tree.DNull /* parentName */, 0 /* colOffset */, pusher.pushRow) }) }) diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index 39ee222f710c..89afea8df562 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/geo/geoindex" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" @@ -89,7 +90,7 @@ func (p *planner) CreateIndex(ctx context.Context, n *tree.CreateIndex) (planNod func (p *planner) setupFamilyAndConstraintForShard( ctx context.Context, tableDesc *tabledesc.Mutable, - shardCol *descpb.ColumnDescriptor, + shardCol catalog.Column, idxColumns []string, buckets int32, ) error { @@ -99,7 +100,7 @@ func (p *planner) setupFamilyAndConstraintForShard( } // Assign shard column to the family of the first column in its index set, and do it // before `AllocateIDs()` assigns it to the primary column family. - if err := tableDesc.AddColumnToFamilyMaybeCreate(shardCol.Name, family, false, false); err != nil { + if err := tableDesc.AddColumnToFamilyMaybeCreate(shardCol.GetName(), family, false, false); err != nil { return err } // Assign an ID to the newly-added shard column, which is needed for the creation @@ -108,7 +109,7 @@ func (p *planner) setupFamilyAndConstraintForShard( return err } - ckDef, err := makeShardCheckConstraintDef(tableDesc, int(buckets), shardCol) + ckDef, err := makeShardCheckConstraintDef(int(buckets), shardCol) if err != nil { return err } @@ -319,7 +320,7 @@ func setupShardedIndex( tableDesc *tabledesc.Mutable, indexDesc *descpb.IndexDescriptor, isNewTable bool, -) (shard *descpb.ColumnDescriptor, newColumns tree.IndexElemList, newColumn bool, err error) { +) (shard catalog.Column, newColumns tree.IndexElemList, newColumn bool, err error) { if !shardedIndexEnabled { return nil, nil, false, hashShardedIndexesDisabledError } @@ -338,13 +339,13 @@ func setupShardedIndex( return nil, nil, false, err } shardIdxElem := tree.IndexElem{ - Column: tree.Name(shardCol.Name), + Column: tree.Name(shardCol.GetName()), Direction: tree.Ascending, } newColumns = append(tree.IndexElemList{shardIdxElem}, columns...) indexDesc.Sharded = descpb.ShardedDescriptor{ IsSharded: true, - Name: shardCol.Name, + Name: shardCol.GetName(), ShardBuckets: buckets, ColumnNames: colNames, } @@ -356,12 +357,12 @@ func setupShardedIndex( // buckets. func maybeCreateAndAddShardCol( shardBuckets int, desc *tabledesc.Mutable, colNames []string, isNewTable bool, -) (col *descpb.ColumnDescriptor, created bool, err error) { - shardCol, err := makeShardColumnDesc(colNames, shardBuckets) +) (col catalog.Column, created bool, err error) { + shardColDesc, err := makeShardColumnDesc(colNames, shardBuckets) if err != nil { return nil, false, err } - existingShardCol, err := desc.FindColumnWithName(tree.Name(shardCol.Name)) + existingShardCol, err := desc.FindColumnWithName(tree.Name(shardColDesc.Name)) if err == nil && !existingShardCol.Dropped() { // TODO(ajwerner): In what ways is existingShardCol allowed to differ from // the newly made shardCol? Should there be some validation of @@ -370,9 +371,9 @@ func maybeCreateAndAddShardCol( // The user managed to reverse-engineer our crazy shard column name, so // we'll return an error here rather than try to be tricky. return nil, false, pgerror.Newf(pgcode.DuplicateColumn, - "column %s already specified; can't be used for sharding", shardCol.Name) + "column %s already specified; can't be used for sharding", shardColDesc.Name) } - return existingShardCol.ColumnDesc(), false, nil + return existingShardCol, false, nil } columnIsUndefined := sqlerrors.IsUndefinedColumnError(err) if err != nil && !columnIsUndefined { @@ -380,13 +381,14 @@ func maybeCreateAndAddShardCol( } if columnIsUndefined || existingShardCol.Dropped() { if isNewTable { - desc.AddColumn(shardCol) + desc.AddColumn(shardColDesc) } else { - desc.AddColumnMutation(shardCol, descpb.DescriptorMutation_ADD) + desc.AddColumnMutation(shardColDesc, descpb.DescriptorMutation_ADD) } created = true } - return shardCol, created, nil + shardCol, err := desc.FindColumnWithName(tree.Name(shardColDesc.Name)) + return shardCol, created, err } var interleavedTableDeprecationError = errors.WithIssueLink( diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 5a16a92a33c6..a5818ca3fc61 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -474,7 +474,7 @@ func (n *createTableNode) startExec(params runParams) error { params.p.txn, params.ExecCfg().Codec, desc.ImmutableCopy().(catalog.TableDescriptor), - desc.Columns, + desc.PublicColumns(), params.p.alloc) if err != nil { return err @@ -1740,7 +1740,7 @@ func NewTableDesc( if err != nil { return nil, err } - checkConstraint, err := makeShardCheckConstraintDef(&desc, int(buckets), shardCol) + checkConstraint, err := makeShardCheckConstraintDef(int(buckets), shardCol) if err != nil { return nil, err } @@ -1825,7 +1825,7 @@ func NewTableDesc( // Now that we've constructed our columns, we pop into any of our computed // columns so that we can dequalify any column references. sourceInfo := colinfo.NewSourceInfoForSingleTable( - n.Table, colinfo.ResultColumnsFromColDescs(desc.GetID(), desc.Columns), + n.Table, colinfo.ResultColumnsFromColumns(desc.GetID(), desc.PublicColumns()), ) for i := range desc.Columns { @@ -1868,7 +1868,7 @@ func NewTableDesc( if err != nil { return nil, err } - checkConstraint, err := makeShardCheckConstraintDef(&desc, int(buckets), shardCol) + checkConstraint, err := makeShardCheckConstraintDef(int(buckets), shardCol) if err != nil { return nil, err } @@ -2715,7 +2715,7 @@ func makeHashShardComputeExpr(colNames []string, buckets int) *string { } func makeShardCheckConstraintDef( - desc *tabledesc.Mutable, buckets int, shardCol *descpb.ColumnDescriptor, + buckets int, shardCol catalog.Column, ) (*tree.CheckConstraintTableDef, error) { values := &tree.Tuple{} for i := 0; i < buckets; i++ { @@ -2729,7 +2729,7 @@ func makeShardCheckConstraintDef( Expr: &tree.ComparisonExpr{ Operator: tree.In, Left: &tree.ColumnItem{ - ColumnName: tree.Name(shardCol.Name), + ColumnName: tree.Name(shardCol.GetName()), }, Right: values, }, diff --git a/pkg/sql/database_region_change_finalizer.go b/pkg/sql/database_region_change_finalizer.go index 4c5bddf69361..6b7a97aaea1d 100644 --- a/pkg/sql/database_region_change_finalizer.go +++ b/pkg/sql/database_region_change_finalizer.go @@ -174,7 +174,7 @@ func (r *databaseRegionChangeFinalizer) repartitionRegionalByRowTables( ctx, txn, tableDesc, - index.IndexDesc(), + index.GetID(), &oldPartitioning, &index.IndexDesc().Partitioning, execCfg, diff --git a/pkg/sql/delete_range.go b/pkg/sql/delete_range.go index 1ff0a3816620..3c31d6023a1c 100644 --- a/pkg/sql/delete_range.go +++ b/pkg/sql/delete_range.go @@ -96,13 +96,13 @@ func (d *deleteRangeNode) startExec(params runParams) error { allTables := make([]row.FetcherTableArgs, len(d.interleavedDesc)+1) allTables[0] = row.FetcherTableArgs{ Desc: d.desc, - Index: d.desc.GetPrimaryIndex().IndexDesc(), + Index: d.desc.GetPrimaryIndex(), Spans: d.spans, } for i, interleaved := range d.interleavedDesc { allTables[i+1] = row.FetcherTableArgs{ Desc: interleaved, - Index: interleaved.GetPrimaryIndex().IndexDesc(), + Index: interleaved.GetPrimaryIndex(), Spans: d.spans, } } diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 30ae83a8de79..8813273ca4fc 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1025,10 +1025,9 @@ func (dsp *DistSQLPlanner) nodeVersionIsCompatible(nodeID roachpb.NodeID) bool { return distsql.FlowVerIsCompatible(dsp.planVersion, v.MinAcceptedVersion, v.Version) } -func getIndexIdx(index *descpb.IndexDescriptor, desc catalog.TableDescriptor) (uint32, error) { - foundIndex, _ := desc.FindIndexWithID(index.ID) - if foundIndex != nil && foundIndex.Public() { - return uint32(foundIndex.Ordinal()), nil +func getIndexIdx(index catalog.Index, desc catalog.TableDescriptor) (uint32, error) { + if index.Public() { + return uint32(index.Ordinal()), nil } return 0, errors.Errorf("invalid index %v (table %s)", index, desc.GetName()) } @@ -2047,7 +2046,7 @@ func (dsp *DistSQLPlanner) createPlanForIndexJoin( plan.PlanToStreamColMap = identityMap(plan.PlanToStreamColMap, len(n.cols)) for i := range n.cols { - ord := tableOrdinal(n.table.desc, n.cols[i].ID, n.table.colCfg.visibility) + ord := tableOrdinal(n.table.desc, n.cols[i].GetID(), n.table.colCfg.visibility) post.OutputColumns[i] = uint32(ord) } diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index cc227e87d866..fc6b2c02b047 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -184,10 +184,10 @@ func (e *distSQLSpecExecFactory) ConstructScan( // 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 + idx := index.(*optIndex).idx colCfg := makeScanColumnsConfig(table, params.NeededCols) - sb := span.MakeBuilder(e.planner.EvalContext(), e.planner.ExecCfg().Codec, tabDesc, indexDesc) + sb := span.MakeBuilder(e.planner.EvalContext(), e.planner.ExecCfg().Codec, tabDesc, idx) // Note that initColsForScan and setting ResultColumns below are equivalent // to what scan.initTable call does in execFactory.ConstructScan. @@ -215,7 +215,7 @@ func (e *distSQLSpecExecFactory) ConstructScan( } isFullTableOrIndexScan := len(spans) == 1 && spans[0].EqualValue( - tabDesc.IndexSpan(e.planner.ExecCfg().Codec, indexDesc.ID), + tabDesc.IndexSpan(e.planner.ExecCfg().Codec, idx.GetID()), ) if err = colCfg.assertValidReqOrdering(reqOrdering); err != nil { return nil, err @@ -244,7 +244,7 @@ func (e *distSQLSpecExecFactory) ConstructScan( trSpec.VirtualColumn = vc.ColumnDesc() } - trSpec.IndexIdx, err = getIndexIdx(indexDesc, tabDesc) + trSpec.IndexIdx, err = getIndexIdx(idx, tabDesc) if err != nil { return nil, err } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index a19a9d385674..5cadfda76506 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -144,11 +144,11 @@ func (n *dropIndexNode) startExec(params runParams) error { // dropShardColumnAndConstraint drops the given shard column and its associated check // constraint. func (n *dropIndexNode) dropShardColumnAndConstraint( - params runParams, tableDesc *tabledesc.Mutable, shardColDesc *descpb.ColumnDescriptor, + params runParams, tableDesc *tabledesc.Mutable, shardCol catalog.Column, ) error { validChecks := tableDesc.Checks[:0] for _, check := range tableDesc.AllActiveAndInactiveChecks() { - if used, err := tableDesc.CheckConstraintUsesColumn(check, shardColDesc.ID); err != nil { + if used, err := tableDesc.CheckConstraintUsesColumn(check, shardCol.GetID()); err != nil { return err } else if used { if check.Validity == descpb.ConstraintValidity_Validating { @@ -164,9 +164,9 @@ func (n *dropIndexNode) dropShardColumnAndConstraint( tableDesc.Checks = validChecks } - tableDesc.AddColumnMutation(shardColDesc, descpb.DescriptorMutation_DROP) + tableDesc.AddColumnMutation(shardCol.ColumnDesc(), descpb.DescriptorMutation_DROP) for i := range tableDesc.Columns { - if tableDesc.Columns[i].ID == shardColDesc.ID { + if tableDesc.Columns[i].ID == shardCol.GetID() { // Note the third slice parameter which will force a copy of the backing // array if the column being removed is not the last column. tableDesc.Columns = append(tableDesc.Columns[:i:i], @@ -206,7 +206,7 @@ func (n *dropIndexNode) maybeDropShardColumn( }) != nil { return nil } - return n.dropShardColumnAndConstraint(params, tableDesc, shardColDesc.ColumnDesc()) + return n.dropShardColumnAndConstraint(params, tableDesc, shardColDesc) } func (*dropIndexNode) Next(runParams) (bool, error) { return false, nil } @@ -240,7 +240,7 @@ func (p *planner) dropIndexByName( constraintBehavior dropIndexConstraintBehavior, jobDesc string, ) error { - idxI, err := tableDesc.FindIndexWithName(string(idxName)) + idx, err := tableDesc.FindIndexWithName(string(idxName)) if err != nil { // Only index names of the form "table@idx" throw an error here if they // don't exist. @@ -251,15 +251,14 @@ func (p *planner) dropIndexByName( // Index does not exist, but we want it to: error out. return pgerror.WithCandidateCode(err, pgcode.UndefinedObject) } - if idxI.Dropped() { + if idx.Dropped() { return nil } - idx := idxI.IndexDesc() - if idx.Unique && behavior != tree.DropCascade && constraintBehavior != ignoreIdxConstraint && !idx.CreatedExplicitly { + if idx.IsUnique() && behavior != tree.DropCascade && constraintBehavior != ignoreIdxConstraint && !idx.IsCreatedExplicitly() { return errors.WithHint( pgerror.Newf(pgcode.DependentObjectsStillExist, - "index %q is in use as unique constraint", idx.Name), + "index %q is in use as unique constraint", idx.GetName()), "use CASCADE if you really want to drop it.", ) } @@ -268,13 +267,13 @@ func (p *planner) dropIndexByName( // necessary for the system tenant, because secondary tenants do not have // zone configs for individual objects. if p.ExecCfg().Codec.ForSystemTenant() { - _, zone, _, err := GetZoneConfigInTxn(ctx, p.txn, config.SystemTenantObjectID(tableDesc.ID), nil, "", false) + _, zone, _, err := GetZoneConfigInTxn(ctx, p.txn, config.SystemTenantObjectID(tableDesc.ID), nil /* index */, "", false) if err != nil { return err } for _, s := range zone.Subzones { - if s.IndexID != uint32(idx.ID) { + if s.IndexID != uint32(idx.GetID()) { _, err = GenerateSubzoneSpans( p.ExecCfg().Settings, p.ExecCfg().ClusterID(), @@ -311,17 +310,17 @@ func (p *planner) dropIndexByName( // Construct a list of all the remaining indexes, so that we can see if there // is another index that could replace the one we are deleting for a given // foreign key constraint. - remainingIndexes := make([]*descpb.IndexDescriptor, 1, len(tableDesc.ActiveIndexes())) - remainingIndexes[0] = tableDesc.GetPrimaryIndex().IndexDesc() + remainingIndexes := make([]catalog.Index, 1, len(tableDesc.ActiveIndexes())) + remainingIndexes[0] = tableDesc.GetPrimaryIndex() for _, index := range tableDesc.PublicNonPrimaryIndexes() { - if index.GetID() != idx.ID { - remainingIndexes = append(remainingIndexes, index.IndexDesc()) + if index.GetID() != idx.GetID() { + remainingIndexes = append(remainingIndexes, index) } } // indexHasReplacementCandidate runs isValidIndex on each index in remainingIndexes and returns // true if at least one index satisfies isValidIndex. - indexHasReplacementCandidate := func(isValidIndex func(*descpb.IndexDescriptor) bool) bool { + indexHasReplacementCandidate := func(isValidIndex func(index catalog.Index) bool) bool { foundReplacement := false for _, index := range remainingIndexes { if isValidIndex(index) { @@ -340,7 +339,7 @@ func (p *planner) dropIndexByName( // foreign key mutation, then make sure that we have another index that // could be used for this mutation. idx.IsValidOriginIndex(c.ForeignKey.OriginColumnIDs) && - !indexHasReplacementCandidate(func(idx *descpb.IndexDescriptor) bool { + !indexHasReplacementCandidate(func(idx catalog.Index) bool { return idx.IsValidOriginIndex(c.ForeignKey.OriginColumnIDs) }) { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, @@ -361,13 +360,13 @@ func (p *planner) dropIndexByName( tableDesc.OutboundFKs[sliceIdx] = tableDesc.OutboundFKs[i] sliceIdx++ fk := &tableDesc.OutboundFKs[i] - canReplace := func(idx *descpb.IndexDescriptor) bool { + canReplace := func(idx catalog.Index) bool { return idx.IsValidOriginIndex(fk.OriginColumnIDs) } // The index being deleted could be used as the origin index for this foreign key. if idx.IsValidOriginIndex(fk.OriginColumnIDs) && !indexHasReplacementCandidate(canReplace) { if behavior != tree.DropCascade && constraintBehavior != ignoreIdxConstraint { - return errors.Errorf("index %q is in use as a foreign key constraint", idx.Name) + return errors.Errorf("index %q is in use as a foreign key constraint", idx.GetName()) } sliceIdx-- if err := p.removeFKBackReference(ctx, tableDesc, fk); err != nil { @@ -391,34 +390,34 @@ func (p *planner) dropIndexByName( return err } - if len(idx.Interleave.Ancestors) > 0 { + if idx.NumInterleaveAncestors() > 0 { if err := p.removeInterleaveBackReference(ctx, tableDesc, idx); err != nil { return err } } - for _, ref := range idx.InterleavedBy { - if err := p.removeInterleave(ctx, ref); err != nil { + for i := 0; i < idx.NumInterleavedBy(); i++ { + if err := p.removeInterleave(ctx, idx.GetInterleavedBy(i)); err != nil { return err } } var droppedViews []string for _, tableRef := range tableDesc.DependedOnBy { - if tableRef.IndexID == idx.ID { + if tableRef.IndexID == idx.GetID() { // Ensure that we have DROP privilege on all dependent views err := p.canRemoveDependentViewGeneric( - ctx, "index", idx.Name, tableDesc.ParentID, tableRef, behavior) + ctx, "index", idx.GetName(), tableDesc.ParentID, tableRef, behavior) if err != nil { return err } viewDesc, err := p.getViewDescForCascade( - ctx, "index", idx.Name, tableDesc.ParentID, tableRef.ID, behavior, + ctx, "index", idx.GetName(), tableDesc.ParentID, tableRef.ID, behavior, ) if err != nil { return err } viewJobDesc := fmt.Sprintf("removing view %q dependent on index %q which is being dropped", - viewDesc.Name, idx.Name) + viewDesc.Name, idx.GetName()) cascadedViews, err := p.removeDependentView(ctx, tableDesc, viewDesc, viewJobDesc) if err != nil { return err @@ -435,12 +434,12 @@ func (p *planner) dropIndexByName( } // Overwriting tableDesc.Index may mess up with the idx object we collected above. Make a copy. - idxCopy := *idx - idx = &idxCopy + idxCopy := *idx.IndexDesc() + idxDesc := &idxCopy // Currently, a replacement primary index must be specified when dropping the primary index, // and this cannot be done with DROP INDEX. - if idx.ID == tableDesc.GetPrimaryIndexID() { + if idxDesc.ID == tableDesc.GetPrimaryIndexID() { return errors.WithHint( pgerror.Newf(pgcode.FeatureNotSupported, "cannot drop the primary index of a table using DROP INDEX"), "instead, use ALTER TABLE ... ALTER PRIMARY KEY or"+ @@ -449,7 +448,7 @@ func (p *planner) dropIndexByName( } foundIndex := catalog.FindPublicNonPrimaryIndex(tableDesc, func(idxEntry catalog.Index) bool { - return idxEntry.GetID() == idx.ID + return idxEntry.GetID() == idxDesc.ID }) if foundIndex == nil { @@ -501,7 +500,7 @@ func (p *planner) dropIndexByName( } tableDesc.RemovePublicNonPrimaryIndex(idxOrdinal) - if err := p.removeIndexComment(ctx, tableDesc.ID, idx.ID); err != nil { + if err := p.removeIndexComment(ctx, tableDesc.ID, idxDesc.ID); err != nil { return err } diff --git a/pkg/sql/drop_sequence.go b/pkg/sql/drop_sequence.go index 796892903c0d..1c42030a1c5a 100644 --- a/pkg/sql/drop_sequence.go +++ b/pkg/sql/drop_sequence.go @@ -143,8 +143,8 @@ func (p *planner) sequenceDependencyError( func (p *planner) canRemoveAllTableOwnedSequences( ctx context.Context, desc *tabledesc.Mutable, behavior tree.DropBehavior, ) error { - for _, col := range desc.Columns { - err := p.canRemoveOwnedSequencesImpl(ctx, desc, &col, behavior, false /* isColumnDrop */) + for _, col := range desc.PublicColumns() { + err := p.canRemoveOwnedSequencesImpl(ctx, desc, col, behavior, false /* isColumnDrop */) if err != nil { return err } @@ -153,10 +153,7 @@ func (p *planner) canRemoveAllTableOwnedSequences( } func (p *planner) canRemoveAllColumnOwnedSequences( - ctx context.Context, - desc *tabledesc.Mutable, - col *descpb.ColumnDescriptor, - behavior tree.DropBehavior, + ctx context.Context, desc *tabledesc.Mutable, col catalog.Column, behavior tree.DropBehavior, ) error { return p.canRemoveOwnedSequencesImpl(ctx, desc, col, behavior, true /* isColumnDrop */) } @@ -164,11 +161,12 @@ func (p *planner) canRemoveAllColumnOwnedSequences( func (p *planner) canRemoveOwnedSequencesImpl( ctx context.Context, desc *tabledesc.Mutable, - col *descpb.ColumnDescriptor, + col catalog.Column, behavior tree.DropBehavior, isColumnDrop bool, ) error { - for _, sequenceID := range col.OwnsSequenceIds { + for i := 0; i < col.NumOwnsSequences(); i++ { + sequenceID := col.GetOwnsSequenceID(i) seqDesc, err := p.LookupTableByID(ctx, sequenceID) if err != nil { // Special case error swallowing for #50711 and #50781, which can cause a @@ -203,7 +201,7 @@ func (p *planner) canRemoveOwnedSequencesImpl( continue } // ...or we're dropping a column in the table of interest. - if len(firstDep.ColumnIDs) == 1 && firstDep.ColumnIDs[0] == col.ID { + if len(firstDep.ColumnIDs) == 1 && firstDep.ColumnIDs[0] == col.GetID() { // The sequence is safe to remove iff it's not depended on by any other // columns in the table other than that one. continue @@ -264,10 +262,9 @@ func dropDependentOnSequence(ctx context.Context, p *planner, seqDesc *tabledesc // Iterate over all columns in the table, drop affected columns' default values // and update back references. - for idx := range tblDesc.Columns { - column := &tblDesc.Columns[idx] - if _, ok := colsToDropDefault[column.ID]; ok { - column.DefaultExpr = nil + for _, column := range tblDesc.PublicColumns() { + if _, ok := colsToDropDefault[column.GetID()]; ok { + column.ColumnDesc().DefaultExpr = nil if err := p.removeSequenceDependencies(ctx, tblDesc, column); err != nil { return err } diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index 84aacde87081..0065f151af04 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -307,7 +307,7 @@ func (p *planner) dropTableImpl( // Remove interleave relationships. for _, idx := range tableDesc.NonDropIndexes() { if idx.NumInterleaveAncestors() > 0 { - if err := p.removeInterleaveBackReference(ctx, tableDesc, idx.IndexDesc()); err != nil { + if err := p.removeInterleaveBackReference(ctx, tableDesc, idx); err != nil { return droppedViews, err } } @@ -320,15 +320,15 @@ func (p *planner) dropTableImpl( } // Remove sequence dependencies. - for i := range tableDesc.Columns { - if err := p.removeSequenceDependencies(ctx, tableDesc, &tableDesc.Columns[i]); err != nil { + for _, col := range tableDesc.PublicColumns() { + if err := p.removeSequenceDependencies(ctx, tableDesc, col); err != nil { return droppedViews, err } } // Drop sequences that the columns of the table own. - for _, col := range tableDesc.Columns { - if err := p.dropSequencesOwnedByCol(ctx, &col, !droppingParent, behavior); err != nil { + for _, col := range tableDesc.PublicColumns() { + if err := p.dropSequencesOwnedByCol(ctx, col, !droppingParent, behavior); err != nil { return droppedViews, err } } @@ -639,12 +639,12 @@ func removeFKBackReferenceFromTable( } func (p *planner) removeInterleaveBackReference( - ctx context.Context, tableDesc *tabledesc.Mutable, idx *descpb.IndexDescriptor, + ctx context.Context, tableDesc *tabledesc.Mutable, idx catalog.Index, ) error { - if len(idx.Interleave.Ancestors) == 0 { + if idx.NumInterleaveAncestors() == 0 { return nil } - ancestor := idx.Interleave.Ancestors[len(idx.Interleave.Ancestors)-1] + ancestor := idx.GetInterleaveAncestor(idx.NumInterleaveAncestors() - 1) var t *tabledesc.Mutable if ancestor.TableID == tableDesc.ID { t = tableDesc @@ -666,10 +666,10 @@ func (p *planner) removeInterleaveBackReference( targetIdx := targetIdxI.IndexDesc() foundAncestor := false for k, ref := range targetIdx.InterleavedBy { - if ref.Table == tableDesc.ID && ref.Index == idx.ID { + if ref.Table == tableDesc.ID && ref.Index == idx.GetID() { if foundAncestor { return errors.AssertionFailedf( - "ancestor entry in %s for %s@%s found more than once", t.Name, tableDesc.Name, idx.Name) + "ancestor entry in %s for %s@%s found more than once", t.Name, tableDesc.Name, idx.GetName()) } targetIdx.InterleavedBy = append(targetIdx.InterleavedBy[:k], targetIdx.InterleavedBy[k+1:]...) foundAncestor = true diff --git a/pkg/sql/exec_factory_util.go b/pkg/sql/exec_factory_util.go index d0a75dc1e966..2e80834b4efa 100644 --- a/pkg/sql/exec_factory_util.go +++ b/pkg/sql/exec_factory_util.go @@ -225,10 +225,10 @@ func constructVirtualScan( if !canQueryVirtualTable(p.EvalContext(), virtual) { return nil, newUnimplementedVirtualTableError(tn.Schema(), tn.Table()) } - indexDesc := index.(*optVirtualIndex).desc + idx := index.(*optVirtualIndex).idx columns, constructor := virtual.getPlanInfo( table.(*optVirtualTable).desc, - indexDesc, params.IndexConstraint, p.execCfg.DistSQLPlanner.stopper) + idx, params.IndexConstraint, p.execCfg.DistSQLPlanner.stopper) n, err := delayedNodeCallback(&delayedNode{ name: fmt.Sprintf("%s@%s", table.Name(), index.Name()), diff --git a/pkg/sql/execinfrapb/flow_diagram.go b/pkg/sql/execinfrapb/flow_diagram.go index 675ea4b86509..3df9745eaf20 100644 --- a/pkg/sql/execinfrapb/flow_diagram.go +++ b/pkg/sql/execinfrapb/flow_diagram.go @@ -149,7 +149,7 @@ func (tr *TableReaderSpec) summary() (string, []string) { tbl := tr.BuildTableDescriptor() // only show the first span idx := tbl.ActiveIndexes()[int(tr.IndexIdx)] - valDirs := catalogkeys.IndexKeyValDirs(idx.IndexDesc()) + valDirs := catalogkeys.IndexKeyValDirs(idx) var spanStr strings.Builder spanStr.WriteString("Spans: ") diff --git a/pkg/sql/explain_plan.go b/pkg/sql/explain_plan.go index 59b064a1289c..ee42a640f2e9 100644 --- a/pkg/sql/explain_plan.go +++ b/pkg/sql/explain_plan.go @@ -175,8 +175,8 @@ func emitExplain( return "<virtual table spans>" } tabDesc := table.(*optTable).desc - idxDesc := index.(*optIndex).desc - spans, err := generateScanSpans(evalCtx, codec, tabDesc, idxDesc, scanParams) + idx := index.(*optIndex).idx + spans, err := generateScanSpans(evalCtx, codec, tabDesc, idx, scanParams) if err != nil { return err.Error() } @@ -192,7 +192,7 @@ func emitExplain( if !codec.ForSystemTenant() { skip = 4 } - return catalogkeys.PrettySpans(idxDesc, spans, skip) + return catalogkeys.PrettySpans(idx, spans, skip) } return explain.Emit(explainPlan, ob, spanFormatFn) diff --git a/pkg/sql/gcjob/gc_job.go b/pkg/sql/gcjob/gc_job.go index c7394e7959d0..70e06d9b01f6 100644 --- a/pkg/sql/gcjob/gc_job.go +++ b/pkg/sql/gcjob/gc_job.go @@ -102,11 +102,15 @@ func (r schemaChangeGCResumer) Resume(ctx context.Context, execCtx interface{}) if err := sql.WaitToUpdateLeases(ctx, execCfg.LeaseManager, details.InterleavedTable.ID); err != nil { return err } + interleavedIndexIDs := make([]descpb.IndexID, len(details.InterleavedIndexes)) + for i := range details.InterleavedIndexes { + interleavedIndexIDs[i] = details.InterleavedIndexes[i].ID + } if err := sql.TruncateInterleavedIndexes( ctx, execCfg, tabledesc.NewBuilder(details.InterleavedTable).BuildImmutableTable(), - details.InterleavedIndexes, + interleavedIndexIDs, ); err != nil { return err } diff --git a/pkg/sql/gcjob/index_garbage_collection.go b/pkg/sql/gcjob/index_garbage_collection.go index 03dc3eb94054..3661bc6ac192 100644 --- a/pkg/sql/gcjob/index_garbage_collection.go +++ b/pkg/sql/gcjob/index_garbage_collection.go @@ -61,9 +61,8 @@ func gcIndexes( continue } - indexDesc := descpb.IndexDescriptor{ID: index.IndexID} - if err := clearIndex(ctx, execCfg, parentTable, indexDesc); err != nil { - return errors.Wrapf(err, "clearing index %d", indexDesc.ID) + if err := clearIndex(ctx, execCfg, parentTable, index.IndexID); err != nil { + return errors.Wrapf(err, "clearing index %d from table %d", index.IndexID, parentTable.GetID()) } // All the data chunks have been removed. Now also removed the @@ -83,16 +82,15 @@ func gcIndexes( if err != nil { return err } - toRemove := []descpb.IndexDescriptor{indexDesc} return sql.RemoveIndexZoneConfigs( - ctx, txn, execCfg, freshParentTableDesc, toRemove, + ctx, txn, execCfg, freshParentTableDesc, []uint32{uint32(index.IndexID)}, ) } lm, ie, db := execCfg.LeaseManager, execCfg.InternalExecutor, execCfg.DB if err := descs.Txn( ctx, execCfg.Settings, lm, ie, db, removeIndexZoneConfigs, ); err != nil { - return errors.Wrapf(err, "removing index %d zone configs", indexDesc.ID) + return errors.Wrapf(err, "removing index %d zone configs", index.IndexID) } if err := completeDroppedIndex( @@ -109,14 +107,11 @@ func clearIndex( ctx context.Context, execCfg *sql.ExecutorConfig, tableDesc catalog.TableDescriptor, - index descpb.IndexDescriptor, + indexID descpb.IndexID, ) error { - log.Infof(ctx, "clearing index %d from table %d", index.ID, tableDesc.GetID()) - if index.IsInterleaved() { - return errors.Errorf("unexpected interleaved index %d", index.ID) - } + log.Infof(ctx, "clearing index %d from table %d", indexID, tableDesc.GetID()) - sp := tableDesc.IndexSpan(execCfg.Codec, index.ID) + sp := tableDesc.IndexSpan(execCfg.Codec, indexID) start, err := keys.Addr(sp.Key) if err != nil { return errors.Errorf("failed to addr index start: %v", err) diff --git a/pkg/sql/index_join.go b/pkg/sql/index_join.go index faac6760b6e9..0742a6a0a952 100644 --- a/pkg/sql/index_join.go +++ b/pkg/sql/index_join.go @@ -13,8 +13,8 @@ package sql import ( "context" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) @@ -31,7 +31,7 @@ type indexJoinNode struct { table *scanNode // The columns returned by this node. - cols []descpb.ColumnDescriptor + cols []catalog.Column // There is a 1-1 correspondence between cols and resultColumns. resultColumns colinfo.ResultColumns diff --git a/pkg/sql/indexbackfiller_test.go b/pkg/sql/indexbackfiller_test.go index 2bc8e339ebc0..7758179fded6 100644 --- a/pkg/sql/indexbackfiller_test.go +++ b/pkg/sql/indexbackfiller_test.go @@ -387,9 +387,9 @@ INSERT INTO foo VALUES (1), (10), (100); row.FetcherTableArgs{ Spans: spans, Desc: table, - Index: idx.IndexDesc(), + Index: idx, ColIdxMap: colIdxMap, - Cols: table.Columns, + Cols: table.PublicColumns(), ValNeededForCol: valsNeeded, IsSecondaryIndex: !idx.Primary(), }, diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 951c3e701095..941108cba0f6 100755 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -1297,16 +1297,16 @@ CREATE TABLE information_schema.statistics ( scNameStr := tree.NewDString(scName) tbNameStr := tree.NewDString(table.GetName()) - appendRow := func(index *descpb.IndexDescriptor, colName string, sequence int, + appendRow := func(index catalog.Index, colName string, sequence int, direction tree.Datum, isStored, isImplicit bool, ) error { return addRow( dbNameStr, // table_catalog scNameStr, // table_schema tbNameStr, // table_name - yesOrNoDatum(!index.Unique), // non_unique + yesOrNoDatum(!index.IsUnique()), // non_unique scNameStr, // index_schema - tree.NewDString(index.Name), // index_name + tree.NewDString(index.GetName()), // index_name tree.NewDInt(tree.DInt(sequence)), // seq_in_index tree.NewDString(colName), // column_name tree.DNull, // collation @@ -1344,7 +1344,7 @@ CREATE TABLE information_schema.statistics ( // We add a row for each column of index. dir := dStringForIndexDirection(index.GetColumnDirection(i)) if err := appendRow( - index.IndexDesc(), + index, col, sequence, dir, @@ -1359,7 +1359,7 @@ CREATE TABLE information_schema.statistics ( for i := 0; i < index.NumStoredColumns(); i++ { col := index.GetStoredColumnName(i) // We add a row for each stored column of index. - if err := appendRow(index.IndexDesc(), col, sequence, + if err := appendRow(index, col, sequence, indexDirectionNA, true, false); err != nil { return err } @@ -1377,7 +1377,7 @@ CREATE TABLE information_schema.statistics ( col := table.GetPrimaryIndex().GetColumnName(i) if _, isImplicit := implicitCols[col]; isImplicit { // We add a row for each implicit column of index. - if err := appendRow(index.IndexDesc(), col, sequence, + if err := appendRow(index, col, sequence, indexDirectionAsc, false, true); err != nil { return err } diff --git a/pkg/sql/insert.go b/pkg/sql/insert.go index 7fa7b5a3e096..2914762884bb 100644 --- a/pkg/sql/insert.go +++ b/pkg/sql/insert.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -53,7 +52,7 @@ type insertRun struct { checkOrds checkSet // insertCols are the columns being inserted into. - insertCols []descpb.ColumnDescriptor + insertCols []catalog.Column // done informs a new call to BatchedNext() that the previous call to // BatchedNext() has completed the work already. @@ -110,7 +109,7 @@ func (r *insertRun) initRowContainer(params runParams, columns colinfo.ResultCol colIDToRetIndex := catalog.ColumnIDToOrdinalMap(r.ti.tableDesc().PublicColumns()) r.rowIdxToTabColIdx = make([]int, len(r.insertCols)) for i, col := range r.insertCols { - if idx, ok := colIDToRetIndex.Get(col.ID); !ok { + if idx, ok := colIDToRetIndex.Get(col.GetID()); !ok { // Column must be write only and not public. r.rowIdxToTabColIdx[i] = -1 } else { diff --git a/pkg/sql/insert_fast_path.go b/pkg/sql/insert_fast_path.go index 0e29d1451a28..73cf48b5b213 100644 --- a/pkg/sql/insert_fast_path.go +++ b/pkg/sql/insert_fast_path.go @@ -87,7 +87,7 @@ type insertFastPathFKCheck struct { exec.InsertFastPathFKCheck tabDesc catalog.TableDescriptor - idxDesc *descpb.IndexDescriptor + idx catalog.Index keyPrefix []byte colMap catalog.TableColMap spanBuilder *span.Builder @@ -96,11 +96,11 @@ type insertFastPathFKCheck struct { func (c *insertFastPathFKCheck) init(params runParams) error { idx := c.ReferencedIndex.(*optIndex) c.tabDesc = c.ReferencedTable.(*optTable).desc - c.idxDesc = idx.desc + c.idx = idx.idx codec := params.ExecCfg().Codec - c.keyPrefix = rowenc.MakeIndexKeyPrefix(codec, c.tabDesc, c.idxDesc.ID) - c.spanBuilder = span.MakeBuilder(params.EvalContext(), codec, c.tabDesc, c.idxDesc) + c.keyPrefix = rowenc.MakeIndexKeyPrefix(codec, c.tabDesc, c.idx.GetID()) + c.spanBuilder = span.MakeBuilder(params.EvalContext(), codec, c.tabDesc, c.idx) if len(c.InsertCols) > idx.numLaxKeyCols { return errors.AssertionFailedf( @@ -109,10 +109,10 @@ func (c *insertFastPathFKCheck) init(params runParams) error { } for i, ord := range c.InsertCols { var colID descpb.ColumnID - if i < len(c.idxDesc.ColumnIDs) { - colID = c.idxDesc.ColumnIDs[i] + if i < c.idx.NumColumns() { + colID = c.idx.GetColumnID(i) } else { - colID = c.idxDesc.ExtraColumnIDs[i-len(c.idxDesc.ColumnIDs)] + colID = c.idx.GetExtraColumnID(i - c.idx.NumColumns()) } c.colMap.Set(colID, int(ord)) @@ -336,23 +336,21 @@ func (n *insertFastPathNode) enableAutoCommit() { // If colNum is not -1, only the colNum'th column in insertCols will be checked // for AlterColumnTypeInProgress, otherwise every column in insertCols will // be checked. -func interceptAlterColumnTypeParseError( - insertCols []descpb.ColumnDescriptor, colNum int, err error, -) error { +func interceptAlterColumnTypeParseError(insertCols []catalog.Column, colNum int, err error) error { // Only intercept the error if the column being inserted into // is an actual column. This is to avoid checking on values that don't // correspond to an actual column, for example a check constraint. if colNum >= len(insertCols) { return err } - var insertCol descpb.ColumnDescriptor + var insertCol catalog.Column // wrapParseError is a helper function that checks if an insertCol has the // AlterColumnTypeInProgress flag and wraps the parse error msg stating // that the error may be because the column is being altered. // Returns if the error msg has been wrapped and the wrapped error msg. - wrapParseError := func(insertCol descpb.ColumnDescriptor, colNum int, err error) (bool, error) { - if insertCol.AlterColumnTypeInProgress { + wrapParseError := func(insertCol catalog.Column, colNum int, err error) (bool, error) { + if insertCol.ColumnDesc().AlterColumnTypeInProgress { code := pgerror.GetPGCode(err) if code == pgcode.InvalidTextRepresentation { if colNum != -1 { diff --git a/pkg/sql/opt/invertedidx/BUILD.bazel b/pkg/sql/opt/invertedidx/BUILD.bazel index 27f522374a4d..d59c54fbe460 100644 --- a/pkg/sql/opt/invertedidx/BUILD.bazel +++ b/pkg/sql/opt/invertedidx/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/geo/geoindex", "//pkg/geo/geopb", "//pkg/geo/geoprojbase", + "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/inverted", "//pkg/sql/opt", diff --git a/pkg/sql/opt/invertedidx/inverted_index_expr.go b/pkg/sql/opt/invertedidx/inverted_index_expr.go index e153575bbb54..ca0e3f0e8ce4 100644 --- a/pkg/sql/opt/invertedidx/inverted_index_expr.go +++ b/pkg/sql/opt/invertedidx/inverted_index_expr.go @@ -15,7 +15,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/geo/geoindex" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/inverted" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" @@ -32,10 +32,11 @@ import ( // NewDatumsToInvertedExpr returns a new DatumsToInvertedExpr. Currently there // is only one possible implementation returned, geoDatumsToInvertedExpr. func NewDatumsToInvertedExpr( - evalCtx *tree.EvalContext, colTypes []*types.T, expr tree.TypedExpr, desc *descpb.IndexDescriptor, + evalCtx *tree.EvalContext, colTypes []*types.T, expr tree.TypedExpr, idx catalog.Index, ) (invertedexpr.DatumsToInvertedExpr, error) { - if !geoindex.IsEmptyConfig(&desc.GeoConfig) { - return NewGeoDatumsToInvertedExpr(evalCtx, colTypes, expr, &desc.GeoConfig) + geoConfig := idx.GetGeoConfig() + if !geoindex.IsEmptyConfig(&geoConfig) { + return NewGeoDatumsToInvertedExpr(evalCtx, colTypes, expr, &geoConfig) } return NewJSONOrArrayDatumsToInvertedExpr(evalCtx, colTypes, expr) diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 0f3907a50516..294d302506b9 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -762,11 +762,11 @@ func newOptTable( ot.indexes = make([]optIndex, 1+len(secondaryIndexes)) for i := range ot.indexes { - var idxDesc *descpb.IndexDescriptor + var idx catalog.Index if i == 0 { - idxDesc = desc.GetPrimaryIndex().IndexDesc() + idx = desc.GetPrimaryIndex() } else { - idxDesc = secondaryIndexes[i-1].IndexDesc() + idx = secondaryIndexes[i-1] } // If there is a subzone that applies to the entire index, use that, else @@ -776,7 +776,7 @@ func newOptTable( partZones := make(map[string]*zonepb.ZoneConfig) for j := range tblZone.Subzones { subzone := &tblZone.Subzones[j] - if subzone.IndexID == uint32(idxDesc.ID) { + if subzone.IndexID == uint32(idx.GetID()) { if subzone.PartitionName == "" { // Subzone applies to the whole index. copyZone := subzone.Config @@ -790,13 +790,13 @@ func newOptTable( } } } - if idxDesc.Type == descpb.IndexDescriptor_INVERTED { + if idx.GetType() == descpb.IndexDescriptor_INVERTED { // The last column of an inverted index is special: in the // descriptors, it looks as if the table column is part of the // index; in fact the key contains values *derived* from that // column. In the catalog, we refer to this key as a separate, // virtual column. - invertedSourceColOrdinal, _ := ot.lookupColumnOrdinal(idxDesc.ColumnIDs[len(idxDesc.ColumnIDs)-1]) + invertedSourceColOrdinal, _ := ot.lookupColumnOrdinal(idx.GetColumnID(idx.NumColumns() - 1)) // Add a virtual column that refers to the inverted index key. virtualCol, virtualColOrd := newColumn() @@ -810,19 +810,19 @@ func newOptTable( false, /* nullable */ invertedSourceColOrdinal, ) - ot.indexes[i].init(ot, i, idxDesc, idxZone, partZones, virtualColOrd) + ot.indexes[i].init(ot, i, idx, idxZone, partZones, virtualColOrd) } else { - ot.indexes[i].init(ot, i, idxDesc, idxZone, partZones, -1 /* virtualColOrd */) + ot.indexes[i].init(ot, i, idx, idxZone, partZones, -1 /* virtualColOrd */) } // Add unique constraints for implicitly partitioned unique indexes. - if idxDesc.Unique && idxDesc.Partitioning.NumImplicitColumns > 0 { + if idx.IsUnique() && idx.GetPartitioning().NumImplicitColumns > 0 { ot.uniqueConstraints = append(ot.uniqueConstraints, optUniqueConstraint{ - name: idxDesc.Name, + name: idx.GetName(), table: ot.ID(), - columns: idxDesc.ColumnIDs[idxDesc.Partitioning.NumImplicitColumns:], + columns: idx.IndexDesc().ColumnIDs[idx.GetPartitioning().NumImplicitColumns:], withoutIndex: true, - predicate: idxDesc.Predicate, + predicate: idx.GetPredicate(), // TODO(rytaft): will we ever support an unvalidated unique constraint // here? validity: descpb.ConstraintValidity_Validated, @@ -1138,11 +1138,11 @@ func (ot *optTable) CollectTypes(ord int) (descpb.IDs, error) { return collectTypes(col) } -// optIndex is a wrapper around descpb.IndexDescriptor that caches some +// optIndex is a wrapper around catalog.Index that caches some // commonly accessed information and keeps a reference to the table wrapper. type optIndex struct { tab *optTable - desc *descpb.IndexDescriptor + idx catalog.Index zone *zonepb.ZoneConfig // storedCols is the set of non-PK columns if this is the primary index, @@ -1171,24 +1171,25 @@ var _ cat.Index = &optIndex{} func (oi *optIndex) init( tab *optTable, indexOrdinal int, - desc *descpb.IndexDescriptor, + idx catalog.Index, zone *zonepb.ZoneConfig, partZones map[string]*zonepb.ZoneConfig, invertedVirtualColOrd int, ) { oi.tab = tab - oi.desc = desc + oi.idx = idx oi.zone = zone oi.indexOrdinal = indexOrdinal oi.invertedVirtualColOrd = invertedVirtualColOrd - if desc == tab.desc.GetPrimaryIndex().IndexDesc() { + if idx.IndexDesc() == tab.desc.GetPrimaryIndex().IndexDesc() { // Although the primary index contains all columns in the table, the index // descriptor does not contain columns that are not explicitly part of the // primary key. Retrieve those columns from the table descriptor. - oi.storedCols = make([]descpb.ColumnID, 0, tab.ColumnCount()-len(desc.ColumnIDs)) + oi.storedCols = make([]descpb.ColumnID, 0, tab.ColumnCount()-idx.NumColumns()) var pkCols util.FastIntSet - for i := range desc.ColumnIDs { - pkCols.Add(int(desc.ColumnIDs[i])) + for i := 0; i < idx.NumColumns(); i++ { + id := idx.GetColumnID(i) + pkCols.Add(int(id)) } for i, n := 0, tab.ColumnCount(); i < n; i++ { if col := tab.Column(i); col.Kind() != cat.VirtualInverted && !col.IsVirtualComputed() { @@ -1197,16 +1198,17 @@ func (oi *optIndex) init( } } } - oi.numCols = len(desc.ColumnIDs) + len(oi.storedCols) + oi.numCols = idx.NumColumns() + len(oi.storedCols) } else { - oi.storedCols = desc.StoreColumnIDs - oi.numCols = len(desc.ColumnIDs) + len(desc.ExtraColumnIDs) + len(desc.StoreColumnIDs) + oi.storedCols = idx.IndexDesc().StoreColumnIDs + oi.numCols = idx.NumColumns() + idx.NumExtraColumns() + idx.NumStoredColumns() } // Collect information about the partitions. - oi.partitions = make([]optPartition, len(desc.Partitioning.List)) - for i := range desc.Partitioning.List { - p := &desc.Partitioning.List[i] + idxPartitioning := idx.GetPartitioning() + oi.partitions = make([]optPartition, len(idxPartitioning.List)) + for i := range idxPartitioning.List { + p := &idxPartitioning.List[i] oi.partitions[i] = optPartition{ name: p.Name, zone: &zonepb.ZoneConfig{}, @@ -1222,7 +1224,7 @@ func (oi *optIndex) init( var a rowenc.DatumAlloc for _, valueEncBuf := range p.Values { t, _, err := rowenc.DecodePartitionTuple( - &a, oi.tab.codec, oi.tab.desc, oi.desc, &oi.desc.Partitioning, + &a, oi.tab.codec, oi.tab.desc, oi.idx, &oi.idx.IndexDesc().Partitioning, valueEncBuf, nil, /* prefixDatums */ ) if err != nil { @@ -1235,9 +1237,10 @@ func (oi *optIndex) init( } } - if desc.Unique { + if idx.IsUnique() { notNull := true - for _, id := range desc.ColumnIDs { + for i := 0; i < idx.NumColumns(); i++ { + id := idx.GetColumnID(i) ord, _ := tab.lookupColumnOrdinal(id) if tab.Column(ord).IsNullable() { notNull = false @@ -1249,41 +1252,41 @@ func (oi *optIndex) init( // Unique index with no null columns: columns from index are sufficient // to form a key without needing extra primary key columns. There is no // separate lax key. - oi.numLaxKeyCols = len(desc.ColumnIDs) + oi.numLaxKeyCols = idx.NumColumns() oi.numKeyCols = oi.numLaxKeyCols } else { // Unique index with at least one nullable column: extra primary key // columns will be added to the row key when one of the unique index // columns has a NULL value. - oi.numLaxKeyCols = len(desc.ColumnIDs) - oi.numKeyCols = oi.numLaxKeyCols + len(desc.ExtraColumnIDs) + oi.numLaxKeyCols = idx.NumColumns() + oi.numKeyCols = oi.numLaxKeyCols + idx.NumExtraColumns() } } else { // Non-unique index: extra primary key columns are always added to the row // key. There is no separate lax key. - oi.numLaxKeyCols = len(desc.ColumnIDs) + len(desc.ExtraColumnIDs) + oi.numLaxKeyCols = idx.NumColumns() + idx.NumExtraColumns() oi.numKeyCols = oi.numLaxKeyCols } } // ID is part of the cat.Index interface. func (oi *optIndex) ID() cat.StableID { - return cat.StableID(oi.desc.ID) + return cat.StableID(oi.idx.GetID()) } // Name is part of the cat.Index interface. func (oi *optIndex) Name() tree.Name { - return tree.Name(oi.desc.Name) + return tree.Name(oi.idx.GetName()) } // IsUnique is part of the cat.Index interface. func (oi *optIndex) IsUnique() bool { - return oi.desc.Unique + return oi.idx.IsUnique() } // IsInverted is part of the cat.Index interface. func (oi *optIndex) IsInverted() bool { - return oi.desc.Type == descpb.IndexDescriptor_INVERTED + return oi.idx.GetType() == descpb.IndexDescriptor_INVERTED } // ColumnCount is part of the cat.Index interface. @@ -1306,29 +1309,29 @@ func (oi *optIndex) NonInvertedPrefixColumnCount() int { if !oi.IsInverted() { panic("non-inverted indexes do not have inverted prefix columns") } - return len(oi.desc.ColumnIDs) - 1 + return oi.idx.NumColumns() - 1 } // Column is part of the cat.Index interface. func (oi *optIndex) Column(i int) cat.IndexColumn { - length := len(oi.desc.ColumnIDs) + length := oi.idx.NumColumns() if i < length { ord := 0 if oi.IsInverted() && i == length-1 { ord = oi.invertedVirtualColOrd } else { - ord, _ = oi.tab.lookupColumnOrdinal(oi.desc.ColumnIDs[i]) + ord, _ = oi.tab.lookupColumnOrdinal(oi.idx.GetColumnID(i)) } return cat.IndexColumn{ Column: oi.tab.Column(ord), - Descending: oi.desc.ColumnDirections[i] == descpb.IndexDescriptor_DESC, + Descending: oi.idx.GetColumnDirection(i) == descpb.IndexDescriptor_DESC, } } i -= length - length = len(oi.desc.ExtraColumnIDs) + length = oi.idx.NumExtraColumns() if i < length { - ord, _ := oi.tab.lookupColumnOrdinal(oi.desc.ExtraColumnIDs[i]) + ord, _ := oi.tab.lookupColumnOrdinal(oi.idx.GetExtraColumnID(i)) return cat.IndexColumn{Column: oi.tab.Column(ord), Descending: false} } @@ -1342,7 +1345,7 @@ func (oi *optIndex) VirtualInvertedColumn() cat.IndexColumn { if !oi.IsInverted() { panic(errors.AssertionFailedf("non-inverted indexes do not have inverted virtual columns")) } - ord := len(oi.desc.ColumnIDs) - 1 + ord := oi.idx.NumColumns() - 1 return oi.Column(ord) } @@ -1350,7 +1353,7 @@ func (oi *optIndex) VirtualInvertedColumn() cat.IndexColumn { // expression and true if the index is a partial index. If the index is not // partial, the empty string and false is returned. func (oi *optIndex) Predicate() (string, bool) { - return oi.desc.Predicate, oi.desc.Predicate != "" + return oi.idx.GetPredicate(), oi.idx.GetPredicate() != "" } // Zone is part of the cat.Index interface. @@ -1366,7 +1369,7 @@ func (oi *optIndex) Span() roachpb.Span { if desc.GetID() <= keys.MaxSystemConfigDescID { return keys.SystemConfigSpan } - return desc.IndexSpan(oi.tab.codec, oi.desc.ID) + return desc.IndexSpan(oi.tab.codec, oi.idx.GetID()) } // Table is part of the cat.Index interface. @@ -1381,39 +1384,39 @@ func (oi *optIndex) Ordinal() int { // ImplicitPartitioningColumnCount is part of the cat.Index interface. func (oi *optIndex) ImplicitPartitioningColumnCount() int { - return int(oi.desc.Partitioning.NumImplicitColumns) + return int(oi.idx.GetPartitioning().NumImplicitColumns) } // InterleaveAncestorCount is part of the cat.Index interface. func (oi *optIndex) InterleaveAncestorCount() int { - return len(oi.desc.Interleave.Ancestors) + return oi.idx.NumInterleaveAncestors() } // InterleaveAncestor is part of the cat.Index interface. func (oi *optIndex) InterleaveAncestor(i int) (table, index cat.StableID, numKeyCols int) { - a := &oi.desc.Interleave.Ancestors[i] + a := oi.idx.GetInterleaveAncestor(i) return cat.StableID(a.TableID), cat.StableID(a.IndexID), int(a.SharedPrefixLen) } // InterleavedByCount is part of the cat.Index interface. func (oi *optIndex) InterleavedByCount() int { - return len(oi.desc.InterleavedBy) + return oi.idx.NumInterleavedBy() } // InterleavedBy is part of the cat.Index interface. func (oi *optIndex) InterleavedBy(i int) (table, index cat.StableID) { - ref := &oi.desc.InterleavedBy[i] + ref := oi.idx.GetInterleavedBy(i) return cat.StableID(ref.Table), cat.StableID(ref.Index) } // GeoConfig is part of the cat.Index interface. func (oi *optIndex) GeoConfig() *geoindex.Config { - return &oi.desc.GeoConfig + return &oi.idx.IndexDesc().GeoConfig } // Version is part of the cat.Index interface. func (oi *optIndex) Version() descpb.IndexDescriptorVersion { - return oi.desc.Version + return oi.idx.GetVersion() } // PartitionCount is part of the cat.Index interface. @@ -1831,11 +1834,6 @@ func newOptVirtualTable( tab: ot, indexOrdinal: 0, numCols: ot.ColumnCount(), - isPrimary: true, - desc: &descpb.IndexDescriptor{ - ID: 0, - Name: "primary", - }, } for _, idx := range ot.desc.PublicNonPrimaryIndexes() { @@ -1846,7 +1844,7 @@ func newOptVirtualTable( // Add 1, since the 0th index will the primary that we added above. ot.indexes[idx.Ordinal()] = optVirtualIndex{ tab: ot, - desc: idx.IndexDesc(), + idx: idx, indexOrdinal: idx.Ordinal(), // The virtual indexes don't return the bogus PK key? numCols: ot.ColumnCount(), @@ -2015,10 +2013,8 @@ func (ot *optVirtualTable) CollectTypes(ord int) (descpb.IDs, error) { type optVirtualIndex struct { tab *optVirtualTable - // isPrimary is set to true if this is the dummy PK index for virtual tables. - isPrimary bool - - desc *descpb.IndexDescriptor + // idx is set to nil if this is the dummy PK index for virtual tables. + idx catalog.Index numCols int @@ -2027,17 +2023,29 @@ type optVirtualIndex struct { // ID is part of the cat.Index interface. func (oi *optVirtualIndex) ID() cat.StableID { - return cat.StableID(oi.desc.ID) + if oi.idx == nil { + // Dummy PK index ID. + return cat.StableID(0) + } + return cat.StableID(oi.idx.GetID()) } // Name is part of the cat.Index interface. func (oi *optVirtualIndex) Name() tree.Name { - return tree.Name(oi.desc.Name) + if oi.idx == nil { + // Dummy PK index name. + return "primary" + } + return tree.Name(oi.idx.GetName()) } // IsUnique is part of the cat.Index interface. func (oi *optVirtualIndex) IsUnique() bool { - return oi.desc.Unique + if oi.idx == nil { + // Dummy PK index is not explicitly UNIQUE. + return false + } + return oi.idx.IsUnique() } // IsInverted is part of the cat.Index interface. @@ -2086,12 +2094,13 @@ func (ot *optVirtualTable) lookupColumnOrdinal(colID descpb.ColumnID) (int, erro // Column is part of the cat.Index interface. func (oi *optVirtualIndex) Column(i int) cat.IndexColumn { - if oi.isPrimary { + if oi.idx == nil { + // Dummy PK index columns. return cat.IndexColumn{Column: oi.tab.Column(i)} } - length := len(oi.desc.ColumnIDs) + length := oi.idx.NumColumns() if i < length { - ord, _ := oi.tab.lookupColumnOrdinal(oi.desc.ColumnIDs[i]) + ord, _ := oi.tab.lookupColumnOrdinal(oi.idx.GetColumnID(i)) return cat.IndexColumn{ Column: oi.tab.Column(ord), } @@ -2103,7 +2112,7 @@ func (oi *optVirtualIndex) Column(i int) cat.IndexColumn { } i -= length + 1 - ord, _ := oi.tab.lookupColumnOrdinal(oi.desc.StoreColumnIDs[i]) + ord, _ := oi.tab.lookupColumnOrdinal(oi.idx.GetStoredColumnID(i)) return cat.IndexColumn{Column: oi.tab.Column(ord)} } diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 6eae14b0ea8c..543c3adfea9e 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -84,7 +84,7 @@ func (ef *execFactory) ConstructScan( } tabDesc := table.(*optTable).desc - indexDesc := index.(*optIndex).desc + idx := index.(*optIndex).idx // Create a scanNode. scan := ef.planner.Scan() colCfg := makeScanColumnsConfig(table, params.NeededCols) @@ -104,20 +104,20 @@ func (ef *execFactory) ConstructScan( return newZeroNode(scan.resultColumns), nil } - scan.index = indexDesc + scan.index = idx scan.hardLimit = params.HardLimit scan.softLimit = params.SoftLimit scan.reverse = params.Reverse scan.parallelize = params.Parallelize var err error - scan.spans, err = generateScanSpans(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, indexDesc, params) + scan.spans, err = generateScanSpans(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, idx, params) if err != nil { return nil, err } scan.isFull = len(scan.spans) == 1 && scan.spans[0].EqualValue( - scan.desc.IndexSpan(ef.planner.ExecCfg().Codec, scan.index.ID), + scan.desc.IndexSpan(ef.planner.ExecCfg().Codec, scan.index.GetID()), ) if err = colCfg.assertValidReqOrdering(reqOrdering); err != nil { return nil, err @@ -136,10 +136,10 @@ func generateScanSpans( evalCtx *tree.EvalContext, codec keys.SQLCodec, tabDesc catalog.TableDescriptor, - indexDesc *descpb.IndexDescriptor, + index catalog.Index, params exec.ScanParams, ) (roachpb.Spans, error) { - sb := span.MakeBuilder(evalCtx, codec, tabDesc, indexDesc) + sb := span.MakeBuilder(evalCtx, codec, tabDesc, index) if params.InvertedConstraint != nil { return sb.SpansFromInvertedSpans(params.InvertedConstraint, params.IndexConstraint) } @@ -547,7 +547,7 @@ func (ef *execFactory) ConstructIndexJoin( ) (exec.Node, error) { tabDesc := table.(*optTable).desc colCfg := makeScanColumnsConfig(table, tableCols) - colDescs := makeColDescList(table, tableCols) + cols := makeColList(table, tableCols) tableScan := ef.planner.Scan() @@ -555,15 +555,14 @@ func (ef *execFactory) ConstructIndexJoin( return nil, err } - primaryIndex := tabDesc.GetPrimaryIndex() - tableScan.index = primaryIndex.IndexDesc() + tableScan.index = tabDesc.GetPrimaryIndex() tableScan.disableBatchLimit() n := &indexJoinNode{ input: input.(planNode), table: tableScan, - cols: colDescs, - resultColumns: colinfo.ResultColumnsFromColDescs(tabDesc.GetID(), colDescs), + cols: cols, + resultColumns: colinfo.ResultColumnsFromColumns(tabDesc.GetID(), cols), reqOrdering: ReqOrdering(reqOrdering), } @@ -594,7 +593,7 @@ func (ef *execFactory) ConstructLookupJoin( return ef.constructVirtualTableLookupJoin(joinType, input, table, index, eqCols, lookupCols, onCond) } tabDesc := table.(*optTable).desc - indexDesc := index.(*optIndex).desc + idx := index.(*optIndex).idx colCfg := makeScanColumnsConfig(table, lookupCols) tableScan := ef.planner.Scan() @@ -602,7 +601,7 @@ func (ef *execFactory) ConstructLookupJoin( return nil, err } - tableScan.index = indexDesc + tableScan.index = idx if locking != nil { tableScan.lockingStrength = descpb.ToScanLockingStrength(locking.Strength) tableScan.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(locking.WaitPolicy) @@ -656,7 +655,7 @@ func (ef *execFactory) constructVirtualTableLookupJoin( if lookupCols.Contains(0) { return nil, errors.Errorf("use of %s column not allowed.", table.Column(0).ColName()) } - indexDesc := index.(*optVirtualIndex).desc + idx := index.(*optVirtualIndex).idx tableDesc := table.(*optVirtualTable).desc // Build the result columns. inputCols := planColumns(input.(planNode)) @@ -672,12 +671,8 @@ func (ef *execFactory) constructVirtualTableLookupJoin( if err := tableScan.initTable(context.TODO(), ef.planner, tableDesc, nil, colCfg); err != nil { return nil, err } - tableScan.index = indexDesc - publicColDescs := make([]descpb.ColumnDescriptor, len(tableDesc.PublicColumns())) - for i, col := range tableDesc.PublicColumns() { - publicColDescs[i] = *col.ColumnDesc() - } - vtableCols := colinfo.ResultColumnsFromColDescs(tableDesc.GetID(), publicColDescs) + tableScan.index = idx + vtableCols := colinfo.ResultColumnsFromColumns(tableDesc.GetID(), tableDesc.PublicColumns()) projectedVtableCols := planColumns(&tableScan) outputCols := make(colinfo.ResultColumns, 0, len(inputCols)+len(projectedVtableCols)) outputCols = append(outputCols, inputCols...) @@ -691,7 +686,7 @@ func (ef *execFactory) constructVirtualTableLookupJoin( virtualTableEntry: virtual, dbName: tn.Catalog(), table: tableDesc, - index: indexDesc, + index: idx, eqCol: int(eqCols[0]), inputCols: inputCols, vtableCols: vtableCols, @@ -715,7 +710,7 @@ func (ef *execFactory) ConstructInvertedJoin( reqOrdering exec.OutputOrdering, ) (exec.Node, error) { tabDesc := table.(*optTable).desc - indexDesc := index.(*optIndex).desc + idx := index.(*optIndex).idx // NB: lookupCols does not include the inverted column, which is only a partial // representation of the original table column. This scan configuration does not // affect what the invertedJoiner implementation retrieves from the inverted @@ -727,7 +722,7 @@ func (ef *execFactory) ConstructInvertedJoin( if err := tableScan.initTable(context.TODO(), ef.planner, tabDesc, nil, colCfg); err != nil { return nil, err } - tableScan.index = indexDesc + tableScan.index = idx n := &invertedJoinNode{ input: input.(planNode), @@ -768,9 +763,7 @@ func (ef *execFactory) ConstructInvertedJoin( // Helper function to create a scanNode from just a table / index descriptor // and requested cols. func (ef *execFactory) constructScanForZigzag( - indexDesc *descpb.IndexDescriptor, - tableDesc catalog.TableDescriptor, - cols exec.TableColumnOrdinalSet, + index catalog.Index, tableDesc catalog.TableDescriptor, cols exec.TableColumnOrdinalSet, ) (*scanNode, error) { colCfg := scanColumnsConfig{ @@ -786,7 +779,7 @@ func (ef *execFactory) constructScanForZigzag( return nil, err } - scan.index = indexDesc + scan.index = index return scan, nil } @@ -806,16 +799,16 @@ func (ef *execFactory) ConstructZigzagJoin( onCond tree.TypedExpr, reqOrdering exec.OutputOrdering, ) (exec.Node, error) { - leftIndexDesc := leftIndex.(*optIndex).desc + leftIdx := leftIndex.(*optIndex).idx leftTabDesc := leftTable.(*optTable).desc - rightIndexDesc := rightIndex.(*optIndex).desc + rightIdx := rightIndex.(*optIndex).idx rightTabDesc := rightTable.(*optTable).desc - leftScan, err := ef.constructScanForZigzag(leftIndexDesc, leftTabDesc, leftCols) + leftScan, err := ef.constructScanForZigzag(leftIdx, leftTabDesc, leftCols) if err != nil { return nil, err } - rightScan, err := ef.constructScanForZigzag(rightIndexDesc, rightTabDesc, rightCols) + rightScan, err := ef.constructScanForZigzag(rightIdx, rightTabDesc, rightCols) if err != nil { return nil, err } @@ -1181,7 +1174,7 @@ func (ef *execFactory) ConstructInsert( // Derive insert table and column descriptors. rowsNeeded := !returnColOrdSet.Empty() tabDesc := table.(*optTable).desc - colDescs := makeColDescList(table, insertColOrdSet) + cols := makeColList(table, insertColOrdSet) if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { return nil, err @@ -1189,7 +1182,7 @@ func (ef *execFactory) ConstructInsert( // Create the table inserter, which does the bulk of the work. ri, err := row.MakeInserter( - ctx, ef.planner.txn, ef.planner.ExecCfg().Codec, tabDesc, colDescs, ef.planner.alloc, + ctx, ef.planner.txn, ef.planner.ExecCfg().Codec, tabDesc, cols, ef.planner.alloc, ) if err != nil { return nil, err @@ -1208,12 +1201,12 @@ func (ef *execFactory) ConstructInsert( // If rows are not needed, no columns are returned. if rowsNeeded { - returnColDescs := makeColDescList(table, returnColOrdSet) - ins.columns = colinfo.ResultColumnsFromColDescs(tabDesc.GetID(), returnColDescs) + returnCols := makeColList(table, returnColOrdSet) + ins.columns = colinfo.ResultColumnsFromColumns(tabDesc.GetID(), returnCols) // Set the tabColIdxToRetIdx for the mutation. Insert always returns // non-mutation columns in the same order they are defined in the table. - ins.run.tabColIdxToRetIdx = makePublicToReturnColumnIndexMapping(tabDesc, returnColDescs) + ins.run.tabColIdxToRetIdx = makePublicToReturnColumnIndexMapping(tabDesc, returnCols) ins.run.rowsNeeded = true } @@ -1247,7 +1240,7 @@ func (ef *execFactory) ConstructInsertFastPath( // Derive insert table and column descriptors. rowsNeeded := !returnColOrdSet.Empty() tabDesc := table.(*optTable).desc - colDescs := makeColDescList(table, insertColOrdSet) + cols := makeColList(table, insertColOrdSet) if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { return nil, err @@ -1255,7 +1248,7 @@ func (ef *execFactory) ConstructInsertFastPath( // Create the table inserter, which does the bulk of the work. ri, err := row.MakeInserter( - ctx, ef.planner.txn, ef.planner.ExecCfg().Codec, tabDesc, colDescs, ef.planner.alloc, + ctx, ef.planner.txn, ef.planner.ExecCfg().Codec, tabDesc, cols, ef.planner.alloc, ) if err != nil { return nil, err @@ -1283,12 +1276,12 @@ func (ef *execFactory) ConstructInsertFastPath( // If rows are not needed, no columns are returned. if rowsNeeded { - returnColDescs := makeColDescList(table, returnColOrdSet) - ins.columns = colinfo.ResultColumnsFromColDescs(tabDesc.GetID(), returnColDescs) + returnCols := makeColList(table, returnColOrdSet) + ins.columns = colinfo.ResultColumnsFromColumns(tabDesc.GetID(), returnCols) // Set the tabColIdxToRetIdx for the mutation. Insert always returns // non-mutation columns in the same order they are defined in the table. - ins.run.tabColIdxToRetIdx = makePublicToReturnColumnIndexMapping(tabDesc, returnColDescs) + ins.run.tabColIdxToRetIdx = makePublicToReturnColumnIndexMapping(tabDesc, returnCols) ins.run.rowsNeeded = true } @@ -1335,7 +1328,7 @@ func (ef *execFactory) ConstructUpdate( // Derive table and column descriptors. rowsNeeded := !returnColOrdSet.Empty() tabDesc := table.(*optTable).desc - fetchColDescs := makeColDescList(table, fetchColOrdSet) + fetchCols := makeColList(table, fetchColOrdSet) if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { return nil, err @@ -1344,10 +1337,10 @@ func (ef *execFactory) ConstructUpdate( // Add each column to update as a sourceSlot. The CBO only uses scalarSlot, // since it compiles tuples and subqueries into a simple sequence of target // columns. - updateColDescs := makeColDescList(table, updateColOrdSet) - sourceSlots := make([]sourceSlot, len(updateColDescs)) + updateCols := makeColList(table, updateColOrdSet) + sourceSlots := make([]sourceSlot, len(updateCols)) for i := range sourceSlots { - sourceSlots[i] = scalarSlot{column: updateColDescs[i], sourceIndex: len(fetchColDescs) + i} + sourceSlots[i] = scalarSlot{column: updateCols[i], sourceIndex: len(fetchCols) + i} } // Create the table updater, which does the bulk of the work. @@ -1356,8 +1349,8 @@ func (ef *execFactory) ConstructUpdate( ef.planner.txn, ef.planner.ExecCfg().Codec, tabDesc, - updateColDescs, - fetchColDescs, + updateCols, + fetchCols, row.UpdaterDefault, ef.planner.alloc, ) @@ -1369,7 +1362,7 @@ func (ef *execFactory) ConstructUpdate( // the explanatory comments in updateRun. var updateColsIdx catalog.TableColMap for i := range ru.UpdateCols { - id := ru.UpdateCols[i].ID + id := ru.UpdateCols[i].GetID() updateColsIdx.Set(id, i) } @@ -1393,9 +1386,9 @@ func (ef *execFactory) ConstructUpdate( // If rows are not needed, no columns are returned. if rowsNeeded { - returnColDescs := makeColDescList(table, returnColOrdSet) + returnCols := makeColList(table, returnColOrdSet) - upd.columns = colinfo.ResultColumnsFromColDescs(tabDesc.GetID(), returnColDescs) + upd.columns = colinfo.ResultColumnsFromColumns(tabDesc.GetID(), returnCols) // Add the passthrough columns to the returning columns. upd.columns = append(upd.columns, passthrough...) @@ -1406,7 +1399,7 @@ func (ef *execFactory) ConstructUpdate( // since the return columns are always a subset of the fetch columns, // we can use use the fetch columns to generate the mapping for the // returned rows. - upd.run.rowIdxToRetIdx = row.ColMapping(ru.FetchCols, returnColDescs) + upd.run.rowIdxToRetIdx = row.ColMapping(ru.FetchCols, returnCols) upd.run.rowsNeeded = true } @@ -1444,9 +1437,9 @@ func (ef *execFactory) ConstructUpsert( // Derive table and column descriptors. rowsNeeded := !returnColOrdSet.Empty() tabDesc := table.(*optTable).desc - insertColDescs := makeColDescList(table, insertColOrdSet) - fetchColDescs := makeColDescList(table, fetchColOrdSet) - updateColDescs := makeColDescList(table, updateColOrdSet) + insertCols := makeColList(table, insertColOrdSet) + fetchCols := makeColList(table, fetchColOrdSet) + updateCols := makeColList(table, updateColOrdSet) if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { return nil, err @@ -1458,7 +1451,7 @@ func (ef *execFactory) ConstructUpsert( ef.planner.txn, ef.planner.ExecCfg().Codec, tabDesc, - insertColDescs, + insertCols, ef.planner.alloc, ) if err != nil { @@ -1471,8 +1464,8 @@ func (ef *execFactory) ConstructUpsert( ef.planner.txn, ef.planner.ExecCfg().Codec, tabDesc, - updateColDescs, - fetchColDescs, + updateCols, + fetchCols, row.UpdaterDefault, ef.planner.alloc, ) @@ -1490,8 +1483,8 @@ func (ef *execFactory) ConstructUpsert( tw: optTableUpserter{ ri: ri, canaryOrdinal: int(canaryCol), - fetchCols: fetchColDescs, - updateCols: updateColDescs, + fetchCols: fetchCols, + updateCols: updateCols, ru: ru, }, }, @@ -1499,14 +1492,14 @@ func (ef *execFactory) ConstructUpsert( // If rows are not needed, no columns are returned. if rowsNeeded { - returnColDescs := makeColDescList(table, returnColOrdSet) - ups.columns = colinfo.ResultColumnsFromColDescs(tabDesc.GetID(), returnColDescs) + returnCols := makeColList(table, returnColOrdSet) + ups.columns = colinfo.ResultColumnsFromColumns(tabDesc.GetID(), returnCols) // Update the tabColIdxToRetIdx for the mutation. Upsert returns // non-mutation columns specified, in the same order they are defined // in the table. - ups.run.tw.tabColIdxToRetIdx = makePublicToReturnColumnIndexMapping(tabDesc, returnColDescs) - ups.run.tw.returnCols = returnColDescs + ups.run.tw.tabColIdxToRetIdx = makePublicToReturnColumnIndexMapping(tabDesc, returnCols) + ups.run.tw.returnCols = returnCols ups.run.tw.rowsNeeded = true } @@ -1536,7 +1529,7 @@ func (ef *execFactory) ConstructDelete( // Derive table and column descriptors. rowsNeeded := !returnColOrdSet.Empty() tabDesc := table.(*optTable).desc - fetchColDescs := makeColDescList(table, fetchColOrdSet) + fetchCols := makeColList(table, fetchColOrdSet) if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { return nil, err @@ -1546,7 +1539,7 @@ func (ef *execFactory) ConstructDelete( // the deleter derives the columns that need to be fetched. By contrast, the // CBO will have already determined the set of fetch columns, and passes // those sets into the deleter (which will basically be a no-op). - rd := row.MakeDeleter(ef.planner.ExecCfg().Codec, tabDesc, fetchColDescs) + rd := row.MakeDeleter(ef.planner.ExecCfg().Codec, tabDesc, fetchCols) // Now make a delete node. We use a pool. del := deleteNodePool.Get().(*deleteNode) @@ -1560,12 +1553,12 @@ func (ef *execFactory) ConstructDelete( // If rows are not needed, no columns are returned. if rowsNeeded { - returnColDescs := makeColDescList(table, returnColOrdSet) + returnCols := makeColList(table, returnColOrdSet) // Delete returns the non-mutation columns specified, in the same // order they are defined in the table. - del.columns = colinfo.ResultColumnsFromColDescs(tabDesc.GetID(), returnColDescs) + del.columns = colinfo.ResultColumnsFromColumns(tabDesc.GetID(), returnCols) - del.run.rowIdxToRetIdx = row.ColMapping(rd.FetchCols, returnColDescs) + del.run.rowIdxToRetIdx = row.ColMapping(rd.FetchCols, returnCols) del.run.rowsNeeded = true } @@ -1593,8 +1586,7 @@ func (ef *execFactory) ConstructDeleteRange( autoCommit bool, ) (exec.Node, error) { tabDesc := table.(*optTable).desc - indexDesc := tabDesc.GetPrimaryIndex().IndexDesc() - sb := span.MakeBuilder(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, indexDesc) + sb := span.MakeBuilder(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, tabDesc.GetPrimaryIndex()) if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { return nil, err @@ -1691,7 +1683,7 @@ func (ef *execFactory) ConstructCreateView( var ref descpb.TableDescriptor_Reference if d.SpecificIndex { idx := d.DataSource.(cat.Table).Index(d.Index) - ref.IndexID = idx.(*optIndex).desc.ID + ref.IndexID = idx.(*optIndex).idx.GetID() } if !d.ColumnOrdinals.Empty() { ref.ColumnIDs = make([]descpb.ColumnID, 0, d.ColumnOrdinals.Len()) @@ -1774,7 +1766,7 @@ func (ef *execFactory) ConstructAlterTableSplit( return &splitNode{ tableDesc: index.Table().(*optTable).desc, - index: index.(*optIndex).desc, + index: index.(*optIndex).idx, rows: input.(planNode), expirationTime: expirationTime, }, nil @@ -1798,7 +1790,7 @@ func (ef *execFactory) ConstructAlterTableUnsplit( return &unsplitNode{ tableDesc: index.Table().(*optTable).desc, - index: index.(*optIndex).desc, + index: index.(*optIndex).idx, rows: input.(planNode), }, nil } @@ -1819,7 +1811,7 @@ func (ef *execFactory) ConstructAlterTableUnsplitAll(index cat.Index) (exec.Node return &unsplitAllNode{ tableDesc: index.Table().(*optTable).desc, - index: index.(*optIndex).desc, + index: index.(*optIndex).idx, }, nil } @@ -1835,7 +1827,7 @@ func (ef *execFactory) ConstructAlterTableRelocate( relocateLease: relocateLease, relocateNonVoters: relocateNonVoters, tableDesc: index.Table().(*optTable).desc, - index: index.(*optIndex).desc, + index: index.(*optIndex).idx, rows: input.(planNode), }, nil } @@ -1975,18 +1967,18 @@ func (rb *renderBuilder) setOutput(exprs tree.TypedExprs, columns colinfo.Result rb.r.columns = columns } -// makeColDescList returns a list of table column descriptors. Columns are +// makeColList returns a list of table column interfaces. Columns are // included if their ordinal position in the table schema is in the cols set. -func makeColDescList(table cat.Table, cols exec.TableColumnOrdinalSet) []descpb.ColumnDescriptor { +func makeColList(table cat.Table, cols exec.TableColumnOrdinalSet) []catalog.Column { tab := table.(optCatalogTableInterface) - colDescs := make([]descpb.ColumnDescriptor, 0, cols.Len()) + ret := make([]catalog.Column, 0, cols.Len()) for i, n := 0, table.ColumnCount(); i < n; i++ { if !cols.Contains(i) { continue } - colDescs = append(colDescs, *tab.getCol(i).ColumnDesc()) + ret = append(ret, tab.getCol(i)) } - return colDescs + return ret } // makePublicToReturnColumnIndexMapping returns a map from the ordinals @@ -1996,12 +1988,7 @@ func makeColDescList(table cat.Table, cols exec.TableColumnOrdinalSet) []descpb. // the i'th public column, or // -1 if the i'th public column is not found in returnColDescs. func makePublicToReturnColumnIndexMapping( - tableDesc catalog.TableDescriptor, returnColDescs []descpb.ColumnDescriptor, + tableDesc catalog.TableDescriptor, returnCols []catalog.Column, ) []int { - publicCols := tableDesc.PublicColumns() - publicColDescs := make([]descpb.ColumnDescriptor, len(publicCols)) - for i, col := range publicCols { - publicColDescs[i] = *col.ColumnDesc() - } - return row.ColMapping(publicColDescs, returnColDescs) + return row.ColMapping(tableDesc.PublicColumns(), returnCols) } diff --git a/pkg/sql/partition.go b/pkg/sql/partition.go index f53880d0606f..567e260d4b7a 100644 --- a/pkg/sql/partition.go +++ b/pkg/sql/partition.go @@ -12,6 +12,7 @@ package sql import ( "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -23,9 +24,8 @@ import ( func partitionByFromTableDesc( codec keys.SQLCodec, tableDesc *tabledesc.Mutable, ) (*tree.PartitionBy, error) { - idxDesc := tableDesc.GetPrimaryIndex().IndexDesc() - partDesc := idxDesc.Partitioning - return partitionByFromTableDescImpl(codec, tableDesc, idxDesc, &partDesc, 0) + idx := tableDesc.GetPrimaryIndex() + return partitionByFromTableDescImpl(codec, tableDesc, idx, &idx.IndexDesc().Partitioning, 0) } // partitionByFromTableDescImpl contains the inner logic of partitionByFromTableDesc. @@ -34,7 +34,7 @@ func partitionByFromTableDesc( func partitionByFromTableDescImpl( codec keys.SQLCodec, tableDesc *tabledesc.Mutable, - idxDesc *descpb.IndexDescriptor, + idx catalog.Index, partDesc *descpb.PartitioningDescriptor, colOffset int, ) (*tree.PartitionBy, error) { @@ -55,7 +55,7 @@ func partitionByFromTableDescImpl( Range: make([]tree.RangePartition, len(partDesc.Range)), } for i := 0; i < int(partDesc.NumColumns); i++ { - partitionBy.Fields[i] = tree.Name(idxDesc.ColumnNames[colOffset+i]) + partitionBy.Fields[i] = tree.Name(idx.GetColumnName(colOffset + i)) } // Copy the LIST of the PARTITION BY clause. @@ -69,7 +69,7 @@ func partitionByFromTableDescImpl( a, codec, tableDesc, - idxDesc, + idx, partDesc, values, fakePrefixDatums, @@ -89,7 +89,7 @@ func partitionByFromTableDescImpl( if partitionBy.List[i].Subpartition, err = partitionByFromTableDescImpl( codec, tableDesc, - idxDesc, + idx, &part.Subpartitioning, colOffset+int(partDesc.NumColumns), ); err != nil { @@ -104,7 +104,7 @@ func partitionByFromTableDescImpl( a, codec, tableDesc, - idxDesc, + idx, partDesc, part.FromInclusive, fakePrefixDatums, @@ -119,7 +119,7 @@ func partitionByFromTableDescImpl( a, codec, tableDesc, - idxDesc, + idx, partDesc, part.ToExclusive, fakePrefixDatums, diff --git a/pkg/sql/partition_utils.go b/pkg/sql/partition_utils.go index 4dfe5626f096..fec2ad869334 100644 --- a/pkg/sql/partition_utils.go +++ b/pkg/sql/partition_utils.go @@ -125,7 +125,7 @@ func GenerateSubzoneSpans( var emptyPrefix []tree.Datum indexPartitionCoverings, err := indexCoveringsForPartitioning( - a, codec, tableDesc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, subzoneIndexByPartition, emptyPrefix) + a, codec, tableDesc, idx, &idx.IndexDesc().Partitioning, subzoneIndexByPartition, emptyPrefix) if err != nil { return err } @@ -185,7 +185,7 @@ func indexCoveringsForPartitioning( a *rowenc.DatumAlloc, codec keys.SQLCodec, tableDesc catalog.TableDescriptor, - idxDesc *descpb.IndexDescriptor, + idx catalog.Index, partDesc *descpb.PartitioningDescriptor, relevantPartitions map[string]int32, prefixDatums []tree.Datum, @@ -209,7 +209,7 @@ func indexCoveringsForPartitioning( for _, p := range partDesc.List { for _, valueEncBuf := range p.Values { t, keyPrefix, err := rowenc.DecodePartitionTuple( - a, codec, tableDesc, idxDesc, partDesc, valueEncBuf, prefixDatums) + a, codec, tableDesc, idx, partDesc, valueEncBuf, prefixDatums) if err != nil { return nil, err } @@ -221,7 +221,7 @@ func indexCoveringsForPartitioning( } newPrefixDatums := append(prefixDatums, t.Datums...) subpartitionCoverings, err := indexCoveringsForPartitioning( - a, codec, tableDesc, idxDesc, &p.Subpartitioning, relevantPartitions, newPrefixDatums) + a, codec, tableDesc, idx, &p.Subpartitioning, relevantPartitions, newPrefixDatums) if err != nil { return nil, err } @@ -241,12 +241,12 @@ func indexCoveringsForPartitioning( continue } _, fromKey, err := rowenc.DecodePartitionTuple( - a, codec, tableDesc, idxDesc, partDesc, p.FromInclusive, prefixDatums) + a, codec, tableDesc, idx, partDesc, p.FromInclusive, prefixDatums) if err != nil { return nil, err } _, toKey, err := rowenc.DecodePartitionTuple( - a, codec, tableDesc, idxDesc, partDesc, p.ToExclusive, prefixDatums) + a, codec, tableDesc, idx, partDesc, p.ToExclusive, prefixDatums) if err != nil { return nil, err } diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 25542d0ec936..ba66270b3fea 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -346,12 +346,12 @@ https://www.postgresql.org/docs/12/catalog-pg-attribute.html`, lookup simpleSchemaResolver, addRow func(...tree.Datum) error) error { // addColumn adds adds either a table or a index column to the pg_attribute table. - addColumn := func(column *descpb.ColumnDescriptor, attRelID tree.Datum, attNum uint32) error { - colTyp := column.Type + addColumn := func(column catalog.Column, attRelID tree.Datum, attNum uint32) error { + colTyp := column.GetType() // Sets the attgenerated column to 's' if the column is generated/ // computed stored, "v" if virtual, zero byte otherwise. var isColumnComputed string - if column.IsComputed() && !column.Virtual { + if column.IsComputed() && !column.IsVirtual() { isColumnComputed = "s" } else if column.IsComputed() { isColumnComputed = "v" @@ -360,7 +360,7 @@ https://www.postgresql.org/docs/12/catalog-pg-attribute.html`, } return addRow( attRelID, // attrelid - tree.NewDName(column.Name), // attname + tree.NewDName(column.GetName()), // attname typOid(colTyp), // atttypid zeroVal, // attstattarget typLen(colTyp), // attlen @@ -371,17 +371,17 @@ https://www.postgresql.org/docs/12/catalog-pg-attribute.html`, tree.DNull, // attbyval (see pg_type.typbyval) tree.DNull, // attstorage tree.DNull, // attalign - tree.MakeDBool(tree.DBool(!column.Nullable)), // attnotnull - tree.MakeDBool(tree.DBool(column.DefaultExpr != nil)), // atthasdef - tree.NewDString(""), // attidentity - tree.NewDString(isColumnComputed), // attgenerated - tree.DBoolFalse, // attisdropped - tree.DBoolTrue, // attislocal - zeroVal, // attinhcount - typColl(colTyp, h), // attcollation - tree.DNull, // attacl - tree.DNull, // attoptions - tree.DNull, // attfdwoptions + tree.MakeDBool(tree.DBool(!column.IsNullable())), // attnotnull + tree.MakeDBool(tree.DBool(column.HasDefault())), // atthasdef + tree.NewDString(""), // attidentity + tree.NewDString(isColumnComputed), // attgenerated + tree.DBoolFalse, // attisdropped + tree.DBoolTrue, // attislocal + zeroVal, // attinhcount + typColl(colTyp, h), // attcollation + tree.DNull, // attacl + tree.DNull, // attoptions + tree.DNull, // attfdwoptions // These columns were automatically created by pg_catalog_test's missing column generator. tree.DNull, // atthasmissing ) @@ -390,7 +390,7 @@ https://www.postgresql.org/docs/12/catalog-pg-attribute.html`, // Columns for table. for _, column := range table.PublicColumns() { tableID := tableOid(table.GetID()) - if err := addColumn(column.ColumnDesc(), tableID, column.GetPGAttributeNum()); err != nil { + if err := addColumn(column, tableID, column.GetPGAttributeNum()); err != nil { return err } } @@ -402,7 +402,7 @@ https://www.postgresql.org/docs/12/catalog-pg-attribute.html`, colID := index.GetColumnID(i) idxID := h.IndexOid(table.GetID(), index.GetID()) column := table.PublicColumns()[columnIdxMap.GetDefault(colID)] - if err := addColumn(column.ColumnDesc(), idxID, column.GetPGAttributeNum()); err != nil { + if err := addColumn(column, idxID, column.GetPGAttributeNum()); err != nil { return err } } @@ -1479,7 +1479,9 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, } // indnkeyatts is the number of attributes without INCLUDED columns. indnkeyatts := len(colIDs) - colIDs = append(colIDs, index.IndexDesc().StoreColumnIDs...) + for i := 0; i < index.NumStoredColumns(); i++ { + colIDs = append(colIDs, index.GetStoredColumnID(i)) + } // indnatts is the number of attributes with INCLUDED columns. indnatts := len(colIDs) indkey, err := colIDArrayToVector(colIDs) @@ -1532,7 +1534,7 @@ https://www.postgresql.org/docs/9.5/view-pg-indexes.html`, scNameName := tree.NewDName(scName) tblName := tree.NewDName(table.GetName()) return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { - def, err := indexDefFromDescriptor(ctx, p, db, scName, table, index.IndexDesc(), tableLookup) + def, err := indexDefFromDescriptor(ctx, p, db, scName, table, index, tableLookup) if err != nil { return err } @@ -1558,33 +1560,34 @@ func indexDefFromDescriptor( db catalog.DatabaseDescriptor, schemaName string, table catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, tableLookup tableLookupFn, ) (string, error) { - colNames := index.ColumnNames[index.ExplicitColumnStartIdx():] + colNames := index.IndexDesc().ColumnNames[index.ExplicitColumnStartIdx():] indexDef := tree.CreateIndex{ - Name: tree.Name(index.Name), + Name: tree.Name(index.GetName()), Table: tree.MakeTableNameWithSchema(tree.Name(db.GetName()), tree.Name(schemaName), tree.Name(table.GetName())), - Unique: index.Unique, + Unique: index.IsUnique(), Columns: make(tree.IndexElemList, len(colNames)), - Storing: make(tree.NameList, len(index.StoreColumnNames)), - Inverted: index.Type == descpb.IndexDescriptor_INVERTED, + Storing: make(tree.NameList, index.NumStoredColumns()), + Inverted: index.GetType() == descpb.IndexDescriptor_INVERTED, } for i, name := range colNames { elem := tree.IndexElem{ Column: tree.Name(name), Direction: tree.Ascending, } - if index.ColumnDirections[index.ExplicitColumnStartIdx()+i] == descpb.IndexDescriptor_DESC { + if index.GetColumnDirection(index.ExplicitColumnStartIdx()+i) == descpb.IndexDescriptor_DESC { elem.Direction = tree.Descending } indexDef.Columns[i] = elem } - for i, name := range index.StoreColumnNames { + for i := 0; i < index.NumStoredColumns(); i++ { + name := index.GetStoredColumnName(i) indexDef.Storing[i] = tree.Name(name) } - if len(index.Interleave.Ancestors) > 0 { - intl := index.Interleave + if index.NumInterleaveAncestors() > 0 { + intl := index.IndexDesc().Interleave parentTable, err := tableLookup.getTableByID(intl.Ancestors[len(intl.Ancestors)-1].TableID) if err != nil { return "", err @@ -1617,7 +1620,7 @@ func indexDefFromDescriptor( // // TODO(mgartner): Avoid parsing the predicate expression twice. It is // parsed in schemaexpr.FormatExprForDisplay and again here. - formattedPred, err := schemaexpr.FormatExprForDisplay(ctx, table, index.Predicate, p.SemaCtx(), tree.FmtPGCatalog) + formattedPred, err := schemaexpr.FormatExprForDisplay(ctx, table, index.GetPredicate(), p.SemaCtx(), tree.FmtPGCatalog) if err != nil { return "", err } diff --git a/pkg/sql/pgwire/testdata/pgtest/notice b/pkg/sql/pgwire/testdata/pgtest/notice index 30d967ea4267..6feffb5c5343 100644 --- a/pkg/sql/pgwire/testdata/pgtest/notice +++ b/pkg/sql/pgwire/testdata/pgtest/notice @@ -55,7 +55,7 @@ Query {"String": "DROP INDEX t_x_idx"} until crdb_only CommandComplete ---- -{"Severity":"NOTICE","SeverityUnlocalized":"","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":519,"Routine":"dropIndexByName","UnknownFields":null} +{"Severity":"NOTICE","SeverityUnlocalized":"","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":518,"Routine":"dropIndexByName","UnknownFields":null} {"Type":"CommandComplete","CommandTag":"DROP INDEX"} until noncrdb_only diff --git a/pkg/sql/physicalplan/fake_span_resolver_test.go b/pkg/sql/physicalplan/fake_span_resolver_test.go index 8f90519a72f5..016902f142eb 100644 --- a/pkg/sql/physicalplan/fake_span_resolver_test.go +++ b/pkg/sql/physicalplan/fake_span_resolver_test.go @@ -57,7 +57,7 @@ func TestFakeSpanResolver(t *testing.T) { it := resolver.NewSpanResolverIterator(txn) tableDesc := catalogkv.TestingGetTableDescriptor(db, keys.SystemSQLCodec, "test", "t") - primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.GetPrimaryIndex().IndexDesc()) + primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.GetPrimaryIndex()) span := tableDesc.PrimaryIndexSpan(keys.SystemSQLCodec) diff --git a/pkg/sql/randgen/mutator.go b/pkg/sql/randgen/mutator.go index bd7890cd11fb..d9d7cbb3ddc9 100644 --- a/pkg/sql/randgen/mutator.go +++ b/pkg/sql/randgen/mutator.go @@ -357,15 +357,9 @@ func encodeInvertedIndexHistogramUpperBounds(colType *types.T, val tree.Datum) ( var err error switch colType.Family() { case types.GeometryFamily: - tempIdx := descpb.IndexDescriptor{ - GeoConfig: *geoindex.DefaultGeometryIndexConfig(), - } - keys, err = rowenc.EncodeGeoInvertedIndexTableKeys(val, nil, &tempIdx) + keys, err = rowenc.EncodeGeoInvertedIndexTableKeys(val, nil, *geoindex.DefaultGeometryIndexConfig()) case types.GeographyFamily: - tempIdx := descpb.IndexDescriptor{ - GeoConfig: *geoindex.DefaultGeographyIndexConfig(), - } - keys, err = rowenc.EncodeGeoInvertedIndexTableKeys(val, nil, &tempIdx) + keys, err = rowenc.EncodeGeoInvertedIndexTableKeys(val, nil, *geoindex.DefaultGeographyIndexConfig()) default: keys, err = rowenc.EncodeInvertedIndexTableKeys(val, nil, descpb.EmptyArraysInInvertedIndexesVersion) } diff --git a/pkg/sql/randgen/schema.go b/pkg/sql/randgen/schema.go index 3b4af8f11eee..956165a94b70 100644 --- a/pkg/sql/randgen/schema.go +++ b/pkg/sql/randgen/schema.go @@ -569,7 +569,7 @@ func TestingMakePrimaryIndexKeyForTenant( } keyPrefix := rowenc.MakeIndexKeyPrefix(codec, desc, index.GetID()) - key, _, err := rowenc.EncodeIndexKey(desc, index.IndexDesc(), colIDToRowIndex, datums, keyPrefix) + key, _, err := rowenc.EncodeIndexKey(desc, index, colIDToRowIndex, datums, keyPrefix) if err != nil { return nil, err } diff --git a/pkg/sql/relocate.go b/pkg/sql/relocate.go index 630eb0547298..7720db1941f2 100644 --- a/pkg/sql/relocate.go +++ b/pkg/sql/relocate.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -32,7 +31,7 @@ type relocateNode struct { relocateLease bool relocateNonVoters bool tableDesc catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index rows planNode run relocateRun diff --git a/pkg/sql/rename_column.go b/pkg/sql/rename_column.go index 6a9fb54069b6..08fa69a252cf 100644 --- a/pkg/sql/rename_column.go +++ b/pkg/sql/rename_column.go @@ -122,7 +122,7 @@ func (p *planner) renameColumn( // Noop. return false, nil } - isShardColumn := tableDesc.IsShardColumn(col.ColumnDesc()) + isShardColumn := tableDesc.IsShardColumn(col) if isShardColumn && !allowRenameOfShardColumn { return false, pgerror.Newf(pgcode.ReservedName, "cannot rename shard column") } @@ -161,9 +161,7 @@ func (p *planner) renameColumn( if err != nil { return false, err } - indexDesc := *index.IndexDesc() - indexDesc.Predicate = newExpr - tableDesc.SetPublicNonPrimaryIndex(index.Ordinal(), indexDesc) + index.IndexDesc().Predicate = newExpr } } @@ -232,7 +230,7 @@ func (p *planner) renameColumn( } // Rename the column in the indexes. - tableDesc.RenameColumnDescriptor(col.ColumnDesc(), string(*newName)) + tableDesc.RenameColumnDescriptor(col, string(*newName)) // Rename any shard columns which need to be renamed because their name was // based on this column. diff --git a/pkg/sql/rename_index.go b/pkg/sql/rename_index.go index 911ade726d63..9a1af5c76fa8 100644 --- a/pkg/sql/rename_index.go +++ b/pkg/sql/rename_index.go @@ -13,6 +13,7 @@ package sql import ( "context" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -26,7 +27,7 @@ var errEmptyIndexName = pgerror.New(pgcode.Syntax, "empty index name") type renameIndexNode struct { n *tree.RenameIndex tableDesc *tabledesc.Mutable - idx *descpb.IndexDescriptor + idx catalog.Index } // RenameIndex renames the index. @@ -65,7 +66,7 @@ func (p *planner) RenameIndex(ctx context.Context, n *tree.RenameIndex) (planNod return nil, err } - return &renameIndexNode{n: n, idx: idx.IndexDesc(), tableDesc: tableDesc}, nil + return &renameIndexNode{n: n, idx: idx, tableDesc: tableDesc}, nil } // ReadingOwnWrites implements the planNodeReadingOwnWrites interface. @@ -80,7 +81,7 @@ func (n *renameIndexNode) startExec(params runParams) error { idx := n.idx for _, tableRef := range tableDesc.DependedOnBy { - if tableRef.IndexID != idx.ID { + if tableRef.IndexID != idx.GetID() { continue } return p.dependentViewError( @@ -97,13 +98,11 @@ func (n *renameIndexNode) startExec(params runParams) error { return nil } - if _, err := tableDesc.FindIndexWithName(string(n.n.NewName)); err == nil { + if foundIndex, _ := tableDesc.FindIndexWithName(string(n.n.NewName)); foundIndex != nil { return pgerror.Newf(pgcode.DuplicateRelation, "index name %q already exists", string(n.n.NewName)) } - if err := tableDesc.RenameIndexDescriptor(idx, string(n.n.NewName)); err != nil { - return err - } + idx.IndexDesc().Name = string(n.n.NewName) if err := validateDescriptor(ctx, p, tableDesc); err != nil { return err diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index cada48f379fe..131e551349a2 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -752,7 +752,7 @@ func expandIndexName( // It can return indexes that are being rolled out. func (p *planner) getTableAndIndex( ctx context.Context, tableWithIndex *tree.TableIndexName, privilege privilege.Kind, -) (*tabledesc.Mutable, *descpb.IndexDescriptor, error) { +) (*tabledesc.Mutable, catalog.Index, error) { var catalog optCatalog catalog.init(p) catalog.reset() @@ -774,7 +774,7 @@ func (p *planner) getTableAndIndex( tableWithIndex.Table = tree.MakeTableNameFromPrefix(qualifiedName.ObjectNamePrefix, qualifiedName.ObjectName) } - return tabledesc.NewBuilder(optIdx.tab.desc.TableDesc()).BuildExistingMutableTable(), optIdx.desc, nil + return tabledesc.NewBuilder(optIdx.tab.desc.TableDesc()).BuildExistingMutableTable(), optIdx.idx, nil } // expandTableGlob expands pattern into a list of objects represented diff --git a/pkg/sql/row/deleter.go b/pkg/sql/row/deleter.go index 66037f0b512c..c492529d6d75 100644 --- a/pkg/sql/row/deleter.go +++ b/pkg/sql/row/deleter.go @@ -26,7 +26,7 @@ import ( // Deleter abstracts the key/value operations for deleting table rows. type Deleter struct { Helper rowHelper - FetchCols []descpb.ColumnDescriptor + FetchCols []catalog.Column // FetchColIDtoRowIndex must be kept in sync with FetchCols. FetchColIDtoRowIndex catalog.TableColMap // For allocation avoidance. @@ -41,15 +41,11 @@ type Deleter struct { // FetchCols; otherwise, all columns that are part of the key of any index // (either primary or secondary) are included in FetchCols. func MakeDeleter( - codec keys.SQLCodec, tableDesc catalog.TableDescriptor, requestedCols []descpb.ColumnDescriptor, + codec keys.SQLCodec, tableDesc catalog.TableDescriptor, requestedCols []catalog.Column, ) Deleter { indexes := tableDesc.DeletableNonPrimaryIndexes() - indexDescs := make([]descpb.IndexDescriptor, len(indexes)) - for i, index := range indexes { - indexDescs[i] = *index.IndexDesc() - } - var fetchCols []descpb.ColumnDescriptor + var fetchCols []catalog.Column var fetchColIDtoRowIndex catalog.TableColMap if requestedCols != nil { fetchCols = requestedCols[:len(requestedCols):len(requestedCols)] @@ -62,7 +58,7 @@ func MakeDeleter( return err } fetchColIDtoRowIndex.Set(col.GetID(), len(fetchCols)) - fetchCols = append(fetchCols, *col.ColumnDesc()) + fetchCols = append(fetchCols, col) } return nil } @@ -90,7 +86,7 @@ func MakeDeleter( } rd := Deleter{ - Helper: newRowHelper(codec, tableDesc, indexDescs), + Helper: newRowHelper(codec, tableDesc, indexes), FetchCols: fetchCols, FetchColIDtoRowIndex: fetchColIDtoRowIndex, } @@ -110,7 +106,7 @@ func (rd *Deleter) DeleteRow( for i := range rd.Helper.Indexes { // If the index ID exists in the set of indexes to ignore, do not // attempt to delete from the index. - if pm.IgnoreForDel.Contains(int(rd.Helper.Indexes[i].ID)) { + if pm.IgnoreForDel.Contains(int(rd.Helper.Indexes[i].GetID())) { continue } @@ -118,7 +114,7 @@ func (rd *Deleter) DeleteRow( entries, err := rowenc.EncodeSecondaryIndex( rd.Helper.Codec, rd.Helper.TableDesc, - &rd.Helper.Indexes[i], + rd.Helper.Indexes[i], rd.FetchColIDtoRowIndex, values, true, /* includeEmpty */ @@ -164,7 +160,7 @@ func (rd *Deleter) DeleteRow( // DeleteIndexRow adds to the batch the kv operations necessary to delete a // table row from the given index. func (rd *Deleter) DeleteIndexRow( - ctx context.Context, b *kv.Batch, idx *descpb.IndexDescriptor, values []tree.Datum, traceKV bool, + ctx context.Context, b *kv.Batch, idx catalog.Index, values []tree.Datum, traceKV bool, ) error { // We want to include empty k/v pairs because we want // to delete all k/v's for this row. By setting includeEmpty diff --git a/pkg/sql/row/errors.go b/pkg/sql/row/errors.go index f2edb2c5c2b4..ebfa67ea921b 100644 --- a/pkg/sql/row/errors.go +++ b/pkg/sql/row/errors.go @@ -101,8 +101,8 @@ func NewUniquenessConstraintViolationError( skipCols := index.ExplicitColumnStartIdx() return errors.WithDetail( pgerror.WithConstraintName(pgerror.Newf(pgcode.UniqueViolation, - "duplicate key value violates unique constraint %q", index.Name, - ), index.Name), + "duplicate key value violates unique constraint %q", index.GetName(), + ), index.GetName()), fmt.Sprintf( "Key (%s)=(%s) already exists.", strings.Join(names[skipCols:], ","), @@ -134,7 +134,7 @@ func NewLockNotAvailableError( strings.Join(colNames, ","), strings.Join(values, ","), tableDesc.GetName(), - index.Name) + index.GetName()) } // DecodeRowInfo takes a table descriptor, a key, and an optional value and @@ -146,7 +146,7 @@ func DecodeRowInfo( key roachpb.Key, value *roachpb.Value, allColumns bool, -) (_ *descpb.IndexDescriptor, columnNames []string, columnValues []string, _ error) { +) (_ catalog.Index, columnNames []string, columnValues []string, _ error) { // Strip the tenant prefix and pretend to use the system tenant's SQL codec // for the rest of this function. This is safe because the key is just used // to decode the corresponding datums and never escapes this function. @@ -188,19 +188,19 @@ func DecodeRowInfo( valNeededForCol.AddRange(0, len(colIDs)-1) var colIdxMap catalog.TableColMap - cols := make([]descpb.ColumnDescriptor, len(colIDs)) + cols := make([]catalog.Column, len(colIDs)) for i, colID := range colIDs { colIdxMap.Set(colID, i) col, err := tableDesc.FindColumnWithID(colID) if err != nil { return nil, nil, nil, err } - cols[i] = *col.ColumnDesc() + cols[i] = col } tableArgs := FetcherTableArgs{ Desc: tableDesc, - Index: index.IndexDesc(), + Index: index, ColIdxMap: colIdxMap, IsSecondaryIndex: indexID != tableDesc.GetPrimaryIndexID(), Cols: cols, @@ -237,13 +237,13 @@ func DecodeRowInfo( names := make([]string, len(cols)) values := make([]string, len(cols)) for i := range cols { - names[i] = cols[i].Name + names[i] = cols[i].GetName() if datums[i] == tree.DNull { continue } values[i] = datums[i].String() } - return index.IndexDesc(), names, values, nil + return index, names, values, nil } func (f *singleKVFetcher) close(context.Context) {} diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index 274bebb17ee2..3efeea503d7e 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -64,7 +64,7 @@ type tableInfo struct { // want to scan. spans roachpb.Spans desc catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index isSecondaryIndex bool indexColumnDirs []descpb.IndexDescriptor_Direction // equivSignature is an equivalence class for each unique table-index @@ -74,7 +74,7 @@ type tableInfo struct { // The table columns to use for fetching, possibly including ones currently in // schema changes. - cols []descpb.ColumnDescriptor + cols []catalog.Column // The set of ColumnIDs that are required. neededCols util.FastIntSet @@ -151,10 +151,10 @@ type FetcherTableArgs struct { // table. Spans roachpb.Spans Desc catalog.TableDescriptor - Index *descpb.IndexDescriptor + Index catalog.Index ColIdxMap catalog.TableColMap IsSecondaryIndex bool - Cols []descpb.ColumnDescriptor + Cols []catalog.Column // The indexes (0 to # of columns - 1) of the columns to return. ValNeededForCol util.FastIntSet } @@ -183,9 +183,9 @@ func (fta *FetcherTableArgs) InitCols( if withSystemColumns { cols = append(cols, desc.SystemColumns()...) } - fta.Cols = make([]descpb.ColumnDescriptor, len(cols)) + fta.Cols = make([]catalog.Column, len(cols)) for i, col := range cols { - fta.Cols[i] = *col.ColumnDesc() + fta.Cols[i] = col } } @@ -377,7 +377,7 @@ func (rf *Fetcher) Init( var err error if multipleTables { // We produce references to every signature's reference. - equivSignatures, err := rowenc.TableEquivSignatures(table.desc.TableDesc(), table.index) + equivSignatures, err := rowenc.TableEquivSignatures(table.desc, table.index) if err != nil { return err } @@ -408,13 +408,13 @@ func (rf *Fetcher) Init( // Scan through the entire columns map to see which columns are // required. for _, col := range table.cols { - idx := table.colIdxMap.GetDefault(col.ID) + idx := table.colIdxMap.GetDefault(col.GetID()) if tableArgs.ValNeededForCol.Contains(idx) { // The idx-th column is required. - table.neededCols.Add(int(col.ID)) + table.neededCols.Add(int(col.GetID())) // Set up any system column metadata, if this column is a system column. - switch colinfo.GetSystemColumnKindFromColumnID(col.ID) { + switch colinfo.GetSystemColumnKindFromColumnID(col.GetID()) { case descpb.SystemColumnKind_MVCCTIMESTAMP: table.timestampOutputIdx = idx rf.mvccDecodeStrategy = MVCCDecodingRequired @@ -426,11 +426,11 @@ func (rf *Fetcher) Init( } table.knownPrefixLength = len( - rowenc.MakeIndexKeyPrefix(codec, table.desc, table.index.ID), + rowenc.MakeIndexKeyPrefix(codec, table.desc, table.index.GetID()), ) var indexColumnIDs []descpb.ColumnID - indexColumnIDs, table.indexColumnDirs = table.index.FullColumnIDs() + indexColumnIDs, table.indexColumnDirs = catalog.FullIndexColumnIDs(table.index) table.neededValueColsByIdx = tableArgs.ValNeededForCol.Copy() neededIndexCols := 0 @@ -475,19 +475,19 @@ func (rf *Fetcher) Init( // - If there are needed columns from the index key, we need to read it. // // Otherwise, we can completely avoid decoding the index key. - if !rf.mustDecodeIndexKey && (neededIndexCols > 0 || len(table.index.InterleavedBy) > 0 || len(table.index.Interleave.Ancestors) > 0) { + if !rf.mustDecodeIndexKey && (neededIndexCols > 0 || table.index.NumInterleavedBy() > 0 || table.index.NumInterleaveAncestors() > 0) { rf.mustDecodeIndexKey = true } // The number of columns we need to read from the value part of the key. // It's the total number of needed columns minus the ones we read from the // index key, except for composite columns. - table.neededValueCols = table.neededCols.Len() - neededIndexCols + len(table.index.CompositeColumnIDs) + table.neededValueCols = table.neededCols.Len() - neededIndexCols + table.index.NumCompositeColumns() if table.isSecondaryIndex { for i := range table.cols { - if table.neededCols.Contains(int(table.cols[i].ID)) && !table.index.ContainsColumnID(table.cols[i].ID) { - return errors.Errorf("requested column %s not in index", table.cols[i].Name) + if table.neededCols.Contains(int(table.cols[i].GetID())) && !table.index.ContainsColumnID(table.cols[i].GetID()) { + return errors.Errorf("requested column %s not in index", table.cols[i].GetName()) } } } @@ -509,8 +509,8 @@ func (rf *Fetcher) Init( // Primary indexes only contain ascendingly-encoded // values. If this ever changes, we'll probably have to // figure out the directions here too. - table.extraTypes, err = colinfo.GetColumnTypes(table.desc, table.index.ExtraColumnIDs, table.extraTypes) - nExtraColumns := len(table.index.ExtraColumnIDs) + table.extraTypes, err = colinfo.GetColumnTypes(table.desc, table.index.IndexDesc().ExtraColumnIDs, table.extraTypes) + nExtraColumns := table.index.NumExtraColumns() if cap(table.extraVals) >= nExtraColumns { table.extraVals = table.extraVals[:nExtraColumns] } else { @@ -523,7 +523,7 @@ func (rf *Fetcher) Init( // Keep track of the maximum keys per row to accommodate a // limitHint when StartScan is invoked. - keysPerRow, err := table.desc.KeysPerRow(table.index.ID) + keysPerRow, err := table.desc.KeysPerRow(table.index.GetID()) if err != nil { return err } @@ -798,8 +798,8 @@ func (rf *Fetcher) NextKey(ctx context.Context) (rowDone bool, err error) { // them when processing the index. The difference with unique secondary indexes // is that the extra columns are not always there, and are used to unique-ify // the index key, rather than provide the primary key column values. - if foundNull && rf.currentTable.isSecondaryIndex && rf.currentTable.index.Unique && len(rf.currentTable.desc.GetFamilies()) != 1 { - for range rf.currentTable.index.ExtraColumnIDs { + if foundNull && rf.currentTable.isSecondaryIndex && rf.currentTable.index.IsUnique() && len(rf.currentTable.desc.GetFamilies()) != 1 { + for i := 0; i < rf.currentTable.index.NumExtraColumns(); i++ { var err error // Slice off an extra encoded column from rf.keyRemainingBytes. rf.keyRemainingBytes, err = rowenc.SkipTableKey(rf.keyRemainingBytes) @@ -960,7 +960,7 @@ func (rf *Fetcher) processKV( prettyKey = fmt.Sprintf( "/%s/%s%s", table.desc.GetName(), - table.index.Name, + table.index.GetName(), rf.prettyEncDatums(table.keyValTypes, table.keyVals), ) } @@ -1015,7 +1015,7 @@ func (rf *Fetcher) processKV( } // For covering secondary indexes, allow for decoding as a primary key. - if table.index.GetEncodingType(table.desc.GetPrimaryIndexID()) == descpb.PrimaryIndexEncoding && + if table.index.GetEncodingType() == descpb.PrimaryIndexEncoding && len(rf.keyRemainingBytes) > 0 { // If familyID is 0, kv.Value contains values for composite key columns. // These columns already have a table.row value assigned above, but that value @@ -1076,7 +1076,8 @@ func (rf *Fetcher) processKV( if err != nil { return "", "", scrub.WrapError(scrub.SecondaryIndexKeyExtraValueDecodingError, err) } - for i, id := range table.index.ExtraColumnIDs { + for i := 0; i < table.index.NumExtraColumns(); i++ { + id := table.index.GetExtraColumnID(i) if table.neededCols.Contains(int(id)) { table.row[table.colIdxMap.GetDefault(id)] = table.extraVals[i] } @@ -1148,7 +1149,7 @@ func (rf *Fetcher) processValueSingle( if len(kv.Value.RawBytes) == 0 { return prettyKey, "", nil } - typ := table.cols[idx].Type + typ := table.cols[idx].GetType() // TODO(arjun): The value is a directly marshaled single value, so we // unmarshal it eagerly here. This can potentially be optimized out, // although that would require changing UnmarshalColumnValue to operate @@ -1228,7 +1229,7 @@ func (rf *Fetcher) processValueBytes( return "", "", err } if rf.traceKV { - err := encValue.EnsureDecoded(table.cols[idx].Type, rf.alloc) + err := encValue.EnsureDecoded(table.cols[idx].GetType(), rf.alloc) if err != nil { return "", "", err } @@ -1268,12 +1269,7 @@ func (rf *Fetcher) processValueTuple( // (relevant when more than one table is specified during initialization). func (rf *Fetcher) NextRow( ctx context.Context, -) ( - row rowenc.EncDatumRow, - table catalog.TableDescriptor, - index *descpb.IndexDescriptor, - err error, -) { +) (row rowenc.EncDatumRow, table catalog.TableDescriptor, index catalog.Index, err error) { if rf.kvEnd { return nil, nil, nil, nil } @@ -1314,7 +1310,7 @@ func (rf *Fetcher) NextRow( // (relevant when more than one table is specified during initialization). func (rf *Fetcher) NextRowDecoded( ctx context.Context, -) (datums tree.Datums, table catalog.TableDescriptor, index *descpb.IndexDescriptor, err error) { +) (datums tree.Datums, table catalog.TableDescriptor, index catalog.Index, err error) { row, table, index, err := rf.NextRow(ctx) if err != nil { err = scrub.UnwrapScrubError(err) @@ -1329,7 +1325,7 @@ func (rf *Fetcher) NextRowDecoded( rf.rowReadyTable.decodedRow[i] = tree.DNull continue } - if err := encDatum.EnsureDecoded(rf.rowReadyTable.cols[i].Type, rf.alloc); err != nil { + if err := encDatum.EnsureDecoded(rf.rowReadyTable.cols[i].GetType(), rf.alloc); err != nil { return nil, nil, nil, err } rf.rowReadyTable.decodedRow[i] = encDatum.Datum @@ -1383,13 +1379,13 @@ func (rf *Fetcher) NextRowWithErrors(ctx context.Context) (rowenc.EncDatumRow, e rf.rowReadyTable.decodedRow[i] = tree.DNull continue } - if err := row[i].EnsureDecoded(rf.rowReadyTable.cols[i].Type, rf.alloc); err != nil { + if err := row[i].EnsureDecoded(rf.rowReadyTable.cols[i].GetType(), rf.alloc); err != nil { return nil, err } rf.rowReadyTable.decodedRow[i] = row[i].Datum } - if index.ID == table.GetPrimaryIndexID() { + if index.GetID() == table.GetPrimaryIndexID() { err = rf.checkPrimaryIndexDatumEncodings(ctx) } else { err = rf.checkSecondaryIndexDatumEncodings(ctx) @@ -1414,12 +1410,7 @@ func (rf *Fetcher) checkPrimaryIndexDatumEncodings(ctx context.Context) error { colIDToColumn[col.GetID()] = col } - indexes := make([]descpb.IndexDescriptor, len(table.desc.PublicNonPrimaryIndexes())) - for i, idx := range table.desc.PublicNonPrimaryIndexes() { - indexes[i] = *idx.IndexDesc() - } - - rh := rowHelper{TableDesc: table.desc, Indexes: indexes} + rh := rowHelper{TableDesc: table.desc, Indexes: table.desc.PublicNonPrimaryIndexes()} return table.desc.ForeachFamily(func(family *descpb.ColumnFamilyDescriptor) error { var lastColID descpb.ColumnID @@ -1475,7 +1466,7 @@ func (rf *Fetcher) checkSecondaryIndexDatumEncodings(ctx context.Context) error colToEncDatum := make(map[descpb.ColumnID]rowenc.EncDatum, len(table.row)) values := make(tree.Datums, len(table.row)) for i, col := range table.cols { - colToEncDatum[col.ID] = table.row[i] + colToEncDatum[col.GetID()] = table.row[i] values[i] = table.row[i].Datum } @@ -1519,10 +1510,11 @@ func (rf *Fetcher) checkKeyOrdering(ctx context.Context) error { // previous row in that column. When the first column with a differing value // is found, compare the values to ensure the ordering matches the column // ordering. - for i, id := range rf.rowReadyTable.index.ColumnIDs { + for i := 0; i < rf.rowReadyTable.index.NumColumns(); i++ { + id := rf.rowReadyTable.index.GetColumnID(i) idx := rf.rowReadyTable.colIdxMap.GetDefault(id) result := rf.rowReadyTable.decodedRow[idx].Compare(&evalCtx, rf.rowReadyTable.lastDatums[idx]) - expectedDirection := rf.rowReadyTable.index.ColumnDirections[i] + expectedDirection := rf.rowReadyTable.index.GetColumnDirection(i) if rf.reverse && expectedDirection == descpb.IndexDescriptor_ASC { expectedDirection = descpb.IndexDescriptor_DESC } else if rf.reverse && expectedDirection == descpb.IndexDescriptor_DESC { @@ -1565,22 +1557,22 @@ func (rf *Fetcher) finalizeRow() error { // Found all cols - done! return nil } - if table.neededCols.Contains(int(table.cols[i].ID)) && table.row[i].IsUnset() { + if table.neededCols.Contains(int(table.cols[i].GetID())) && table.row[i].IsUnset() { // If the row was deleted, we'll be missing any non-primary key // columns, including nullable ones, but this is expected. - if !table.cols[i].Nullable && !table.rowIsDeleted && !rf.IgnoreUnexpectedNulls { + if !table.cols[i].IsNullable() && !table.rowIsDeleted && !rf.IgnoreUnexpectedNulls { var indexColValues []string for _, idx := range table.indexColIdx { if idx != -1 { - indexColValues = append(indexColValues, table.row[idx].String(table.cols[idx].Type)) + indexColValues = append(indexColValues, table.row[idx].String(table.cols[idx].GetType())) } else { indexColValues = append(indexColValues, "?") } } err := errors.AssertionFailedf( "Non-nullable column \"%s:%s\" with no value! Index scanned was %q with the index key columns (%s) and the values (%s)", - table.desc.GetName(), table.cols[i].Name, table.index.Name, - strings.Join(table.index.ColumnNames, ","), strings.Join(indexColValues, ",")) + table.desc.GetName(), table.cols[i].GetName(), table.index.GetName(), + strings.Join(table.index.IndexDesc().ColumnNames, ","), strings.Join(indexColValues, ",")) if rf.isCheck { return scrub.WrapError(scrub.UnexpectedNullValueError, err) @@ -1629,7 +1621,7 @@ func (rf *Fetcher) GetBytesRead() int64 { // Only unique secondary indexes have extra columns to decode (namely the // primary index columns). func hasExtraCols(table *tableInfo) bool { - return table.isSecondaryIndex && table.index.Unique + return table.isSecondaryIndex && table.index.IsUnique() } // consumeIndexKeyWithoutTableIDIndexIDPrefix consumes an index key that's @@ -1641,11 +1633,12 @@ func hasExtraCols(table *tableInfo) bool { // would include the trailing table ID index ID pair, since that's a more // precise key: /Table/60/1/6/7/#/61/1. func consumeIndexKeyWithoutTableIDIndexIDPrefix( - index *descpb.IndexDescriptor, nCols int, key []byte, + index catalog.Index, nCols int, key []byte, ) (int, error) { origKeyLen := len(key) consumedCols := 0 - for _, ancestor := range index.Interleave.Ancestors { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + ancestor := index.GetInterleaveAncestor(i) length := int(ancestor.SharedPrefixLen) // Skip up to length values. for j := 0; j < length; j++ { diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index b75a7f933729..029e5a29d577 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -92,19 +92,17 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { for _, desc := range []catalog.TableDescriptor{parentDesc, childDesc} { var colIdxMap catalog.TableColMap var valNeededForCol util.FastIntSet - colDescs := make([]descpb.ColumnDescriptor, len(desc.PublicColumns())) for i, col := range desc.PublicColumns() { colIdxMap.Set(col.GetID(), i) valNeededForCol.Add(i) - colDescs[i] = *col.ColumnDesc() } args = append(args, row.FetcherTableArgs{ Spans: desc.AllIndexSpans(keys.SystemSQLCodec), Desc: desc, - Index: desc.GetPrimaryIndex().IndexDesc(), + Index: desc.GetPrimaryIndex(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, - Cols: colDescs, + Cols: desc.PublicColumns(), ValNeededForCol: valNeededForCol, }) } diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index 32783fa8efe9..17c0ee1a83ea 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -53,15 +53,12 @@ func makeFetcherArgs(entries []initFetcherArgs) []FetcherTableArgs { fetcherArgs[i] = FetcherTableArgs{ Spans: entry.spans, Desc: entry.tableDesc, - Index: index.IndexDesc(), + Index: index, ColIdxMap: catalog.ColumnIDToOrdinalMap(entry.tableDesc.PublicColumns()), IsSecondaryIndex: !index.Primary(), - Cols: make([]descpb.ColumnDescriptor, len(entry.tableDesc.PublicColumns())), + Cols: entry.tableDesc.PublicColumns(), ValNeededForCol: entry.valNeededForCol, } - for j, col := range entry.tableDesc.PublicColumns() { - fetcherArgs[i].Cols[j] = *col.ColumnDesc() - } } return fetcherArgs } @@ -195,11 +192,11 @@ func TestNextRowSingle(t *testing.T) { count++ - if desc.GetID() != tableDesc.GetID() || index.ID != tableDesc.GetPrimaryIndexID() { + if desc.GetID() != tableDesc.GetID() || index.GetID() != tableDesc.GetPrimaryIndexID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", tableDesc.GetName(), tableDesc.GetPrimaryIndex().GetName(), - desc.GetName(), index.Name, + desc.GetName(), index.GetName(), ) } @@ -316,11 +313,11 @@ func TestNextRowBatchLimiting(t *testing.T) { count++ - if desc.GetID() != tableDesc.GetID() || index.ID != tableDesc.GetPrimaryIndexID() { + if desc.GetID() != tableDesc.GetID() || index.GetID() != tableDesc.GetPrimaryIndexID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", tableDesc.GetName(), tableDesc.GetPrimaryIndex().GetName(), - desc.GetName(), index.Name, + desc.GetName(), index.GetName(), ) } @@ -676,11 +673,11 @@ func TestNextRowSecondaryIndex(t *testing.T) { count++ - if desc.GetID() != tableDesc.GetID() || index.ID != tableDesc.PublicNonPrimaryIndexes()[0].GetID() { + if desc.GetID() != tableDesc.GetID() || index.GetID() != tableDesc.PublicNonPrimaryIndexes()[0].GetID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", tableDesc.GetName(), tableDesc.PublicNonPrimaryIndexes()[0].GetName(), - desc.GetName(), index.Name, + desc.GetName(), index.GetName(), ) } @@ -1032,11 +1029,11 @@ func TestNextRowInterleaved(t *testing.T) { break } - entry, found := idLookups[idLookupKey(desc.GetID(), index.ID)] + entry, found := idLookups[idLookupKey(desc.GetID(), index.GetID())] if !found { t.Fatalf( "unexpected row from table %s - index %s", - desc.GetName(), index.Name, + desc.GetName(), index.GetName(), ) } diff --git a/pkg/sql/row/helper.go b/pkg/sql/row/helper.go index 8a83c055455b..8589d034fe88 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -29,7 +29,7 @@ type rowHelper struct { TableDesc catalog.TableDescriptor // Secondary indexes. - Indexes []descpb.IndexDescriptor + Indexes []catalog.Index indexEntries []rowenc.IndexEntry // Computed during initialization for pretty-printing. @@ -43,17 +43,17 @@ type rowHelper struct { } func newRowHelper( - codec keys.SQLCodec, desc catalog.TableDescriptor, indexes []descpb.IndexDescriptor, + codec keys.SQLCodec, desc catalog.TableDescriptor, indexes []catalog.Index, ) rowHelper { rh := rowHelper{Codec: codec, TableDesc: desc, Indexes: indexes} // Pre-compute the encoding directions of the index key values for // pretty-printing in traces. - rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.GetPrimaryIndex().IndexDesc()) + rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.GetPrimaryIndex()) rh.secIndexValDirs = make([][]encoding.Direction, len(rh.Indexes)) for i := range rh.Indexes { - rh.secIndexValDirs[i] = catalogkeys.IndexKeyValDirs(&rh.Indexes[i]) + rh.secIndexValDirs[i] = catalogkeys.IndexKeyValDirs(rh.Indexes[i]) } return rh @@ -89,7 +89,7 @@ func (rh *rowHelper) encodePrimaryIndex( rh.TableDesc.GetPrimaryIndexID()) } primaryIndexKey, _, err = rowenc.EncodeIndexKey( - rh.TableDesc, rh.TableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) + rh.TableDesc, rh.TableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) return primaryIndexKey, err } @@ -117,8 +117,8 @@ func (rh *rowHelper) encodeSecondaryIndexes( rh.indexEntries = rh.indexEntries[:0] for i := range rh.Indexes { - index := &rh.Indexes[i] - if !ignoreIndexes.Contains(int(index.ID)) { + index := rh.Indexes[i] + if !ignoreIndexes.Contains(int(index.GetID())) { entries, err := rowenc.EncodeSecondaryIndex(rh.Codec, rh.TableDesc, index, colIDtoRowIndex, values, includeEmpty) if err != nil { return nil, err diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index 9e66fa40a0af..61b641374c43 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -28,7 +27,7 @@ import ( // Inserter abstracts the key/value operations for inserting table rows. type Inserter struct { Helper rowHelper - InsertCols []descpb.ColumnDescriptor + InsertCols []catalog.Column InsertColIDtoRowIndex catalog.TableColMap // For allocation avoidance. @@ -47,17 +46,11 @@ func MakeInserter( txn *kv.Txn, codec keys.SQLCodec, tableDesc catalog.TableDescriptor, - insertCols []descpb.ColumnDescriptor, + insertCols []catalog.Column, alloc *rowenc.DatumAlloc, ) (Inserter, error) { - writableIndexes := tableDesc.WritableNonPrimaryIndexes() - writableIndexDescs := make([]descpb.IndexDescriptor, len(writableIndexes)) - for i, index := range writableIndexes { - writableIndexDescs[i] = *index.IndexDesc() - } - ri := Inserter{ - Helper: newRowHelper(codec, tableDesc, writableIndexDescs), + Helper: newRowHelper(codec, tableDesc, tableDesc.WritableNonPrimaryIndexes()), InsertCols: insertCols, InsertColIDtoRowIndex: ColIDtoRowIndexFromCols(insertCols), marshaled: make([]roachpb.Value, len(insertCols)), @@ -146,7 +139,7 @@ func (ri *Inserter) InsertRow( for i, val := range values { // Make sure the value can be written to the column before proceeding. var err error - if ri.marshaled[i], err = rowenc.MarshalColumnValue(&ri.InsertCols[i], val); err != nil { + if ri.marshaled[i], err = rowenc.MarshalColumnValue(ri.InsertCols[i], val); err != nil { return err } } diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index b7422675e637..4f1a76c45e0a 100644 --- a/pkg/sql/row/row_converter.go +++ b/pkg/sql/row/row_converter.go @@ -90,8 +90,8 @@ func (i KVInserter) InitPut(key, value interface{}, failOnTombstones bool) { func GenerateInsertRow( defaultExprs []tree.TypedExpr, computeExprs []tree.TypedExpr, - insertCols []descpb.ColumnDescriptor, - computedColsLookup []descpb.ColumnDescriptor, + insertCols []catalog.Column, + computedColsLookup []catalog.Column, evalCtx *tree.EvalContext, tableDesc catalog.TableDescriptor, rowVals tree.Datums, @@ -131,15 +131,16 @@ func GenerateInsertRow( // columns, all the columns which could possibly be referenced *are* // available. col := computedColsLookup[i] - computeIdx := rowContainerForComputedVals.Mapping.GetDefault(col.ID) + computeIdx := rowContainerForComputedVals.Mapping.GetDefault(col.GetID()) if !col.IsComputed() { continue } d, err := computeExprs[computeIdx].Eval(evalCtx) if err != nil { + name := col.GetName() return nil, errors.Wrapf(err, "computed column %s", - tree.ErrString((*tree.Name)(&col.Name))) + tree.ErrString((*tree.Name)(&name))) } rowVals[computeIdx] = d } @@ -172,7 +173,7 @@ func GenerateInsertRow( // Ensure that the values honor the specified column widths. for i := 0; i < len(insertCols); i++ { - outVal, err := tree.AdjustValueToType(insertCols[i].Type, rowVals[i]) + outVal, err := tree.AdjustValueToType(insertCols[i].GetType(), rowVals[i]) if err != nil { return nil, err } @@ -215,8 +216,8 @@ type DatumRowConverter struct { // The rest of these are derived from tableDesc, just cached here. ri Inserter EvalCtx *tree.EvalContext - cols []descpb.ColumnDescriptor - VisibleCols []descpb.ColumnDescriptor + cols []catalog.Column + VisibleCols []catalog.Column VisibleColTypes []*types.T computedExprs []tree.TypedExpr defaultCache []tree.TypedExpr @@ -247,12 +248,13 @@ func TestingSetDatumRowConverterBatchSize(newSize int) func() { // related to the sequence which will be used when evaluating the default // expression using the sequence. func (c *DatumRowConverter) getSequenceAnnotation( - evalCtx *tree.EvalContext, cols []descpb.ColumnDescriptor, + evalCtx *tree.EvalContext, cols []catalog.Column, ) (map[string]*SequenceMetadata, map[descpb.ID]*SequenceMetadata, error) { // Identify the sequences used in all the columns. sequenceIDs := make(map[descpb.ID]struct{}) for _, col := range cols { - for _, id := range col.UsesSequenceIds { + for i := 0; i < col.NumUsesSequences(); i++ { + id := col.GetUsesSequenceID(i) sequenceIDs[id] = struct{}{} } } @@ -319,21 +321,18 @@ func NewDatumRowConverter( targetCols = tableDesc.VisibleColumns() } - targetColDescriptors := make([]descpb.ColumnDescriptor, len(targetCols)) var targetColIDs catalog.TableColSet for i, col := range targetCols { c.TargetColOrds.Add(i) targetColIDs.Add(col.GetID()) - targetColDescriptors[i] = *col.ColumnDesc() } var txCtx transform.ExprTransformContext semaCtx := tree.MakeSemaContext() - relevantColumns := func(col *descpb.ColumnDescriptor) bool { + relevantColumns := func(col catalog.Column) bool { return col.HasDefault() || col.IsComputed() } - cols := schemaexpr.ProcessColumnSet( - targetColDescriptors, tableDesc, relevantColumns) + cols := schemaexpr.ProcessColumnSet(targetCols, tableDesc, relevantColumns) defaultExprs, err := schemaexpr.MakeDefaultExprs(ctx, cols, &txCtx, c.EvalCtx, &semaCtx) if err != nil { return nil, errors.Wrap(err, "process default and computed columns") @@ -354,13 +353,13 @@ func NewDatumRowConverter( c.ri = ri c.cols = cols - c.VisibleCols = targetColDescriptors + c.VisibleCols = targetCols c.VisibleColTypes = make([]*types.T, len(c.VisibleCols)) for i := range c.VisibleCols { - c.VisibleColTypes[i] = c.VisibleCols[i].Type + c.VisibleColTypes[i] = c.VisibleCols[i].GetType() } - c.Datums = make([]tree.Datum, len(targetColDescriptors), len(cols)) + c.Datums = make([]tree.Datum, len(targetCols), len(cols)) c.defaultCache = make([]tree.TypedExpr, len(cols)) annot := make(tree.Annotations, 1) @@ -383,12 +382,11 @@ func NewDatumRowConverter( // In addition, check for non-targeted columns with non-null DEFAULT expressions. // If the DEFAULT expression is immutable, we can store it in the cache so that it // doesn't have to be reevaluated for every row. - for i := range cols { - col := &cols[i] - if col.DefaultExpr != nil { + for i, col := range cols { + if col.HasDefault() { // Placeholder for columns with default values that will be evaluated when // each import row is being created. - typedExpr, volatile, err := sanitizeExprsForImport(ctx, c.EvalCtx, defaultExprs[i], col.Type) + typedExpr, volatile, err := sanitizeExprsForImport(ctx, c.EvalCtx, defaultExprs[i], col.GetType()) if err != nil { // This expression may not be safe for import but we don't want to // call the user out at this stage: targeted columns may not have @@ -409,11 +407,11 @@ func NewDatumRowConverter( } } } - if !targetColIDs.Contains(col.ID) { + if !targetColIDs.Contains(col.GetID()) { c.Datums = append(c.Datums, nil) } } - if col.IsComputed() && !targetColIDs.Contains(col.ID) { + if col.IsComputed() && !targetColIDs.Contains(col.GetID()) { c.Datums = append(c.Datums, nil) } } @@ -425,13 +423,11 @@ func NewDatumRowConverter( c.BatchCap = kvDatumRowConverterBatchSize + padding c.KvBatch.KVs = make([]roachpb.KeyValue, 0, c.BatchCap) - colDescs := make([]descpb.ColumnDescriptor, len(c.tableDesc.PublicColumns())) - colsOrdered := make([]descpb.ColumnDescriptor, len(cols)) - for i, col := range c.tableDesc.PublicColumns() { - colDescs[i] = *col.ColumnDesc() + colsOrdered := make([]catalog.Column, len(cols)) + for _, col := range c.tableDesc.PublicColumns() { // We prefer to have the order of columns that will be sent into // MakeComputedExprs to map that of Datums. - colsOrdered[ri.InsertColIDtoRowIndex.GetDefault(col.GetID())] = colDescs[i] + colsOrdered[ri.InsertColIDtoRowIndex.GetDefault(col.GetID())] = col } // Here, computeExprs will be nil if there's no computed column, or // the list of computed expressions (including nil, for those columns @@ -439,7 +435,7 @@ func NewDatumRowConverter( c.computedExprs, _, err = schemaexpr.MakeComputedExprs( ctx, colsOrdered, - colDescs, + c.tableDesc.PublicColumns(), c.tableDesc, tree.NewUnqualifiedTableName(tree.Name(c.tableDesc.GetName())), c.EvalCtx, @@ -450,10 +446,7 @@ func NewDatumRowConverter( c.computedIVarContainer = schemaexpr.RowIndexedVarContainer{ Mapping: ri.InsertColIDtoRowIndex, - Cols: make([]descpb.ColumnDescriptor, len(tableDesc.PublicColumns())), - } - for i, col := range tableDesc.PublicColumns() { - c.computedIVarContainer.Cols[i] = *col.ColumnDesc() + Cols: tableDesc.PublicColumns(), } return c, nil } @@ -464,9 +457,8 @@ const rowIDBits = 64 - builtins.NodeIDBits // if necessary. func (c *DatumRowConverter) Row(ctx context.Context, sourceID int32, rowIndex int64) error { getCellInfoAnnotation(c.EvalCtx.Annotations).reset(sourceID, rowIndex) - for i := range c.cols { - col := &c.cols[i] - if col.DefaultExpr != nil { + for i, col := range c.cols { + if col.HasDefault() { // If this column is targeted, then the evaluation is a no-op except to // make one evaluation just in case we have random() default expression // to ensure that the positions we advance in a row is the same as the @@ -477,20 +469,16 @@ func (c *DatumRowConverter) Row(ctx context.Context, sourceID int32, rowIndex in if !c.TargetColOrds.Contains(i) { if err != nil { return errors.Wrapf( - err, "error evaluating default expression %q", *col.DefaultExpr) + err, "error evaluating default expression %q", col.GetDefaultExpr()) } c.Datums[i] = datum } } } - var computedColsLookup []descpb.ColumnDescriptor + var computedColsLookup []catalog.Column if len(c.computedExprs) > 0 { - cols := c.tableDesc.PublicColumns() - computedColsLookup = make([]descpb.ColumnDescriptor, len(cols)) - for i, col := range cols { - computedColsLookup[i] = *col.ColumnDesc() - } + computedColsLookup = c.tableDesc.PublicColumns() } insertRow, err := GenerateInsertRow( diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index b2246e8958d9..c20935bf0b15 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -31,10 +31,10 @@ import ( type Updater struct { Helper rowHelper DeleteHelper *rowHelper - FetchCols []descpb.ColumnDescriptor + FetchCols []catalog.Column // FetchColIDtoRowIndex must be kept in sync with FetchCols. FetchColIDtoRowIndex catalog.TableColMap - UpdateCols []descpb.ColumnDescriptor + UpdateCols []catalog.Column UpdateColIDtoRowIndex catalog.TableColMap primaryKeyColChange bool @@ -85,8 +85,8 @@ func MakeUpdater( txn *kv.Txn, codec keys.SQLCodec, tableDesc catalog.TableDescriptor, - updateCols []descpb.ColumnDescriptor, - requestedCols []descpb.ColumnDescriptor, + updateCols []catalog.Column, + requestedCols []catalog.Column, updateType rowUpdaterType, alloc *rowenc.DatumAlloc, ) (Updater, error) { @@ -104,7 +104,7 @@ func MakeUpdater( var primaryKeyColChange bool for _, c := range updateCols { - if primaryIndexCols.Contains(c.ID) { + if primaryIndexCols.Contains(c.GetID()) { primaryKeyColChange = true break } @@ -141,20 +141,20 @@ func MakeUpdater( }) != nil } - includeIndexes := make([]descpb.IndexDescriptor, 0, len(tableDesc.WritableNonPrimaryIndexes())) - var deleteOnlyIndexes []descpb.IndexDescriptor + includeIndexes := make([]catalog.Index, 0, len(tableDesc.WritableNonPrimaryIndexes())) + var deleteOnlyIndexes []catalog.Index for _, index := range tableDesc.DeletableNonPrimaryIndexes() { if !needsUpdate(index) { continue } if !index.DeleteOnly() { - includeIndexes = append(includeIndexes, *index.IndexDesc()) + includeIndexes = append(includeIndexes, index) } else { if deleteOnlyIndexes == nil { // Allocate at most once. - deleteOnlyIndexes = make([]descpb.IndexDescriptor, 0, len(tableDesc.DeleteOnlyNonPrimaryIndexes())) + deleteOnlyIndexes = make([]catalog.Index, 0, len(tableDesc.DeleteOnlyNonPrimaryIndexes())) } - deleteOnlyIndexes = append(deleteOnlyIndexes, *index.IndexDesc()) + deleteOnlyIndexes = append(deleteOnlyIndexes, index) } } @@ -243,7 +243,7 @@ func (ru *Updater) UpdateRow( // happen before index encoding because certain datum types (i.e. tuple) // cannot be used as index values. for i, val := range updateValues { - if ru.marshaled[i], err = rowenc.MarshalColumnValue(&ru.UpdateCols[i], val); err != nil { + if ru.marshaled[i], err = rowenc.MarshalColumnValue(ru.UpdateCols[i], val); err != nil { return nil, err } } @@ -251,7 +251,7 @@ func (ru *Updater) UpdateRow( // Update the row values. copy(ru.newValues, oldValues) for i, updateCol := range ru.UpdateCols { - idx, ok := ru.FetchColIDtoRowIndex.Get(updateCol.ID) + idx, ok := ru.FetchColIDtoRowIndex.Get(updateCol.GetID()) if !ok { return nil, errors.AssertionFailedf("update column without a corresponding fetch column") } @@ -269,8 +269,7 @@ func (ru *Updater) UpdateRow( rowPrimaryKeyChanged = !bytes.Equal(primaryIndexKey, newPrimaryIndexKey) } - for i := range ru.Helper.Indexes { - index := &ru.Helper.Indexes[i] + for i, index := range ru.Helper.Indexes { // We don't want to insert any empty k/v's, so set includeEmpty to false. // Consider the following case: // TABLE t ( @@ -289,7 +288,7 @@ func (ru *Updater) UpdateRow( // exists in ignoreIndexesForDel and ignoreIndexesForPut, respectively. // Index IDs in these sets indicate that old and new values for the row // do not satisfy a partial index's predicate expression. - if pm.IgnoreForDel.Contains(int(index.ID)) { + if pm.IgnoreForDel.Contains(int(index.GetID())) { ru.oldIndexEntries[i] = nil } else { ru.oldIndexEntries[i], err = rowenc.EncodeSecondaryIndex( @@ -304,7 +303,7 @@ func (ru *Updater) UpdateRow( return nil, err } } - if pm.IgnoreForPut.Contains(int(index.ID)) { + if pm.IgnoreForPut.Contains(int(index.GetID())) { ru.newIndexEntries[i] = nil } else { ru.newIndexEntries[i], err = rowenc.EncodeSecondaryIndex( @@ -319,7 +318,7 @@ func (ru *Updater) UpdateRow( return nil, err } } - if ru.Helper.Indexes[i].Type == descpb.IndexDescriptor_INVERTED { + if ru.Helper.Indexes[i].GetType() == descpb.IndexDescriptor_INVERTED { // Deduplicate the keys we're adding and removing if we're updating an // inverted index. For example, imagine a table with an inverted index on j: // @@ -381,9 +380,8 @@ func (ru *Updater) UpdateRow( // Update secondary indexes. // We're iterating through all of the indexes, which should have corresponding entries // in the new and old values. - for i := range ru.Helper.Indexes { - index := &ru.Helper.Indexes[i] - if index.Type == descpb.IndexDescriptor_FORWARD { + for i, index := range ru.Helper.Indexes { + if index.GetType() == descpb.IndexDescriptor_FORWARD { oldIdx, newIdx := 0, 0 oldEntries, newEntries := ru.oldIndexEntries[i], ru.newIndexEntries[i] // The index entries for a particular index are stored in @@ -432,7 +430,7 @@ func (ru *Updater) UpdateRow( if oldEntry.Family == descpb.FamilyID(0) { return nil, errors.AssertionFailedf( "index entry for family 0 for table %s, index %s was not generated", - ru.Helper.TableDesc.GetName(), index.Name, + ru.Helper.TableDesc.GetName(), index.GetName(), ) } // In this case, the index has a k/v for a family that does not exist in @@ -446,7 +444,7 @@ func (ru *Updater) UpdateRow( if newEntry.Family == descpb.FamilyID(0) { return nil, errors.AssertionFailedf( "index entry for family 0 for table %s, index %s was not generated", - ru.Helper.TableDesc.GetName(), index.Name, + ru.Helper.TableDesc.GetName(), index.GetName(), ) } // In this case, the index now has a k/v that did not exist in the diff --git a/pkg/sql/row/writer.go b/pkg/sql/row/writer.go index 66757a5a1d45..f3cb820dc909 100644 --- a/pkg/sql/row/writer.go +++ b/pkg/sql/row/writer.go @@ -29,10 +29,10 @@ import ( // ColIDtoRowIndexFromCols groups a slice of ColumnDescriptors by their ID // field, returning a map from ID to the index of the column in the input slice. // It assumes there are no duplicate descriptors in the input. -func ColIDtoRowIndexFromCols(cols []descpb.ColumnDescriptor) catalog.TableColMap { +func ColIDtoRowIndexFromCols(cols []catalog.Column) catalog.TableColMap { var colIDtoRowIndex catalog.TableColMap for i := range cols { - colIDtoRowIndex.Set(cols[i].ID, i) + colIDtoRowIndex.Set(cols[i].GetID(), i) } return colIDtoRowIndex } @@ -42,11 +42,11 @@ func ColIDtoRowIndexFromCols(cols []descpb.ColumnDescriptor) catalog.TableColMap // // result[i] = j such that fromCols[i].ID == toCols[j].ID, or // -1 if the column is not part of toCols. -func ColMapping(fromCols, toCols []descpb.ColumnDescriptor) []int { +func ColMapping(fromCols, toCols []catalog.Column) []int { // colMap is a map from ColumnID to ordinal into fromCols. var colMap util.FastIntMap for i := range fromCols { - colMap.Set(int(fromCols[i].ID), i) + colMap.Set(int(fromCols[i].GetID()), i) } result := make([]int, len(fromCols)) @@ -57,7 +57,7 @@ func ColMapping(fromCols, toCols []descpb.ColumnDescriptor) []int { // Set the appropriate index values for the returning columns. for toOrd := range toCols { - if fromOrd, ok := colMap.Get(int(toCols[toOrd].ID)); ok { + if fromOrd, ok := colMap.Get(int(toCols[toOrd].GetID())); ok { result[fromOrd] = toOrd } } @@ -94,7 +94,7 @@ func prepareInsertOrUpdateBatch( batch putter, helper *rowHelper, primaryIndexKey []byte, - fetchedCols []descpb.ColumnDescriptor, + fetchedCols []catalog.Column, values []tree.Datum, valColIDMapping catalog.TableColMap, marshaledValues []roachpb.Value, @@ -180,12 +180,12 @@ func prepareInsertOrUpdateBatch( continue } - col := &fetchedCols[idx] - if lastColID > col.ID { - return nil, errors.AssertionFailedf("cannot write column id %d after %d", col.ID, lastColID) + col := fetchedCols[idx] + if lastColID > col.GetID() { + return nil, errors.AssertionFailedf("cannot write column id %d after %d", col.GetID(), lastColID) } - colIDDiff := col.ID - lastColID - lastColID = col.ID + colIDDiff := col.GetID() - lastColID + lastColID = col.GetID() var err error rawValueBuf, err = rowenc.EncodeTableValue(rawValueBuf, colIDDiff, values[idx], nil) if err != nil { diff --git a/pkg/sql/rowenc/client_index_encoding_test.go b/pkg/sql/rowenc/client_index_encoding_test.go index ce179f665852..1040c1cc1636 100644 --- a/pkg/sql/rowenc/client_index_encoding_test.go +++ b/pkg/sql/rowenc/client_index_encoding_test.go @@ -230,7 +230,7 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { t.Run(strconv.Itoa(i), func(t *testing.T) { codec := keys.SystemSQLCodec actual := EncodeTestKey(t, kvDB, codec, ShortToLongKeyFmt(tc.input)) - actual, err := rowenc.AdjustStartKeyForInterleave(codec, tc.index.IndexDesc(), actual) + actual, err := rowenc.AdjustStartKeyForInterleave(codec, tc.index, actual) if err != nil { t.Fatal(err) } @@ -652,7 +652,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { t.Run(strconv.Itoa(i), func(t *testing.T) { codec := keys.SystemSQLCodec actual := EncodeTestKey(t, kvDB, codec, ShortToLongKeyFmt(tc.input)) - actual, err := rowenc.AdjustEndKeyForInterleave(codec, tc.table, tc.index.IndexDesc(), actual, tc.inclusive) + actual, err := rowenc.AdjustEndKeyForInterleave(codec, tc.table, tc.index, actual, tc.inclusive) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/rowenc/column_type_encoding.go b/pkg/sql/rowenc/column_type_encoding.go index cf09f20d95f1..54b5e3c77915 100644 --- a/pkg/sql/rowenc/column_type_encoding.go +++ b/pkg/sql/rowenc/column_type_encoding.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/apd/v2" "github.com/cockroachdb/cockroach/pkg/geo" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -691,14 +692,21 @@ func DecodeUntaggedDatum(a *DatumAlloc, t *types.T, buf []byte) (tree.Datum, []b // // If val's type is incompatible with col, or if col's type is not yet // implemented by this function, an error is returned. -func MarshalColumnValue(col *descpb.ColumnDescriptor, val tree.Datum) (roachpb.Value, error) { +func MarshalColumnValue(col catalog.Column, val tree.Datum) (roachpb.Value, error) { + return MarshalColumnTypeValue(col.GetName(), col.GetType(), val) +} + +// MarshalColumnTypeValue is called by MarshalColumnValue and in tests. +func MarshalColumnTypeValue( + colName string, colType *types.T, val tree.Datum, +) (roachpb.Value, error) { var r roachpb.Value if val == tree.DNull { return r, nil } - switch col.Type.Family() { + switch colType.Family() { case types.BitFamily: if v, ok := val.(*tree.DBitArray); ok { r.SetBitArray(v.BitArray) @@ -801,7 +809,7 @@ func MarshalColumnValue(col *descpb.ColumnDescriptor, val tree.Datum) (roachpb.V } case types.ArrayFamily: if v, ok := val.(*tree.DArray); ok { - if err := checkElementType(v.ParamTyp, col.Type.ArrayContents()); err != nil { + if err := checkElementType(v.ParamTyp, colType.ArrayContents()); err != nil { return r, err } b, err := encodeArray(v, nil) @@ -813,7 +821,7 @@ func MarshalColumnValue(col *descpb.ColumnDescriptor, val tree.Datum) (roachpb.V } case types.CollatedStringFamily: if v, ok := val.(*tree.DCollatedString); ok { - if lex.LocaleNamesAreEqual(v.Locale, col.Type.Locale()) { + if lex.LocaleNamesAreEqual(v.Locale, colType.Locale()) { r.SetString(v.Contents) return r, nil } @@ -822,7 +830,7 @@ func MarshalColumnValue(col *descpb.ColumnDescriptor, val tree.Datum) (roachpb.V // the mutation planning code. return r, errors.AssertionFailedf( "locale mismatch %q vs %q for column %q", - v.Locale, col.Type.Locale(), tree.ErrNameString(col.Name)) + v.Locale, colType.Locale(), tree.ErrNameString(colName)) } case types.OidFamily: if v, ok := val.(*tree.DOid); ok { @@ -835,10 +843,10 @@ func MarshalColumnValue(col *descpb.ColumnDescriptor, val tree.Datum) (roachpb.V return r, nil } default: - return r, errors.AssertionFailedf("unsupported column type: %s", col.Type.Family()) + return r, errors.AssertionFailedf("unsupported column type: %s", colType.Family()) } return r, errors.AssertionFailedf("mismatched type %q vs %q for column %q", - val.ResolvedType(), col.Type.Family(), tree.ErrNameString(col.Name)) + val.ResolvedType(), colType.Family(), tree.ErrNameString(colName)) } // UnmarshalColumnValue is the counterpart to MarshalColumnValues. diff --git a/pkg/sql/rowenc/column_type_encoding_test.go b/pkg/sql/rowenc/column_type_encoding_test.go index afe3874f511d..a39b69672570 100644 --- a/pkg/sql/rowenc/column_type_encoding_test.go +++ b/pkg/sql/rowenc/column_type_encoding_test.go @@ -280,10 +280,7 @@ func TestMarshalColumnValueRoundtrip(t *testing.T) { return "error generating datum" } datum := d.(tree.Datum) - desc := descpb.ColumnDescriptor{ - Type: typ, - } - value, err := rowenc.MarshalColumnValue(&desc, datum) + value, err := rowenc.MarshalColumnTypeValue("testcol", typ, datum) if err != nil { return "error marshaling: " + err.Error() } diff --git a/pkg/sql/rowenc/index_encoding.go b/pkg/sql/rowenc/index_encoding.go index 1f6b44586552..35e9b8ad3d3a 100644 --- a/pkg/sql/rowenc/index_encoding.go +++ b/pkg/sql/rowenc/index_encoding.go @@ -63,7 +63,7 @@ func MakeIndexKeyPrefix( // full index key. func EncodeIndexKey( tableDesc catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, colMap catalog.TableColMap, values []tree.Datum, keyPrefix []byte, @@ -71,7 +71,7 @@ func EncodeIndexKey( return EncodePartialIndexKey( tableDesc, index, - len(index.ColumnIDs), /* encode all columns */ + index.NumColumns(), /* encode all columns */ colMap, values, keyPrefix, @@ -83,7 +83,7 @@ func EncodeIndexKey( // EncodePartialIndexKey. func EncodePartialIndexSpan( tableDesc catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, numCols int, colMap catalog.TableColMap, values []tree.Datum, @@ -95,7 +95,7 @@ func EncodePartialIndexSpan( if err != nil { return span, containsNull, err } - if numCols == len(index.ColumnIDs) { + if numCols == index.NumColumns() { // If all values in the input index were specified, append an interleave // marker instead of PrefixEnding the key, to avoid including any child // interleaves of the input key. @@ -112,21 +112,21 @@ func EncodePartialIndexSpan( // - append(index.ColumnIDs, index.ExtraColumnIDs) for non-unique indexes. func EncodePartialIndexKey( tableDesc catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, numCols int, colMap catalog.TableColMap, values []tree.Datum, keyPrefix []byte, ) (key []byte, containsNull bool, err error) { var colIDs, extraColIDs []descpb.ColumnID - if numCols <= len(index.ColumnIDs) { - colIDs = index.ColumnIDs[:numCols] + if numCols <= index.NumColumns() { + colIDs = index.IndexDesc().ColumnIDs[:numCols] } else { - if index.Unique || numCols > len(index.ColumnIDs)+len(index.ExtraColumnIDs) { + if index.IsUnique() || numCols > index.NumColumns()+index.NumExtraColumns() { return nil, false, errors.Errorf("encoding too many columns (%d)", numCols) } - colIDs = index.ColumnIDs - extraColIDs = index.ExtraColumnIDs[:numCols-len(index.ColumnIDs)] + colIDs = index.IndexDesc().ColumnIDs + extraColIDs = index.IndexDesc().ExtraColumnIDs[:numCols-index.NumColumns()] } // We know we will append to the key which will cause the capacity to grow so @@ -134,12 +134,13 @@ func EncodePartialIndexKey( // Add the length of the key prefix as an initial guess. // Add 3 bytes for every ancestor: table,index id + interleave sentinel. // Add 2 bytes for every column value. An underestimate for all but low integers. - key = growKey(keyPrefix, len(keyPrefix)+3*len(index.Interleave.Ancestors)+2*len(values)) + key = growKey(keyPrefix, len(keyPrefix)+3*index.NumInterleaveAncestors()+2*len(values)) - dirs := directions(index.ColumnDirections) + dirs := directions(index.IndexDesc().ColumnDirections) - if len(index.Interleave.Ancestors) > 0 { - for i, ancestor := range index.Interleave.Ancestors { + if index.NumInterleaveAncestors() > 0 { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + ancestor := index.GetInterleaveAncestor(i) // The first ancestor is assumed to already be encoded in keyPrefix. if i != 0 { key = EncodePartialTableIDIndexID(key, ancestor.TableID, ancestor.IndexID) @@ -169,7 +170,7 @@ func EncodePartialIndexKey( key = encoding.EncodeInterleavedSentinel(key) } - key = EncodePartialTableIDIndexID(key, tableDesc.GetID(), index.ID) + key = EncodePartialTableIDIndexID(key, tableDesc.GetID(), index.GetID()) } var n bool @@ -213,7 +214,7 @@ func MakeSpanFromEncDatums( types []*types.T, dirs []descpb.IndexDescriptor_Direction, tableDesc catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, alloc *DatumAlloc, keyPrefix []byte, ) (_ roachpb.Span, containsNull bool, _ error) { @@ -223,7 +224,7 @@ func MakeSpanFromEncDatums( } var endKey roachpb.Key - if complete && index.Unique { + if complete && index.IsUnique() { // If all values in the input index were specified and the input index is // unique, indicating that it might have child interleaves, append an // interleave marker instead of PrefixEnding the key, to avoid including @@ -246,7 +247,7 @@ func MakeSpanFromEncDatums( // retrieve neededCols for the specified table and index. The returned descpb.FamilyIDs // are in sorted order. func NeededColumnFamilyIDs( - neededColOrdinals util.FastIntSet, table catalog.TableDescriptor, index *descpb.IndexDescriptor, + neededColOrdinals util.FastIntSet, table catalog.TableDescriptor, index catalog.Index, ) []descpb.FamilyID { if table.NumFamilies() == 1 { return []descpb.FamilyID{table.GetFamilies()[0].ID} @@ -258,15 +259,18 @@ func NeededColumnFamilyIDs( var indexedCols util.FastIntSet var compositeCols util.FastIntSet var extraCols util.FastIntSet - for _, columnID := range index.ColumnIDs { + for i := 0; i < index.NumColumns(); i++ { + columnID := index.GetColumnID(i) columnOrdinal := colIdxMap.GetDefault(columnID) indexedCols.Add(columnOrdinal) } - for _, columnID := range index.CompositeColumnIDs { + for i := 0; i < index.NumCompositeColumns(); i++ { + columnID := index.GetCompositeColumnID(i) columnOrdinal := colIdxMap.GetDefault(columnID) compositeCols.Add(columnOrdinal) } - for _, columnID := range index.ExtraColumnIDs { + for i := 0; i < index.NumExtraColumns(); i++ { + columnID := index.GetExtraColumnID(i) columnOrdinal := colIdxMap.GetDefault(columnID) extraCols.Add(columnOrdinal) } @@ -278,7 +282,7 @@ func NeededColumnFamilyIDs( // values here for composite and "extra" columns. ("Extra" means primary key // columns which are not indexed.) var family0 *descpb.ColumnFamilyDescriptor - hasSecondaryEncoding := index.GetEncodingType(table.GetPrimaryIndexID()) == descpb.SecondaryIndexEncoding + hasSecondaryEncoding := index.GetEncodingType() == descpb.SecondaryIndexEncoding // First iterate over the needed columns and look for a few special cases: // * columns which can be decoded from the key and columns whose value is stored @@ -424,7 +428,7 @@ func MakeKeyFromEncDatums( types []*types.T, dirs []descpb.IndexDescriptor_Direction, tableDesc catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, alloc *DatumAlloc, keyPrefix []byte, ) (_ roachpb.Key, complete bool, containsNull bool, _ error) { @@ -440,8 +444,9 @@ func MakeKeyFromEncDatums( key := make(roachpb.Key, len(keyPrefix), len(keyPrefix)*2) copy(key, keyPrefix) - if len(index.Interleave.Ancestors) > 0 { - for i, ancestor := range index.Interleave.Ancestors { + if index.NumInterleaveAncestors() > 0 { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + ancestor := index.GetInterleaveAncestor(i) // The first ancestor is assumed to already be encoded in keyPrefix. if i != 0 { key = EncodePartialTableIDIndexID(key, ancestor.TableID, ancestor.IndexID) @@ -474,7 +479,7 @@ func MakeKeyFromEncDatums( key = encoding.EncodeInterleavedSentinel(key) } - key = EncodePartialTableIDIndexID(key, tableDesc.GetID(), index.ID) + key = EncodePartialTableIDIndexID(key, tableDesc.GetID(), index.GetID()) } var ( err error @@ -485,7 +490,7 @@ func MakeKeyFromEncDatums( return key, false, false, err } containsNull = containsNull || n - return key, len(types) == len(index.ColumnIDs), containsNull, err + return key, len(types) == index.NumColumns(), containsNull, err } // findColumnValue returns the value corresponding to the column. If @@ -617,7 +622,7 @@ func DecodeIndexKeyPrefix( func DecodeIndexKey( codec keys.SQLCodec, desc catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, types []*types.T, vals []EncDatum, colDirs []descpb.IndexDescriptor_Direction, @@ -639,7 +644,7 @@ func DecodeIndexKey( // id / index id key prefix. func DecodeIndexKeyWithoutTableIDIndexIDPrefix( desc catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, types []*types.T, vals []EncDatum, colDirs []descpb.IndexDescriptor_Direction, @@ -649,8 +654,9 @@ func DecodeIndexKeyWithoutTableIDIndexIDPrefix( var decodedIndexID descpb.IndexID var err error - if len(index.Interleave.Ancestors) > 0 { - for i, ancestor := range index.Interleave.Ancestors { + if index.NumInterleaveAncestors() > 0 { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + ancestor := index.GetInterleaveAncestor(i) // Our input key had its first table id / index id chopped off, so // don't try to decode those for the first ancestor. if i != 0 { @@ -684,7 +690,7 @@ func DecodeIndexKeyWithoutTableIDIndexIDPrefix( if err != nil { return nil, false, false, err } - if decodedTableID != desc.GetID() || decodedIndexID != index.ID { + if decodedTableID != desc.GetID() || decodedIndexID != index.GetID() { return nil, false, false, nil } } @@ -762,7 +768,7 @@ func (a byID) Less(i, j int) bool { return a[i].id < a[j].id } // concatenating keyPrefix with the encodings of the column in the // index. func EncodeInvertedIndexKeys( - index *descpb.IndexDescriptor, colMap catalog.TableColMap, values []tree.Datum, keyPrefix []byte, + index catalog.Index, colMap catalog.TableColMap, values []tree.Datum, keyPrefix []byte, ) (key [][]byte, err error) { keyPrefix, err = EncodeInvertedIndexPrefixKeys(index, colMap, values, keyPrefix) if err != nil { @@ -775,26 +781,27 @@ func EncodeInvertedIndexKeys( } else { val = tree.DNull } - if !geoindex.IsEmptyConfig(&index.GeoConfig) { - return EncodeGeoInvertedIndexTableKeys(val, keyPrefix, index) + indexGeoConfig := index.GetGeoConfig() + if !geoindex.IsEmptyConfig(&indexGeoConfig) { + return EncodeGeoInvertedIndexTableKeys(val, keyPrefix, indexGeoConfig) } - return EncodeInvertedIndexTableKeys(val, keyPrefix, index.Version) + return EncodeInvertedIndexTableKeys(val, keyPrefix, index.GetVersion()) } // EncodeInvertedIndexPrefixKeys encodes the non-inverted prefix columns if // the given index is a multi-column inverted index. func EncodeInvertedIndexPrefixKeys( - index *descpb.IndexDescriptor, colMap catalog.TableColMap, values []tree.Datum, keyPrefix []byte, + index catalog.Index, colMap catalog.TableColMap, values []tree.Datum, keyPrefix []byte, ) (_ []byte, err error) { - numColumns := len(index.ColumnIDs) + numColumns := index.NumColumns() // If the index is a multi-column inverted index, we encode the non-inverted // columns in the key prefix. if numColumns > 1 { // Do not encode the last column, which is the inverted column, here. It // is encoded below this block. - colIDs := index.ColumnIDs[:numColumns-1] - dirs := directions(index.ColumnDirections) + colIDs := index.IndexDesc().ColumnIDs[:numColumns-1] + dirs := directions(index.IndexDesc().ColumnDirections) // Double the size of the key to make the imminent appends more // efficient. @@ -1033,21 +1040,21 @@ func encodeContainedArrayInvertedIndexSpans( // EncodeGeoInvertedIndexTableKeys is the equivalent of EncodeInvertedIndexTableKeys // for Geography and Geometry. func EncodeGeoInvertedIndexTableKeys( - val tree.Datum, inKey []byte, index *descpb.IndexDescriptor, + val tree.Datum, inKey []byte, indexGeoConfig geoindex.Config, ) (key [][]byte, err error) { if val == tree.DNull { return nil, nil } switch val.ResolvedType().Family() { case types.GeographyFamily: - index := geoindex.NewS2GeographyIndex(*index.GeoConfig.S2Geography) + index := geoindex.NewS2GeographyIndex(*indexGeoConfig.S2Geography) intKeys, bbox, err := index.InvertedIndexKeys(context.TODO(), val.(*tree.DGeography).Geography) if err != nil { return nil, err } return encodeGeoKeys(encoding.EncodeGeoInvertedAscending(inKey), intKeys, bbox) case types.GeometryFamily: - index := geoindex.NewS2GeometryIndex(*index.GeoConfig.S2Geometry) + index := geoindex.NewS2GeometryIndex(*indexGeoConfig.S2Geometry) intKeys, bbox, err := index.InvertedIndexKeys(context.TODO(), val.(*tree.DGeometry).Geometry) if err != nil { return nil, err @@ -1089,19 +1096,20 @@ func encodeGeoKeys( func EncodePrimaryIndex( codec keys.SQLCodec, tableDesc catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, colMap catalog.TableColMap, values []tree.Datum, includeEmpty bool, ) ([]IndexEntry, error) { - keyPrefix := MakeIndexKeyPrefix(codec, tableDesc, index.ID) + keyPrefix := MakeIndexKeyPrefix(codec, tableDesc, index.GetID()) indexKey, _, err := EncodeIndexKey(tableDesc, index, colMap, values, keyPrefix) if err != nil { return nil, err } // This information should be precomputed on the table descriptor. var indexedColumns catalog.TableColSet - for _, colID := range index.ColumnIDs { + for i := 0; i < index.NumColumns(); i++ { + colID := index.GetColumnID(i) indexedColumns.Add(colID) } var entryValue []byte @@ -1128,7 +1136,7 @@ func EncodePrimaryIndex( if err != nil { return err } - value, err := MarshalColumnValue(col.ColumnDesc(), datum) + value, err := MarshalColumnValue(col, datum) if err != nil { return err } @@ -1176,22 +1184,22 @@ func EncodePrimaryIndex( func EncodeSecondaryIndex( codec keys.SQLCodec, tableDesc catalog.TableDescriptor, - secondaryIndex *descpb.IndexDescriptor, + secondaryIndex catalog.Index, colMap catalog.TableColMap, values []tree.Datum, includeEmpty bool, ) ([]IndexEntry, error) { - secondaryIndexKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc, secondaryIndex.ID) + secondaryIndexKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc, secondaryIndex.GetID()) // Use the primary key encoding for covering indexes. - if secondaryIndex.GetEncodingType(tableDesc.GetPrimaryIndexID()) == descpb.PrimaryIndexEncoding { + if secondaryIndex.GetEncodingType() == descpb.PrimaryIndexEncoding { return EncodePrimaryIndex(codec, tableDesc, secondaryIndex, colMap, values, includeEmpty) } var containsNull = false var secondaryKeys [][]byte var err error - if secondaryIndex.Type == descpb.IndexDescriptor_INVERTED { + if secondaryIndex.GetType() == descpb.IndexDescriptor_INVERTED { secondaryKeys, err = EncodeInvertedIndexKeys(secondaryIndex, colMap, values, secondaryIndexKeyPrefix) } else { var secondaryIndexKey []byte @@ -1206,7 +1214,7 @@ func EncodeSecondaryIndex( // Add the extra columns - they are encoded in ascending order which is done // by passing nil for the encoding directions. - extraKey, _, err := EncodeColumns(secondaryIndex.ExtraColumnIDs, nil, + extraKey, _, err := EncodeColumns(secondaryIndex.IndexDesc().ExtraColumnIDs, nil, colMap, values, nil) if err != nil { return []IndexEntry{}, err @@ -1216,15 +1224,15 @@ func EncodeSecondaryIndex( // len(secondaryKeys) positions to avoid allocations from appending. entries := make([]IndexEntry, 0, len(secondaryKeys)) for _, key := range secondaryKeys { - if !secondaryIndex.Unique || containsNull { + if !secondaryIndex.IsUnique() || containsNull { // If the index is not unique or it contains a NULL value, append // extraKey to the key in order to make it unique. key = append(key, extraKey...) } if tableDesc.NumFamilies() == 1 || - secondaryIndex.Type == descpb.IndexDescriptor_INVERTED || - secondaryIndex.Version == descpb.BaseIndexFormatVersion { + secondaryIndex.GetType() == descpb.IndexDescriptor_INVERTED || + secondaryIndex.GetVersion() == descpb.BaseIndexFormatVersion { // We do all computation that affects indexes with families in a separate code path to avoid performance // regression for tables without column families. entry, err := encodeSecondaryIndexNoFamilies(secondaryIndex, colMap, key, values, extraKey) @@ -1248,11 +1256,13 @@ func EncodeSecondaryIndex( // Ensure that column family 0 always generates a k/v pair. familyToColumns[0] = []valueEncodedColumn{} // All composite columns are stored in family 0. - for _, id := range secondaryIndex.CompositeColumnIDs { + for i := 0; i < secondaryIndex.NumCompositeColumns(); i++ { + id := secondaryIndex.GetCompositeColumnID(i) addToFamilyColMap(0, valueEncodedColumn{id: id, isComposite: true}) } _ = tableDesc.ForeachFamily(func(family *descpb.ColumnFamilyDescriptor) error { - for _, id := range secondaryIndex.StoreColumnIDs { + for i := 0; i < secondaryIndex.NumStoredColumns(); i++ { + id := secondaryIndex.GetStoredColumnID(i) for _, col := range family.ColumnIDs { if id == col { addToFamilyColMap(family.ID, valueEncodedColumn{id: id, isComposite: false}) @@ -1278,7 +1288,7 @@ func EncodeSecondaryIndex( // The returned indexEntries are in family sorted order. func encodeSecondaryIndexWithFamilies( familyMap map[descpb.FamilyID][]valueEncodedColumn, - index *descpb.IndexDescriptor, + index catalog.Index, colMap catalog.TableColMap, key []byte, row []tree.Datum, @@ -1315,7 +1325,7 @@ func encodeSecondaryIndexWithFamilies( sort.Sort(byID(storedColsInFam)) key = keys.MakeFamilyKey(key, uint32(familyID)) - if index.Unique && familyID == 0 { + if index.IsUnique() && familyID == 0 { // Note that a unique secondary index that contains a NULL column value // will have extraKey appended to the key and stored in the value. We // require extraKey to be appended to the key in order to make the key @@ -1358,7 +1368,7 @@ func encodeSecondaryIndexWithFamilies( // performs the index encoding version before column // families were introduced onto secondary indexes. func encodeSecondaryIndexNoFamilies( - index *descpb.IndexDescriptor, + index catalog.Index, colMap catalog.TableColMap, key []byte, row []tree.Datum, @@ -1370,7 +1380,7 @@ func encodeSecondaryIndexNoFamilies( ) // If we aren't encoding index keys with families, all index keys use the sentinel family 0. key = keys.MakeFamilyKey(key, 0) - if index.Unique { + if index.IsUnique() { // Note that a unique secondary index that contains a NULL column value // will have extraKey appended to the key and stored in the value. We // require extraKey to be appended to the key in order to make the key @@ -1384,13 +1394,15 @@ func encodeSecondaryIndexNoFamilies( } var cols []valueEncodedColumn // Since we aren't encoding data with families, we just encode all stored and composite columns in the value. - for _, id := range index.StoreColumnIDs { + for i := 0; i < index.NumStoredColumns(); i++ { + id := index.GetStoredColumnID(i) cols = append(cols, valueEncodedColumn{id: id, isComposite: false}) } - for _, id := range index.CompositeColumnIDs { + for i := 0; i < index.NumCompositeColumns(); i++ { + id := index.GetCompositeColumnID(i) // Inverted indexes on a composite type (i.e. an array of composite types) // should not add the indexed column to the value. - if index.Type == descpb.IndexDescriptor_INVERTED && id == index.ColumnIDs[0] { + if index.GetType() == descpb.IndexDescriptor_INVERTED && id == index.GetColumnID(0) { continue } cols = append(cols, valueEncodedColumn{id: id, isComposite: true}) @@ -1438,7 +1450,7 @@ func EncodeSecondaryIndexes( ctx context.Context, codec keys.SQLCodec, tableDesc catalog.TableDescriptor, - indexes []*descpb.IndexDescriptor, + indexes []catalog.Index, colMap catalog.TableColMap, values []tree.Datum, secondaryIndexEntries []IndexEntry, @@ -1612,22 +1624,23 @@ func IndexKeyEquivSignature( // TableEquivSignatures returns the equivalence signatures for each interleave // ancestor and itself. See IndexKeyEquivSignature for more info. func TableEquivSignatures( - desc *descpb.TableDescriptor, index *descpb.IndexDescriptor, + desc catalog.TableDescriptor, index catalog.Index, ) (signatures [][]byte, err error) { // signatures contains the slice reference to the signature of every // ancestor of the current table-index. // The last slice reference is the given table-index's signature. - signatures = make([][]byte, len(index.Interleave.Ancestors)+1) + signatures = make([][]byte, index.NumInterleaveAncestors()+1) // fullSignature is the backing byte slice for each individual signature // as it buffers each block of table and index IDs. // We eagerly allocate 4 bytes for each of the two IDs per ancestor // (which can fit Uvarint IDs up to 2^17-1 without another allocation), // 1 byte for each interleave sentinel, and 4 bytes each for the given // table's and index's ID. - fullSignature := make([]byte, 0, len(index.Interleave.Ancestors)*9+8) + fullSignature := make([]byte, 0, index.NumInterleaveAncestors()*9+8) // Encode the table's ancestors' TableIDs and descpb.IndexIDs. - for i, ancestor := range index.Interleave.Ancestors { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + ancestor := index.GetInterleaveAncestor(i) fullSignature = EncodePartialTableIDIndexID(fullSignature, ancestor.TableID, ancestor.IndexID) // Create a reference up to this point for the ancestor's // signature. @@ -1637,7 +1650,7 @@ func TableEquivSignatures( } // Encode the table's table and index IDs. - fullSignature = EncodePartialTableIDIndexID(fullSignature, desc.ID, index.ID) + fullSignature = EncodePartialTableIDIndexID(fullSignature, desc.GetID(), index.GetID()) // Create a reference for the given table's signature as the last // element of signatures. signatures[len(signatures)-1] = fullSignature @@ -1666,16 +1679,16 @@ func TableEquivSignatures( // /table/index/<parent-pk1>/.../<parent-pkX> // // This returns the maximum number of <tokens> in this prefix. -func maxKeyTokens(index *descpb.IndexDescriptor, containsNull bool) int { - nTables := len(index.Interleave.Ancestors) + 1 - nKeyCols := len(index.ColumnIDs) +func maxKeyTokens(index catalog.Index, containsNull bool) int { + nTables := index.NumInterleaveAncestors() + 1 + nKeyCols := index.NumColumns() // Non-unique secondary indexes or unique secondary indexes with a NULL // value have additional columns in the key that may appear in a span // (e.g. primary key columns not part of the index). // See EncodeSecondaryIndex. - if !index.Unique || containsNull { - nKeyCols += len(index.ExtraColumnIDs) + if !index.IsUnique() || containsNull { + nKeyCols += index.NumExtraColumns() } // To illustrate how we compute max # of key tokens, take the @@ -1711,7 +1724,7 @@ func maxKeyTokens(index *descpb.IndexDescriptor, containsNull bool) int { // was /1, we cannot push this forwards since that is the first key we want // to read. func AdjustStartKeyForInterleave( - codec keys.SQLCodec, index *descpb.IndexDescriptor, start roachpb.Key, + codec keys.SQLCodec, index catalog.Index, start roachpb.Key, ) (roachpb.Key, error) { // Remove the tenant prefix before decomposing. strippedStart, err := codec.StripTenantPrefix(start) @@ -1757,11 +1770,11 @@ func AdjustStartKeyForInterleave( func AdjustEndKeyForInterleave( codec keys.SQLCodec, table catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, end roachpb.Key, inclusive bool, ) (roachpb.Key, error) { - if index.Type == descpb.IndexDescriptor_INVERTED { + if index.GetType() == descpb.IndexDescriptor_INVERTED { return end.PrefixEnd(), nil } @@ -1793,7 +1806,7 @@ func AdjustEndKeyForInterleave( // adjust the sibling key such that we add or remove child (the current // index's) rows from our span. - if index.ID != table.GetPrimaryIndexID() || len(keyTokens) < nIndexTokens { + if index.GetID() != table.GetPrimaryIndexID() || len(keyTokens) < nIndexTokens { // Case 1: secondary index, parent key or partial child key: // Secondary indexes cannot have interleaved rows. // We cannot adjust or tighten parent keys with respect to a diff --git a/pkg/sql/rowenc/index_encoding_test.go b/pkg/sql/rowenc/index_encoding_test.go index e832257f0fed..8d64ff3afdb8 100644 --- a/pkg/sql/rowenc/index_encoding_test.go +++ b/pkg/sql/rowenc/index_encoding_test.go @@ -112,14 +112,14 @@ func makeTableDescForTest(test indexKeyTest) (catalog.TableDescriptor, catalog.T } func decodeIndex( - codec keys.SQLCodec, tableDesc catalog.TableDescriptor, index *descpb.IndexDescriptor, key []byte, + codec keys.SQLCodec, tableDesc catalog.TableDescriptor, index catalog.Index, key []byte, ) ([]tree.Datum, error) { - types, err := colinfo.GetColumnTypes(tableDesc, index.ColumnIDs, nil) + types, err := colinfo.GetColumnTypes(tableDesc, index.IndexDesc().ColumnIDs, nil) if err != nil { return nil, err } - values := make([]EncDatum, len(index.ColumnIDs)) - colDirs := index.ColumnDirections + values := make([]EncDatum, index.NumColumns()) + colDirs := index.IndexDesc().ColumnDirections _, ok, _, err := DecodeIndexKey(codec, tableDesc, index, types, values, colDirs, key) if err != nil { return nil, err @@ -227,7 +227,7 @@ func TestIndexKey(t *testing.T) { codec := keys.SystemSQLCodec primaryKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc, tableDesc.GetPrimaryIndexID()) - primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colMap, testValues, primaryKeyPrefix) + primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.GetPrimaryIndex(), colMap, testValues, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -235,7 +235,7 @@ func TestIndexKey(t *testing.T) { primaryIndexKV := kv.KeyValue{Key: primaryKey, Value: &primaryValue} secondaryIndexEntry, err := EncodeSecondaryIndex( - codec, tableDesc, tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), colMap, testValues, true /* includeEmpty */) + codec, tableDesc, tableDesc.PublicNonPrimaryIndexes()[0], colMap, testValues, true /* includeEmpty */) if len(secondaryIndexEntry) != 1 { t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndexEntry), secondaryIndexEntry) } @@ -247,14 +247,14 @@ func TestIndexKey(t *testing.T) { Value: &secondaryIndexEntry[0].Value, } - checkEntry := func(index *descpb.IndexDescriptor, entry kv.KeyValue) { + checkEntry := func(index catalog.Index, entry kv.KeyValue) { values, err := decodeIndex(codec, tableDesc, index, entry.Key) if err != nil { t.Fatal(err) } for j, value := range values { - testValue := testValues[colMap.GetDefault(index.ColumnIDs[j])] + testValue := testValues[colMap.GetDefault(index.GetColumnID(j))] if value.Compare(evalCtx, testValue) != 0 { t.Fatalf("%d: value %d got %q but expected %q", i, j, value, testValue) } @@ -264,7 +264,7 @@ func TestIndexKey(t *testing.T) { if err != nil { t.Fatal(err) } - if indexID != index.ID { + if indexID != index.GetID() { t.Errorf("%d", i) } @@ -277,8 +277,8 @@ func TestIndexKey(t *testing.T) { } } - checkEntry(tableDesc.GetPrimaryIndex().IndexDesc(), primaryIndexKV) - checkEntry(tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), secondaryIndexKV) + checkEntry(tableDesc.GetPrimaryIndex(), primaryIndexKV) + checkEntry(tableDesc.PublicNonPrimaryIndexes()[0], secondaryIndexKV) } } @@ -388,7 +388,7 @@ func TestInvertedIndexKey(t *testing.T) { codec := keys.SystemSQLCodec secondaryIndexEntries, err := EncodeSecondaryIndex( - codec, tableDesc, tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), colMap, testValues, true /* includeEmpty */) + codec, tableDesc, tableDesc.PublicNonPrimaryIndexes()[0], colMap, testValues, true /* includeEmpty */) if err != nil { t.Fatal(err) } @@ -943,9 +943,7 @@ func TestMarshalColumnValue(t *testing.T) { for i, testCase := range tests { typ := testCase.typ - col := descpb.ColumnDescriptor{ID: descpb.ColumnID(typ.Family() + 1), Type: typ} - - if actual, err := MarshalColumnValue(&col, testCase.datum); err != nil { + if actual, err := MarshalColumnTypeValue("testcol", typ, testCase.datum); err != nil { t.Errorf("%d: unexpected error with column type %v: %v", i, typ, err) } else if !reflect.DeepEqual(actual, testCase.exp) { t.Errorf("%d: MarshalColumnValue() got %v, expected %v", i, actual, testCase.exp) @@ -1070,7 +1068,7 @@ func TestIndexKeyEquivSignature(t *testing.T) { desc, colMap := makeTableDescForTest(tc.table.indexKeyArgs) primaryKeyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPrimaryIndexID()) primaryKey, _, err := EncodeIndexKey( - desc, desc.GetPrimaryIndex().IndexDesc(), colMap, tc.table.values, primaryKeyPrefix) + desc, desc.GetPrimaryIndex(), colMap, tc.table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -1127,7 +1125,7 @@ func TestTableEquivSignatures(t *testing.T) { tc.table.indexKeyArgs.primaryValues = tc.table.values // Setup descriptors and form an index key. desc, _ := makeTableDescForTest(tc.table.indexKeyArgs) - equivSigs, err := TableEquivSignatures(desc.TableDesc(), desc.GetPrimaryIndex().IndexDesc()) + equivSigs, err := TableEquivSignatures(desc, desc.GetPrimaryIndex()) if err != nil { t.Fatal(err) } @@ -1212,13 +1210,13 @@ func TestEquivSignature(t *testing.T) { desc, colMap := makeTableDescForTest(table.indexKeyArgs) primaryKeyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPrimaryIndexID()) primaryKey, _, err := EncodeIndexKey( - desc, desc.GetPrimaryIndex().IndexDesc(), colMap, table.values, primaryKeyPrefix) + desc, desc.GetPrimaryIndex(), colMap, table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } // Extract out the table's equivalence signature. - tempEquivSigs, err := TableEquivSignatures(desc.TableDesc(), desc.GetPrimaryIndex().IndexDesc()) + tempEquivSigs, err := TableEquivSignatures(desc, desc.GetPrimaryIndex()) if err != nil { t.Fatal(err) } @@ -1371,20 +1369,19 @@ func ExtractIndexKey( return entry.Key, nil } - indexI, err := tableDesc.FindIndexWithID(indexID) + index, err := tableDesc.FindIndexWithID(indexID) if err != nil { return nil, err } - index := indexI.IndexDesc() // Extract the values for index.ColumnIDs. - indexTypes, err := colinfo.GetColumnTypes(tableDesc, index.ColumnIDs, nil) + indexTypes, err := colinfo.GetColumnTypes(tableDesc, index.IndexDesc().ColumnIDs, nil) if err != nil { return nil, err } - values := make([]EncDatum, len(index.ColumnIDs)) - dirs := index.ColumnDirections - if len(index.Interleave.Ancestors) > 0 { + values := make([]EncDatum, index.NumColumns()) + dirs := index.IndexDesc().ColumnDirections + if index.NumInterleaveAncestors() > 0 { // TODO(dan): In the interleaved index case, we parse the key twice; once to // find the index id so we can look up the descriptor, and once to extract // the values. Only parse once. @@ -1404,18 +1401,18 @@ func ExtractIndexKey( } // Extract the values for index.ExtraColumnIDs - extraTypes, err := colinfo.GetColumnTypes(tableDesc, index.ExtraColumnIDs, nil) + extraTypes, err := colinfo.GetColumnTypes(tableDesc, index.IndexDesc().ExtraColumnIDs, nil) if err != nil { return nil, err } - extraValues := make([]EncDatum, len(index.ExtraColumnIDs)) - dirs = make([]descpb.IndexDescriptor_Direction, len(index.ExtraColumnIDs)) - for i := range index.ExtraColumnIDs { + extraValues := make([]EncDatum, index.NumExtraColumns()) + dirs = make([]descpb.IndexDescriptor_Direction, index.NumExtraColumns()) + for i := 0; i < index.NumExtraColumns(); i++ { // Implicit columns are always encoded Ascending. dirs[i] = descpb.IndexDescriptor_ASC } extraKey := key - if index.Unique { + if index.IsUnique() { extraKey, err = entry.Value.GetBytes() if err != nil { return nil, err @@ -1428,11 +1425,13 @@ func ExtractIndexKey( // Encode the index key from its components. var colMap catalog.TableColMap - for i, columnID := range index.ColumnIDs { + for i := 0; i < index.NumColumns(); i++ { + columnID := index.GetColumnID(i) colMap.Set(columnID, i) } - for i, columnID := range index.ExtraColumnIDs { - colMap.Set(columnID, i+len(index.ColumnIDs)) + for i := 0; i < index.NumExtraColumns(); i++ { + columnID := index.GetExtraColumnID(i) + colMap.Set(columnID, i+index.NumColumns()) } indexKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc, tableDesc.GetPrimaryIndexID()) @@ -1452,6 +1451,6 @@ func ExtractIndexKey( decodedValues[len(values)+i] = value.Datum } indexKey, _, err := EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colMap, decodedValues, indexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), colMap, decodedValues, indexKeyPrefix) return indexKey, err } diff --git a/pkg/sql/rowenc/partition.go b/pkg/sql/rowenc/partition.go index 4684d3c5b5a5..99a28c5ccd59 100644 --- a/pkg/sql/rowenc/partition.go +++ b/pkg/sql/rowenc/partition.go @@ -106,12 +106,12 @@ func DecodePartitionTuple( a *DatumAlloc, codec keys.SQLCodec, tableDesc catalog.TableDescriptor, - idxDesc *descpb.IndexDescriptor, + index catalog.Index, partDesc *descpb.PartitioningDescriptor, valueEncBuf []byte, prefixDatums tree.Datums, ) (*PartitionTuple, []byte, error) { - if len(prefixDatums)+int(partDesc.NumColumns) > len(idxDesc.ColumnIDs) { + if len(prefixDatums)+int(partDesc.NumColumns) > index.NumColumns() { return nil, nil, fmt.Errorf("not enough columns in index for this partitioning") } @@ -119,8 +119,8 @@ func DecodePartitionTuple( Datums: make(tree.Datums, 0, int(partDesc.NumColumns)), } - colIDs := idxDesc.ColumnIDs[len(prefixDatums) : len(prefixDatums)+int(partDesc.NumColumns)] - for _, colID := range colIDs { + for i := len(prefixDatums); i < index.NumColumns() && i < len(prefixDatums)+int(partDesc.NumColumns); i++ { + colID := index.GetColumnID(i) col, err := tableDesc.FindColumnWithID(colID) if err != nil { return nil, nil, err @@ -161,12 +161,12 @@ func DecodePartitionTuple( allDatums := append(prefixDatums, t.Datums...) var colMap catalog.TableColMap for i := range allDatums { - colMap.Set(idxDesc.ColumnIDs[i], i) + colMap.Set(index.GetColumnID(i), i) } - indexKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc, idxDesc.ID) + indexKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc, index.GetID()) key, _, err := EncodePartialIndexKey( - tableDesc, idxDesc, len(allDatums), colMap, allDatums, indexKeyPrefix) + tableDesc, index, len(allDatums), colMap, allDatums, indexKeyPrefix) if err != nil { return nil, nil, err } diff --git a/pkg/sql/rowexec/inverted_joiner.go b/pkg/sql/rowexec/inverted_joiner.go index ea0a1a47b5ae..fc14057fe804 100644 --- a/pkg/sql/rowexec/inverted_joiner.go +++ b/pkg/sql/rowexec/inverted_joiner.go @@ -66,7 +66,7 @@ type invertedJoiner struct { desc catalog.TableDescriptor // The map from ColumnIDs in the table to the column position. colIdxMap catalog.TableColMap - index *descpb.IndexDescriptor + index catalog.Index // The ColumnID of the inverted column. Confusingly, this is also the id of // the table column that was indexed. invertedColID descpb.ColumnID @@ -200,12 +200,12 @@ func newInvertedJoiner( if indexIdx >= len(ij.desc.ActiveIndexes()) { return nil, errors.Errorf("invalid indexIdx %d", indexIdx) } - ij.index = ij.desc.ActiveIndexes()[indexIdx].IndexDesc() + ij.index = ij.desc.ActiveIndexes()[indexIdx] ij.invertedColID = ij.index.InvertedColumnID() // Initialize tableRow, indexRow, indexRowTypes, and indexRowToTableRowMap, // a mapping from indexRow column ordinal to tableRow column ordinals. - indexColumnIDs, _ := ij.index.FullColumnIDs() + indexColumnIDs, _ := catalog.FullIndexColumnIDs(ij.index) // Inverted joins are not used for mutations. ij.tableRow = make(rowenc.EncDatumRow, len(ij.desc.PublicColumns())) ij.indexRow = make(rowenc.EncDatumRow, len(indexColumnIDs)-1) @@ -446,7 +446,7 @@ func (ij *invertedJoiner) readInput() (invertedJoinerState, *execinfrapb.Produce prefixKey, _, _, err := rowenc.MakeKeyFromEncDatums( ij.indexRow[:len(ij.prefixEqualityCols)], ij.indexRowTypes[:len(ij.prefixEqualityCols)], - ij.index.ColumnDirections, + ij.index.IndexDesc().ColumnDirections, ij.desc, ij.index, &ij.alloc, @@ -540,7 +540,7 @@ func (ij *invertedJoiner) performScan() (invertedJoinerState, *execinfrapb.Produ prefixKey, _, _, err := rowenc.MakeKeyFromEncDatums( ij.indexRow[:len(ij.prefixEqualityCols)], ij.indexRowTypes[:len(ij.prefixEqualityCols)], - ij.index.ColumnDirections, + ij.index.IndexDesc().ColumnDirections, ij.desc, ij.index, &ij.alloc, diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index bbb20988f516..74fb795bc36f 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -74,7 +74,7 @@ type joinReader struct { diskMonitor *mon.BytesMonitor desc catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index colIdxMap catalog.TableColMap maintainOrdering bool @@ -220,9 +220,8 @@ func newJoinReader( if indexIdx >= len(jr.desc.ActiveIndexes()) { return nil, errors.Errorf("invalid indexIdx %d", indexIdx) } - indexI := jr.desc.ActiveIndexes()[indexIdx] - jr.index = indexI.IndexDesc() - isSecondary = !indexI.Primary() + jr.index = jr.desc.ActiveIndexes()[indexIdx] + isSecondary = !jr.index.Primary() cols := jr.desc.PublicColumns() if spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic { cols = jr.desc.DeletableColumns() @@ -230,7 +229,7 @@ func newJoinReader( jr.colIdxMap = catalog.ColumnIDToOrdinalMap(cols) columnTypes := catalog.ColumnTypes(cols) - columnIDs, _ := jr.index.FullColumnIDs() + columnIDs, _ := catalog.FullIndexColumnIDs(jr.index) indexCols := make([]uint32, len(columnIDs)) for i, columnID := range columnIDs { indexCols[i] = uint32(columnID) @@ -369,7 +368,7 @@ func (jr *joinReader) initJoinReaderStrategy( // Since jr.lookupExpr is set, we need to use multiSpanGenerator, which // supports looking up multiple spans per input row. tableOrdToIndexOrd := util.FastIntMap{} - columnIDs, _ := jr.index.FullColumnIDs() + columnIDs, _ := catalog.FullIndexColumnIDs(jr.index) for i, colID := range columnIDs { tabOrd := jr.colIdxMap.GetDefault(colID) tableOrdToIndexOrd.Set(tabOrd, i) @@ -439,11 +438,9 @@ func (jr *joinReader) initJoinReaderStrategy( } // getIndexColSet returns a set of all column indices for the given index. -func getIndexColSet( - index *descpb.IndexDescriptor, colIdxMap catalog.TableColMap, -) (util.FastIntSet, error) { +func getIndexColSet(index catalog.Index, colIdxMap catalog.TableColMap) (util.FastIntSet, error) { cols := util.MakeFastIntSet() - err := index.RunOverAllColumns(func(id descpb.ColumnID) error { + err := index.ForEachColumnID(func(id descpb.ColumnID) error { cols.Add(colIdxMap.GetDefault(id)) return nil }) diff --git a/pkg/sql/rowexec/rowfetcher.go b/pkg/sql/rowexec/rowfetcher.go index ffaf2c231021..d9aabae5f620 100644 --- a/pkg/sql/rowexec/rowfetcher.go +++ b/pkg/sql/rowexec/rowfetcher.go @@ -48,7 +48,7 @@ type rowFetcher interface { ) error NextRow(ctx context.Context) ( - rowenc.EncDatumRow, catalog.TableDescriptor, *descpb.IndexDescriptor, error) + rowenc.EncDatumRow, catalog.TableDescriptor, catalog.Index, error) // PartialKey is not stat-related but needs to be supported. PartialKey(int) (roachpb.Key, error) @@ -76,13 +76,12 @@ func initRowFetcher( lockWaitPolicy descpb.ScanLockingWaitPolicy, withSystemColumns bool, virtualColumn catalog.Column, -) (index *descpb.IndexDescriptor, isSecondaryIndex bool, err error) { +) (index catalog.Index, isSecondaryIndex bool, err error) { if indexIdx >= len(desc.ActiveIndexes()) { return nil, false, errors.Errorf("invalid indexIdx %d", indexIdx) } - indexI := desc.ActiveIndexes()[indexIdx] - index = indexI.IndexDesc() - isSecondaryIndex = !indexI.Primary() + index = desc.ActiveIndexes()[indexIdx] + isSecondaryIndex = !index.Primary() tableArgs := row.FetcherTableArgs{ Desc: desc, diff --git a/pkg/sql/rowexec/sampler.go b/pkg/sql/rowexec/sampler.go index 3cc036db5ccf..f7cdfbd1866c 100644 --- a/pkg/sql/rowexec/sampler.go +++ b/pkg/sql/rowexec/sampler.go @@ -342,7 +342,7 @@ func (s *samplerProcessor) mainLoop(ctx context.Context) (earlyExit bool, err er } switch s.outTypes[col].Family() { case types.GeographyFamily, types.GeometryFamily: - invKeys, err = rowenc.EncodeGeoInvertedIndexTableKeys(row[col].Datum, nil /* inKey */, index) + invKeys, err = rowenc.EncodeGeoInvertedIndexTableKeys(row[col].Datum, nil /* inKey */, index.GeoConfig) default: invKeys, err = rowenc.EncodeInvertedIndexTableKeys(row[col].Datum, nil /* inKey */, index.Version) } diff --git a/pkg/sql/rowexec/stats.go b/pkg/sql/rowexec/stats.go index f811fbdefedb..e94223c01649 100644 --- a/pkg/sql/rowexec/stats.go +++ b/pkg/sql/rowexec/stats.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" @@ -94,7 +93,7 @@ func newRowFetcherStatCollector(f *row.Fetcher) *rowFetcherStatCollector { // NextRow is part of the rowFetcher interface. func (c *rowFetcherStatCollector) NextRow( ctx context.Context, -) (rowenc.EncDatumRow, catalog.TableDescriptor, *descpb.IndexDescriptor, error) { +) (rowenc.EncDatumRow, catalog.TableDescriptor, catalog.Index, error) { start := timeutil.Now() row, t, i, err := c.Fetcher.NextRow(ctx) if row != nil { diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index ffd4c322606d..0aa21824e763 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -382,7 +382,7 @@ type zigzagJoinerInfo struct { rowsRead int64 alloc *rowenc.DatumAlloc table catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index indexTypes []*types.T indexDirs []descpb.IndexDescriptor_Direction @@ -428,15 +428,15 @@ func (z *zigzagJoiner) setupInfo( info.table = tables[side] info.eqColumns = spec.EqColumns[side].Columns indexOrdinal := spec.IndexOrdinals[side] - info.index = info.table.ActiveIndexes()[indexOrdinal].IndexDesc() + info.index = info.table.ActiveIndexes()[indexOrdinal] var columnIDs []descpb.ColumnID - columnIDs, info.indexDirs = info.index.FullColumnIDs() + columnIDs, info.indexDirs = catalog.FullIndexColumnIDs(info.index) info.indexTypes = make([]*types.T, len(columnIDs)) columnTypes := catalog.ColumnTypes(info.table.PublicColumns()) colIdxMap := catalog.ColumnIDToOrdinalMap(info.table.PublicColumns()) for i, columnID := range columnIDs { - if info.index.Type == descpb.IndexDescriptor_INVERTED && + if info.index.GetType() == descpb.IndexDescriptor_INVERTED && columnID == info.index.InvertedColumnID() { // Inverted key columns have type Bytes. info.indexTypes[i] = types.Bytes @@ -499,7 +499,7 @@ func (z *zigzagJoiner) setupInfo( info.fetcher = &fetcher } - info.prefix = rowenc.MakeIndexKeyPrefix(flowCtx.Codec(), info.table, info.index.ID) + info.prefix = rowenc.MakeIndexKeyPrefix(flowCtx.Codec(), info.table, info.index.GetID()) span, err := z.produceSpanFromBaseRow() if err != nil { @@ -519,9 +519,9 @@ func (z *zigzagJoiner) close() { } } -func findColumnID(s []descpb.ColumnID, t descpb.ColumnID) int { - for i := range s { - if s[i] == t { +func findColumnOrdinalInIndex(index catalog.Index, t descpb.ColumnID) int { + for i := 0; i < index.NumColumns(); i++ { + if index.GetColumnID(i) == t { return i } } @@ -592,12 +592,12 @@ func (z *zigzagJoiner) produceInvertedIndexKey( } decodedDatums[i] = encDatum.Datum - if i < len(info.index.ColumnIDs) { - colMap.Set(info.index.ColumnIDs[i], i) + if i < info.index.NumColumns() { + colMap.Set(info.index.GetColumnID(i), i) } else { // This column's value will be encoded in the second part (i.e. // EncodeColumns). - colMap.Set(info.index.ExtraColumnIDs[i-len(info.index.ColumnIDs)], i) + colMap.Set(info.index.GetExtraColumnID(i-info.index.NumColumns()), i) } } @@ -625,7 +625,7 @@ func (z *zigzagJoiner) produceInvertedIndexKey( // Append remaining (non-JSON) datums to the key. keyBytes, _, err := rowenc.EncodeColumns( - info.index.ExtraColumnIDs[:len(datums)-1], + info.index.IndexDesc().ExtraColumnIDs[:len(datums)-1], info.indexDirs[1:], colMap, decodedDatums, @@ -647,7 +647,7 @@ func (z *zigzagJoiner) produceSpanFromBaseRow() (roachpb.Span, error) { // Construct correct row by concatenating right fixed datums with // primary key extracted from `row`. - if info.index.Type == descpb.IndexDescriptor_INVERTED { + if info.index.GetType() == descpb.IndexDescriptor_INVERTED { return z.produceInvertedIndexKey(info, neededDatums) } @@ -674,12 +674,12 @@ func (zi *zigzagJoinerInfo) eqOrdering() (colinfo.ColumnOrdering, error) { // the current column, 'colID'. var direction encoding.Direction var err error - if idx := findColumnID(zi.index.ColumnIDs, colID); idx != -1 { - direction, err = zi.index.ColumnDirections[idx].ToEncodingDirection() + if idx := findColumnOrdinalInIndex(zi.index, colID); idx != -1 { + direction, err = zi.index.GetColumnDirection(idx).ToEncodingDirection() if err != nil { return nil, err } - } else if idx := findColumnID(zi.table.GetPrimaryIndex().IndexDesc().ColumnIDs, colID); idx != -1 { + } else if idx := findColumnOrdinalInIndex(zi.table.GetPrimaryIndex(), colID); idx != -1 { direction, err = zi.table.GetPrimaryIndex().GetColumnDirection(idx).ToEncodingDirection() if err != nil { return nil, err diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 726bf7b7636d..13785f751e94 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -46,10 +46,10 @@ type scanNode struct { _ util.NoCopy desc catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index // Set if an index was explicitly specified. - specifiedIndex *descpb.IndexDescriptor + specifiedIndex catalog.Index // Set if the NO_INDEX_JOIN hint was given. noIndexJoin bool @@ -231,14 +231,14 @@ func (n *scanNode) lookupSpecifiedIndex(indexFlags *tree.IndexFlags) error { if foundIndex == nil || !foundIndex.Public() { return errors.Errorf("index %q not found", tree.ErrString(&indexFlags.Index)) } - n.specifiedIndex = foundIndex.IndexDesc() + n.specifiedIndex = foundIndex } else if indexFlags.IndexID != 0 { // Search index by ID. foundIndex, _ := n.desc.FindIndexWithID(descpb.IndexID(indexFlags.IndexID)) if foundIndex == nil || !foundIndex.Public() { return errors.Errorf("index [%d] not found", indexFlags.IndexID) } - n.specifiedIndex = foundIndex.IndexDesc() + n.specifiedIndex = foundIndex } return nil } @@ -301,7 +301,7 @@ func initColsForScan( // Initializes the column structures. func (n *scanNode) initDescDefaults(colCfg scanColumnsConfig) error { n.colCfg = colCfg - n.index = n.desc.GetPrimaryIndex().IndexDesc() + n.index = n.desc.GetPrimaryIndex() var err error n.cols, err = initColsForScan(n.desc, n.colCfg) diff --git a/pkg/sql/scatter.go b/pkg/sql/scatter.go index 0344d5195942..4b0cf6576065 100644 --- a/pkg/sql/scatter.go +++ b/pkg/sql/scatter.go @@ -45,24 +45,25 @@ func (p *planner) Scatter(ctx context.Context, n *tree.Scatter) (planNode, error var span roachpb.Span if n.From == nil { // No FROM/TO specified; the span is the entire table/index. - span = tableDesc.IndexSpan(p.ExecCfg().Codec, index.ID) + span = tableDesc.IndexSpan(p.ExecCfg().Codec, index.GetID()) } else { switch { case len(n.From) == 0: return nil, errors.Errorf("no columns in SCATTER FROM expression") - case len(n.From) > len(index.ColumnIDs): + case len(n.From) > index.NumColumns(): return nil, errors.Errorf("too many columns in SCATTER FROM expression") case len(n.To) == 0: return nil, errors.Errorf("no columns in SCATTER TO expression") - case len(n.To) > len(index.ColumnIDs): + case len(n.To) > index.NumColumns(): return nil, errors.Errorf("too many columns in SCATTER TO expression") } // Calculate the desired types for the select statement: // - column values; it is OK if the select statement returns fewer columns // (the relevant prefix is used). - desiredTypes := make([]*types.T, len(index.ColumnIDs)) - for i, colID := range index.ColumnIDs { + desiredTypes := make([]*types.T, index.NumColumns()) + for i := 0; i < index.NumColumns(); i++ { + colID := index.GetColumnID(i) c, err := tableDesc.FindColumnWithID(colID) if err != nil { return nil, err diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 4b604eaa4eb0..4e7f3e3adb24 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1079,7 +1079,7 @@ func (sc *SchemaChanger) done(ctx context.Context) error { } isRollback = m.IsRollback() if idx := m.AsIndex(); m.Dropped() && idx != nil { - if canClearRangeForDrop(idx.IndexDesc()) { + if canClearRangeForDrop(idx) { // how we keep track of dropped index names (for, e.g., zone config // lookups), even though in the absence of a GC job there's nothing to // clean them up. diff --git a/pkg/sql/schemachanger/scbuild/builder.go b/pkg/sql/schemachanger/scbuild/builder.go index bb8688a39a7d..eb71d52080ac 100644 --- a/pkg/sql/schemachanger/scbuild/builder.go +++ b/pkg/sql/schemachanger/scbuild/builder.go @@ -484,7 +484,7 @@ func (b *Builder) addOrUpdatePrimaryIndexTargetsForAddColumn( // Create a new primary index, identical to the existing one except for its // ID and name. idxID = b.nextIndexID(table) - newIdx := protoutil.Clone(table.GetPrimaryIndex().IndexDesc()).(*descpb.IndexDescriptor) + newIdx := table.GetPrimaryIndex().IndexDescDeepCopy() newIdx.Name = tabledesc.GenerateUniqueConstraintName( "new_primary_key", func(name string) bool { @@ -513,7 +513,7 @@ func (b *Builder) addOrUpdatePrimaryIndexTargetsForAddColumn( b.addNode(scpb.Target_ADD, &scpb.PrimaryIndex{ TableID: table.GetID(), - Index: *newIdx, + Index: newIdx, OtherPrimaryIndexID: table.GetPrimaryIndexID(), StoreColumnIDs: append(storeColIDs, colID), StoreColumnNames: append(storeColNames, colName), @@ -522,7 +522,7 @@ func (b *Builder) addOrUpdatePrimaryIndexTargetsForAddColumn( // Drop the existing primary index. b.addNode(scpb.Target_DROP, &scpb.PrimaryIndex{ TableID: table.GetID(), - Index: *(protoutil.Clone(table.GetPrimaryIndex().IndexDesc()).(*descpb.IndexDescriptor)), + Index: table.GetPrimaryIndex().IndexDescDeepCopy(), OtherPrimaryIndexID: newIdx.ID, StoreColumnIDs: storeColIDs, StoreColumnNames: storeColNames, diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index c464cfc5fd06..169592576e16 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -284,13 +284,13 @@ func (n *scrubNode) startScrubTable( // getPrimaryColIdxs returns a list of the primary index columns and // their corresponding index in the columns list. func getPrimaryColIdxs( - tableDesc catalog.TableDescriptor, columns []*descpb.ColumnDescriptor, + tableDesc catalog.TableDescriptor, columns []catalog.Column, ) (primaryColIdxs []int, err error) { for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { colID := tableDesc.GetPrimaryIndex().GetColumnID(i) rowIdx := -1 for idx, col := range columns { - if col.ID == colID { + if col.GetID() == colID { rowIdx = idx break } @@ -347,7 +347,7 @@ func createPhysicalCheckOperations( tableDesc catalog.TableDescriptor, tableName *tree.TableName, ) (checks []checkOperation) { for _, idx := range tableDesc.ActiveIndexes() { - checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, idx.IndexDesc())) + checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, idx)) } return checks } @@ -371,7 +371,7 @@ func createIndexCheckOperations( results = append(results, newIndexCheckOperation( tableName, tableDesc, - idx.IndexDesc(), + idx, asOf, )) } @@ -388,7 +388,7 @@ func createIndexCheckOperations( results = append(results, newIndexCheckOperation( tableName, tableDesc, - idx.IndexDesc(), + idx, asOf, )) delete(names, idx.GetName()) diff --git a/pkg/sql/scrub_constraint.go b/pkg/sql/scrub_constraint.go index 845617642f9b..72f46ff99cf2 100644 --- a/pkg/sql/scrub_constraint.go +++ b/pkg/sql/scrub_constraint.go @@ -34,7 +34,7 @@ type sqlCheckConstraintCheckOperation struct { // columns is a list of the columns returned in the query result // tree.Datums. - columns []*descpb.ColumnDescriptor + columns []catalog.Column // primaryColIdxs maps PrimaryIndex.Columns to the row // indexes in the query result tree.Datums. primaryColIdxs []int @@ -107,11 +107,8 @@ func (o *sqlCheckConstraintCheckOperation) Start(params runParams) error { o.run.started = true o.run.rows = rows - // Collect all the columns. - for _, c := range o.tableDesc.PublicColumns() { - o.columns = append(o.columns, c.ColumnDesc()) - } + o.columns = o.tableDesc.PublicColumns() // Find the row indexes for all of the primary index columns. o.primaryColIdxs, err = getPrimaryColIdxs(o.tableDesc, o.columns) return err @@ -140,7 +137,7 @@ func (o *sqlCheckConstraintCheckOperation) Next(params runParams) (tree.Datums, details["constraint_name"] = o.checkDesc.Name for rowIdx, col := range o.columns { // TODO(joey): We should maybe try to get the underlying type. - rowDetails[col.Name] = row[rowIdx].String() + rowDetails[col.GetName()] = row[rowIdx].String() } detailsJSON, err := tree.MakeDJSON(details) if err != nil { diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index 3a002b25eac5..2e00f6e024fe 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -102,15 +102,6 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { o.run.rows = append(o.run.rows, rows...) } - // Collect the expected types for the query results. This is all - // columns and extra columns in the secondary index used for foreign - // key referencing. This also implicitly includes all primary index - // columns. - columnsByID := make(map[descpb.ColumnID]*descpb.ColumnDescriptor, len(o.tableDesc.PublicColumns())) - for _, c := range o.tableDesc.PublicColumns() { - columnsByID[c.GetID()] = c.ColumnDesc() - } - // Get primary key columns not included in the FK. var colIDs []descpb.ColumnID colIDs = append(colIDs, o.constraint.FK.OriginColumnIDs...) diff --git a/pkg/sql/scrub_index.go b/pkg/sql/scrub_index.go index fbecb3cd09dc..4bda5f728503 100644 --- a/pkg/sql/scrub_index.go +++ b/pkg/sql/scrub_index.go @@ -33,13 +33,13 @@ import ( type indexCheckOperation struct { tableName *tree.TableName tableDesc catalog.TableDescriptor - indexDesc *descpb.IndexDescriptor + index catalog.Index asOf hlc.Timestamp // columns is a list of the columns returned by one side of the // queries join. The actual resulting rows from the RowContainer is // twice this. - columns []*descpb.ColumnDescriptor + columns []catalog.Column // primaryColIdxs maps PrimaryIndex.Columns to the row // indexes in the query result tree.Datums. primaryColIdxs []int @@ -58,13 +58,13 @@ type indexCheckRun struct { func newIndexCheckOperation( tableName *tree.TableName, tableDesc catalog.TableDescriptor, - indexDesc *descpb.IndexDescriptor, + index catalog.Index, asOf hlc.Timestamp, ) *indexCheckOperation { return &indexCheckOperation{ tableName: tableName, tableDesc: tableDesc, - indexDesc: indexDesc, + index: index, asOf: asOf, } } @@ -79,48 +79,41 @@ func (o *indexCheckOperation) Start(params runParams) error { colToIdx.Set(c.GetID(), c.Ordinal()) } - var pkColumns, otherColumns []*descpb.ColumnDescriptor + var pkColumns, otherColumns []catalog.Column for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { colID := o.tableDesc.GetPrimaryIndex().GetColumnID(i) col := o.tableDesc.PublicColumns()[colToIdx.GetDefault(colID)] - pkColumns = append(pkColumns, col.ColumnDesc()) + pkColumns = append(pkColumns, col) colToIdx.Set(colID, -1) } - maybeAddOtherCol := func(colID descpb.ColumnID) { + maybeAddOtherCol := func(colID descpb.ColumnID) error { pos := colToIdx.GetDefault(colID) if pos == -1 { // Skip PK column. - return + return nil } col := o.tableDesc.PublicColumns()[pos] - otherColumns = append(otherColumns, col.ColumnDesc()) + otherColumns = append(otherColumns, col) + return nil } // Collect all of the columns we are fetching from the index. This // includes the columns involved in the index: columns, extra columns, // and store columns. - for _, colID := range o.indexDesc.ColumnIDs { - maybeAddOtherCol(colID) - } - for _, colID := range o.indexDesc.ExtraColumnIDs { - maybeAddOtherCol(colID) - } - for _, colID := range o.indexDesc.StoreColumnIDs { - maybeAddOtherCol(colID) - } + _ = o.index.ForEachColumnID(maybeAddOtherCol) - colNames := func(cols []*descpb.ColumnDescriptor) []string { + colNames := func(cols []catalog.Column) []string { res := make([]string, len(cols)) for i := range cols { - res[i] = cols[i].Name + res[i] = cols[i].GetName() } return res } checkQuery := createIndexCheckQuery( - colNames(pkColumns), colNames(otherColumns), o.tableDesc.GetID(), o.indexDesc.ID, + colNames(pkColumns), colNames(otherColumns), o.tableDesc.GetID(), o.index.GetID(), ) rows, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.QueryBuffered( @@ -181,12 +174,12 @@ func (o *indexCheckOperation) Next(params runParams) (tree.Datums, error) { details := make(map[string]interface{}) rowDetails := make(map[string]interface{}) details["row_data"] = rowDetails - details["index_name"] = o.indexDesc.Name + details["index_name"] = o.index.GetName() if isMissingIndexReferenceError { // Fetch the primary index values from the primary index row data. for rowIdx, col := range o.columns { // TODO(joey): We should maybe try to get the underlying type. - rowDetails[col.Name] = row[rowIdx].String() + rowDetails[col.GetName()] = row[rowIdx].String() } } else { // Fetch the primary index values from the secondary index row data, @@ -195,7 +188,7 @@ func (o *indexCheckOperation) Next(params runParams) (tree.Datums, error) { // set of columns is for the primary index. for rowIdx, col := range o.columns { // TODO(joey): We should maybe try to get the underlying type. - rowDetails[col.Name] = row[rowIdx+colLen].String() + rowDetails[col.GetName()] = row[rowIdx+colLen].String() } } diff --git a/pkg/sql/scrub_physical.go b/pkg/sql/scrub_physical.go index 1ba8357a1979..517ca3795a65 100644 --- a/pkg/sql/scrub_physical.go +++ b/pkg/sql/scrub_physical.go @@ -29,11 +29,11 @@ var _ checkOperation = &physicalCheckOperation{} type physicalCheckOperation struct { tableName *tree.TableName tableDesc catalog.TableDescriptor - indexDesc *descpb.IndexDescriptor + index catalog.Index // columns is a list of the columns returned in the query result // tree.Datums. - columns []*descpb.ColumnDescriptor + columns []catalog.Column // primaryColIdxs maps PrimaryIndex.Columns to the row // indexes in the query result tree.Datums. primaryColIdxs []int @@ -50,12 +50,12 @@ type physicalCheckRun struct { } func newPhysicalCheckOperation( - tableName *tree.TableName, tableDesc catalog.TableDescriptor, indexDesc *descpb.IndexDescriptor, + tableName *tree.TableName, tableDesc catalog.TableDescriptor, index catalog.Index, ) *physicalCheckOperation { return &physicalCheckOperation{ tableName: tableName, tableDesc: tableDesc, - indexDesc: indexDesc, + index: index, } } @@ -67,28 +67,31 @@ func (o *physicalCheckOperation) Start(params runParams) error { // Collect all of the columns, their types, and their IDs. var columnIDs []tree.ColumnID colIDToIdx := catalog.ColumnIDToOrdinalMap(o.tableDesc.PublicColumns()) - columns := make([]*descpb.ColumnDescriptor, len(columnIDs)) + columns := make([]catalog.Column, len(columnIDs)) // Collect all of the columns being scanned. - if o.indexDesc.ID == o.tableDesc.GetPrimaryIndexID() { + if o.index.GetID() == o.tableDesc.GetPrimaryIndexID() { for _, c := range o.tableDesc.PublicColumns() { columnIDs = append(columnIDs, tree.ColumnID(c.GetID())) } } else { - for _, id := range o.indexDesc.ColumnIDs { + for i := 0; i < o.index.NumColumns(); i++ { + id := o.index.GetColumnID(i) columnIDs = append(columnIDs, tree.ColumnID(id)) } - for _, id := range o.indexDesc.ExtraColumnIDs { + for i := 0; i < o.index.NumExtraColumns(); i++ { + id := o.index.GetExtraColumnID(i) columnIDs = append(columnIDs, tree.ColumnID(id)) } - for _, id := range o.indexDesc.StoreColumnIDs { + for i := 0; i < o.index.NumStoredColumns(); i++ { + id := o.index.GetStoredColumnID(i) columnIDs = append(columnIDs, tree.ColumnID(id)) } } for i := range columnIDs { idx := colIDToIdx.GetDefault(descpb.ColumnID(columnIDs[i])) - columns = append(columns, o.tableDesc.PublicColumns()[idx].ColumnDesc()) + columns = append(columns, o.tableDesc.PublicColumns()[idx]) } // Find the row indexes for all of the primary index columns. @@ -98,7 +101,7 @@ func (o *physicalCheckOperation) Start(params runParams) error { } indexFlags := &tree.IndexFlags{ - IndexID: tree.IndexID(o.indexDesc.ID), + IndexID: tree.IndexID(o.index.GetID()), NoIndexJoin: true, } scan := params.p.Scan() @@ -108,7 +111,7 @@ func (o *physicalCheckOperation) Start(params runParams) error { return err } scan.index = scan.specifiedIndex - sb := span.MakeBuilder(params.EvalContext(), params.ExecCfg().Codec, o.tableDesc, o.indexDesc) + sb := span.MakeBuilder(params.EvalContext(), params.ExecCfg().Codec, o.tableDesc, o.index) scan.spans, err = sb.UnconstrainedSpans() if err != nil { return err diff --git a/pkg/sql/scrub_test.go b/pkg/sql/scrub_test.go index 972adb6bdcd0..e538b09ec161 100644 --- a/pkg/sql/scrub_test.go +++ b/pkg/sql/scrub_test.go @@ -69,7 +69,7 @@ INSERT INTO t."tEst" VALUES (10, 20); // Construct the secondary index key that is currently in the // database. secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex, colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -138,7 +138,7 @@ CREATE INDEX secondary ON t.test (v); // Construct datums and secondary k/v for our row values (k, v). values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(314)} secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex, colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -233,7 +233,7 @@ INSERT INTO t.test VALUES (10, 20, 1337); // Generate the existing secondary index key. values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(20), tree.NewDInt(1337)} secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex, colIDtoRowIndex, values, true /* includeEmpty */) if len(secondaryIndexKey) != 1 { t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndexKey), secondaryIndexKey) @@ -250,7 +250,7 @@ INSERT INTO t.test VALUES (10, 20, 1337); // Generate a secondary index k/v that has a different value. values = []tree.Datum{tree.NewDInt(10), tree.NewDInt(20), tree.NewDInt(314)} secondaryIndexKey, err = rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex, colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -353,7 +353,7 @@ INSERT INTO t.test VALUES (10, 2); primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -454,7 +454,7 @@ func TestScrubFKConstraintFKMissing(t *testing.T) { // Construct the secondary index key entry as it exists in the // database. secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex, colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -474,7 +474,7 @@ func TestScrubFKConstraintFKMissing(t *testing.T) { // Construct the new secondary index key that will be inserted. secondaryIndexKey, err = rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex, colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -593,7 +593,7 @@ INSERT INTO t.test VALUES (217, 314); primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -676,7 +676,7 @@ INSERT INTO t.test VALUES (217, 314, 1337); primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -781,7 +781,7 @@ CREATE TABLE t.test ( primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -886,7 +886,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v1 INT, v2 INT); primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( keys.SystemSQLCodec, tableDesc, tableDesc.GetPrimaryIndexID()) primaryIndexKey, _, err := rowenc.EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index ae8d7e17d4ec..301bc563d073 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -3968,25 +3968,30 @@ may increase either contention or retry errors, or both.`, if err != nil { return nil, err } - indexDesc := index.IndexDesc() // Collect the index columns. If the index is a non-unique secondary // index, it might have some extra key columns. - indexColIDs := indexDesc.ColumnIDs - if indexDesc.ID != tableDesc.GetPrimaryIndexID() && !indexDesc.Unique { - indexColIDs = append(indexColIDs, indexDesc.ExtraColumnIDs...) + indexColIDs := make([]descpb.ColumnID, index.NumColumns(), index.NumColumns()+index.NumExtraColumns()) + for i := 0; i < index.NumColumns(); i++ { + indexColIDs[i] = index.GetColumnID(i) + } + if index.GetID() != tableDesc.GetPrimaryIndexID() && !index.IsUnique() { + for i := 0; i < index.NumExtraColumns(); i++ { + indexColIDs = append(indexColIDs, index.GetExtraColumnID(i)) + } } // Ensure that the input tuple length equals the number of index cols. if len(rowDatums.D) != len(indexColIDs) { err := errors.Newf( "number of values must equal number of columns in index %q", - indexDesc.Name, + index.GetName(), ) // If the index has some extra key columns, then output an error // message with some extra information to explain the subtlety. - if indexDesc.ID != tableDesc.GetPrimaryIndexID() && !indexDesc.Unique && len(indexDesc.ExtraColumnIDs) > 0 { + if index.GetID() != tableDesc.GetPrimaryIndexID() && !index.IsUnique() && index.NumExtraColumns() > 0 { var extraColNames []string - for _, id := range indexDesc.ExtraColumnIDs { + for i := 0; i < index.NumExtraColumns(); i++ { + id := index.GetExtraColumnID(i) col, colErr := tableDesc.FindColumnWithID(id) if colErr != nil { return nil, errors.CombineErrors(err, colErr) @@ -4005,7 +4010,7 @@ may increase either contention or retry errors, or both.`, err, "columns %v are implicitly part of index %q's key, include columns %v in this order", extraColNames, - indexDesc.Name, + index.GetName(), allColNames, ) } @@ -4047,8 +4052,8 @@ may increase either contention or retry errors, or both.`, colMap.Set(id, i) } // Finally, encode the index key using the provided datums. - keyPrefix := rowenc.MakeIndexKeyPrefix(ctx.Codec, tableDesc, indexDesc.ID) - res, _, err := rowenc.EncodePartialIndexKey(tableDesc, indexDesc, len(datums), colMap, datums, keyPrefix) + keyPrefix := rowenc.MakeIndexKeyPrefix(ctx.Codec, tableDesc, index.GetID()) + res, _, err := rowenc.EncodePartialIndexKey(tableDesc, index, len(datums), colMap, datums, keyPrefix) if err != nil { return nil, err } @@ -4460,7 +4465,7 @@ may increase either contention or retry errors, or both.`, if index.GetGeoConfig().S2Geography == nil { return nil, errors.Errorf("index_id %d is not a geography inverted index", indexID) } - keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.IndexDesc()) + keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.GetGeoConfig()) if err != nil { return nil, err } @@ -4494,7 +4499,7 @@ may increase either contention or retry errors, or both.`, if index.GetGeoConfig().S2Geometry == nil { return nil, errors.Errorf("index_id %d is not a geometry inverted index", indexID) } - keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.IndexDesc()) + keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.GetGeoConfig()) if err != nil { return nil, err } diff --git a/pkg/sql/sequence.go b/pkg/sql/sequence.go index ae0fb9fa5e60..e10eaf693317 100644 --- a/pkg/sql/sequence.go +++ b/pkg/sql/sequence.go @@ -495,7 +495,7 @@ func assignSequenceOptions( // We only want to trigger schema changes if the owner is not what we // want it to be. if opts.SequenceOwner.OwnerTableID != tableDesc.ID || - opts.SequenceOwner.OwnerColumnID != col.ID { + opts.SequenceOwner.OwnerColumnID != col.GetID() { if err := removeSequenceOwnerIfExists(params.ctx, params.p, sequenceID, opts); err != nil { return err } @@ -559,17 +559,17 @@ func removeSequenceOwnerIfExists( return err } // Find an item in colDesc.OwnsSequenceIds which references SequenceID. - refIdx := -1 + newOwnsSequenceIDs := make([]descpb.ID, 0, col.NumOwnsSequences()) for i := 0; i < col.NumOwnsSequences(); i++ { id := col.GetOwnsSequenceID(i) - if id == sequenceID { - refIdx = i + if id != sequenceID { + newOwnsSequenceIDs = append(newOwnsSequenceIDs, id) } } - if refIdx == -1 { + if len(newOwnsSequenceIDs) == col.NumOwnsSequences() { return errors.AssertionFailedf("couldn't find reference from column to this sequence") } - col.ColumnDesc().OwnsSequenceIds = append(col.ColumnDesc().OwnsSequenceIds[:refIdx], col.ColumnDesc().OwnsSequenceIds[refIdx+1:]...) + col.ColumnDesc().OwnsSequenceIds = newOwnsSequenceIDs if err := p.writeSchemaChange( ctx, tableDesc, descpb.InvalidMutationID, fmt.Sprintf("removing sequence owner %s(%d) for sequence %d", @@ -585,7 +585,7 @@ func removeSequenceOwnerIfExists( func resolveColumnItemToDescriptors( ctx context.Context, p *planner, columnItem *tree.ColumnItem, -) (*tabledesc.Mutable, *descpb.ColumnDescriptor, error) { +) (*tabledesc.Mutable, catalog.Column, error) { if columnItem.TableName == nil { err := pgerror.New(pgcode.Syntax, "invalid OWNED BY option") return nil, nil, errors.WithHint(err, "Specify OWNED BY table.column or OWNED BY NONE.") @@ -599,7 +599,7 @@ func resolveColumnItemToDescriptors( if err != nil { return nil, nil, err } - return tableDesc, col.ColumnDesc(), nil + return tableDesc, col, nil } func addSequenceOwner( @@ -614,9 +614,9 @@ func addSequenceOwner( return err } - col.OwnsSequenceIds = append(col.OwnsSequenceIds, sequenceID) + col.ColumnDesc().OwnsSequenceIds = append(col.ColumnDesc().OwnsSequenceIds, sequenceID) - opts.SequenceOwner.OwnerColumnID = col.ID + opts.SequenceOwner.OwnerColumnID = col.GetID() opts.SequenceOwner.OwnerTableID = tableDesc.GetID() return p.writeSchemaChange( ctx, tableDesc, descpb.InvalidMutationID, fmt.Sprintf( @@ -736,12 +736,16 @@ func GetSequenceDescFromIdentifier( // dropSequencesOwnedByCol drops all the sequences from col.OwnsSequenceIDs. // Called when the respective column (or the whole table) is being dropped. func (p *planner) dropSequencesOwnedByCol( - ctx context.Context, col *descpb.ColumnDescriptor, queueJob bool, behavior tree.DropBehavior, + ctx context.Context, col catalog.Column, queueJob bool, behavior tree.DropBehavior, ) error { // Copy out the sequence IDs as the code to drop the sequence will reach // back around and update the descriptor from underneath us. - ownsSequenceIDs := append([]descpb.ID(nil), col.OwnsSequenceIds...) - for _, sequenceID := range ownsSequenceIDs { + colOwnsSequenceIDs := make([]descpb.ID, col.NumOwnsSequences()) + for i := 0; i < col.NumOwnsSequences(); i++ { + colOwnsSequenceIDs[i] = col.GetOwnsSequenceID(i) + } + + for _, sequenceID := range colOwnsSequenceIDs { seqDesc, err := p.Descriptors().GetMutableTableVersionByID(ctx, sequenceID, p.txn) // Special case error swallowing for #50781, which can cause a // column to own sequences that do not exist. @@ -776,9 +780,10 @@ func (p *planner) dropSequencesOwnedByCol( // - writes the sequence descriptor and notifies a schema change. // The column descriptor is mutated but not saved to persistent storage; the caller must save it. func (p *planner) removeSequenceDependencies( - ctx context.Context, tableDesc *tabledesc.Mutable, col *descpb.ColumnDescriptor, + ctx context.Context, tableDesc *tabledesc.Mutable, col catalog.Column, ) error { - for _, sequenceID := range col.UsesSequenceIds { + for i := 0; i < col.NumUsesSequences(); i++ { + sequenceID := col.GetUsesSequenceID(i) // Get the sequence descriptor so we can remove the reference from it. seqDesc, err := p.Descriptors().GetMutableTableVersionByID(ctx, sequenceID, p.txn) if err != nil { @@ -803,7 +808,7 @@ func (p *planner) removeSequenceDependencies( if reference.ID == tableDesc.ID { refTableIdx = i for j, colRefID := range seqDesc.DependedOnBy[i].ColumnIDs { - if colRefID == col.ID { + if colRefID == col.GetID() { refColIdx = j break found } @@ -842,6 +847,6 @@ func (p *planner) removeSequenceDependencies( } } // Remove the reference from the column descriptor to the sequence descriptor. - col.UsesSequenceIds = []descpb.ID{} + col.ColumnDesc().UsesSequenceIds = []descpb.ID{} return nil } diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index 2253762e7241..2a37fcecb92d 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -427,7 +427,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { var indexes []catalog.Index for _, idx := range table.NonDropIndexes() { - if tabledesc.FindIndexPartitionByName(idx.IndexDesc(), partitionName) != nil { + if tabledesc.FindIndexPartitionByName(idx, partitionName) != nil { indexes = append(indexes, idx) } } @@ -453,7 +453,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { if n.zoneSpecifier.TargetsPartition() && n.allIndexes { sqltelemetry.IncrementPartitioningCounter(sqltelemetry.AlterAllPartitions) for _, idx := range table.NonDropIndexes() { - if p := tabledesc.FindIndexPartitionByName(idx.IndexDesc(), string(n.zoneSpecifier.Partition)); p != nil { + if p := tabledesc.FindIndexPartitionByName(idx, string(n.zoneSpecifier.Partition)); p != nil { zs := n.zoneSpecifier zs.TableOrIndex.Index = tree.UnrestrictedName(idx.GetName()) specifiers = append(specifiers, zs) @@ -507,7 +507,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { var partialSubzone *zonepb.Subzone if index != nil { - partialSubzone = partialZone.GetSubzoneExact(uint32(index.ID), partition) + partialSubzone = partialZone.GetSubzoneExact(uint32(index.GetID()), partition) if partialSubzone == nil { partialSubzone = &zonepb.Subzone{Config: *zonepb.NewZoneConfig()} } @@ -573,7 +573,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // In the case of updating a partition, we need try inheriting fields // from the subzone's index, and inherit the remainder from the zone. subzoneInheritedFields := zonepb.ZoneConfig{} - if indexSubzone := completeZone.GetSubzone(uint32(index.ID), ""); indexSubzone != nil { + if indexSubzone := completeZone.GetSubzone(uint32(index.GetID()), ""); indexSubzone != nil { subzoneInheritedFields.InheritFromParent(&indexSubzone.Config) } subzoneInheritedFields.InheritFromParent(&zoneInheritedFields) @@ -586,8 +586,8 @@ func (n *setZoneConfigNode) startExec(params runParams) error { if deleteZone { if index != nil { - didDelete := completeZone.DeleteSubzone(uint32(index.ID), partition) - _ = partialZone.DeleteSubzone(uint32(index.ID), partition) + didDelete := completeZone.DeleteSubzone(uint32(index.GetID()), partition) + _ = partialZone.DeleteSubzone(uint32(index.GetID()), partition) if !didDelete { // If we didn't do any work, return early. We'd otherwise perform an // update that would make it look like one row was affected. @@ -719,7 +719,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { completeZone = zonepb.NewZoneConfig() } completeZone.SetSubzone(zonepb.Subzone{ - IndexID: uint32(index.ID), + IndexID: uint32(index.GetID()), PartitionName: partition, Config: newZone, }) @@ -731,7 +731,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { } partialZone.SetSubzone(zonepb.Subzone{ - IndexID: uint32(index.ID), + IndexID: uint32(index.GetID()), PartitionName: partition, Config: finalZone, }) @@ -1052,7 +1052,7 @@ func RemoveIndexZoneConfigs( txn *kv.Txn, execCfg *ExecutorConfig, tableDesc catalog.TableDescriptor, - indexDescs []descpb.IndexDescriptor, + indexIDs []uint32, ) error { if !execCfg.Codec.ForSystemTenant() { // Tenants are agnostic to zone configs. @@ -1071,12 +1071,12 @@ func RemoveIndexZoneConfigs( // of them. We only want to rewrite the zone config below if there's actual // work to be done here. zcRewriteNecessary := false - for _, indexDesc := range indexDescs { + for _, indexID := range indexIDs { for _, s := range zone.Subzones { - if s.IndexID == uint32(indexDesc.ID) { + if s.IndexID == indexID { // We've found an subzone that matches the given indexID. Delete all of // this index's subzones and move on to the next index. - zone.DeleteIndexSubzones(uint32(indexDesc.ID)) + zone.DeleteIndexSubzones(indexID) zcRewriteNecessary = true break } diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index cd1d550489d5..187c1705b560 100644 --- a/pkg/sql/show_create.go +++ b/pkg/sql/show_create.go @@ -87,7 +87,7 @@ func ShowCreateTable( f.WriteString(",") } f.WriteString("\n\t") - colstr, err := schemaexpr.FormatColumnForDisplay(ctx, desc, col.ColumnDesc(), &p.RunParams(ctx).p.semaCtx) + colstr, err := schemaexpr.FormatColumnForDisplay(ctx, desc, col, &p.RunParams(ctx).p.semaCtx) if err != nil { return "", err } @@ -155,7 +155,7 @@ func ShowCreateTable( // Build the PARTITION BY clause. var partitionBuf bytes.Buffer if err := ShowCreatePartitioning( - a, p.ExecCfg().Codec, desc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, &partitionBuf, 1 /* indent */, 0, /* colOffset */ + a, p.ExecCfg().Codec, desc, idx, &idx.IndexDesc().Partitioning, &partitionBuf, 1 /* indent */, 0, /* colOffset */ ); err != nil { return "", err } @@ -166,7 +166,7 @@ func ShowCreateTable( if includeInterleaveClause { // TODO(mgartner): The logic in showCreateInterleave can be // moved to catformat.IndexForDisplay. - if err := showCreateInterleave(idx.IndexDesc(), &interleaveBuf, dbPrefix, lCtx); err != nil { + if err := showCreateInterleave(idx, &interleaveBuf, dbPrefix, lCtx); err != nil { return "", err } } @@ -176,7 +176,7 @@ func ShowCreateTable( ctx, desc, &descpb.AnonymousTable, - idx.IndexDesc(), + idx, partitionBuf.String(), interleaveBuf.String(), p.RunParams(ctx).p.SemaCtx(), @@ -195,11 +195,11 @@ func ShowCreateTable( return "", err } - if err := showCreateInterleave(desc.GetPrimaryIndex().IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { + if err := showCreateInterleave(desc.GetPrimaryIndex(), &f.Buffer, dbPrefix, lCtx); err != nil { return "", err } if err := ShowCreatePartitioning( - a, p.ExecCfg().Codec, desc, desc.GetPrimaryIndex().IndexDesc(), &desc.GetPrimaryIndex().IndexDesc().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ + a, p.ExecCfg().Codec, desc, desc.GetPrimaryIndex(), &desc.GetPrimaryIndex().IndexDesc().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ ); err != nil { return "", err } diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index 0ab31e5a10b3..7807c7d04e77 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -332,12 +332,12 @@ func showCreateLocality(desc catalog.TableDescriptor, f *tree.FmtCtx) error { // it is equal to the given dbPrefix. This allows us to elide the prefix // when the given index is interleaved in a table of the current database. func showCreateInterleave( - idx *descpb.IndexDescriptor, buf *bytes.Buffer, dbPrefix string, lCtx simpleSchemaResolver, + idx catalog.Index, buf *bytes.Buffer, dbPrefix string, lCtx simpleSchemaResolver, ) error { - if len(idx.Interleave.Ancestors) == 0 { + if idx.NumInterleaveAncestors() == 0 { return nil } - intl := idx.Interleave + intl := idx.IndexDesc().Interleave parentTableID := intl.Ancestors[len(intl.Ancestors)-1].TableID var err error var parentName tree.TableName @@ -360,7 +360,7 @@ func showCreateInterleave( fmtCtx.FormatNode(&parentName) buf.WriteString(fmtCtx.CloseAndGetString()) buf.WriteString(" (") - formatQuoteNames(buf, idx.ColumnNames[:sharedPrefixLen]...) + formatQuoteNames(buf, idx.IndexDesc().ColumnNames[:sharedPrefixLen]...) buf.WriteString(")") return nil } @@ -371,21 +371,21 @@ func ShowCreatePartitioning( a *rowenc.DatumAlloc, codec keys.SQLCodec, tableDesc catalog.TableDescriptor, - idxDesc *descpb.IndexDescriptor, + idx catalog.Index, partDesc *descpb.PartitioningDescriptor, buf *bytes.Buffer, indent int, colOffset int, ) error { isPrimaryKeyOfPartitionAllByTable := - tableDesc.IsPartitionAllBy() && tableDesc.GetPrimaryIndexID() == idxDesc.ID && colOffset == 0 + tableDesc.IsPartitionAllBy() && tableDesc.GetPrimaryIndexID() == idx.GetID() && colOffset == 0 if partDesc.NumColumns == 0 && !isPrimaryKeyOfPartitionAllByTable { return nil } // Do not print PARTITION BY clauses of non-primary indexes belonging to a table // that is PARTITION BY ALL. The ALL will be printed for the PRIMARY INDEX clause. - if tableDesc.IsPartitionAllBy() && tableDesc.GetPrimaryIndexID() != idxDesc.ID { + if tableDesc.IsPartitionAllBy() && tableDesc.GetPrimaryIndexID() != idx.GetID() { return nil } // Do not print PARTITION ALL BY if we are a REGIONAL BY ROW table. @@ -424,7 +424,7 @@ func ShowCreatePartitioning( if i != 0 { buf.WriteString(", ") } - buf.WriteString(idxDesc.ColumnNames[colOffset+i]) + buf.WriteString(idx.GetColumnName(colOffset + i)) } buf.WriteString(`) (`) fmtCtx := tree.NewFmtCtx(tree.FmtSimple) @@ -444,7 +444,7 @@ func ShowCreatePartitioning( buf.WriteString(`, `) } tuple, _, err := rowenc.DecodePartitionTuple( - a, codec, tableDesc, idxDesc, partDesc, values, fakePrefixDatums) + a, codec, tableDesc, idx, partDesc, values, fakePrefixDatums) if err != nil { return err } @@ -452,7 +452,7 @@ func ShowCreatePartitioning( } buf.WriteString(`)`) if err := ShowCreatePartitioning( - a, codec, tableDesc, idxDesc, &part.Subpartitioning, buf, indent+1, + a, codec, tableDesc, idx, &part.Subpartitioning, buf, indent+1, colOffset+int(partDesc.NumColumns), ); err != nil { return err @@ -468,14 +468,14 @@ func ShowCreatePartitioning( buf.WriteString(part.Name) buf.WriteString(" VALUES FROM ") fromTuple, _, err := rowenc.DecodePartitionTuple( - a, codec, tableDesc, idxDesc, partDesc, part.FromInclusive, fakePrefixDatums) + a, codec, tableDesc, idx, partDesc, part.FromInclusive, fakePrefixDatums) if err != nil { return err } buf.WriteString(fromTuple.String()) buf.WriteString(" TO ") toTuple, _, err := rowenc.DecodePartitionTuple( - a, codec, tableDesc, idxDesc, partDesc, part.ToExclusive, fakePrefixDatums) + a, codec, tableDesc, idx, partDesc, part.ToExclusive, fakePrefixDatums) if err != nil { return err } diff --git a/pkg/sql/show_fingerprints.go b/pkg/sql/show_fingerprints.go index 00aa5f502291..1543c52cd9d3 100644 --- a/pkg/sql/show_fingerprints.go +++ b/pkg/sql/show_fingerprints.go @@ -29,7 +29,7 @@ type showFingerprintsNode struct { optColumnsSlot tableDesc catalog.TableDescriptor - indexes []*descpb.IndexDescriptor + indexes []catalog.Index run showFingerprintsRun } @@ -65,15 +65,9 @@ func (p *planner) ShowFingerprints( return nil, err } - indexes := tableDesc.NonDropIndexes() - indexDescs := make([]*descpb.IndexDescriptor, len(indexes)) - for i, index := range indexes { - indexDescs[i] = index.IndexDesc() - } - return &showFingerprintsNode{ tableDesc: tableDesc, - indexes: indexDescs, + indexes: tableDesc.NonDropIndexes(), }, nil } @@ -97,30 +91,34 @@ func (n *showFingerprintsNode) Next(params runParams) (bool, error) { index := n.indexes[n.run.rowIdx] cols := make([]string, 0, len(n.tableDesc.PublicColumns())) - addColumn := func(col *descpb.ColumnDescriptor) { + addColumn := func(col catalog.Column) { // TODO(dan): This is known to be a flawed way to fingerprint. Any datum // with the same string representation is fingerprinted the same, even // if they're different types. - switch col.Type.Family() { + name := col.GetName() + switch col.GetType().Family() { case types.BytesFamily: - cols = append(cols, fmt.Sprintf("%s:::bytes", tree.NameStringP(&col.Name))) + cols = append(cols, fmt.Sprintf("%s:::bytes", tree.NameStringP(&name))) default: - cols = append(cols, fmt.Sprintf("%s::string::bytes", tree.NameStringP(&col.Name))) + cols = append(cols, fmt.Sprintf("%s::string::bytes", tree.NameStringP(&name))) } } - if index.ID == n.tableDesc.GetPrimaryIndexID() { + if index.GetID() == n.tableDesc.GetPrimaryIndexID() { for _, col := range n.tableDesc.PublicColumns() { - addColumn(col.ColumnDesc()) + addColumn(col) } } else { - colsByID := make(map[descpb.ColumnID]*descpb.ColumnDescriptor) - for _, col := range n.tableDesc.PublicColumns() { - colsByID[col.GetID()] = col.ColumnDesc() - } - colIDs := append(append(index.ColumnIDs, index.ExtraColumnIDs...), index.StoreColumnIDs...) - for _, colID := range colIDs { - addColumn(colsByID[colID]) + err := index.ForEachColumnID(func(id descpb.ColumnID) error { + col, err := n.tableDesc.FindColumnWithID(id) + if err != nil { + return err + } + addColumn(col) + return nil + }) + if err != nil { + return false, err } } @@ -139,7 +137,7 @@ func (n *showFingerprintsNode) Next(params runParams) (bool, error) { sql := fmt.Sprintf(`SELECT xor_agg(fnv64(%s))::string AS fingerprint FROM [%d AS t]@{FORCE_INDEX=[%d]} - `, strings.Join(cols, `,`), n.tableDesc.GetID(), index.ID) + `, strings.Join(cols, `,`), n.tableDesc.GetID(), index.GetID()) // If were'in in an AOST context, propagate it to the inner statement so that // the inner statement gets planned with planner.avoidCachedDescriptors set, // like the outter one. @@ -165,7 +163,7 @@ func (n *showFingerprintsNode) Next(params runParams) (bool, error) { } fingerprint := fingerprintCols[0] - n.run.values[0] = tree.NewDString(index.Name) + n.run.values[0] = tree.NewDString(index.GetName()) n.run.values[1] = fingerprint n.run.rowIdx++ return true, nil diff --git a/pkg/sql/span/span_builder.go b/pkg/sql/span/span_builder.go index 0b6da0edfdb6..cc5dc3dbff63 100644 --- a/pkg/sql/span/span_builder.go +++ b/pkg/sql/span/span_builder.go @@ -34,7 +34,7 @@ type Builder struct { evalCtx *tree.EvalContext codec keys.SQLCodec table catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index indexColTypes []*types.T indexColDirs []descpb.IndexDescriptor_Direction @@ -60,20 +60,20 @@ func MakeBuilder( evalCtx *tree.EvalContext, codec keys.SQLCodec, table catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, ) *Builder { s := &Builder{ evalCtx: evalCtx, codec: codec, table: table, index: index, - KeyPrefix: rowenc.MakeIndexKeyPrefix(codec, table, index.ID), - interstices: make([][]byte, len(index.ColumnDirections)+len(index.ExtraColumnIDs)+1), + KeyPrefix: rowenc.MakeIndexKeyPrefix(codec, table, index.GetID()), + interstices: make([][]byte, index.NumColumns()+index.NumExtraColumns()+1), neededFamilies: nil, } var columnIDs descpb.ColumnIDs - columnIDs, s.indexColDirs = index.FullColumnIDs() + columnIDs, s.indexColDirs = catalog.FullIndexColumnIDs(index) s.indexColTypes = make([]*types.T, len(columnIDs)) for i, colID := range columnIDs { col, _ := table.FindColumnWithID(colID) @@ -85,10 +85,11 @@ func MakeBuilder( // Set up the interstices for encoding interleaved tables later. s.interstices[0] = s.KeyPrefix - if len(index.Interleave.Ancestors) > 0 { + if index.NumInterleaveAncestors() > 0 { // TODO(rohany): too much of this code is copied from EncodePartialIndexKey. sharedPrefixLen := 0 - for i, ancestor := range index.Interleave.Ancestors { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + ancestor := index.GetInterleaveAncestor(i) // The first ancestor is already encoded in interstices[0]. if i != 0 { s.interstices[sharedPrefixLen] = rowenc.EncodePartialTableIDIndexID( @@ -99,7 +100,7 @@ func MakeBuilder( s.interstices[sharedPrefixLen]) } s.interstices[sharedPrefixLen] = rowenc.EncodePartialTableIDIndexID( - s.interstices[sharedPrefixLen], table.GetID(), index.ID) + s.interstices[sharedPrefixLen], table.GetID(), index.GetID()) } return s @@ -191,12 +192,12 @@ func (s *Builder) CanSplitSpanIntoFamilySpans( } // * The index is unique. - if !s.index.Unique { + if !s.index.IsUnique() { return false } // * The index is fully constrained. - if prefixLen != len(s.index.ColumnIDs) { + if prefixLen != s.index.NumColumns() { return false } @@ -209,24 +210,24 @@ func (s *Builder) CanSplitSpanIntoFamilySpans( } // If we're looking at a secondary index... - if s.index.ID != s.table.GetPrimaryIndexID() { + if s.index.GetID() != s.table.GetPrimaryIndexID() { // * The index constraint must not contain null, since that would cause the // index key to not be completely knowable. if containsNull { return false } // * The index cannot be inverted. - if s.index.Type != descpb.IndexDescriptor_FORWARD { + if s.index.GetType() != descpb.IndexDescriptor_FORWARD { return false } // * The index must store some columns. - if len(s.index.StoreColumnIDs) == 0 { + if s.index.NumStoredColumns() == 0 { return false } // * The index is a new enough version. - if s.index.Version < descpb.SecondaryIndexFamilyFormatVersion { + if s.index.GetVersion() < descpb.SecondaryIndexFamilyFormatVersion { return false } } @@ -341,8 +342,8 @@ func (s *Builder) encodeConstraintKey( // For extra columns (like implicit columns), the direction // is ascending. dir := encoding.Ascending - if i < len(s.index.ColumnDirections) { - dir, err = s.index.ColumnDirections[i].ToEncodingDirection() + if i < s.index.NumColumns() { + dir, err = s.index.GetColumnDirection(i).ToEncodingDirection() if err != nil { return nil, false, err } diff --git a/pkg/sql/span/span_builder_test.go b/pkg/sql/span/span_builder_test.go index 926202318d4c..0c12b5faa96d 100644 --- a/pkg/sql/span/span_builder_test.go +++ b/pkg/sql/span/span_builder_test.go @@ -23,7 +23,7 @@ func TestSpanBuilderDoesNotSplitSystemTableFamilySpans(t *testing.T) { st := cluster.MakeTestingClusterSettings() evalCtx := tree.MakeTestingEvalContext(st) builder := MakeBuilder(&evalCtx, keys.SystemSQLCodec, systemschema.DescriptorTable, - systemschema.DescriptorTable.GetPrimaryIndex().IndexDesc()) + systemschema.DescriptorTable.GetPrimaryIndex()) if res := builder.CanSplitSpanIntoFamilySpans( 1, 1, false); res { diff --git a/pkg/sql/span_builder_test.go b/pkg/sql/span_builder_test.go index 49e3621b7d67..a9cef0991178 100644 --- a/pkg/sql/span_builder_test.go +++ b/pkg/sql/span_builder_test.go @@ -107,7 +107,7 @@ func TestSpanBuilderCanSplitSpan(t *testing.T) { if err != nil { t.Fatal(err) } - builder := span.MakeBuilder(evalCtx, execCfg.Codec, desc, idx.IndexDesc()) + builder := span.MakeBuilder(evalCtx, execCfg.Codec, desc, idx) if res := builder.CanSplitSpanIntoFamilySpans( tc.numNeededFamilies, tc.prefixLen, tc.containsNull); res != tc.canSplit { t.Errorf("expected result to be %v, but found %v", tc.canSplit, res) diff --git a/pkg/sql/split.go b/pkg/sql/split.go index 04ec4c4cd660..bbb0650c9279 100644 --- a/pkg/sql/split.go +++ b/pkg/sql/split.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -29,7 +28,7 @@ type splitNode struct { optColumnsSlot tableDesc catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index rows planNode run splitRun expirationTime hlc.Timestamp @@ -88,19 +87,16 @@ func (n *splitNode) Close(ctx context.Context) { // getRowKey generates a key that corresponds to a row (or prefix of a row) in a table or index. // Both tableDesc and index are required (index can be the primary index). func getRowKey( - codec keys.SQLCodec, - tableDesc catalog.TableDescriptor, - index *descpb.IndexDescriptor, - values []tree.Datum, + codec keys.SQLCodec, tableDesc catalog.TableDescriptor, index catalog.Index, values []tree.Datum, ) ([]byte, error) { - if len(index.ColumnIDs) < len(values) { - return nil, pgerror.Newf(pgcode.Syntax, "excessive number of values provided: expected %d, got %d", len(index.ColumnIDs), len(values)) + if index.NumColumns() < len(values) { + return nil, pgerror.Newf(pgcode.Syntax, "excessive number of values provided: expected %d, got %d", index.NumColumns(), len(values)) } var colMap catalog.TableColMap for i := range values { - colMap.Set(index.ColumnIDs[i], i) + colMap.Set(index.GetColumnID(i), i) } - prefix := rowenc.MakeIndexKeyPrefix(codec, tableDesc, index.ID) + prefix := rowenc.MakeIndexKeyPrefix(codec, tableDesc, index.GetID()) key, _, err := rowenc.EncodePartialIndexKey( tableDesc, index, len(values), colMap, values, prefix, ) diff --git a/pkg/sql/stats/stats_cache.go b/pkg/sql/stats/stats_cache.go index b94ff46b5695..8a7c2f4cc94b 100644 --- a/pkg/sql/stats/stats_cache.go +++ b/pkg/sql/stats/stats_cache.go @@ -186,7 +186,7 @@ func decodeTableStatisticsKV( dirs := []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC, descpb.IndexDescriptor_ASC} keyVals := make([]rowenc.EncDatum, 2) _, matches, _, err := rowenc.DecodeIndexKey( - codec, tbl, tbl.GetPrimaryIndex().IndexDesc(), types, keyVals, dirs, kv.Key, + codec, tbl, tbl.GetPrimaryIndex(), types, keyVals, dirs, kv.Key, ) if err != nil { return 0, err diff --git a/pkg/sql/tablewriter_delete.go b/pkg/sql/tablewriter_delete.go index 80b4fe64dc6e..483bfd8dc831 100644 --- a/pkg/sql/tablewriter_delete.go +++ b/pkg/sql/tablewriter_delete.go @@ -116,14 +116,14 @@ func (td *tableDeleter) deleteAllRowsScan( var valNeededForCol util.FastIntSet for i := range td.rd.FetchCols { - col := td.rd.FetchCols[i].ID + col := td.rd.FetchCols[i].GetID() valNeededForCol.Add(td.rd.FetchColIDtoRowIndex.GetDefault(col)) } var rf row.Fetcher tableArgs := row.FetcherTableArgs{ Desc: td.tableDesc(), - Index: td.tableDesc().GetPrimaryIndex().IndexDesc(), + Index: td.tableDesc().GetPrimaryIndex(), ColIdxMap: td.rd.FetchColIDtoRowIndex, Cols: td.rd.FetchCols, ValNeededForCol: valNeededForCol, @@ -189,7 +189,7 @@ func (td *tableDeleter) deleteAllRowsScan( // // limit is a limit on the number of index entries deleted in the operation. func (td *tableDeleter) deleteIndex( - ctx context.Context, idx *descpb.IndexDescriptor, resume roachpb.Span, limit int64, traceKV bool, + ctx context.Context, idx catalog.Index, resume roachpb.Span, limit int64, traceKV bool, ) (roachpb.Span, error) { if idx.IsInterleaved() { if log.V(2) { @@ -201,10 +201,10 @@ func (td *tableDeleter) deleteIndex( } func (td *tableDeleter) deleteIndexFast( - ctx context.Context, idx *descpb.IndexDescriptor, resume roachpb.Span, limit int64, traceKV bool, + ctx context.Context, idx catalog.Index, resume roachpb.Span, limit int64, traceKV bool, ) (roachpb.Span, error) { if resume.Key == nil { - resume = td.tableDesc().IndexSpan(td.rd.Helper.Codec, idx.ID) + resume = td.tableDesc().IndexSpan(td.rd.Helper.Codec, idx.GetID()) } if traceKV { @@ -221,12 +221,12 @@ func (td *tableDeleter) deleteIndexFast( return td.b.Results[0].ResumeSpanAsValue(), nil } -func (td *tableDeleter) clearIndex(ctx context.Context, idx *descpb.IndexDescriptor) error { +func (td *tableDeleter) clearIndex(ctx context.Context, idx catalog.Index) error { if idx.IsInterleaved() { - return errors.Errorf("unexpected interleaved index %d", idx.ID) + return errors.Errorf("unexpected interleaved index %d", idx.GetID()) } - sp := td.tableDesc().IndexSpan(td.rd.Helper.Codec, idx.ID) + sp := td.tableDesc().IndexSpan(td.rd.Helper.Codec, idx.GetID()) // ClearRange cannot be run in a transaction, so create a // non-transactional batch to send the request. @@ -241,7 +241,7 @@ func (td *tableDeleter) clearIndex(ctx context.Context, idx *descpb.IndexDescrip } func (td *tableDeleter) deleteIndexScan( - ctx context.Context, idx *descpb.IndexDescriptor, resume roachpb.Span, limit int64, traceKV bool, + ctx context.Context, idx catalog.Index, resume roachpb.Span, limit int64, traceKV bool, ) (roachpb.Span, error) { if resume.Key == nil { resume = td.tableDesc().PrimaryIndexSpan(td.rd.Helper.Codec) @@ -249,14 +249,14 @@ func (td *tableDeleter) deleteIndexScan( var valNeededForCol util.FastIntSet for i := range td.rd.FetchCols { - col := td.rd.FetchCols[i].ID + col := td.rd.FetchCols[i].GetID() valNeededForCol.Add(td.rd.FetchColIDtoRowIndex.GetDefault(col)) } var rf row.Fetcher tableArgs := row.FetcherTableArgs{ Desc: td.tableDesc(), - Index: td.tableDesc().GetPrimaryIndex().IndexDesc(), + Index: td.tableDesc().GetPrimaryIndex(), ColIdxMap: td.rd.FetchColIDtoRowIndex, Cols: td.rd.FetchCols, ValNeededForCol: valNeededForCol, diff --git a/pkg/sql/tablewriter_upsert_opt.go b/pkg/sql/tablewriter_upsert_opt.go index 57fc008efafa..a4f6f1fde8a1 100644 --- a/pkg/sql/tablewriter_upsert_opt.go +++ b/pkg/sql/tablewriter_upsert_opt.go @@ -64,13 +64,13 @@ type optTableUpserter struct { // fetchCols indicate which columns need to be fetched from the target table, // in order to detect whether a conflict has occurred, as well as to provide // existing values for updates. - fetchCols []descpb.ColumnDescriptor + fetchCols []catalog.Column // updateCols indicate which columns need an update during a conflict. - updateCols []descpb.ColumnDescriptor + updateCols []catalog.Column // returnCols indicate which columns need to be returned by the Upsert. - returnCols []descpb.ColumnDescriptor + returnCols []catalog.Column // canaryOrdinal is the ordinal position of the column within the input row // that is used to decide whether to execute an insert or update operation. @@ -107,7 +107,7 @@ func (tu *optTableUpserter) init( tu.resultRow = make(tree.Datums, len(tu.returnCols)) tu.rows = rowcontainer.NewRowContainer( evalCtx.Mon.MakeBoundAccount(), - colinfo.ColTypeInfoFromColDescs(tu.returnCols), + colinfo.ColTypeInfoFromColumns(tu.returnCols), ) // Create the map from colIds to the expected columns. @@ -117,7 +117,7 @@ func (tu *optTableUpserter) init( tu.colIDToReturnIndex = catalog.ColumnIDToOrdinalMap(tu.tableDesc().PublicColumns()) if tu.ri.InsertColIDtoRowIndex.Len() == tu.colIDToReturnIndex.Len() { for i := range tu.ri.InsertCols { - colID := tu.ri.InsertCols[i].ID + colID := tu.ri.InsertCols[i].GetID() resultIndex, ok := tu.colIDToReturnIndex.Get(colID) if !ok || resultIndex != tu.ri.InsertColIDtoRowIndex.GetDefault(colID) { tu.insertReorderingRequired = true diff --git a/pkg/sql/tests/hash_sharded_test.go b/pkg/sql/tests/hash_sharded_test.go index 4d114da27992..fe24fbabaa8b 100644 --- a/pkg/sql/tests/hash_sharded_test.go +++ b/pkg/sql/tests/hash_sharded_test.go @@ -118,7 +118,8 @@ func TestBasicHashShardedIndexes(t *testing.T) { t.Fatal(err) } foundShardColumn := false - for _, colID := range foo.IndexDesc().ExtraColumnIDs { + for i := 0; i < foo.NumExtraColumns(); i++ { + colID := foo.GetExtraColumnID(i) if colID == shardColID { foundShardColumn = true break diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 9701bce4dfc1..9cf0efe81464 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -531,6 +531,6 @@ func (p *planner) reassignIndexComments( // key from a single span. // This determines whether an index is dropped during a schema change, or if // it is only deleted upon GC. -func canClearRangeForDrop(index *descpb.IndexDescriptor) bool { +func canClearRangeForDrop(index catalog.Index) bool { return !index.IsInterleaved() } diff --git a/pkg/sql/unsplit.go b/pkg/sql/unsplit.go index 2036726403e2..7eb48460306b 100644 --- a/pkg/sql/unsplit.go +++ b/pkg/sql/unsplit.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/errors" @@ -26,7 +25,7 @@ type unsplitNode struct { optColumnsSlot tableDesc catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index run unsplitRun rows planNode } @@ -77,7 +76,7 @@ type unsplitAllNode struct { optColumnsSlot tableDesc catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index run unsplitAllRun } @@ -104,8 +103,8 @@ func (n *unsplitAllNode) startExec(params runParams) error { return err } indexName := "" - if n.index.ID != n.tableDesc.GetPrimaryIndexID() { - indexName = n.index.Name + if n.index.GetID() != n.tableDesc.GetPrimaryIndexID() { + indexName = n.index.GetName() } it, err := params.p.ExtendedEvalContext().InternalExecutor.(*InternalExecutor).QueryIteratorEx( params.ctx, "split points query", params.p.txn, sessiondata.InternalExecutorOverride{}, diff --git a/pkg/sql/update.go b/pkg/sql/update.go index 262e78a90520..02dd7958f2cb 100644 --- a/pkg/sql/update.go +++ b/pkg/sql/update.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" @@ -58,7 +57,7 @@ type updateRun struct { // computedCols are the columns that need to be (re-)computed as // the result of updating some of the columns in updateCols. - computedCols []descpb.ColumnDescriptor + computedCols []catalog.Column // computeExprs are the expressions to evaluate to re-compute the // columns in computedCols. computeExprs []tree.TypedExpr @@ -252,7 +251,7 @@ func (u *updateNode) processSourceRow(params runParams, sourceVals tree.Datums) // iVarContainerForComputedCols does this. copy(u.run.iVarContainerForComputedCols.CurSourceRow, oldValues) for i := range u.run.tu.ru.UpdateCols { - id := u.run.tu.ru.UpdateCols[i].ID + id := u.run.tu.ru.UpdateCols[i].GetID() idx := u.run.tu.ru.FetchColIDtoRowIndex.GetDefault(id) u.run.iVarContainerForComputedCols.CurSourceRow[idx] = u.run. updateValues[i] @@ -266,9 +265,10 @@ func (u *updateNode) processSourceRow(params runParams, sourceVals tree.Datums) d, err := u.run.computeExprs[i].Eval(params.EvalContext()) if err != nil { params.EvalContext().IVarContainer = nil - return errors.Wrapf(err, "computed column %s", tree.ErrString((*tree.Name)(&u.run.computedCols[i].Name))) + name := u.run.computedCols[i].GetName() + return errors.Wrapf(err, "computed column %s", tree.ErrString((*tree.Name)(&name))) } - idx := u.run.updateColsIdx.GetDefault(u.run.computedCols[i].ID) + idx := u.run.updateColsIdx.GetDefault(u.run.computedCols[i].GetID()) u.run.updateValues[idx] = d } params.EvalContext().PopIVarContainer() @@ -391,7 +391,7 @@ type sourceSlot interface { } type scalarSlot struct { - column descpb.ColumnDescriptor + column catalog.Column sourceIndex int } @@ -402,7 +402,7 @@ func (ss scalarSlot) extractValues(row tree.Datums) tree.Datums { func (ss scalarSlot) checkColumnTypes(row []tree.TypedExpr) error { renderedResult := row[ss.sourceIndex] typ := renderedResult.ResolvedType() - return colinfo.CheckDatumTypeFitsColumnType(&ss.column, typ) + return colinfo.CheckDatumTypeFitsColumnType(ss.column, typ) } // enforceLocalColumnConstraints asserts the column constraints that @@ -417,13 +417,12 @@ func (ss scalarSlot) checkColumnTypes(row []tree.TypedExpr) error { // // The row buffer is modified in-place with the result of the // checks. -func enforceLocalColumnConstraints(row tree.Datums, cols []descpb.ColumnDescriptor) error { - for i := range cols { - col := &cols[i] - if !col.Nullable && row[i] == tree.DNull { - return sqlerrors.NewNonNullViolationError(col.Name) +func enforceLocalColumnConstraints(row tree.Datums, cols []catalog.Column) error { + for i, col := range cols { + if !col.IsNullable() && row[i] == tree.DNull { + return sqlerrors.NewNonNullViolationError(col.GetName()) } - outVal, err := tree.AdjustValueToType(col.Type, row[i]) + outVal, err := tree.AdjustValueToType(col.GetType(), row[i]) if err != nil { return err } diff --git a/pkg/sql/upsert.go b/pkg/sql/upsert.go index 8cafaf2212cf..2fb8aa0cd947 100644 --- a/pkg/sql/upsert.go +++ b/pkg/sql/upsert.go @@ -14,8 +14,8 @@ import ( "context" "sync" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) @@ -43,7 +43,7 @@ type upsertRun struct { checkOrds checkSet // insertCols are the columns being inserted/upserted into. - insertCols []descpb.ColumnDescriptor + insertCols []catalog.Column // done informs a new call to BatchedNext() that the previous call to // BatchedNext() has completed the work already. diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index 0e69bdca11cc..f6fdeeec77fe 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -478,7 +478,7 @@ func (e *virtualDefEntry) validateRow(datums tree.Datums, columns colinfo.Result // where we can't guarantee it will be Close()d in case of error. func (e *virtualDefEntry) getPlanInfo( table catalog.TableDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, idxConstraint *constraint.Constraint, stopper *stop.Stopper, ) (colinfo.ResultColumns, virtualTableConstructor) { @@ -541,14 +541,14 @@ func (e *virtualDefEntry) getPlanInfo( // We are now dealing with a constrained virtual index scan. - if index.ID == 1 { + if index.GetID() == 1 { return nil, errors.AssertionFailedf( "programming error: can't constrain scan on primary virtual index of table %s", e.desc.GetName()) } // Figure out the ordinal position of the column that we're filtering on. columnIdxMap := catalog.ColumnIDToOrdinalMap(table.PublicColumns()) - indexKeyDatums := make([]tree.Datum, len(index.ColumnIDs)) + indexKeyDatums := make([]tree.Datum, index.NumColumns()) generator, cleanup, setupError := setupGenerator(ctx, e.makeConstrainedRowsGenerator( ctx, p, dbDesc, index, indexKeyDatums, columnIdxMap, idxConstraint, columns), stopper) @@ -572,7 +572,7 @@ func (e *virtualDefEntry) makeConstrainedRowsGenerator( ctx context.Context, p *planner, dbDesc catalog.DatabaseDescriptor, - index *descpb.IndexDescriptor, + index catalog.Index, indexKeyDatums []tree.Datum, columnIdxMap catalog.TableColMap, idxConstraint *constraint.Constraint, @@ -583,7 +583,8 @@ func (e *virtualDefEntry) makeConstrainedRowsGenerator( var span constraint.Span addRowIfPassesFilter := func(idxConstraint *constraint.Constraint) func(datums ...tree.Datum) error { return func(datums ...tree.Datum) error { - for i, id := range index.ColumnIDs { + for i := 0; i < index.NumColumns(); i++ { + id := index.GetColumnID(i) indexKeyDatums[i] = datums[columnIdxMap.GetDefault(id)] } // Construct a single key span out of the current row, so that @@ -620,7 +621,7 @@ func (e *virtualDefEntry) makeConstrainedRowsGenerator( break } constraintDatum := span.StartKey().Value(0) - virtualIndex := def.getIndex(index.ID) + virtualIndex := def.getIndex(index.GetID()) // For each span, run the index's populate method, constrained to the // constraint span's value. diff --git a/pkg/sql/virtual_table.go b/pkg/sql/virtual_table.go index 81d9e8ac1c22..9d701b750d98 100644 --- a/pkg/sql/virtual_table.go +++ b/pkg/sql/virtual_table.go @@ -206,7 +206,7 @@ type vTableLookupJoinNode struct { dbName string db catalog.DatabaseDescriptor table catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index // eqCol is the single equality column ordinal into the lookup table. Virtual // indexes only support a single indexed column currently. eqCol int diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index f004c89c1353..ee96a2bcb723 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -216,7 +216,7 @@ func GetZoneConfigInTxn( ctx context.Context, txn *kv.Txn, id config.SystemTenantObjectID, - index *descpb.IndexDescriptor, + index catalog.Index, partition string, getInheritedDefault bool, ) (config.SystemTenantObjectID, *zonepb.ZoneConfig, *zonepb.Subzone, error) { @@ -237,17 +237,18 @@ func GetZoneConfigInTxn( } var subzone *zonepb.Subzone if index != nil { + indexID := uint32(index.GetID()) if placeholder != nil { - if subzone = placeholder.GetSubzone(uint32(index.ID), partition); subzone != nil { - if indexSubzone := placeholder.GetSubzone(uint32(index.ID), ""); indexSubzone != nil { + if subzone = placeholder.GetSubzone(indexID, partition); subzone != nil { + if indexSubzone := placeholder.GetSubzone(indexID, ""); indexSubzone != nil { subzone.Config.InheritFromParent(&indexSubzone.Config) } subzone.Config.InheritFromParent(zone) return placeholderID, placeholder, subzone, nil } } else { - if subzone = zone.GetSubzone(uint32(index.ID), partition); subzone != nil { - if indexSubzone := zone.GetSubzone(uint32(index.ID), ""); indexSubzone != nil { + if subzone = zone.GetSubzone(indexID, partition); subzone != nil { + if indexSubzone := zone.GetSubzone(indexID, ""); indexSubzone != nil { subzone.Config.InheritFromParent(&indexSubzone.Config) } subzone.Config.InheritFromParent(zone) @@ -328,7 +329,7 @@ func resolveZone(ctx context.Context, txn *kv.Txn, zs *tree.ZoneSpecifier) (desc func resolveSubzone( zs *tree.ZoneSpecifier, table catalog.TableDescriptor, -) (*descpb.IndexDescriptor, string, error) { +) (catalog.Index, string, error) { if !zs.TargetsTable() || zs.TableOrIndex.Index == "" && zs.Partition == "" { return nil, "", nil } @@ -348,19 +349,19 @@ func resolveSubzone( partitionName := string(zs.Partition) if partitionName != "" { - if partitioning := tabledesc.FindIndexPartitionByName(index.IndexDesc(), partitionName); partitioning == nil { + if partitioning := tabledesc.FindIndexPartitionByName(index, partitionName); partitioning == nil { return nil, "", fmt.Errorf("partition %q does not exist on index %q", partitionName, indexName) } } - return index.IndexDesc(), partitionName, nil + return index, partitionName, nil } func deleteRemovedPartitionZoneConfigs( ctx context.Context, txn *kv.Txn, tableDesc catalog.TableDescriptor, - idxDesc *descpb.IndexDescriptor, + indexID descpb.IndexID, oldPartDesc *descpb.PartitioningDescriptor, newPartDesc *descpb.PartitioningDescriptor, execCfg *ExecutorConfig, @@ -385,7 +386,7 @@ func deleteRemovedPartitionZoneConfigs( zone = zonepb.NewZoneConfig() } for _, n := range removedNames { - zone.DeleteSubzone(uint32(idxDesc.ID), n) + zone.DeleteSubzone(uint32(indexID), n) } hasNewSubzones := false _, err = writeZoneConfig(ctx, txn, tableDesc.GetID(), tableDesc, zone, execCfg, hasNewSubzones) diff --git a/pkg/sql/zone_config_test.go b/pkg/sql/zone_config_test.go index c143cddaf3db..7780caa34ccc 100644 --- a/pkg/sql/zone_config_test.go +++ b/pkg/sql/zone_config_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -136,9 +137,10 @@ func TestGetZoneConfig(t *testing.T) { } // Verify sql.GetZoneConfigInTxn. + dummyIndex := systemschema.CommentsTable.GetPrimaryIndex() if err := s.DB().Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { _, zoneCfg, subzone, err := sql.GetZoneConfigInTxn( - ctx, txn, config.SystemTenantObjectID(tc.objectID), &descpb.IndexDescriptor{}, tc.partitionName, false, + ctx, txn, config.SystemTenantObjectID(tc.objectID), dummyIndex, tc.partitionName, false, ) if err != nil { return err @@ -259,8 +261,8 @@ func TestGetZoneConfig(t *testing.T) { tb21Cfg.NumReplicas = proto.Int32(1) tb21Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1"}}}} tb21Cfg.Subzones = []zonepb.Subzone{ - {PartitionName: "p0", Config: p211Cfg}, - {PartitionName: "p1", Config: p212Cfg}, + {IndexID: 1, PartitionName: "p0", Config: p211Cfg}, + {IndexID: 1, PartitionName: "p1", Config: p212Cfg}, } tb21Cfg.SubzoneSpans = []zonepb.SubzoneSpan{ {SubzoneIndex: 0, Key: []byte{1}}, @@ -275,7 +277,7 @@ func TestGetZoneConfig(t *testing.T) { // Subzone Placeholder tb22Cfg := *zonepb.NewZoneConfig() tb22Cfg.NumReplicas = proto.Int32(0) - tb22Cfg.Subzones = []zonepb.Subzone{{PartitionName: "p0", Config: p221Cfg}} + tb22Cfg.Subzones = []zonepb.Subzone{{IndexID: 1, PartitionName: "p0", Config: p221Cfg}} tb22Cfg.SubzoneSpans = []zonepb.SubzoneSpan{ {SubzoneIndex: 0, Key: []byte{1}, EndKey: []byte{255}}, } @@ -374,9 +376,10 @@ func TestCascadingZoneConfig(t *testing.T) { } // Verify sql.GetZoneConfigInTxn. + dummyIndex := systemschema.CommentsTable.GetPrimaryIndex() if err := s.DB().Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { _, zoneCfg, subzone, err := sql.GetZoneConfigInTxn( - ctx, txn, config.SystemTenantObjectID(tc.objectID), &descpb.IndexDescriptor{}, tc.partitionName, false, + ctx, txn, config.SystemTenantObjectID(tc.objectID), dummyIndex, tc.partitionName, false, ) if err != nil { return err @@ -516,8 +519,8 @@ func TestCascadingZoneConfig(t *testing.T) { tb21Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1"}}}} tb21Cfg.InheritedConstraints = false tb21Cfg.Subzones = []zonepb.Subzone{ - {PartitionName: "p0", Config: p211Cfg}, - {PartitionName: "p1", Config: p212Cfg}, + {IndexID: 1, PartitionName: "p0", Config: p211Cfg}, + {IndexID: 1, PartitionName: "p1", Config: p212Cfg}, } tb21Cfg.SubzoneSpans = []zonepb.SubzoneSpan{ {SubzoneIndex: 0, Key: []byte{1}}, @@ -529,8 +532,8 @@ func TestCascadingZoneConfig(t *testing.T) { expectedTb21Cfg := defaultZoneConfig expectedTb21Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1"}}}} expectedTb21Cfg.Subzones = []zonepb.Subzone{ - {PartitionName: "p0", Config: p211Cfg}, - {PartitionName: "p1", Config: p212Cfg}, + {IndexID: 1, PartitionName: "p0", Config: p211Cfg}, + {IndexID: 1, PartitionName: "p1", Config: p212Cfg}, } expectedTb21Cfg.SubzoneSpans = []zonepb.SubzoneSpan{ {SubzoneIndex: 0, Key: []byte{1}}, @@ -549,7 +552,7 @@ func TestCascadingZoneConfig(t *testing.T) { // Subzone Placeholder tb22Cfg := *zonepb.NewZoneConfig() tb22Cfg.NumReplicas = proto.Int32(0) - tb22Cfg.Subzones = []zonepb.Subzone{{PartitionName: "p0", Config: p221Cfg}} + tb22Cfg.Subzones = []zonepb.Subzone{{IndexID: 1, PartitionName: "p0", Config: p221Cfg}} tb22Cfg.SubzoneSpans = []zonepb.SubzoneSpan{ {SubzoneIndex: 0, Key: []byte{1}, EndKey: []byte{255}}, } From 8f1f38223e190cc240a19f73b647feede59276ad Mon Sep 17 00:00:00 2001 From: Radu Berinde <radu@cockroachlabs.com> Date: Fri, 16 Apr 2021 09:32:08 -0700 Subject: [PATCH 23/37] opt: add opttester facility to test placeholder assignment We like to assume that the result of building a memo with placeholders followed by AssignPlaceholders is equivalent to building the query with the values directly. This is not necessarily the case - it is possible that some normalization rules act on a higher part of the tree in a way that would not happen if we had fully normalized a lower part of the tree. This commit adds two new opttester directives: `assign-placeholders-norm` and `assign-placeholders-opt`. These take a query that has placeholders and simulates the prepared query planning path. We use these facilities to add some tests that reproduce a customer issue. Release note: None --- .../norm/testdata/rules/assign_placeholders | 135 ++++++++++++++++++ pkg/sql/opt/testutils/opttester/BUILD.bazel | 1 + .../opt/testutils/opttester/memo_groups.go | 6 +- pkg/sql/opt/testutils/opttester/opt_tester.go | 99 ++++++++++++- 4 files changed, 237 insertions(+), 4 deletions(-) create mode 100644 pkg/sql/opt/norm/testdata/rules/assign_placeholders diff --git a/pkg/sql/opt/norm/testdata/rules/assign_placeholders b/pkg/sql/opt/norm/testdata/rules/assign_placeholders new file mode 100644 index 000000000000..e6705cdfc2a7 --- /dev/null +++ b/pkg/sql/opt/norm/testdata/rules/assign_placeholders @@ -0,0 +1,135 @@ +exec-ddl +CREATE TABLE kv (k INT PRIMARY KEY, v INT) +---- + +exec-ddl +CREATE TABLE abcd (a INT, b INT, c INT, d INT, PRIMARY KEY (a,b,c)) +---- + +assign-placeholders-norm query-args=(1) +SELECT v FROM kv WHERE k = $1 +---- +project + ├── columns: v:2 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(2) + └── select + ├── columns: k:1!null v:2 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(1,2) + ├── scan kv + │ ├── columns: k:1!null v:2 + │ ├── key: (1) + │ └── fd: (1)-->(2) + └── filters + └── k:1 = 1 [outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] + +assign-placeholders-opt query-args=(1) +SELECT v FROM kv WHERE k = $1 +---- +project + ├── columns: v:2 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(2) + └── scan kv + ├── columns: k:1!null v:2 + ├── constraint: /1: [/1 - /1] + ├── cardinality: [0 - 1] + ├── key: () + └── fd: ()-->(1,2) + +# This is what we ideally want to obtain after assigning placeholders in the +# test below. +norm +SELECT * FROM abcd WHERE (a, b) IN ( + SELECT unnest('{1}'::INT[]), + unnest('{2}'::INT[]) +) +---- +select + ├── columns: a:1!null b:2!null c:3!null d:4 + ├── key: (3) + ├── fd: ()-->(1,2), (3)-->(4) + ├── scan abcd + │ ├── columns: a:1!null b:2!null c:3!null d:4 + │ ├── key: (1-3) + │ └── fd: (1-3)-->(4) + └── filters + └── (a:1, b:2) IN ((1, 2),) [outer=(1,2), constraints=(/1/2: [/1/2 - /1/2]; /2: [/2 - /2]; tight), fd=()-->(1,2)] + +# TODO(radu): hoisting of the subquery results in projecting the tuple, which +# gets in the way of simplifying the expression. In particular, we can't +# convert this to a lookup join (see next testcase), which is unfortunate. +assign-placeholders-norm query-args=('{1}','{2}') +SELECT * FROM abcd WHERE (a, b) IN ( + SELECT unnest($1:::STRING::INT[]), + unnest($2:::STRING::INT[]) +) +---- +project + ├── columns: a:1!null b:2!null c:3!null d:4 + ├── immutable + ├── key: (1-3) + ├── fd: (1-3)-->(4) + └── semi-join (hash) + ├── columns: a:1!null b:2!null c:3!null d:4 column9:9!null + ├── immutable + ├── key: (1-3) + ├── fd: (1-3)-->(4), (1,2)-->(9) + ├── project + │ ├── columns: column9:9!null a:1!null b:2!null c:3!null d:4 + │ ├── key: (1-3) + │ ├── fd: (1-3)-->(4), (1,2)-->(9) + │ ├── scan abcd + │ │ ├── columns: a:1!null b:2!null c:3!null d:4 + │ │ ├── key: (1-3) + │ │ └── fd: (1-3)-->(4) + │ └── projections + │ └── (a:1, b:2) [as=column9:9, outer=(1,2)] + ├── values + │ ├── columns: column8:8 + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ ├── fd: ()-->(8) + │ └── ((1, 2),) + └── filters + └── column9:9 = column8:8 [outer=(8,9), immutable, constraints=(/8: (/NULL - ]; /9: (/NULL - ]), fd=(8)==(9), (9)==(8)] + +# TODO(radu): the end result involves a full table scan. +assign-placeholders-opt query-args=('{1}','{2}') +SELECT * FROM abcd WHERE (a, b) IN ( + SELECT unnest($1:::STRING::INT[]), + unnest($2:::STRING::INT[]) +) +---- +project + ├── columns: a:1!null b:2!null c:3!null d:4 + ├── immutable + ├── key: (1-3) + ├── fd: (1-3)-->(4) + └── semi-join (hash) + ├── columns: a:1!null b:2!null c:3!null d:4 column9:9!null + ├── immutable + ├── key: (1-3) + ├── fd: (1-3)-->(4), (1,2)-->(9) + ├── project + │ ├── columns: column9:9!null a:1!null b:2!null c:3!null d:4 + │ ├── key: (1-3) + │ ├── fd: (1-3)-->(4), (1,2)-->(9) + │ ├── scan abcd + │ │ ├── columns: a:1!null b:2!null c:3!null d:4 + │ │ ├── key: (1-3) + │ │ └── fd: (1-3)-->(4) + │ └── projections + │ └── (a:1, b:2) [as=column9:9, outer=(1,2)] + ├── values + │ ├── columns: column8:8 + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ ├── fd: ()-->(8) + │ └── ((1, 2),) + └── filters + └── column9:9 = column8:8 [outer=(8,9), immutable, constraints=(/8: (/NULL - ]; /9: (/NULL - ]), fd=(8)==(9), (9)==(8)] diff --git a/pkg/sql/opt/testutils/opttester/BUILD.bazel b/pkg/sql/opt/testutils/opttester/BUILD.bazel index d64580c37683..350879818575 100644 --- a/pkg/sql/opt/testutils/opttester/BUILD.bazel +++ b/pkg/sql/opt/testutils/opttester/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/security", "//pkg/settings/cluster", "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/schemaexpr", "//pkg/sql/opt", "//pkg/sql/opt/cat", "//pkg/sql/opt/exec/execbuilder", diff --git a/pkg/sql/opt/testutils/opttester/memo_groups.go b/pkg/sql/opt/testutils/opttester/memo_groups.go index 47ed5f578a48..8efe553fec0f 100644 --- a/pkg/sql/opt/testutils/opttester/memo_groups.go +++ b/pkg/sql/opt/testutils/opttester/memo_groups.go @@ -109,9 +109,9 @@ func (g *memoGroups) depthFirstSearch( return nil } - // There are various scalar leaf singletons that won't be registered as - // groups; ignore them. - if scalar, ok := start.(opt.ScalarExpr); ok && scalar.ChildCount() == 0 { + // There are various scalars that won't be registered as groups (e.g. + // singletons). Ignore them (rather than panicking in firstInGroup). + if scalar, ok := start.(opt.ScalarExpr); ok { if _, found := g.exprMap[scalar]; !found { return nil } diff --git a/pkg/sql/opt/testutils/opttester/opt_tester.go b/pkg/sql/opt/testutils/opttester/opt_tester.go index 4c4e311c0ba5..b03823279459 100644 --- a/pkg/sql/opt/testutils/opttester/opt_tester.go +++ b/pkg/sql/opt/testutils/opttester/opt_tester.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" _ "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder" // for ExprFmtHideScalars. @@ -220,6 +221,9 @@ type Flags struct { // MemoGroupLimit is used by the check-size command to check whether // more than MemoGroupLimit memo groups are constructed during optimization. MemoGroupLimit int64 + + // QueryArgs are values for placeholders, used for assign-placeholders-*. + QueryArgs []string } // New constructs a new instance of the OptTester for the given SQL statement. @@ -276,6 +280,17 @@ func New(catalog cat.Catalog, sql string) *OptTester { // Builds an expression tree from a SQL query, fully optimizes it using the // memo, and then outputs the lowest cost tree. // +// - assign-placeholders-norm query-args=(...) +// +// Builds a query that has placeholders (with normalization enabled), then +// assigns placeholders to the given query arguments. Normalization rules are +// enabled when assigning placeholders. +// +// - assign-placeholders-opt query-args=(...) +// +// Builds a query that has placeholders (with normalization enabled), then +// assigns placeholders to the given query arguments and fully optimizes it. +// // - build-cascades [flags] // // Builds a query and then recursively builds cascading queries. Outputs all @@ -441,14 +456,17 @@ func (ot *OptTester) RunCommand(tb testing.TB, d *datadriven.TestData) string { d.Fatalf(tb, "%+v", err) } } + ot.Flags.Verbose = datadriven.Verbose() + + ot.semaCtx.Placeholders = tree.PlaceholderInfo{} ot.evalCtx.SessionData.ReorderJoinsLimit = ot.Flags.JoinLimit ot.evalCtx.SessionData.PreferLookupJoinsForFKs = ot.Flags.PreferLookupJoinsForFKs - ot.Flags.Verbose = datadriven.Verbose() ot.evalCtx.TestingKnobs.OptimizerCostPerturbation = ot.Flags.PerturbCost ot.evalCtx.Locality = ot.Flags.Locality ot.evalCtx.SessionData.SaveTablesPrefix = ot.Flags.SaveTablesPrefix + ot.evalCtx.Placeholders = nil switch d.Cmd { case "exec-ddl": @@ -510,6 +528,15 @@ func (ot *OptTester) RunCommand(tb testing.TB, d *datadriven.TestData) string { ot.postProcess(tb, d, e) return ot.FormatExpr(e) + case "assign-placeholders-norm", "assign-placeholders-opt": + explore := d.Cmd == "assign-placeholders-opt" + e, err := ot.AssignPlaceholders(ot.Flags.QueryArgs, explore) + if err != nil { + d.Fatalf(tb, "%+v", err) + } + ot.postProcess(tb, d, e) + return ot.FormatExpr(e) + case "build-cascades": o := ot.makeOptimizer() o.DisableOptimizations() @@ -926,6 +953,9 @@ func (f *Flags) Set(arg datadriven.CmdArg) error { } f.MemoGroupLimit = limit + case "query-args": + f.QueryArgs = arg.Vals + default: return fmt.Errorf("unknown argument: %s", arg.Key) } @@ -983,6 +1013,73 @@ func (ot *OptTester) Optimize() (opt.Expr, error) { return ot.optimizeExpr(o) } +// AssignPlaceholders builds the given query with placeholders, then assigns the +// placeholders to the given argument values, and optionally runs exploration. +// +// The arguments are parsed as SQL expressions. +func (ot *OptTester) AssignPlaceholders(queryArgs []string, explore bool) (opt.Expr, error) { + o := ot.makeOptimizer() + + // Build the prepared memo. Note that placeholders don't have values yet, so + // they won't be replaced. + err := ot.buildExpr(o.Factory()) + if err != nil { + return nil, err + } + prepMemo := o.DetachMemo() + + // Construct placeholder values. + if exp := len(ot.semaCtx.Placeholders.Types); len(queryArgs) != exp { + return nil, errors.Errorf("expected %d arguments, got %d", exp, len(queryArgs)) + } + ot.semaCtx.Placeholders.Values = make(tree.QueryArguments, len(queryArgs)) + for i, arg := range queryArgs { + var parg tree.Expr + parg, err := parser.ParseExpr(fmt.Sprintf("%v", arg)) + if err != nil { + return nil, err + } + + id := tree.PlaceholderIdx(i) + typ, _ := ot.semaCtx.Placeholders.ValueType(id) + texpr, err := schemaexpr.SanitizeVarFreeExpr( + context.Background(), + parg, + typ, + "", /* context */ + &ot.semaCtx, + tree.VolatilityVolatile, + ) + if err != nil { + return nil, err + } + + ot.semaCtx.Placeholders.Values[i] = texpr + } + ot.evalCtx.Placeholders = &ot.semaCtx.Placeholders + + // Now assign placeholders. + o = ot.makeOptimizer() + o.NotifyOnMatchedRule(func(ruleName opt.RuleName) bool { + if !explore && !ruleName.IsNormalize() { + return false + } + if ot.Flags.DisableRules.Contains(int(ruleName)) { + return false + } + return true + }) + o.NotifyOnAppliedRule(func(ruleName opt.RuleName, source, target opt.Expr) { + ot.appliedRules.Add(int(ruleName)) + }) + + o.Factory().FoldingControl().AllowStableFolds() + if err := o.Factory().AssignPlaceholders(prepMemo); err != nil { + return nil, err + } + return o.Optimize() +} + // Memo returns a string that shows the memo data structure that is constructed // by the optimizer. func (ot *OptTester) Memo() (string, error) { From e6fd187af623549e99c546615342a30415253848 Mon Sep 17 00:00:00 2001 From: Radu Berinde <radu@cockroachlabs.com> Date: Fri, 16 Apr 2021 09:32:08 -0700 Subject: [PATCH 24/37] opt: allow IN subquery to be converted to lookup join This change adds a rule that handles a case which prevents Exists subqueries from becoming lookup joins. Fixes #43198. Release note (performance improvement): certain queries containing `<tuple> IN (<subquery>)` conditions may run significantly faster. --- pkg/sql/opt/norm/rules/scalar.opt | 54 ++++++ pkg/sql/opt/norm/scalar_funcs.go | 24 +++ .../norm/testdata/rules/assign_placeholders | 144 ++++++++++------ pkg/sql/opt/norm/testdata/rules/scalar | 160 +++++++++++++++--- pkg/sql/opt/xform/testdata/rules/join | 18 ++ 5 files changed, 322 insertions(+), 78 deletions(-) diff --git a/pkg/sql/opt/norm/rules/scalar.opt b/pkg/sql/opt/norm/rules/scalar.opt index 23c4baa43409..e72005f6e873 100644 --- a/pkg/sql/opt/norm/rules/scalar.opt +++ b/pkg/sql/opt/norm/rules/scalar.opt @@ -126,6 +126,60 @@ $input => (Exists $input $subqueryPrivate) +# InlineExistsSelectTuple splits a tuple equality filter into multiple +# (per-column) equalities, in the case where the tuple on one side is being +# projected. +# +# We are specifically handling the case when this is under Exists because we +# don't have to keep the same output columns for the Select. This case is +# important because it is produced for an IN subquery: +# +# SELECT * FROM ab WHERE (a, b) IN (SELECT c, d FROM cd) +# +# Without this rule, we would not be able to produce a lookup join plan for such +# a query. +# +[InlineExistsSelectTuple, Normalize] +(Exists + (Select + (Project + $input:* + [ + ... + (ProjectionsItem $tuple:(Tuple) $tupleCol:*) + ... + ] + ) + $filters:[ + ... + $item:(FiltersItem + (Eq + # CommuteVar ensures that the variable is on the left. + (Variable + $varCol:* & + (EqualsColumn $varCol $tupleCol) + ) + $rhs:(Tuple) & + (TuplesHaveSameLength $tuple $rhs) + ) + ) + ... + ] + ) + $subqueryPrivate:* +) +=> +(Exists + (Select + $input + (ConcatFilters + (RemoveFiltersItem $filters $item) + (SplitTupleEq $tuple $rhs) + ) + ) + $subqueryPrivate +) + # IntroduceExistsLimit inserts a LIMIT 1 "under" Exists so as to save resources # to make the EXISTS determination. # diff --git a/pkg/sql/opt/norm/scalar_funcs.go b/pkg/sql/opt/norm/scalar_funcs.go index f942d264c378..19ae601f8680 100644 --- a/pkg/sql/opt/norm/scalar_funcs.go +++ b/pkg/sql/opt/norm/scalar_funcs.go @@ -305,3 +305,27 @@ func (c *CustomFuncs) IsTupleOfVars(t *memo.TupleExpr, cols opt.ColList) bool { func (c *CustomFuncs) VarsAreSame(left, right *memo.VariableExpr) bool { return left.Col == right.Col } + +// EqualsColumn returns true if the two column IDs are the same. +func (c *CustomFuncs) EqualsColumn(left, right opt.ColumnID) bool { + return left == right +} + +// TuplesHaveSameLength returns true if two tuples have the same number of +// elements. +func (c *CustomFuncs) TuplesHaveSameLength(a, b *memo.TupleExpr) bool { + return len(a.Elems) == len(b.Elems) +} + +// SplitTupleEq splits an equality condition between two tuples into multiple +// equalities, one for each tuple column. +func (c *CustomFuncs) SplitTupleEq(lhs, rhs *memo.TupleExpr) memo.FiltersExpr { + if len(lhs.Elems) != len(rhs.Elems) { + panic(errors.AssertionFailedf("unequal tuple lengths")) + } + res := make(memo.FiltersExpr, len(lhs.Elems)) + for i := range res { + res[i] = c.f.ConstructFiltersItem(c.f.ConstructEq(lhs.Elems[i], rhs.Elems[i])) + } + return res +} diff --git a/pkg/sql/opt/norm/testdata/rules/assign_placeholders b/pkg/sql/opt/norm/testdata/rules/assign_placeholders index e6705cdfc2a7..8d27ff305c64 100644 --- a/pkg/sql/opt/norm/testdata/rules/assign_placeholders +++ b/pkg/sql/opt/norm/testdata/rules/assign_placeholders @@ -60,46 +60,83 @@ select └── filters └── (a:1, b:2) IN ((1, 2),) [outer=(1,2), constraints=(/1/2: [/1/2 - /1/2]; /2: [/2 - /2]; tight), fd=()-->(1,2)] -# TODO(radu): hoisting of the subquery results in projecting the tuple, which -# gets in the way of simplifying the expression. In particular, we can't -# convert this to a lookup join (see next testcase), which is unfortunate. +# The normalized expression above can be explored into a constrained scan. +opt +SELECT * FROM abcd WHERE (a, b) IN ( + SELECT unnest('{1}'::INT[]), + unnest('{2}'::INT[]) +) +---- +scan abcd + ├── columns: a:1!null b:2!null c:3!null d:4 + ├── constraint: /1/2/3: [/1/2 - /1/2] + ├── key: (3) + └── fd: ()-->(1,2), (3)-->(4) + assign-placeholders-norm query-args=('{1}','{2}') SELECT * FROM abcd WHERE (a, b) IN ( SELECT unnest($1:::STRING::INT[]), unnest($2:::STRING::INT[]) ) ---- -project +select ├── columns: a:1!null b:2!null c:3!null d:4 - ├── immutable + ├── key: (3) + ├── fd: ()-->(1,2), (3)-->(4) + ├── scan abcd + │ ├── columns: a:1!null b:2!null c:3!null d:4 + │ ├── key: (1-3) + │ └── fd: (1-3)-->(4) + └── filters + ├── a:1 = 1 [outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] + └── b:2 = 2 [outer=(2), constraints=(/2: [/2 - /2]; tight), fd=()-->(2)] + +# We want this query to be optimized into a constrained scan, just like the +# no-placeholders variant above. +assign-placeholders-opt query-args=('{1}','{2}') +SELECT * FROM abcd WHERE (a, b) IN ( + SELECT unnest($1:::STRING::INT[]), + unnest($2:::STRING::INT[]) +) +---- +scan abcd + ├── columns: a:1!null b:2!null c:3!null d:4 + ├── constraint: /1/2/3: [/1/2 - /1/2] + ├── key: (3) + └── fd: ()-->(1,2), (3)-->(4) + +# Note: \x2c is a comma; we can't use a comma directly because of the +# datadriven parser. +assign-placeholders-norm query-args=('{1\x2c 2}','{3\x2c 4}') +SELECT * FROM abcd WHERE (a, b) IN ( + SELECT unnest($1:::STRING::INT[]), + unnest($2:::STRING::INT[]) +) +---- +semi-join (hash) + ├── columns: a:1!null b:2!null c:3!null d:4 + ├── stable ├── key: (1-3) ├── fd: (1-3)-->(4) - └── semi-join (hash) - ├── columns: a:1!null b:2!null c:3!null d:4 column9:9!null - ├── immutable - ├── key: (1-3) - ├── fd: (1-3)-->(4), (1,2)-->(9) - ├── project - │ ├── columns: column9:9!null a:1!null b:2!null c:3!null d:4 - │ ├── key: (1-3) - │ ├── fd: (1-3)-->(4), (1,2)-->(9) - │ ├── scan abcd - │ │ ├── columns: a:1!null b:2!null c:3!null d:4 - │ │ ├── key: (1-3) - │ │ └── fd: (1-3)-->(4) - │ └── projections - │ └── (a:1, b:2) [as=column9:9, outer=(1,2)] - ├── values - │ ├── columns: column8:8 - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ ├── fd: ()-->(8) - │ └── ((1, 2),) - └── filters - └── column9:9 = column8:8 [outer=(8,9), immutable, constraints=(/8: (/NULL - ]; /9: (/NULL - ]), fd=(8)==(9), (9)==(8)] + ├── scan abcd + │ ├── columns: a:1!null b:2!null c:3!null d:4 + │ ├── key: (1-3) + │ └── fd: (1-3)-->(4) + ├── project-set + │ ├── columns: unnest:6 unnest:7 + │ ├── stable + │ ├── values + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ └── () + │ └── zip + │ ├── unnest(e'{1\\x2c 2}'::INT8[]) [stable] + │ └── unnest(e'{3\\x2c 4}'::INT8[]) [stable] + └── filters + ├── unnest:6 = a:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── unnest:7 = b:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] -# TODO(radu): the end result involves a full table scan. -assign-placeholders-opt query-args=('{1}','{2}') +assign-placeholders-opt query-args=('{1\x2c 2}','{3\x2c 4}') SELECT * FROM abcd WHERE (a, b) IN ( SELECT unnest($1:::STRING::INT[]), unnest($2:::STRING::INT[]) @@ -107,29 +144,28 @@ SELECT * FROM abcd WHERE (a, b) IN ( ---- project ├── columns: a:1!null b:2!null c:3!null d:4 - ├── immutable + ├── stable ├── key: (1-3) ├── fd: (1-3)-->(4) - └── semi-join (hash) - ├── columns: a:1!null b:2!null c:3!null d:4 column9:9!null - ├── immutable - ├── key: (1-3) - ├── fd: (1-3)-->(4), (1,2)-->(9) - ├── project - │ ├── columns: column9:9!null a:1!null b:2!null c:3!null d:4 - │ ├── key: (1-3) - │ ├── fd: (1-3)-->(4), (1,2)-->(9) - │ ├── scan abcd - │ │ ├── columns: a:1!null b:2!null c:3!null d:4 - │ │ ├── key: (1-3) - │ │ └── fd: (1-3)-->(4) - │ └── projections - │ └── (a:1, b:2) [as=column9:9, outer=(1,2)] - ├── values - │ ├── columns: column8:8 - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ ├── fd: ()-->(8) - │ └── ((1, 2),) - └── filters - └── column9:9 = column8:8 [outer=(8,9), immutable, constraints=(/8: (/NULL - ]; /9: (/NULL - ]), fd=(8)==(9), (9)==(8)] + └── inner-join (lookup abcd) + ├── columns: a:1!null b:2!null c:3!null d:4 unnest:6!null unnest:7!null + ├── key columns: [6 7] = [1 2] + ├── stable + ├── key: (3,6,7) + ├── fd: (1-3)-->(4), (1)==(6), (6)==(1), (2)==(7), (7)==(2) + ├── distinct-on + │ ├── columns: unnest:6 unnest:7 + │ ├── grouping columns: unnest:6 unnest:7 + │ ├── stable + │ ├── key: (6,7) + │ └── project-set + │ ├── columns: unnest:6 unnest:7 + │ ├── stable + │ ├── values + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ └── () + │ └── zip + │ ├── unnest(e'{1\\x2c 2}'::INT8[]) [stable] + │ └── unnest(e'{3\\x2c 4}'::INT8[]) [stable] + └── filters (true) diff --git a/pkg/sql/opt/norm/testdata/rules/scalar b/pkg/sql/opt/norm/testdata/rules/scalar index 9e805e2627da..297087215077 100644 --- a/pkg/sql/opt/norm/testdata/rules/scalar +++ b/pkg/sql/opt/norm/testdata/rules/scalar @@ -6,6 +6,10 @@ exec-ddl CREATE TABLE xy (x INT PRIMARY KEY, y INT) ---- +exec-ddl +CREATE TABLE abcd (a INT, b INT, c INT, d INT) +---- + # -------------------------------------------------- # CommuteVar # -------------------------------------------------- @@ -458,6 +462,126 @@ select │ └── limit hint: 1.00 └── 1 +# -------------------------------------------------- +# InlineExistsSelectTuple +# -------------------------------------------------- +norm expect=InlineExistsSelectTuple +SELECT * FROM a WHERE (k, i) IN (SELECT x, y FROM xy) +---- +semi-join (hash) + ├── columns: k:1!null i:2 f:3 s:4 arr:5 + ├── key: (1) + ├── fd: (1)-->(2-5) + ├── scan a + │ ├── columns: k:1!null i:2 f:3 s:4 arr:5 + │ ├── key: (1) + │ └── fd: (1)-->(2-5) + ├── scan xy + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) + └── filters + ├── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + +norm expect=InlineExistsSelectTuple +SELECT * FROM a WHERE (k, i) IN (SELECT x, 2 FROM xy) +---- +semi-join (hash) + ├── columns: k:1!null i:2!null f:3 s:4 arr:5 + ├── key: (1) + ├── fd: ()-->(2), (1)-->(3-5) + ├── select + │ ├── columns: k:1!null i:2!null f:3 s:4 arr:5 + │ ├── key: (1) + │ ├── fd: ()-->(2), (1)-->(3-5) + │ ├── scan a + │ │ ├── columns: k:1!null i:2 f:3 s:4 arr:5 + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5) + │ └── filters + │ └── i:2 = 2 [outer=(2), constraints=(/2: [/2 - /2]; tight), fd=()-->(2)] + ├── scan xy + │ ├── columns: x:7!null + │ └── key: (7) + └── filters + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + +norm expect=InlineExistsSelectTuple +SELECT * FROM a WHERE f>1 AND (k, i) IN (SELECT x, 2 FROM xy) AND s = 'foo' +---- +semi-join (hash) + ├── columns: k:1!null i:2!null f:3!null s:4!null arr:5 + ├── key: (1) + ├── fd: ()-->(2,4), (1)-->(3,5) + ├── select + │ ├── columns: k:1!null i:2!null f:3!null s:4!null arr:5 + │ ├── key: (1) + │ ├── fd: ()-->(2,4), (1)-->(3,5) + │ ├── scan a + │ │ ├── columns: k:1!null i:2 f:3 s:4 arr:5 + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5) + │ └── filters + │ ├── i:2 = 2 [outer=(2), constraints=(/2: [/2 - /2]; tight), fd=()-->(2)] + │ ├── f:3 > 1.0 [outer=(3), constraints=(/3: [/1.0000000000000002 - ]; tight)] + │ └── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] + ├── scan xy + │ ├── columns: x:7!null + │ └── key: (7) + └── filters + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + +# Verify that we handle multiple tuples. +norm expect=InlineExistsSelectTuple +SELECT * FROM abcd WHERE (a, b) IN (SELECT x, y FROM xy) AND (c, d) IN (SELECT k, i FROM a) +---- +semi-join (hash) + ├── columns: a:1 b:2 c:3 d:4 + ├── semi-join (hash) + │ ├── columns: a:1 b:2 c:3 d:4 + │ ├── scan abcd + │ │ └── columns: a:1 b:2 c:3 d:4 + │ ├── scan a + │ │ ├── columns: k:10!null i:11 + │ │ ├── key: (10) + │ │ └── fd: (10)-->(11) + │ └── filters + │ ├── k:10 = c:3 [outer=(3,10), constraints=(/3: (/NULL - ]; /10: (/NULL - ]), fd=(3)==(10), (10)==(3)] + │ └── i:11 = d:4 [outer=(4,11), constraints=(/4: (/NULL - ]; /11: (/NULL - ]), fd=(4)==(11), (11)==(4)] + ├── scan xy + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) + └── filters + ├── x:7 = a:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── y:8 = b:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + +# Make sure we check that the left-hand side is the correct tuple; the result +# would be bad if we didn't check that the variable is for the tuple in the +# projection. +norm expect=InlineExistsSelectTuple +SELECT * FROM abcd WHERE EXISTS(SELECT * FROM (SELECT (x, y), (x+1,y+1) FROM xy) AS v(tup1,tup2) WHERE tup2 = (a, b)) +---- +semi-join (hash) + ├── columns: a:1 b:2 c:3 d:4 + ├── immutable + ├── scan abcd + │ └── columns: a:1 b:2 c:3 d:4 + ├── project + │ ├── columns: column13:13 column12:12!null + │ ├── immutable + │ ├── scan xy + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) + │ └── projections + │ ├── y:8 + 1 [as=column13:13, outer=(8), immutable] + │ └── x:7 + 1 [as=column12:12, outer=(7), immutable] + └── filters + ├── a:1 = column12:12 [outer=(1,12), constraints=(/1: (/NULL - ]; /12: (/NULL - ]), fd=(1)==(12), (12)==(1)] + └── b:2 = column13:13 [outer=(2,13), constraints=(/2: (/NULL - ]; /13: (/NULL - ]), fd=(2)==(13), (13)==(2)] + # -------------------------------------------------- # IntroduceExistsLimit # -------------------------------------------------- @@ -1148,36 +1272,24 @@ 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 + ├── columns: k:1!null i:2 ├── key: (1) - ├── fd: (1)-->(10) - ├── project - │ ├── columns: column10:10 k:1!null + ├── fd: (1)-->(2) + ├── scan a + │ ├── columns: k:1!null i:2 │ ├── key: (1) - │ ├── fd: (1)-->(10) - │ ├── scan a - │ │ ├── columns: k:1!null i:2 - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2) - │ └── projections - │ └── (k:1, i:2) [as=column10:10, outer=(1,2)] - ├── project - │ ├── columns: column9:9!null + │ └── fd: (1)-->(2) + ├── values + │ ├── columns: column1:7!null column2:8!null │ ├── cardinality: [3 - 3] - │ ├── values - │ │ ├── columns: column1:7!null column2:8!null - │ │ ├── cardinality: [3 - 3] - │ │ ├── (1, 1) - │ │ ├── (2, 2) - │ │ └── (3, 3) - │ └── projections - │ └── (column2:8, column1:7) [as=column9:9, outer=(7,8)] + │ ├── (1, 1) + │ ├── (2, 2) + │ └── (3, 3) └── filters - └── column10:10 = column9:9 [outer=(9,10), immutable, constraints=(/9: (/NULL - ]; /10: (/NULL - ]), fd=(9)==(10), (10)==(9)] + ├── column2:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + └── column1:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # -------------------------------------------------- # SimplifyEqualsAnyTuple diff --git a/pkg/sql/opt/xform/testdata/rules/join b/pkg/sql/opt/xform/testdata/rules/join index e13cdd634bb4..44fc9e50fa88 100644 --- a/pkg/sql/opt/xform/testdata/rules/join +++ b/pkg/sql/opt/xform/testdata/rules/join @@ -8349,3 +8349,21 @@ semi-join (lookup abc_part) │ │ └── fd: ()-->(16-19) │ └── filters (true) └── filters (true) + +# A multi-column IN query must be able to become a lookup join. +opt +SELECT * FROM abc WHERE (a, b) IN (SELECT m, n FROM small) +---- +project + ├── columns: a:1 b:2 c:3 + └── inner-join (lookup abc@ab) + ├── columns: a:1!null b:2!null c:3 m:6!null n:7!null + ├── key columns: [6 7] = [1 2] + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) + ├── distinct-on + │ ├── columns: m:6 n:7 + │ ├── grouping columns: m:6 n:7 + │ ├── key: (6,7) + │ └── scan small + │ └── columns: m:6 n:7 + └── filters (true) From 467ae98aa8b11377e94cbb6a468564c187fa0f9f Mon Sep 17 00:00:00 2001 From: Andrei Matei <andrei@cockroachlabs.com> Date: Wed, 14 Apr 2021 15:26:44 -0400 Subject: [PATCH 25/37] kvserver: fix write below closedts bug This patch fixes a bug in our closed timestamp management. This bug was making it possible for a command to close a timestamp even though other requests writing at lower timestamps are currently evaluating. The problem was that we were assuming that, if a replica is proposing a new lease, there can't be any requests in flight and every future write evaluated on the range will wait for the new lease and the evaluate above the lease start time. Based on that reasoning, the proposal buffer was recording the lease start time as its assignedClosedTimestamp. This was matching what it does for every write, where assignedClosedTimestamp corresponds to the the closed timestamp carried by the command. It turns out that the replica's reasoning was wrong. It is, in fact, possible for writes to be evaluating on the range when the lease acquisition is proposed. And these evaluations might be done at timestamps below the would-be lease's start time. This happens when the replica has already received a lease through a lease transfer. The transfer must have applied after the previous lease expired and the replica decided to start acquiring a new one. This fixes one of the assertion failures seen in #62655. Release note (bug fix): A bug leading to crashes with the message "writing below closed ts" has been fixed. --- pkg/kv/kvserver/helpers_test.go | 9 + pkg/kv/kvserver/replica_closedts_test.go | 243 ++++++++++++++++++- pkg/kv/kvserver/replica_init.go | 1 + pkg/kv/kvserver/replica_proposal_buf.go | 39 ++- pkg/kv/kvserver/replica_proposal_buf_test.go | 90 ++++--- pkg/kv/kvserver/testing_knobs.go | 5 + 6 files changed, 346 insertions(+), 41 deletions(-) diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index a32673dc1a39..eeea580b89a5 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -42,6 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" + "github.com/cockroachdb/logtags" "go.etcd.io/etcd/raft/v3" ) @@ -546,3 +547,11 @@ func WatchForDisappearingReplicas(t testing.TB, store *Store) { } } } + +// AcquireLease is redirectOnOrAcquireLease exposed for tests. +func (r *Replica) AcquireLease(ctx context.Context) (kvserverpb.LeaseStatus, error) { + ctx = r.AnnotateCtx(ctx) + ctx = logtags.AddTag(ctx, "lease-acq", nil) + l, pErr := r.redirectOnOrAcquireLease(ctx) + return l, pErr.GoError() +} diff --git a/pkg/kv/kvserver/replica_closedts_test.go b/pkg/kv/kvserver/replica_closedts_test.go index 87a0b7d95638..a68484da4668 100644 --- a/pkg/kv/kvserver/replica_closedts_test.go +++ b/pkg/kv/kvserver/replica_closedts_test.go @@ -13,6 +13,7 @@ package kvserver_test import ( "context" "sync" + "sync/atomic" "testing" "github.com/cockroachdb/cockroach/pkg/base" @@ -33,9 +34,9 @@ import ( ) // TestBumpSideTransportClosed tests the various states that a replica can find -// itself in when its TestBumpSideTransportClosed is called. It verifies that -// the method only returns successfully if it can bump its closed timestamp to -// the target. +// itself in when its BumpSideTransportClosed is called. It verifies that the +// method only returns successfully if it can bump its closed timestamp to the +// target. func TestBumpSideTransportClosed(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -413,6 +414,242 @@ func TestBumpSideTransportClosed(t *testing.T) { } } +// Test that a lease proposal that gets rejected doesn't erroneously dictate the +// closed timestamp of further requests. If it would, then writes could violate +// that closed timestamp. The tricky scenario tested is the following: +// +// 1. A lease held by rep1 is getting close to its expiration. +// 2. Rep1 begins the process of transferring its lease to rep2 with a start +// time of 100. +// 3. The transfer goes slowly. From the perspective of rep2, the original lease +// expires, so it begins acquiring a new lease with a start time of 200. The +// lease acquisition is slow to propose. +// 4. The lease transfer finally applies. Rep2 is the new leaseholder and bumps +// its tscache to 100. +// 5. Two writes start evaluating on rep2 under the new lease. They bump their +// write timestamp to 100,1. +// 6. Rep2's lease acquisition from step 3 is proposed. Here's where the +// regression that this test is protecting against comes in: if rep2 was to +// mechanically bump its assignedClosedTimestamp to 200, that'd be incorrect +// because there are in-flight writes at 100. If those writes get proposed +// after the lease acquisition request, the second of them to get proposed +// would violate the closed time carried by the first (see below). +// 7. The lease acquisition gets rejected below Raft because the previous lease +// it asserts doesn't correspond to the lease that it applies under. +// 8. The two writes from step 5 are proposed. The closed timestamp that they +// each carry has a lower bound of rep2.assignedClosedTimestmap. If this was +// 200, then the second one would violate the closed timestamp carried by the +// first one - the first one says that 200 is closed, but then the second +// tries to write at 100. Note that the first write is OK writing at 100 even +// though it carries a closed timestamp of 200 - the closed timestamp carried +// by a command only binds future commands. +// +// The test simulates the scenario and verifies that we don't crash with a +// closed timestamp violation assertion. We avoid the violation because, in step +// 6, the lease proposal doesn't bump the assignedClosedTimestamp. +func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + // We're going to orchestrate the scenario by controlling the timing of the + // lease transfer, the lease acquisition and the writes. Note that we'll block + // the lease acquisition and the writes after they evaluate but before they + // get proposed, but we'll block the lease transfer when it's just about to be + // proposed, after it gets assigned the closed timestamp that it will carry. + // We want it to carry a relatively low closed timestamp, so we want its + // closed timestamp to be assigned before we bump the clock to expire the + // original lease. + + // leaseTransferCh is used to block the lease transfer. + leaseTransferCh := make(chan struct{}) + // leaseAcqCh is used to block the lease acquisition. + leaseAcqCh := make(chan struct{}) + // writeCh is used to wait for the two writes to block. + writeCh := make(chan struct{}) + // unblockWritesCh is used to unblock the two writes. + unblockWritesCh := make(chan struct{}) + var writeKey1, writeKey2 atomic.Value + // Initialize the atomics so they get bound to a specific type. + writeKey1.Store(roachpb.Key{}) + writeKey2.Store(roachpb.Key{}) + var blockedRangeID int64 + var trappedLeaseAcquisition int64 + + blockLeaseAcquisition := func(args kvserverbase.FilterArgs) { + blockedRID := roachpb.RangeID(atomic.LoadInt64(&blockedRangeID)) + leaseReq, ok := args.Req.(*roachpb.RequestLeaseRequest) + if !ok || args.Hdr.RangeID != blockedRID || leaseReq.Lease.Replica.NodeID != 2 { + return + } + if atomic.CompareAndSwapInt64(&trappedLeaseAcquisition, 0, 1) { + leaseAcqCh <- struct{}{} + <-leaseAcqCh + } + } + + blockWrites := func(args kvserverbase.FilterArgs) { + wk1 := writeKey1.Load().(roachpb.Key) + wk2 := writeKey2.Load().(roachpb.Key) + if put, ok := args.Req.(*roachpb.PutRequest); ok && (put.Key.Equal(wk1) || put.Key.Equal(wk2)) { + writeCh <- struct{}{} + <-unblockWritesCh + } + } + + blockTransfer := func(p *kvserver.ProposalData) { + blockedRID := roachpb.RangeID(atomic.LoadInt64(&blockedRangeID)) + ba := p.Request + if ba.RangeID != blockedRID { + return + } + _, ok := p.Request.GetArg(roachpb.TransferLease) + if !ok { + return + } + leaseTransferCh <- struct{}{} + <-leaseTransferCh + } + + manual := hlc.NewHybridManualClock() + tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manual.UnixNano, + }, + Store: &kvserver.StoreTestingKnobs{ + DisableConsistencyQueue: true, + EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ + TestingPostEvalFilter: func(args kvserverbase.FilterArgs) *roachpb.Error { + blockWrites(args) + blockLeaseAcquisition(args) + return nil + }, + }, + TestingProposalSubmitFilter: func(p *kvserver.ProposalData) (drop bool, _ error) { + blockTransfer(p) + return false, nil + }, + }, + }, + }}) + defer tc.Stopper().Stop(ctx) + + manual.Pause() + // Upreplicate a range. + n1, n2 := tc.Servers[0], tc.Servers[1] + // One of the filters hardcodes a node id. + require.Equal(t, roachpb.NodeID(2), n2.NodeID()) + key := tc.ScratchRangeWithExpirationLease(t) + s1 := tc.GetFirstStoreFromServer(t, 0) + t1, t2 := tc.Target(0), tc.Target(1) + repl0 := s1.LookupReplica(keys.MustAddr(key)) + desc := *repl0.Desc() + require.NotNil(t, repl0) + tc.AddVotersOrFatal(t, key, t2) + require.NoError(t, tc.WaitForVoters(key, t2)) + // Make sure the lease starts off on n1. + lease, _ /* now */, err := tc.FindRangeLease(desc, &t1 /* hint */) + require.NoError(t, err) + require.Equal(t, n1.NodeID(), lease.Replica.NodeID) + + // Advance the time a bit. We'll then initiate a transfer, and we want the + // transferred lease to be valid for a while after the original lease expires. + remainingNanos := lease.GetExpiration().WallTime - manual.UnixNano() + // NOTE: We don't advance the clock past the mid-point of the lease, otherwise + // it gets extended. + pause1 := remainingNanos / 3 + manual.Increment(pause1) + + // Start a lease transfer from n1 to n2. We'll block the proposal of the transfer for a while. + atomic.StoreInt64(&blockedRangeID, int64(desc.RangeID)) + transferErrCh := make(chan error) + go func() { + transferErrCh <- tc.TransferRangeLease(desc, t2) + }() + defer func() { + require.NoError(t, <-transferErrCh) + }() + // Wait for the lease transfer to evaluate and then block. + <-leaseTransferCh + // With the lease transfer still blocked, we now advance the clock beyond the + // original lease's expiration and we make n2 try to acquire a lease. This + // lease acquisition request will also be blocked. + manual.Increment(remainingNanos - pause1 + 1) + leaseAcqErrCh := make(chan error) + go func() { + r, _, err := n2.Stores().GetReplicaForRangeID(ctx, desc.RangeID) + if err != nil { + leaseAcqErrCh <- err + return + } + _, err = r.AcquireLease(ctx) + leaseAcqErrCh <- err + }() + // Wait for the lease acquisition to be blocked. + select { + case <-leaseAcqCh: + case err := <-leaseAcqErrCh: + t.Fatalf("lease request unexpectedly finished. err: %v", err) + } + // Let the previously blocked transfer succeed. n2's lease acquisition remains + // blocked. + close(leaseTransferCh) + // Wait until n2 has applied the lease transfer. + desc = *repl0.Desc() + testutils.SucceedsSoon(t, func() error { + li, _ /* now */, err := tc.FindRangeLeaseEx(ctx, desc, &t2 /* hint */) + if err != nil { + return err + } + lease = li.Current() + if !lease.OwnedBy(n2.GetFirstStoreID()) { + return errors.Errorf("n2 still unaware of its lease: %s", li.Current()) + } + return nil + }) + + // Now we send two writes. We'll block them after evaluation. Then we'll + // unblock the lease acquisition, let the respective command fail to apply, + // and then we'll unblock the writes. + err1 := make(chan error) + err2 := make(chan error) + go func() { + writeKey1.Store(key) + sender := n2.DB().NonTransactionalSender() + pArgs := putArgs(key, []byte("test val")) + _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: lease.Start.ToTimestamp()}, pArgs) + err1 <- pErr.GoError() + }() + go func() { + k := key.Next() + writeKey2.Store(k) + sender := n2.DB().NonTransactionalSender() + pArgs := putArgs(k, []byte("test val2")) + _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: lease.Start.ToTimestamp()}, pArgs) + err2 <- pErr.GoError() + }() + // Wait for the writes to evaluate and block before proposal. + <-writeCh + <-writeCh + + // Unblock the lease acquisition. + close(leaseAcqCh) + if err := <-leaseAcqErrCh; err != nil { + close(unblockWritesCh) + t.Fatal(err) + } + + // Now unblock the writes. + close(unblockWritesCh) + require.NoError(t, <-err1) + require.NoError(t, <-err2) + // Not crashing with a closed timestamp violation assertion marks the success + // of this test. +} + // BenchmarkBumpSideTransportClosed measures the latency of a single call to // (*Replica).BumpSideTransportClosed. The closed timestamp side-transport was // designed with a performance expectation of this check taking no more than diff --git a/pkg/kv/kvserver/replica_init.go b/pkg/kv/kvserver/replica_init.go index c64027597c26..203762749be1 100644 --- a/pkg/kv/kvserver/replica_init.go +++ b/pkg/kv/kvserver/replica_init.go @@ -99,6 +99,7 @@ func newUnloadedReplica( r.mu.proposalBuf.Init((*replicaProposer)(r), tracker.NewLockfreeTracker(), r.Clock(), r.ClusterSettings()) r.mu.proposalBuf.testing.allowLeaseProposalWhenNotLeader = store.cfg.TestingKnobs.AllowLeaseRequestProposalsWhenNotLeader r.mu.proposalBuf.testing.dontCloseTimestamps = store.cfg.TestingKnobs.DontCloseTimestamps + r.mu.proposalBuf.testing.submitProposalFilter = store.cfg.TestingKnobs.TestingProposalSubmitFilter if leaseHistoryMaxEntries > 0 { r.leaseHistory = newLeaseHistory() diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 134b8b4e11c7..cccb15f2ada6 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -726,6 +726,32 @@ func (b *propBuf) assignClosedTimestampToProposalLocked( // For brand new leases, we close the lease start time. Since this proposing // replica is not the leaseholder, the previous target is meaningless. closedTSTarget = newLease.Start.ToTimestamp() + // We forward closedTSTarget to b.assignedClosedTimestamp. We surprisingly + // might have previously closed a timestamp above the lease start time - + // when we close timestamps in the future, then attempt to transfer our + // lease away (and thus proscribe it) but the transfer fails and we're now + // acquiring a new lease to replace the proscribed one. + // + // TODO(andrei,nvanbenschoten): add a test with scenario: + // - Acquire lease @ 1 + // - Close future timestamp @ 3 + // - Attempt to transfer lease @ 2 + // - Reject + // - Reacquire lease @ 2 + closedTSTarget.Forward(b.assignedClosedTimestamp) + + // Note that we're not bumping b.assignedClosedTimestamp here (we're not + // calling forwardClosedTimestampLocked). Bumping it to the lease start time + // would (surprisingly?) be illegal: just because we're proposing a lease + // starting at timestamp 100, doesn't mean we're sure to not be in the + // process of evaluating requests writing below 100. This can happen if a + // lease transfer has already applied while we were evaluating this lease + // request, and if we've already started evaluating writes under the + // transferred lease. Such a transfer can give us the lease starting at + // timestamp 50. If such a transfer applied, then our lease request that + // we're proposing now is sure to not apply. But if we were to bump + // b.assignedClosedTimestamp, the damage would be done. See + // TestRejectedLeaseDoesntDictateClosedTimestamp. } else { // Sanity check that this command is not violating the closed timestamp. It // must be writing at a timestamp above assignedClosedTimestamp @@ -752,15 +778,18 @@ func (b *propBuf) assignClosedTimestampToProposalLocked( } // We can't close timestamps above the current lease's expiration. closedTSTarget.Backward(p.leaseStatus.ClosedTimestampUpperBound()) + + // We're about to close closedTSTarget. The propBuf needs to remember that + // in order for incoming requests to be bumped above it (through + // TrackEvaluatingRequest). + if !b.forwardClosedTimestampLocked(closedTSTarget) { + closedTSTarget = b.assignedClosedTimestamp + } } - // We're about to close closedTSTarget. The propBuf needs to remember that in - // order for incoming requests to be bumped above it (through - // TrackEvaluatingRequest). - b.forwardClosedTimestampLocked(closedTSTarget) // Fill in the closed ts in the proposal. f := &b.tmpClosedTimestampFooter - f.ClosedTimestamp = b.assignedClosedTimestamp + f.ClosedTimestamp = closedTSTarget footerLen := f.Size() if log.ExpensiveLogEnabled(ctx, 4) { log.VEventf(ctx, 4, "attaching closed timestamp %s to proposal %x", b.assignedClosedTimestamp, p.idKey) diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index 7046a2a008c8..b032f44d9076 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -762,37 +762,48 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { // proposed. lease roachpb.Lease + // expClosed is the expected closed timestamp carried by the proposal. Empty + // means the proposal is not expected to carry a closed timestamp update. expClosed hlc.Timestamp + // expAssignedClosedBumped, if set, means that the test expects + // b.assignedClosedTimestamp to be bumped before proposing. If not set, then + // the test expects b.assignedClosedTimestamp to be left at + // prevClosedTimestamp, regardless of whether the proposal carries a closed + // timestamp or not (expClosed). + expAssignedClosedBumped bool }{ { - name: "basic", - reqType: regularWrite, - trackerLowerBound: hlc.Timestamp{}, - leaseExp: hlc.MaxTimestamp, - rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, - prevClosedTimestamp: hlc.Timestamp{}, - expClosed: nowMinusClosedLag, + name: "basic", + reqType: regularWrite, + trackerLowerBound: hlc.Timestamp{}, + leaseExp: hlc.MaxTimestamp, + rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, + prevClosedTimestamp: hlc.Timestamp{}, + expClosed: nowMinusClosedLag, + expAssignedClosedBumped: true, }, { // The request tracker will prevent us from closing below its lower bound. - name: "not closing below evaluating requests", - reqType: regularWrite, - trackerLowerBound: nowMinusTwiceClosedLag, - leaseExp: hlc.MaxTimestamp, - rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, - prevClosedTimestamp: hlc.Timestamp{}, - expClosed: nowMinusTwiceClosedLag.FloorPrev(), + name: "not closing below evaluating requests", + reqType: regularWrite, + trackerLowerBound: nowMinusTwiceClosedLag, + leaseExp: hlc.MaxTimestamp, + rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, + prevClosedTimestamp: hlc.Timestamp{}, + expClosed: nowMinusTwiceClosedLag.FloorPrev(), + expAssignedClosedBumped: true, }, { // Like the basic test, except that we can't close timestamp below what // we've already closed previously. - name: "no regression", - reqType: regularWrite, - trackerLowerBound: hlc.Timestamp{}, - leaseExp: hlc.MaxTimestamp, - rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, - prevClosedTimestamp: someClosedTS, - expClosed: someClosedTS, + name: "no regression", + reqType: regularWrite, + trackerLowerBound: hlc.Timestamp{}, + leaseExp: hlc.MaxTimestamp, + rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, + prevClosedTimestamp: someClosedTS, + expClosed: someClosedTS, + expAssignedClosedBumped: false, }, { name: "brand new lease", @@ -808,6 +819,11 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { leaseExp: expiredLeaseTimestamp, rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, expClosed: now.ToTimestamp(), + // Check that the lease proposal does not bump b.assignedClosedTimestamp. + // The proposer cannot make promises about the write timestamps of further + // requests based on the start time of a proposed lease. See comments in + // propBuf.assignClosedTimestampToProposalLocked(). + expAssignedClosedBumped: false, }, { name: "lease extension", @@ -824,7 +840,8 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, // Lease extensions don't carry closed timestamps because they don't get // MLAIs, and so they can be reordered. - expClosed: hlc.Timestamp{}, + expClosed: hlc.Timestamp{}, + expAssignedClosedBumped: false, }, { // Lease transfers behave just like regular writes. The lease start time @@ -835,21 +852,23 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { Sequence: curLease.Sequence + 1, Start: now, }, - trackerLowerBound: hlc.Timestamp{}, - leaseExp: hlc.MaxTimestamp, - rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, - expClosed: nowMinusClosedLag, + trackerLowerBound: hlc.Timestamp{}, + leaseExp: hlc.MaxTimestamp, + rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, + expClosed: nowMinusClosedLag, + expAssignedClosedBumped: true, }, { // With the LEAD_FOR_GLOBAL_READS policy, we're expecting to close // timestamps in the future. - name: "global range", - reqType: regularWrite, - trackerLowerBound: hlc.Timestamp{}, - leaseExp: hlc.MaxTimestamp, - rangePolicy: roachpb.LEAD_FOR_GLOBAL_READS, - prevClosedTimestamp: hlc.Timestamp{}, - expClosed: nowPlusGlobalReadLead, + name: "global range", + reqType: regularWrite, + trackerLowerBound: hlc.Timestamp{}, + leaseExp: hlc.MaxTimestamp, + rangePolicy: roachpb.LEAD_FOR_GLOBAL_READS, + prevClosedTimestamp: hlc.Timestamp{}, + expClosed: nowPlusGlobalReadLead, + expAssignedClosedBumped: true, }, } { t.Run(tc.name, func(t *testing.T) { @@ -899,6 +918,11 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { require.NoError(t, err) require.NoError(t, b.flushLocked(ctx)) checkClosedTS(t, r, tc.expClosed) + if tc.expAssignedClosedBumped { + require.Equal(t, tc.expClosed, b.assignedClosedTimestamp) + } else { + require.Equal(t, tc.prevClosedTimestamp, b.assignedClosedTimestamp) + } }) } } diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 59378df61af4..811aa512980e 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -47,6 +47,11 @@ type StoreTestingKnobs struct { // TestingProposalFilter is called before proposing each command. TestingProposalFilter kvserverbase.ReplicaProposalFilter + // TestingProposalSubmitFilter can be used by tests to observe and optionally + // drop Raft proposals before they are handed to etcd/raft to begin the + // process of replication. Dropped proposals are still eligible to be + // reproposed due to ticks. + TestingProposalSubmitFilter func(*ProposalData) (drop bool, err error) // TestingApplyFilter is called before applying the results of a // command on each replica. If it returns an error, the command will From 3d0303324ec932661425b10998639f4bd26aaebf Mon Sep 17 00:00:00 2001 From: Ricky Stewart <ricky@cockroachlabs.com> Date: Mon, 19 Apr 2021 10:22:21 -0500 Subject: [PATCH 26/37] build: upgrade go to 1.15.11 Pick up fixes to the following Go bugs: * https://github.com/golang/go/issues/45076 * https://github.com/golang/go/issues/45187 * https://github.com/golang/go/issues/42884 * [ ] Adjust the Pebble tests to run in new version. * [x] Adjust version in Docker image ([source](./builder/Dockerfile)). * [x] Rebuild and push the Docker image (following [Basic Process](#basic-process)) * [x] Bump the version in `WORKSPACE` under `go_register_toolchains`. You may need to bump [rules_go](https://github.com/bazelbuild/rules_go/releases). * [x] Bump the version in `builder.sh` accordingly ([source](./builder.sh#L6)). * [ ] Bump the version in `go-version-check.sh` ([source](./go-version-check.sh)), unless bumping to a new patch release. * [ ] Bump the go version in `go.mod`. You may also need to rerun `make vendor_rebuild` if vendoring has changed. * [x] Bump the default installed version of Go in `bootstrap-debian.sh` ([source](./bootstrap/bootstrap-debian.sh#L40-42)). * [x] Replace other mentions of the older version of go (grep for `golang:<old_version>` and `go<old_version>`). * [ ] Update the `builder.dockerImage` parameter in the TeamCity [`Cockroach`](https://teamcity.cockroachdb.com/admin/editProject.html?projectId=Cockroach&tab=projectParams) and [`Internal`](https://teamcity.cockroachdb.com/admin/editProject.html?projectId=Internal&tab=projectParams) projects. * [x] Adjust `GO_VERSION` in the TeamCity agent image ([setup script](./packer/teamcity-agent.sh)) and ask the Developer Infrastructure team to deploy new images. Resolves #63836. Release note (general change): Upgrade the CRDB binary to Go 1.15.10 --- WORKSPACE | 2 +- build/bootstrap/bootstrap-debian.sh | 4 ++-- build/builder.sh | 2 +- build/builder/Dockerfile | 4 ++-- build/packer/teamcity-agent.sh | 4 ++-- pkg/cmd/roachtest/go_helpers.go | 4 ++-- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/WORKSPACE b/WORKSPACE index dca36599ba4a..e9102c1adb1e 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -108,7 +108,7 @@ load("@io_bazel_rules_go//go:deps.bzl", "go_register_toolchains", "go_rules_depe go_rules_dependencies() -go_register_toolchains(go_version = "1.15.10") +go_register_toolchains(go_version = "1.15.11") # Configure nodeJS. load("@build_bazel_rules_nodejs//:index.bzl", "yarn_install") diff --git a/build/bootstrap/bootstrap-debian.sh b/build/bootstrap/bootstrap-debian.sh index 07dea0a03ca5..4783eea4e84b 100755 --- a/build/bootstrap/bootstrap-debian.sh +++ b/build/bootstrap/bootstrap-debian.sh @@ -46,9 +46,9 @@ sudo tar -C /usr -zxf /tmp/cmake.tgz && rm /tmp/cmake.tgz # Install Go. trap 'rm -f /tmp/go.tgz' EXIT -curl -fsSL https://dl.google.com/go/go1.15.10.linux-amd64.tar.gz > /tmp/go.tgz +curl -fsSL https://dl.google.com/go/go1.15.11.linux-amd64.tar.gz > /tmp/go.tgz sha256sum -c - <<EOF -4aa1267517df32f2bf1cc3d55dfc27d0c6b2c2b0989449c96dd19273ccca051d /tmp/go.tgz +8825b72d74b14e82b54ba3697813772eb94add3abf70f021b6bdebe193ed01ec /tmp/go.tgz EOF sudo tar -C /usr/local -zxf /tmp/go.tgz && rm /tmp/go.tgz diff --git a/build/builder.sh b/build/builder.sh index 3baeb5dff35d..3082c828c848 100755 --- a/build/builder.sh +++ b/build/builder.sh @@ -3,7 +3,7 @@ set -euo pipefail image=cockroachdb/builder -version=20210330-194147 +version=20210419-170223 function init() { docker build --tag="${image}" "$(dirname "${0}")/builder" diff --git a/build/builder/Dockerfile b/build/builder/Dockerfile index f855ee3b2714..07e14df5a10c 100644 --- a/build/builder/Dockerfile +++ b/build/builder/Dockerfile @@ -235,8 +235,8 @@ RUN curl -fsSL https://github.com/Kitware/CMake/releases/download/v3.17.0/cmake- # releases of Go will no longer be run in CI once it is changed. Consider # bumping the minimum allowed version of Go in /build/go-version-chech.sh. RUN DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends golang \ - && curl -fsSL https://storage.googleapis.com/golang/go1.15.10.src.tar.gz -o golang.tar.gz \ - && echo 'c1dbca6e0910b41d61a95bf9878f6d6e93d15d884c226b91d9d4b1113c10dd65 golang.tar.gz' | sha256sum -c - \ + && curl -fsSL https://storage.googleapis.com/golang/go1.15.11.src.tar.gz -o golang.tar.gz \ + && echo 'f25b2441d4c76cf63cde94d59bab237cc33e8a2a139040d904c8630f46d061e5 golang.tar.gz' | sha256sum -c - \ && tar -C /usr/local -xzf golang.tar.gz \ && rm golang.tar.gz \ && cd /usr/local/go/src \ diff --git a/build/packer/teamcity-agent.sh b/build/packer/teamcity-agent.sh index eca335fd0ce4..0430c6e1c7a4 100644 --- a/build/packer/teamcity-agent.sh +++ b/build/packer/teamcity-agent.sh @@ -42,9 +42,9 @@ apt-get install --yes \ pass \ unzip -curl -fsSL https://dl.google.com/go/go1.15.10.linux-amd64.tar.gz > /tmp/go.tgz +curl -fsSL https://dl.google.com/go/go1.15.11.linux-amd64.tar.gz > /tmp/go.tgz sha256sum -c - <<EOF -4aa1267517df32f2bf1cc3d55dfc27d0c6b2c2b0989449c96dd19273ccca051d /tmp/go.tgz +8825b72d74b14e82b54ba3697813772eb94add3abf70f021b6bdebe193ed01ec /tmp/go.tgz EOF tar -C /usr/local -zxf /tmp/go.tgz && rm /tmp/go.tgz diff --git a/pkg/cmd/roachtest/go_helpers.go b/pkg/cmd/roachtest/go_helpers.go index 11c3ff1783b7..7f1535441fcd 100644 --- a/pkg/cmd/roachtest/go_helpers.go +++ b/pkg/cmd/roachtest/go_helpers.go @@ -18,13 +18,13 @@ const goPath = `/mnt/data1/go` // "node". func installGolang(ctx context.Context, t *test, c *cluster, node nodeListOption) { if err := repeatRunE( - ctx, c, node, "download go", `curl -fsSL https://dl.google.com/go/go1.15.10.linux-amd64.tar.gz > /tmp/go.tgz`, + ctx, c, node, "download go", `curl -fsSL https://dl.google.com/go/go1.15.11.linux-amd64.tar.gz > /tmp/go.tgz`, ); err != nil { t.Fatal(err) } if err := repeatRunE( ctx, c, node, "verify tarball", `sha256sum -c - <<EOF -4aa1267517df32f2bf1cc3d55dfc27d0c6b2c2b0989449c96dd19273ccca051d /tmp/go.tgz +8825b72d74b14e82b54ba3697813772eb94add3abf70f021b6bdebe193ed01ec /tmp/go.tgz EOF`, ); err != nil { t.Fatal(err) From b38b69fd381743ccc458a3c98f8c54e44a510248 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy <yevgeniy@cockroachlabs.com> Date: Mon, 19 Apr 2021 12:23:14 -0400 Subject: [PATCH 27/37] changefeedccl: Make `kafka_sink_config` a valid option. Add `kafka_sink_config` to the list of allowed changefeed options. Release Notes: None --- pkg/ccl/changefeedccl/changefeed_test.go | 5 ++++- pkg/ccl/changefeedccl/changefeedbase/options.go | 1 + 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 753deedb548b..50abef916fd4 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -2254,7 +2254,10 @@ func TestChangefeedErrors(t *testing.T) { t, `param sasl_mechanism must be one of SCRAM-SHA-256, SCRAM-SHA-512, or PLAIN`, `CREATE CHANGEFEED FOR foo INTO $1`, `kafka://nope/?sasl_enabled=true&sasl_mechanism=unsuppported`, ) - + sqlDB.ExpectErr( + t, `client has run out of available brokers`, + `CREATE CHANGEFEED FOR foo INTO 'kafka://nope/' WITH kafka_sink_config='{"Flush": {"Messages": 100}}'`, + ) // The avro format doesn't support key_in_value yet. sqlDB.ExpectErr( t, `key_in_value is not supported with format=experimental_avro`, diff --git a/pkg/ccl/changefeedccl/changefeedbase/options.go b/pkg/ccl/changefeedccl/changefeedbase/options.go index f114914895b7..27eaeb1991be 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/options.go +++ b/pkg/ccl/changefeedccl/changefeedbase/options.go @@ -126,4 +126,5 @@ var ChangefeedOptionExpectValues = map[string]sql.KVStringOptValidate{ OptInitialScan: sql.KVStringOptRequireNoValue, OptNoInitialScan: sql.KVStringOptRequireNoValue, OptProtectDataFromGCOnPause: sql.KVStringOptRequireNoValue, + OptKafkaSinkConfig: sql.KVStringOptRequireValue, } From e757cb5b9cd0c785ce49cfa71a5b6b90feebccd3 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss <knz@thaumogen.net> Date: Mon, 19 Apr 2021 13:34:51 +0200 Subject: [PATCH 28/37] cli: bump the cobra dependency and add autocompletion for fish This updates the cobra dep which hadn't been updated since 2017. Also picks up the new support for fish autocompletions. Release note (cli change): Certain errors caused by invalid command-line arguments are now printed on the process' standard error stream, instead of standard output. Release note (cli change): The `cockroach gen autocomplete` command has been updated and can now produce autocompletion definitions for the `fish` shell. --- DEPS.bzl | 20 ++++++++++++++++++-- go.mod | 2 +- go.sum | 10 ++++++++-- pkg/cli/cli_test.go | 1 + pkg/cli/flags.go | 9 +++++++++ pkg/cli/gen.go | 10 ++++++++-- vendor | 2 +- 7 files changed, 46 insertions(+), 8 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 9ad635be5f4d..0c96711652aa 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -710,6 +710,14 @@ def go_deps(): sum = "h1:BSKMNlYxDvnunlTymqtgONjNnaRV1sTpcovwwjF22jk=", version = "v1.0.10", ) + go_repository( + name = "com_github_cpuguy83_go_md2man_v2", + build_file_proto_mode = "disable_global", + importpath = "github.com/cpuguy83/go-md2man/v2", + sum = "h1:EoUDS0afbrsXAZ9YQ9jdu/mZ2sXgT1/2yyNng4PGlyM=", + version = "v2.0.0", + ) + go_repository( name = "com_github_creack_pty", build_file_proto_mode = "disable_global", @@ -3270,6 +3278,14 @@ def go_deps(): sum = "h1:HyvC0ARfnZBqnXwABFeSZHpKvJHJJfPz81GNueLj0oo=", version = "v1.5.2", ) + go_repository( + name = "com_github_russross_blackfriday_v2", + build_file_proto_mode = "disable_global", + importpath = "github.com/russross/blackfriday/v2", + sum = "h1:lPqVAte+HuHNfhJ/0LC98ESWRz8afy9tM/0RK8m9o+Q=", + version = "v2.0.1", + ) + go_repository( name = "com_github_ryanuber_columnize", build_file_proto_mode = "disable_global", @@ -3425,8 +3441,8 @@ def go_deps(): name = "com_github_spf13_cobra", build_file_proto_mode = "disable_global", importpath = "github.com/spf13/cobra", - sum = "h1:f0B+LkLX6DtmRH1isoNA9VTtNUK9K8xYd28JNNfOv/s=", - version = "v0.0.5", + sum = "h1:xghbfqPkxzxP3C/f3n5DdpAbdKLj4ZE4BWQI362l53M=", + version = "v1.1.3", ) go_repository( name = "com_github_spf13_jwalterweatherman", diff --git a/go.mod b/go.mod index 06b12511f1a1..51b58c3b9c95 100644 --- a/go.mod +++ b/go.mod @@ -136,7 +136,7 @@ require ( github.com/sasha-s/go-deadlock v0.2.0 github.com/shirou/gopsutil v2.20.9+incompatible github.com/shopspring/decimal v1.2.0 // indirect - github.com/spf13/cobra v0.0.5 + github.com/spf13/cobra v1.1.3 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.6.1 github.com/twpayne/go-geom v1.3.7-0.20210228220813-9d9885b50d3e diff --git a/go.sum b/go.sum index ac0ccfd25bd4..8d5fc5856005 100644 --- a/go.sum +++ b/go.sum @@ -290,6 +290,8 @@ github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfc github.com/corpix/uarand v0.1.1/go.mod h1:SFKZvkcRoLqVRFZ4u25xPmp6m9ktANfbpXZ7SJ0/FNU= github.com/cpuguy83/go-md2man v1.0.10 h1:BSKMNlYxDvnunlTymqtgONjNnaRV1sTpcovwwjF22jk= github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= +github.com/cpuguy83/go-md2man/v2 v2.0.0 h1:EoUDS0afbrsXAZ9YQ9jdu/mZ2sXgT1/2yyNng4PGlyM= +github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.11/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= @@ -568,7 +570,6 @@ github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18h github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= @@ -1123,6 +1124,8 @@ github.com/rs/zerolog v1.13.0/go.mod h1:YbFCdg8HfsridGWAh22vktObvhZbQsZXe4/zB0OK github.com/rs/zerolog v1.15.0/go.mod h1:xYTKnLHcpfU2225ny5qZjxnj9NvkumZYjJHlAThCjNc= github.com/russross/blackfriday v1.5.2 h1:HyvC0ARfnZBqnXwABFeSZHpKvJHJJfPz81GNueLj0oo= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= +github.com/russross/blackfriday/v2 v2.0.1 h1:lPqVAte+HuHNfhJ/0LC98ESWRz8afy9tM/0RK8m9o+Q= +github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/samuel/go-zookeeper v0.0.0-20200724154423-2164a8ac840e/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= @@ -1140,6 +1143,7 @@ github.com/shirou/gopsutil v2.20.9+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4= github.com/shopspring/decimal v1.2.0 h1:abSATXmQEYyShuxI4/vyW3tV1MrKAJzCZ/0zLUXYbsQ= github.com/shopspring/decimal v1.2.0/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= +github.com/shurcooL/sanitized_anchor_name v1.0.0 h1:PdmoCO6wvbs+7yrJyMORt4/BmY5IYyJwS/kOiWx8mHo= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= @@ -1163,8 +1167,9 @@ github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkU github.com/spf13/cast v1.3.1 h1:nFm6S0SMdyzrzcmThSipiEubIDy8WEXKNZ0UOgiRpng= github.com/spf13/cast v1.3.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= -github.com/spf13/cobra v0.0.5 h1:f0B+LkLX6DtmRH1isoNA9VTtNUK9K8xYd28JNNfOv/s= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= +github.com/spf13/cobra v1.1.3 h1:xghbfqPkxzxP3C/f3n5DdpAbdKLj4ZE4BWQI362l53M= +github.com/spf13/cobra v1.1.3/go.mod h1:pGADOWyqRD/YMrPZigI/zbliZ2wVD/23d+is3pSWzOo= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= github.com/spf13/jwalterweatherman v1.1.0 h1:ue6voC5bR5F8YxI5S67j9i582FU4Qvo2bmqnqMYADFk= github.com/spf13/jwalterweatherman v1.1.0/go.mod h1:aNWZUN0dPAAO/Ljvb5BEdw96iTZ0EXowPYD95IqWIGo= @@ -1174,6 +1179,7 @@ github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnIn github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= +github.com/spf13/viper v1.7.0/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg= github.com/spf13/viper v1.7.1 h1:pM5oEahlgWv/WnHXpgbKz7iLIxRf65tye2Ci+XFK5sk= github.com/spf13/viper v1.7.1/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= diff --git a/pkg/cli/cli_test.go b/pkg/cli/cli_test.go index c5fa26176cd5..4f8b25eb3550 100644 --- a/pkg/cli/cli_test.go +++ b/pkg/cli/cli_test.go @@ -1549,6 +1549,7 @@ func TestGenAutocomplete(t *testing.T) { }{ {shell: ""}, {shell: "bash"}, + {shell: "fish"}, {shell: "zsh"}, {shell: "bad", expErr: `invalid argument "bad" for "cockroach gen autocomplete"`}, } { diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index b13d5edc8666..fa41beb9f232 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -284,6 +284,15 @@ func init() { pf.AddFlag(flag) }) + { + // Since cobra v0.0.7, cobra auto-adds `-v` if not defined. We don't + // want that: we will likely want to add --verbose for some sub-commands, + // and -v should remain reserved as an alias for --verbose. + var unused bool + pf.BoolVarP(&unused, "verbose", "v", false, "") + _ = pf.MarkHidden("verbose") + } + // Logging flags common to all commands. { // Logging configuration. diff --git a/pkg/cli/gen.go b/pkg/cli/gen.go index 34eb31e9e226..295c1ef76b5a 100644 --- a/pkg/cli/gen.go +++ b/pkg/cli/gen.go @@ -85,7 +85,8 @@ var genAutocompleteCmd = &cobra.Command{ Long: `Generate autocompletion script for CockroachDB. If no arguments are passed, or if 'bash' is passed, a bash completion file is -written to ./cockroach.bash. If 'zsh' is passed, a zsh completion file is written +written to ./cockroach.bash. If 'fish' is passed, a fish completion file +is written to ./cockroach.fish. If 'zsh' is passed, a zsh completion file is written to ./_cockroach. Use "--out=/path/to/file" to override the output file location. Note that for the generated file to work on OS X with bash, you'll need to install @@ -93,7 +94,7 @@ Homebrew's bash-completion package (or an equivalent) and follow the post-instal instructions. `, Args: cobra.OnlyValidArgs, - ValidArgs: []string{"bash", "zsh"}, + ValidArgs: []string{"bash", "zsh", "fish"}, RunE: MaybeDecorateGRPCError(runGenAutocompleteCmd), } @@ -112,6 +113,11 @@ func runGenAutocompleteCmd(cmd *cobra.Command, args []string) error { autoCompletePath = "cockroach.bash" } err = cmd.Root().GenBashCompletionFile(autoCompletePath) + case "fish": + if autoCompletePath == "" { + autoCompletePath = "cockroach.fish" + } + err = cmd.Root().GenFishCompletionFile(autoCompletePath, true /* include description */) case "zsh": if autoCompletePath == "" { autoCompletePath = "_cockroach" diff --git a/vendor b/vendor index c12592169796..c14695753233 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit c12592169796038ce96f88e32dd47b7571245e83 +Subproject commit c14695753233d2aa260a7c591cfcda466b09bd2a From 26effeeb68934e6e297614fc0103c94e14f90b67 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss <knz@thaumogen.net> Date: Mon, 19 Apr 2021 16:04:20 +0200 Subject: [PATCH 29/37] cli/interactive_tests: complete the end-to-end test for `cockroach connect` The end-to-end test for the new `connect` command was incomplete, because of issue #61624 that was blocking the functionality. Now that #63589 is in, we can add the missing test. Release note: None --- pkg/cli/interactive_tests/test_connect.tcl | 33 ++++++++++++++++------ 1 file changed, 24 insertions(+), 9 deletions(-) diff --git a/pkg/cli/interactive_tests/test_connect.tcl b/pkg/cli/interactive_tests/test_connect.tcl index 7752e8137df5..82d21c779fe7 100644 --- a/pkg/cli/interactive_tests/test_connect.tcl +++ b/pkg/cli/interactive_tests/test_connect.tcl @@ -67,6 +67,7 @@ eexpect "cert files generated in: certs/n1" eexpect ":/# " end_test +# Keep the generated certs for both nodes to the artifacts directory. system "cp -a certs logs/" # NB: we will be able to remove the manual generation of root certs @@ -74,15 +75,29 @@ system "cp -a certs logs/" system "$argv cert create-client root --ca-key=certs/n1/ca-client.key --certs-dir=certs/n1" system "$argv cert create-client root --ca-key=certs/n2/ca-client.key --certs-dir=certs/n2" -# TODO(knz): Also test multi-server start once the advertise addresses are populated. -# -# start_test "Check that we can start two servers using the newly minted certs." -# send "$argv start --listen-addr=`cat hostname.txt`:26257 --http-addr=`cat hostname.txt`:8080 --join=`cat hostname.txt`:26258 --certs-dir=certs/n1 --store=logs/db1 --vmodule='*=1'\r" -# eexpect "initial startup completed" -# -# set spawn_id $shell2_spawn_id -# send "$argv start --listen-addr=`cat hostname.txt`:26258 --http-addr=`cat hostname.txt`:8081 --join=`cat hostname.txt`:26257 --certs-dir=certs/n2 --store=logs/db2 --vmodule='*=1'\r" -# eexpect "initial startup completed" +start_test "Check that we can start two servers using the newly minted certs." +send "$argv start --listen-addr=`cat hostname.txt`:26257 --http-addr=`cat hostname.txt`:8080 --join=`cat hostname.txt`:26258 --certs-dir=certs/n1 --store=logs/db1 --pid-file=server_pid1 --vmodule='*=1'\r" +eexpect "initial startup completed" +set spawn_id $shell2_spawn_id +send "$argv start --listen-addr=`cat hostname.txt`:26258 --http-addr=`cat hostname.txt`:8081 --join=`cat hostname.txt`:26257 --certs-dir=certs/n2 --store=logs/db2 --pid-file=server_pid2 --vmodule='*=1'\r" +eexpect "initial startup completed" + +# Now initialize the cluster to trigger generation of the node IDs. +system "$argv init --certs-dir=certs/n1 --host `cat hostname.txt`" + +# Now expect the startup messages on both process outputs. +eexpect "CockroachDB node starting" +set spawn_id $shell1_spawn_id +eexpect "CockroachDB node starting" end_test +start_test "Check we can connect a SQL client to the newly initialized two nodes" +system "$argv sql --certs-dir=certs/n1 --host=`cat hostname.txt`:26257 -e 'select 1'" +system "$argv sql --certs-dir=certs/n2 --host=`cat hostname.txt`:26258 -e 'select 1'" +end_test + +# Stop the servers. We do not care about graceful shutdown here since we are not +# using the server files again beyond this point. +system "kill -9 `cat server_pid1 server_pid2`" + From f430c4a8071d7287a8af59d9680a578e82dffbbe Mon Sep 17 00:00:00 2001 From: Marylia Gutierrez <marylia@cockroachlabs.com> Date: Mon, 12 Apr 2021 15:40:15 -0400 Subject: [PATCH 30/37] sql: Add Last Execution Timestamp Release note: None --- pkg/roachpb/app_stats.go | 4 + pkg/roachpb/app_stats.pb.go | 227 +++++++++++------- pkg/roachpb/app_stats.proto | 3 + pkg/sql/app_stats.go | 1 + pkg/ui/package.json | 2 +- pkg/ui/src/js/package.json | 2 +- pkg/ui/src/util/appStats.spec.ts | 5 + pkg/ui/src/util/appStats.ts | 4 + .../src/views/statements/statements.spec.tsx | 4 + pkg/ui/yarn-vendor | 2 +- pkg/ui/yarn.lock | 141 ++++++++++- 11 files changed, 290 insertions(+), 105 deletions(-) diff --git a/pkg/roachpb/app_stats.go b/pkg/roachpb/app_stats.go index 3bf87bd9e2ec..2822838b1cd8 100644 --- a/pkg/roachpb/app_stats.go +++ b/pkg/roachpb/app_stats.go @@ -145,6 +145,10 @@ func (s *StatementStatistics) Add(other *StatementStatistics) { s.SensitiveInfo = other.SensitiveInfo } + if s.LastExecTimestamp.Before(other.LastExecTimestamp) { + s.LastExecTimestamp = other.LastExecTimestamp + } + s.Count += other.Count } diff --git a/pkg/roachpb/app_stats.pb.go b/pkg/roachpb/app_stats.pb.go index a70cf74967bc..d35227990b81 100644 --- a/pkg/roachpb/app_stats.pb.go +++ b/pkg/roachpb/app_stats.pb.go @@ -89,6 +89,8 @@ type StatementStatistics struct { ExecStats ExecStats `protobuf:"bytes,21,opt,name=exec_stats,json=execStats" json:"exec_stats"` // SQLType is the type of the sql (DDL, DML, DCL or TCL) SQLType string `protobuf:"bytes,22,opt,name=sql_type,json=sqlType" json:"sql_type"` + // LastExecTimestamp is the last timestamp the statement was executed. + LastExecTimestamp time.Time `protobuf:"bytes,23,opt,name=last_exec_timestamp,json=lastExecTimestamp,stdtime" json:"last_exec_timestamp"` } func (m *StatementStatistics) Reset() { *m = StatementStatistics{} } @@ -561,90 +563,92 @@ func init() { func init() { proto.RegisterFile("roachpb/app_stats.proto", fileDescriptor_81c296505f9d1940) } var fileDescriptor_81c296505f9d1940 = []byte{ - // 1325 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x3b, 0x6f, 0x1b, 0x47, - 0x10, 0xd6, 0x89, 0x27, 0xf1, 0x38, 0x7c, 0x88, 0x5e, 0x49, 0xf6, 0x85, 0x10, 0x48, 0x99, 0xb0, - 0x61, 0x39, 0x0f, 0x0a, 0x10, 0xd2, 0x24, 0x81, 0x5f, 0x34, 0x1d, 0x80, 0xb2, 0x6c, 0xc4, 0xa4, - 0xd2, 0xa4, 0x39, 0xac, 0xef, 0x86, 0xf2, 0x41, 0xf7, 0xd2, 0xee, 0x52, 0x26, 0xfb, 0x20, 0xb5, - 0x8b, 0x34, 0xe9, 0xf2, 0x3f, 0x52, 0xa6, 0x71, 0x13, 0xc0, 0xe9, 0x5c, 0x29, 0x89, 0xdc, 0xe4, - 0x17, 0xa4, 0x48, 0x15, 0xec, 0xde, 0x1d, 0x4d, 0xd2, 0x74, 0x72, 0x8e, 0xbb, 0xe3, 0xcc, 0x7c, - 0xdf, 0xee, 0xce, 0x7e, 0x33, 0xb3, 0x84, 0x4b, 0x2c, 0xa4, 0xf6, 0x93, 0xe8, 0xf1, 0x2e, 0x8d, - 0x22, 0x8b, 0x0b, 0x2a, 0x78, 0x2b, 0x62, 0xa1, 0x08, 0x49, 0xd9, 0x0e, 0xed, 0x63, 0xe5, 0x6c, - 0xf1, 0x13, 0xaf, 0xb6, 0x71, 0x14, 0x1e, 0x85, 0xca, 0xb3, 0x2b, 0xbf, 0xe2, 0xa0, 0x5a, 0xe3, - 0x28, 0x0c, 0x8f, 0x3c, 0xdc, 0x55, 0xbf, 0x1e, 0x0f, 0x07, 0xbb, 0xc2, 0xf5, 0x91, 0x0b, 0xea, - 0x47, 0x71, 0x40, 0xf3, 0xd7, 0x3c, 0xac, 0xf7, 0x05, 0x15, 0xe8, 0x63, 0x20, 0xe4, 0x87, 0xcb, - 0x85, 0x6b, 0x73, 0x52, 0x83, 0x15, 0x3b, 0x1c, 0x06, 0xc2, 0xd4, 0xb6, 0xb5, 0x9d, 0x5c, 0x5b, - 0x7f, 0x7e, 0xd6, 0x58, 0xea, 0xc5, 0x26, 0xf2, 0x29, 0xac, 0x0f, 0x5c, 0xc6, 0x85, 0x45, 0x85, - 0x40, 0x3f, 0x12, 0x56, 0x1c, 0xb9, 0x3c, 0x15, 0x79, 0x41, 0x05, 0xdc, 0x89, 0xfd, 0x77, 0x15, - 0xea, 0x2a, 0x14, 0x7d, 0x3a, 0xb2, 0x18, 0x0a, 0xe6, 0x22, 0x37, 0x73, 0x53, 0xd1, 0xe0, 0xd3, - 0x51, 0x2f, 0xb6, 0x93, 0x8f, 0x61, 0xcd, 0xc3, 0x23, 0x6a, 0x8f, 0x2d, 0x8f, 0x72, 0x61, 0x21, - 0x63, 0xa6, 0xbe, 0xad, 0xed, 0x14, 0x92, 0xd0, 0x72, 0xec, 0x3c, 0xa0, 0x5c, 0xdc, 0x63, 0x8c, - 0xdc, 0x00, 0x73, 0x2e, 0xda, 0x62, 0xe8, 0x50, 0x5b, 0xa0, 0x63, 0x16, 0xa7, 0x60, 0x9b, 0x33, - 0xb0, 0x5e, 0x12, 0x42, 0xbe, 0x00, 0x23, 0x18, 0xfa, 0x16, 0x0b, 0x9f, 0x72, 0x73, 0x65, 0x5b, - 0xdb, 0x29, 0xee, 0xd5, 0x5a, 0x33, 0x69, 0x6d, 0x3d, 0x1c, 0xfa, 0xc8, 0x5c, 0x5b, 0x66, 0x26, - 0xa1, 0xca, 0x07, 0x43, 0xbf, 0x17, 0x3e, 0xe5, 0xe4, 0x06, 0x14, 0x22, 0xca, 0x38, 0x5a, 0x1e, - 0x15, 0xe6, 0x6a, 0x46, 0xb4, 0xa1, 0x20, 0x07, 0x54, 0xc8, 0xb5, 0x23, 0x8f, 0x06, 0x0a, 0x9d, - 0xcf, 0xba, 0xb6, 0x44, 0x48, 0xf0, 0x67, 0x90, 0x67, 0xc3, 0x18, 0x6b, 0x64, 0xc4, 0xae, 0xb2, - 0xa1, 0x82, 0xde, 0x81, 0x22, 0x47, 0x76, 0xea, 0xda, 0xf1, 0xc6, 0x0b, 0x19, 0xe1, 0x90, 0x80, - 0x24, 0xc5, 0x5d, 0x28, 0x85, 0xa7, 0xc8, 0x9e, 0x20, 0x75, 0x14, 0x07, 0x64, 0xe4, 0x28, 0xa6, - 0x28, 0x49, 0xd2, 0x85, 0x0a, 0xc7, 0x80, 0xbb, 0xc2, 0x3d, 0x45, 0xcb, 0x0d, 0x06, 0xa1, 0x59, - 0x52, 0x34, 0x5b, 0x73, 0x34, 0xfd, 0x34, 0xa8, 0x1b, 0x0c, 0xc2, 0x54, 0x05, 0x7c, 0xda, 0x48, - 0x6e, 0x01, 0x3c, 0x1e, 0x0b, 0xe4, 0x16, 0x43, 0xea, 0x98, 0x6b, 0x19, 0x77, 0x53, 0x50, 0x98, - 0x1e, 0x52, 0x47, 0x5e, 0xa5, 0xd4, 0x40, 0x8c, 0xaf, 0x66, 0xbd, 0x4a, 0x09, 0x49, 0xe0, 0x80, - 0x23, 0xb4, 0xe3, 0xf2, 0x34, 0x37, 0x15, 0xde, 0x9c, 0xc3, 0xdf, 0x1b, 0xa1, 0x02, 0xf3, 0x74, - 0x75, 0x4c, 0x0d, 0xe4, 0x43, 0x30, 0xf8, 0x89, 0x67, 0x89, 0x71, 0x84, 0xe6, 0x45, 0x25, 0xda, - 0x35, 0x19, 0x72, 0x7e, 0xd6, 0xc8, 0xf7, 0x1f, 0x1d, 0x1c, 0x8e, 0x23, 0xec, 0xe5, 0xf9, 0x89, - 0x27, 0x3f, 0xf6, 0x75, 0xa3, 0x5c, 0xad, 0xec, 0xeb, 0x46, 0xa5, 0xba, 0xb6, 0xaf, 0x1b, 0x17, - 0xaa, 0x64, 0x5f, 0x37, 0x48, 0x75, 0x7d, 0x5f, 0x37, 0xd6, 0xab, 0x1b, 0xfb, 0xba, 0xb1, 0x51, - 0xdd, 0x6c, 0xfe, 0xa0, 0xc3, 0xe6, 0x21, 0xa3, 0x01, 0xa7, 0xb6, 0x70, 0xc3, 0x20, 0x63, 0x55, - 0xcf, 0xd5, 0xe7, 0xf2, 0x5b, 0xea, 0x73, 0xba, 0x64, 0x72, 0xef, 0x5a, 0x32, 0x73, 0xda, 0xd3, - 0xff, 0x87, 0xf6, 0xe4, 0x55, 0xa1, 0x60, 0x63, 0x45, 0xb0, 0x92, 0xf9, 0xaa, 0x24, 0x44, 0xc2, - 0x6f, 0x01, 0xd8, 0xa1, 0xef, 0xbb, 0xe2, 0x9d, 0xaa, 0xb6, 0x10, 0x63, 0x12, 0x82, 0x29, 0xad, - 0xe5, 0xdf, 0x53, 0x6b, 0xc6, 0x7b, 0x6a, 0xad, 0xf0, 0x8e, 0x5a, 0x6b, 0x7e, 0xbb, 0x0c, 0xe5, - 0x99, 0x8a, 0x22, 0x0d, 0x30, 0x26, 0x9d, 0x56, 0x9b, 0x6a, 0x99, 0x79, 0x2f, 0xe9, 0xb1, 0x2e, - 0x6c, 0xf9, 0x21, 0x17, 0x16, 0x43, 0x1b, 0x03, 0x61, 0xa9, 0xa6, 0xe5, 0x20, 0xb7, 0x99, 0x1b, - 0x49, 0x79, 0x29, 0xa5, 0x14, 0xf7, 0x9a, 0x6f, 0xec, 0x21, 0xf2, 0xa8, 0x1b, 0x1c, 0x32, 0xc4, - 0xaf, 0x3c, 0x1a, 0x3c, 0x0c, 0x1d, 0x4c, 0x88, 0x3f, 0x90, 0x6c, 0x3d, 0x45, 0x26, 0x3d, 0x9d, - 0xd7, 0x54, 0x84, 0x42, 0xed, 0x8d, 0xa5, 0x26, 0x13, 0x6b, 0x22, 0xb7, 0x78, 0xa6, 0xb5, 0xd2, - 0x99, 0xd6, 0x3a, 0x4c, 0x23, 0xda, 0x86, 0x5c, 0xe0, 0xd9, 0x6f, 0x0d, 0xad, 0x77, 0x69, 0x76, - 0x91, 0x49, 0xc8, 0xe7, 0xfa, 0x9f, 0x3f, 0x36, 0xb4, 0x66, 0x0f, 0x8a, 0x53, 0x49, 0x26, 0x26, - 0xe8, 0x3e, 0xd2, 0x40, 0x9d, 0x5f, 0x4b, 0xb6, 0xa9, 0x2c, 0xe4, 0x3a, 0x94, 0xf9, 0xc9, 0x90, - 0x32, 0x74, 0x2c, 0xc7, 0x1d, 0x0c, 0xe2, 0xba, 0x48, 0x43, 0x4a, 0x89, 0xab, 0x23, 0x3d, 0xcd, - 0xef, 0x96, 0xe1, 0xe2, 0x82, 0x51, 0x7a, 0x1f, 0xc7, 0xb2, 0xee, 0x4e, 0x86, 0xc8, 0xc6, 0x33, - 0x09, 0x8e, 0x4d, 0xe4, 0x22, 0xe4, 0x68, 0x14, 0x29, 0xde, 0xd4, 0x23, 0x0d, 0xa4, 0x0e, 0x79, - 0xc7, 0xe5, 0xa2, 0xff, 0xe8, 0x40, 0x1d, 0xdc, 0x48, 0xaf, 0x25, 0x31, 0x92, 0x2d, 0x58, 0x1d, - 0x50, 0xd7, 0x43, 0x47, 0x95, 0x51, 0xea, 0x4e, 0x6c, 0x92, 0x35, 0x8c, 0xe2, 0x02, 0x49, 0x5d, - 0xd2, 0x40, 0xae, 0x41, 0xc9, 0xf5, 0x23, 0xcf, 0xb5, 0x5d, 0x61, 0x89, 0x51, 0xa0, 0x2a, 0x20, - 0x0d, 0x28, 0xa6, 0x9e, 0xc3, 0x51, 0x20, 0x09, 0x4e, 0xd1, 0x56, 0x02, 0x9f, 0x10, 0x9c, 0xa2, - 0x4d, 0x2e, 0x43, 0x61, 0x30, 0xf4, 0x3c, 0x8b, 0xdb, 0x34, 0x50, 0xf2, 0x4d, 0xbd, 0x86, 0x34, - 0xf7, 0x6d, 0x1a, 0x34, 0x7f, 0xd6, 0x60, 0xeb, 0x6e, 0xe8, 0x79, 0x28, 0x67, 0xec, 0xa2, 0xc7, - 0xc5, 0x15, 0x58, 0x76, 0x1d, 0x75, 0x2a, 0xbd, 0xbd, 0x91, 0xb4, 0xba, 0xe5, 0x6e, 0xe7, 0xef, - 0xb3, 0xc6, 0x6a, 0x5f, 0xf8, 0xa2, 0xdb, 0xe9, 0x2d, 0xbb, 0x52, 0xe9, 0xb9, 0x63, 0x8c, 0x53, - 0x56, 0xdc, 0xbb, 0x3a, 0x3f, 0x15, 0x16, 0x26, 0x3a, 0xdd, 0xe8, 0x31, 0x8e, 0xc9, 0x4d, 0x58, - 0x89, 0x6b, 0x64, 0xb1, 0x3e, 0x17, 0x10, 0xa4, 0xf7, 0xa2, 0x60, 0xcd, 0x9f, 0x34, 0xa8, 0x4f, - 0x4e, 0xb1, 0xb8, 0x9d, 0xde, 0x82, 0x32, 0x4f, 0x69, 0x2c, 0xd7, 0xe1, 0xa6, 0xb6, 0x9d, 0xdb, - 0xd1, 0xdb, 0xb5, 0xf3, 0xb3, 0x46, 0x69, 0xc2, 0xdf, 0xed, 0xf0, 0xa9, 0x83, 0x95, 0x26, 0x80, - 0xae, 0xc3, 0xdf, 0x7a, 0xf7, 0xb7, 0xd3, 0xbd, 0xc7, 0x92, 0xbf, 0x32, 0xb7, 0xf7, 0x85, 0xbb, - 0x99, 0xdd, 0xfd, 0x5f, 0x1a, 0xac, 0x2f, 0x28, 0x41, 0xa9, 0xf4, 0x80, 0xfa, 0x38, 0x23, 0x44, - 0x65, 0x21, 0x37, 0x60, 0x85, 0x0a, 0xc1, 0x64, 0xbe, 0x72, 0x3b, 0xc5, 0xbd, 0x6b, 0xff, 0x5d, - 0xcf, 0xad, 0x3b, 0x42, 0xb0, 0x5e, 0x8c, 0x22, 0x37, 0xc1, 0xb0, 0x9f, 0xb8, 0x9e, 0xc3, 0x30, - 0x30, 0x73, 0x8a, 0x21, 0x43, 0x47, 0xe8, 0x4d, 0x30, 0xb5, 0xdb, 0xa0, 0x4b, 0x3a, 0x99, 0x92, - 0xf4, 0xd6, 0x0b, 0xd3, 0xd7, 0x59, 0x83, 0x95, 0x53, 0xea, 0x0d, 0x71, 0x26, 0x59, 0xb1, 0x29, - 0xae, 0xe9, 0xa4, 0xb2, 0x7f, 0xd1, 0xc0, 0x38, 0x1c, 0x05, 0xf1, 0x64, 0xbd, 0x0c, 0x05, 0x31, - 0x0a, 0xac, 0x37, 0x67, 0x9e, 0x21, 0x46, 0x41, 0xfc, 0x2c, 0x6d, 0x43, 0x49, 0x86, 0xc8, 0x2e, - 0x63, 0x71, 0xb4, 0x13, 0xb5, 0x64, 0x98, 0x49, 0x62, 0xa4, 0xba, 0x4a, 0x1f, 0x6d, 0xf2, 0x09, - 0xac, 0xc5, 0x03, 0x42, 0xa0, 0x93, 0x2c, 0x36, 0xfd, 0xbc, 0xad, 0x4c, 0x9c, 0xf1, 0x92, 0x1f, - 0x41, 0x65, 0x52, 0x83, 0x71, 0xb4, 0x3e, 0x15, 0x5d, 0x4e, 0x7d, 0x2a, 0xb8, 0xf9, 0x7d, 0x0e, - 0x0a, 0x93, 0x7e, 0xfe, 0xaf, 0x03, 0xfc, 0x1e, 0x94, 0x03, 0x14, 0x4f, 0x43, 0x76, 0x6c, 0xa9, - 0x69, 0x93, 0xf9, 0x28, 0xa5, 0x04, 0xd6, 0x96, 0x28, 0xd2, 0x81, 0xb2, 0x7c, 0x07, 0xf8, 0xe8, - 0x5b, 0x43, 0x4e, 0x8f, 0x30, 0xf3, 0x94, 0x97, 0xcf, 0x87, 0x07, 0xe8, 0x7f, 0x2d, 0x41, 0xa4, - 0x2b, 0x53, 0x12, 0x08, 0x0c, 0xa4, 0x48, 0x55, 0x76, 0x33, 0x4f, 0xfb, 0xca, 0x6b, 0xa0, 0x4c, - 0x30, 0xb9, 0x0f, 0xd5, 0xf4, 0x5c, 0x3e, 0x72, 0xc9, 0x9e, 0xfd, 0xb1, 0xbe, 0x96, 0x20, 0x1f, - 0x24, 0x40, 0xf2, 0x25, 0x54, 0xe4, 0xe9, 0x1c, 0x97, 0x1f, 0x27, 0xc7, 0xcb, 0xfa, 0x06, 0x28, - 0xf9, 0x74, 0xd4, 0x71, 0xf9, 0xb1, 0x3a, 0x5f, 0xfb, 0xfa, 0xf3, 0x3f, 0xea, 0x4b, 0xcf, 0xcf, - 0xeb, 0xda, 0x8b, 0xf3, 0xba, 0xf6, 0xf2, 0xbc, 0xae, 0xfd, 0x7e, 0x5e, 0xd7, 0x9e, 0xbd, 0xaa, - 0x2f, 0xbd, 0x78, 0x55, 0x5f, 0x7a, 0xf9, 0xaa, 0xbe, 0xf4, 0x4d, 0x3e, 0xf9, 0xe3, 0xf6, 0x4f, - 0x00, 0x00, 0x00, 0xff, 0xff, 0xc5, 0xcc, 0x85, 0x98, 0xc2, 0x0d, 0x00, 0x00, + // 1349 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0xbf, 0x6f, 0x1b, 0xc7, + 0x12, 0xd6, 0x89, 0x27, 0xf1, 0x38, 0x24, 0x25, 0x7a, 0x25, 0xd9, 0xf7, 0x08, 0x81, 0x94, 0x09, + 0x1b, 0x96, 0xdf, 0x7b, 0x91, 0x00, 0x21, 0x4d, 0x12, 0xf8, 0x97, 0x2c, 0x07, 0x90, 0x2c, 0x1b, + 0x31, 0xc9, 0x34, 0x69, 0x0e, 0xeb, 0xbb, 0xa1, 0x7c, 0xd0, 0xfd, 0xd2, 0xee, 0x52, 0x26, 0xfb, + 0x20, 0xb5, 0x8b, 0x34, 0xe9, 0xd2, 0xe5, 0x8f, 0x48, 0x99, 0xc6, 0x4d, 0x00, 0x97, 0xae, 0x94, + 0x44, 0x6e, 0xf2, 0x17, 0xa4, 0x48, 0x15, 0xec, 0xee, 0x1d, 0x4d, 0xd2, 0x74, 0x72, 0x8a, 0xbb, + 0xe3, 0xcc, 0x7c, 0xdf, 0xee, 0xce, 0x7e, 0x33, 0xb3, 0x84, 0x2b, 0x2c, 0xa6, 0xee, 0xb3, 0xe4, + 0xe9, 0x36, 0x4d, 0x12, 0x87, 0x0b, 0x2a, 0xf8, 0x56, 0xc2, 0x62, 0x11, 0x93, 0xaa, 0x1b, 0xbb, + 0xc7, 0xca, 0xb9, 0xc5, 0x4f, 0x82, 0xfa, 0xea, 0x51, 0x7c, 0x14, 0x2b, 0xcf, 0xb6, 0xfc, 0xd2, + 0x41, 0xf5, 0xe6, 0x51, 0x1c, 0x1f, 0x05, 0xb8, 0xad, 0x7e, 0x3d, 0xed, 0xf7, 0xb6, 0x85, 0x1f, + 0x22, 0x17, 0x34, 0x4c, 0x74, 0x40, 0xeb, 0x07, 0x0b, 0x56, 0x3a, 0x82, 0x0a, 0x0c, 0x31, 0x12, + 0xf2, 0xc3, 0xe7, 0xc2, 0x77, 0x39, 0xa9, 0xc3, 0x82, 0x1b, 0xf7, 0x23, 0x61, 0x1b, 0x1b, 0xc6, + 0x66, 0x61, 0xd7, 0x7c, 0x79, 0xd6, 0x9c, 0x6b, 0x6b, 0x13, 0xf9, 0x18, 0x56, 0x7a, 0x3e, 0xe3, + 0xc2, 0xa1, 0x42, 0x60, 0x98, 0x08, 0x47, 0x47, 0xce, 0x8f, 0x45, 0x5e, 0x52, 0x01, 0xf7, 0xb4, + 0xff, 0xbe, 0x42, 0x5d, 0x87, 0x72, 0x48, 0x07, 0x0e, 0x43, 0xc1, 0x7c, 0xe4, 0x76, 0x61, 0x2c, + 0x1a, 0x42, 0x3a, 0x68, 0x6b, 0x3b, 0xf9, 0x3f, 0x2c, 0x07, 0x78, 0x44, 0xdd, 0xa1, 0x13, 0x50, + 0x2e, 0x1c, 0x64, 0xcc, 0x36, 0x37, 0x8c, 0xcd, 0x52, 0x1a, 0x5a, 0xd5, 0xce, 0x43, 0xca, 0xc5, + 0x03, 0xc6, 0xc8, 0x2d, 0xb0, 0xa7, 0xa2, 0x1d, 0x86, 0x1e, 0x75, 0x05, 0x7a, 0x76, 0x79, 0x0c, + 0xb6, 0x36, 0x01, 0x6b, 0xa7, 0x21, 0xe4, 0x33, 0xb0, 0xa2, 0x7e, 0xe8, 0xb0, 0xf8, 0x39, 0xb7, + 0x17, 0x36, 0x8c, 0xcd, 0xf2, 0x4e, 0x7d, 0x6b, 0x22, 0xad, 0x5b, 0x8f, 0xfb, 0x21, 0x32, 0xdf, + 0x95, 0x99, 0x49, 0xa9, 0x8a, 0x51, 0x3f, 0x6c, 0xc7, 0xcf, 0x39, 0xb9, 0x05, 0xa5, 0x84, 0x32, + 0x8e, 0x4e, 0x40, 0x85, 0xbd, 0x98, 0x13, 0x6d, 0x29, 0xc8, 0x21, 0x15, 0x72, 0xed, 0x24, 0xa0, + 0x91, 0x42, 0x17, 0xf3, 0xae, 0x2d, 0x11, 0x12, 0xfc, 0x09, 0x14, 0x59, 0x5f, 0x63, 0xad, 0x9c, + 0xd8, 0x45, 0xd6, 0x57, 0xd0, 0x7b, 0x50, 0xe6, 0xc8, 0x4e, 0x7d, 0x57, 0x6f, 0xbc, 0x94, 0x13, + 0x0e, 0x29, 0x48, 0x52, 0xdc, 0x87, 0x4a, 0x7c, 0x8a, 0xec, 0x19, 0x52, 0x4f, 0x71, 0x40, 0x4e, + 0x8e, 0x72, 0x86, 0x92, 0x24, 0xfb, 0xb0, 0xc4, 0x31, 0xe2, 0xbe, 0xf0, 0x4f, 0xd1, 0xf1, 0xa3, + 0x5e, 0x6c, 0x57, 0x14, 0xcd, 0xfa, 0x14, 0x4d, 0x27, 0x0b, 0xda, 0x8f, 0x7a, 0x71, 0xa6, 0x02, + 0x3e, 0x6e, 0x24, 0x77, 0x00, 0x9e, 0x0e, 0x05, 0x72, 0x87, 0x21, 0xf5, 0xec, 0xe5, 0x9c, 0xbb, + 0x29, 0x29, 0x4c, 0x1b, 0xa9, 0x27, 0xaf, 0x52, 0x6a, 0x40, 0xe3, 0x6b, 0x79, 0xaf, 0x52, 0x42, + 0x52, 0x38, 0xe0, 0x00, 0x5d, 0x5d, 0x9e, 0xf6, 0x9a, 0xc2, 0xdb, 0x53, 0xf8, 0x07, 0x03, 0x54, + 0x60, 0x9e, 0xad, 0x8e, 0x99, 0x81, 0xfc, 0x17, 0x2c, 0x7e, 0x12, 0x38, 0x62, 0x98, 0xa0, 0x7d, + 0x59, 0x89, 0x76, 0x59, 0x86, 0x9c, 0x9f, 0x35, 0x8b, 0x9d, 0x27, 0x87, 0xdd, 0x61, 0x82, 0xed, + 0x22, 0x3f, 0x09, 0xe4, 0x07, 0xe9, 0xc2, 0x8a, 0x56, 0xba, 0x5c, 0x6f, 0x54, 0xcc, 0xf6, 0x95, + 0x74, 0xcf, 0xba, 0xdc, 0xb7, 0xb2, 0x72, 0xdf, 0xea, 0x66, 0x11, 0xbb, 0x96, 0xa4, 0x7c, 0xf1, + 0x4b, 0xd3, 0x68, 0x5f, 0x92, 0x04, 0x72, 0x3b, 0x23, 0xe7, 0x81, 0x69, 0x55, 0x6b, 0x4b, 0x07, + 0xa6, 0xb5, 0x54, 0x5b, 0x3e, 0x30, 0xad, 0x4b, 0x35, 0x72, 0x60, 0x5a, 0xa4, 0xb6, 0x72, 0x60, + 0x5a, 0x2b, 0xb5, 0xd5, 0x03, 0xd3, 0x5a, 0xad, 0xad, 0xb5, 0xbe, 0x33, 0x61, 0xad, 0xcb, 0x68, + 0xc4, 0xa9, 0x2b, 0xfc, 0x38, 0xca, 0xd9, 0x2b, 0xa6, 0xaa, 0x7e, 0xfe, 0x3d, 0x55, 0x3f, 0x5e, + 0x88, 0x85, 0x8b, 0x16, 0xe2, 0x94, 0xa2, 0xcd, 0x7f, 0xa1, 0x68, 0x29, 0x00, 0x14, 0x6c, 0xa8, + 0x08, 0x16, 0x72, 0x0b, 0x40, 0x42, 0x24, 0xfc, 0x0e, 0x80, 0x1b, 0x87, 0xa1, 0x2f, 0x2e, 0xd4, + 0x0b, 0x4a, 0x1a, 0x93, 0x12, 0x8c, 0x29, 0xb8, 0xf8, 0x81, 0x0a, 0xb6, 0x3e, 0x50, 0xc1, 0xa5, + 0x0b, 0x2a, 0xb8, 0xf5, 0xf5, 0x3c, 0x54, 0x27, 0xea, 0x94, 0x34, 0xc1, 0x1a, 0xf5, 0x6f, 0x63, + 0xac, 0x11, 0x17, 0x83, 0xb4, 0x73, 0xfb, 0xb0, 0x1e, 0xc6, 0x5c, 0x38, 0x0c, 0x5d, 0x8c, 0x84, + 0xa3, 0x5a, 0xa1, 0x87, 0xdc, 0x65, 0x7e, 0x22, 0xe5, 0xa5, 0x94, 0x52, 0xde, 0x69, 0xbd, 0xb3, + 0x87, 0x24, 0xa0, 0x7e, 0xd4, 0x65, 0x88, 0x5f, 0x04, 0x34, 0x7a, 0x1c, 0x7b, 0x98, 0x12, 0xff, + 0x47, 0xb2, 0xb5, 0x15, 0x99, 0xf4, 0xec, 0xbd, 0xa5, 0x22, 0x14, 0xea, 0xef, 0x2c, 0xf5, 0xb6, + 0x74, 0x0a, 0x17, 0x28, 0x9d, 0x2b, 0x93, 0x8b, 0x8c, 0x42, 0x3e, 0x35, 0x7f, 0xff, 0xbe, 0x69, + 0xb4, 0xda, 0x50, 0x1e, 0x4b, 0x32, 0xb1, 0xc1, 0x0c, 0x91, 0x46, 0xea, 0xfc, 0x46, 0xba, 0x4d, + 0x65, 0x21, 0x37, 0xa1, 0xca, 0x4f, 0xfa, 0x94, 0xa1, 0xe7, 0x78, 0x7e, 0xaf, 0xa7, 0xeb, 0x22, + 0x0b, 0xa9, 0xa4, 0xae, 0x3d, 0xe9, 0x69, 0x7d, 0x33, 0x0f, 0x97, 0x67, 0x0c, 0xe8, 0x87, 0x38, + 0x94, 0x75, 0x77, 0xd2, 0x47, 0x36, 0x9c, 0x48, 0xb0, 0x36, 0x91, 0xcb, 0x50, 0xa0, 0x49, 0xa2, + 0x78, 0x33, 0x8f, 0x34, 0x90, 0x06, 0x14, 0x3d, 0x9f, 0x8b, 0xce, 0x93, 0x43, 0x75, 0x70, 0x2b, + 0xbb, 0x96, 0xd4, 0x48, 0xd6, 0x61, 0xb1, 0x47, 0xfd, 0x00, 0x3d, 0x55, 0x46, 0x99, 0x3b, 0xb5, + 0x49, 0xd6, 0x38, 0xd1, 0x05, 0x92, 0xb9, 0xa4, 0x81, 0xdc, 0x80, 0x8a, 0x1f, 0x26, 0x81, 0xef, + 0xfa, 0xc2, 0x11, 0x83, 0x48, 0x55, 0x40, 0x16, 0x50, 0xce, 0x3c, 0xdd, 0x41, 0x24, 0x09, 0x4e, + 0xd1, 0x55, 0x02, 0x1f, 0x11, 0x9c, 0xa2, 0x4b, 0xae, 0x42, 0xa9, 0xd7, 0x0f, 0x02, 0x87, 0xbb, + 0x34, 0x52, 0xf2, 0xcd, 0xbc, 0x96, 0x34, 0x77, 0x5c, 0x1a, 0xb5, 0x7e, 0x32, 0x60, 0xfd, 0x7e, + 0x1c, 0x04, 0x28, 0x27, 0xf7, 0xac, 0x27, 0xcb, 0x35, 0x98, 0xf7, 0x3d, 0x75, 0x2a, 0x73, 0x77, + 0x35, 0x6d, 0xa0, 0xf3, 0xfb, 0x7b, 0x7f, 0x9e, 0x35, 0x17, 0x3b, 0x22, 0x14, 0xfb, 0x7b, 0xed, + 0x79, 0x5f, 0x2a, 0xbd, 0x70, 0x8c, 0x3a, 0x65, 0xe5, 0x9d, 0xeb, 0xd3, 0xb3, 0x66, 0x66, 0xa2, + 0xb3, 0x8d, 0x1e, 0xe3, 0x90, 0xdc, 0x86, 0x05, 0x5d, 0x23, 0xb3, 0xf5, 0x39, 0x83, 0x20, 0xbb, + 0x17, 0x05, 0x6b, 0xfd, 0x68, 0x40, 0x63, 0x74, 0x8a, 0xd9, 0xed, 0xf4, 0x0e, 0x54, 0x79, 0x46, + 0xe3, 0xf8, 0x1e, 0xb7, 0x8d, 0x8d, 0xc2, 0xa6, 0xb9, 0x5b, 0x3f, 0x3f, 0x6b, 0x56, 0x46, 0xfc, + 0xfb, 0x7b, 0x7c, 0xec, 0x60, 0x95, 0x11, 0x60, 0xdf, 0xe3, 0xef, 0xbd, 0xfb, 0xbb, 0xd9, 0xde, + 0xb5, 0xe4, 0xaf, 0x4d, 0xed, 0x7d, 0xe6, 0x6e, 0x26, 0x77, 0xff, 0x87, 0x01, 0x2b, 0x33, 0x4a, + 0x50, 0x2a, 0x3d, 0xa2, 0x21, 0x4e, 0x08, 0x51, 0x59, 0xc8, 0x2d, 0x58, 0xa0, 0x42, 0x30, 0x99, + 0xaf, 0xc2, 0x66, 0x79, 0xe7, 0xc6, 0x3f, 0xd7, 0xf3, 0xd6, 0x3d, 0x21, 0x58, 0x5b, 0xa3, 0xc8, + 0x6d, 0xb0, 0xdc, 0x67, 0x7e, 0xe0, 0x31, 0x8c, 0xec, 0x82, 0x62, 0xc8, 0xd1, 0x11, 0xda, 0x23, + 0x4c, 0xfd, 0x2e, 0x98, 0x92, 0x4e, 0xa6, 0x24, 0xbb, 0xf5, 0xd2, 0xf8, 0x75, 0xd6, 0x61, 0xe1, + 0x94, 0x06, 0x7d, 0x9c, 0x48, 0x96, 0x36, 0xe9, 0x9a, 0x4e, 0x2b, 0xfb, 0x67, 0x03, 0xac, 0xee, + 0x20, 0xd2, 0xf3, 0xfa, 0x2a, 0x94, 0xc4, 0x20, 0x72, 0xde, 0x9d, 0x79, 0x96, 0x18, 0x44, 0xfa, + 0xb1, 0xbb, 0x0b, 0x15, 0x19, 0x22, 0xbb, 0x8c, 0xc3, 0xd1, 0x4d, 0xd5, 0x92, 0x63, 0x26, 0x89, + 0x81, 0xea, 0x2a, 0x1d, 0x74, 0xc9, 0x47, 0xb0, 0xac, 0x07, 0x84, 0x40, 0x2f, 0x5d, 0x6c, 0xfc, + 0xd1, 0xbc, 0x34, 0x72, 0xea, 0x25, 0xff, 0x07, 0x4b, 0xa3, 0x1a, 0xd4, 0xd1, 0xe6, 0x58, 0x74, + 0x35, 0xf3, 0xa9, 0xe0, 0xd6, 0xb7, 0x05, 0x28, 0x8d, 0xfa, 0xf9, 0xdf, 0x0e, 0xf0, 0x07, 0x50, + 0x8d, 0x50, 0x3c, 0x8f, 0xd9, 0xb1, 0xa3, 0xa6, 0x4d, 0xee, 0xa3, 0x54, 0x52, 0xd8, 0xae, 0x44, + 0x91, 0x3d, 0xa8, 0xca, 0x77, 0x40, 0x88, 0xa1, 0xd3, 0xe7, 0xf4, 0x08, 0x73, 0x4f, 0x79, 0xf9, + 0x7c, 0x78, 0x84, 0xe1, 0x97, 0x12, 0x44, 0xf6, 0x65, 0x4a, 0x22, 0x81, 0x91, 0x14, 0xa9, 0xca, + 0x6e, 0xee, 0x69, 0xbf, 0xf4, 0x16, 0x28, 0x13, 0x4c, 0x1e, 0x42, 0x2d, 0x3b, 0x57, 0x88, 0x5c, + 0xb2, 0xe7, 0xff, 0x0b, 0xb0, 0x9c, 0x22, 0x1f, 0xa5, 0x40, 0xf2, 0x39, 0x2c, 0xc9, 0xd3, 0x79, + 0x3e, 0x3f, 0x4e, 0x8f, 0x97, 0xf7, 0x0d, 0x50, 0x09, 0xe9, 0x60, 0xcf, 0xe7, 0xc7, 0xea, 0x7c, + 0xbb, 0x37, 0x5f, 0xfe, 0xd6, 0x98, 0x7b, 0x79, 0xde, 0x30, 0x5e, 0x9d, 0x37, 0x8c, 0xd7, 0xe7, + 0x0d, 0xe3, 0xd7, 0xf3, 0x86, 0xf1, 0xe2, 0x4d, 0x63, 0xee, 0xd5, 0x9b, 0xc6, 0xdc, 0xeb, 0x37, + 0x8d, 0xb9, 0xaf, 0x8a, 0xe9, 0xdf, 0xc1, 0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, 0x04, 0xf1, 0x3a, + 0x34, 0x18, 0x0e, 0x00, 0x00, } func (this *SensitiveInfo) Equal(that interface{}) bool { @@ -764,6 +768,16 @@ func (m *StatementStatistics) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + n1, err1 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.LastExecTimestamp, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.LastExecTimestamp):]) + if err1 != nil { + return 0, err1 + } + i -= n1 + i = encodeVarintAppStats(dAtA, i, uint64(n1)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xba i -= len(m.SQLType) copy(dAtA[i:], m.SQLType) i = encodeVarintAppStats(dAtA, i, uint64(len(m.SQLType))) @@ -1016,12 +1030,12 @@ func (m *SensitiveInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n18, err18 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.MostRecentPlanTimestamp, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.MostRecentPlanTimestamp):]) - if err18 != nil { - return 0, err18 + n19, err19 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.MostRecentPlanTimestamp, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.MostRecentPlanTimestamp):]) + if err19 != nil { + return 0, err19 } - i -= n18 - i = encodeVarintAppStats(dAtA, i, uint64(n18)) + i -= n19 + i = encodeVarintAppStats(dAtA, i, uint64(n19)) i-- dAtA[i] = 0x1a { @@ -1498,6 +1512,8 @@ func (m *StatementStatistics) Size() (n int) { n += 2 + l + sovAppStats(uint64(l)) l = len(m.SQLType) n += 2 + l + sovAppStats(uint64(l)) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.LastExecTimestamp) + n += 2 + l + sovAppStats(uint64(l)) return n } @@ -2191,6 +2207,39 @@ func (m *StatementStatistics) Unmarshal(dAtA []byte) error { } m.SQLType = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 23: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LastExecTimestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAppStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthAppStats + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthAppStats + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.LastExecTimestamp, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipAppStats(dAtA[iNdEx:]) diff --git a/pkg/roachpb/app_stats.proto b/pkg/roachpb/app_stats.proto index d7068733600e..ad4d90d83fd0 100644 --- a/pkg/roachpb/app_stats.proto +++ b/pkg/roachpb/app_stats.proto @@ -94,6 +94,9 @@ message StatementStatistics { // SQLType is the type of the sql (DDL, DML, DCL or TCL) optional string sql_type = 22 [(gogoproto.nullable) = false, (gogoproto.customname) = "SQLType"]; + // LastExecTimestamp is the last timestamp the statement was executed. + optional google.protobuf.Timestamp last_exec_timestamp = 23 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true]; + // Note: be sure to update `sql/app_stats.go` when adding/removing fields here! reserved 13, 14, 17, 18, 19, 20; diff --git a/pkg/sql/app_stats.go b/pkg/sql/app_stats.go index 52053be84c5d..176608395870 100644 --- a/pkg/sql/app_stats.go +++ b/pkg/sql/app_stats.go @@ -244,6 +244,7 @@ func (a *appStats) recordStatement( s.mu.data.OverheadLat.Record(s.mu.data.Count, ovhLat) s.mu.data.BytesRead.Record(s.mu.data.Count, float64(stats.bytesRead)) s.mu.data.RowsRead.Record(s.mu.data.Count, float64(stats.rowsRead)) + s.mu.data.LastExecTimestamp = timeutil.Now() // Note that some fields derived from tracing statements (such as // BytesSentOverNetwork) are not updated here because they are collected // on-demand. diff --git a/pkg/ui/package.json b/pkg/ui/package.json index 59955fe666bc..48f130126610 100644 --- a/pkg/ui/package.json +++ b/pkg/ui/package.json @@ -15,7 +15,7 @@ "cypress:update-snapshots": "yarn cypress run --env updateSnapshots=true --spec 'cypress/integration/**/*.visual.spec.ts'" }, "dependencies": { - "@cockroachlabs/cluster-ui": "^0.2.28", + "@cockroachlabs/cluster-ui": "^0.2.30", "analytics-node": "^3.5.0", "antd": "^3.25.2", "babel-polyfill": "^6.26.0", diff --git a/pkg/ui/src/js/package.json b/pkg/ui/src/js/package.json index abb7d68cdc33..d1853e8e090e 100644 --- a/pkg/ui/src/js/package.json +++ b/pkg/ui/src/js/package.json @@ -1,6 +1,6 @@ { "name": "@cockroachlabs/crdb-protobuf-client", - "version": "0.0.9", + "version": "0.0.10", "description": "Javascript client for use by CockroachDB UI", "main": "./protos.js", "types": "./protos.d.ts", diff --git a/pkg/ui/src/util/appStats.spec.ts b/pkg/ui/src/util/appStats.spec.ts index 30328d1af83f..622ca276e858 100644 --- a/pkg/ui/src/util/appStats.spec.ts +++ b/pkg/ui/src/util/appStats.spec.ts @@ -179,6 +179,11 @@ function randomStats(sensitiveInfo?: ISensitiveInfo): StatementStatistics { rows_read: randomStat(), sensitive_info: sensitiveInfo || makeSensitiveInfo(null, null), exec_stats: randomExecStats(), + sql_type: "DDL", + last_exec_timestamp: { + seconds: Long.fromInt(1599670292), + nanos: 111613000, + }, }; } diff --git a/pkg/ui/src/util/appStats.ts b/pkg/ui/src/util/appStats.ts index cd8b8fe91e67..5f2fffcde41a 100644 --- a/pkg/ui/src/util/appStats.ts +++ b/pkg/ui/src/util/appStats.ts @@ -86,6 +86,10 @@ export function addStatementStats( legacy_last_err_redacted: "", exec_stats: addExecStats(a.exec_stats, b.exec_stats), sql_type: a.sql_type, + last_exec_timestamp: + a.last_exec_timestamp.seconds > b.last_exec_timestamp.seconds + ? a.last_exec_timestamp + : b.last_exec_timestamp, }; } diff --git a/pkg/ui/src/views/statements/statements.spec.tsx b/pkg/ui/src/views/statements/statements.spec.tsx index 86c0faeee7b8..90bcff56042f 100644 --- a/pkg/ui/src/views/statements/statements.spec.tsx +++ b/pkg/ui/src/views/statements/statements.spec.tsx @@ -498,6 +498,10 @@ function makeStats(): Required<StatementStatistics> { bytes_read: makeStat(), exec_stats: makeExecStats(), sql_type: "DDL", + last_exec_timestamp: { + seconds: Long.fromInt(1599670292), + nanos: 111613000, + }, }; } diff --git a/pkg/ui/yarn-vendor b/pkg/ui/yarn-vendor index c276e8479317..e2d632216ff9 160000 --- a/pkg/ui/yarn-vendor +++ b/pkg/ui/yarn-vendor @@ -1 +1 @@ -Subproject commit c276e8479317c8fe63cd6ba5f14cc77efb80fe59 +Subproject commit e2d632216ff9cf5f4d9136a4b3acf137fb914c80 diff --git a/pkg/ui/yarn.lock b/pkg/ui/yarn.lock index 3d9a12b587ef..6f681ca6deda 100644 --- a/pkg/ui/yarn.lock +++ b/pkg/ui/yarn.lock @@ -1717,7 +1717,7 @@ dependencies: regenerator-runtime "^0.13.2" -"@babel/runtime@^7.12.13": +"@babel/runtime@^7.12.0", "@babel/runtime@^7.12.13", "@babel/runtime@^7.8.7": version "7.13.10" resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.13.10.tgz#47d42a57b6095f4468da440388fdbad8bebf0d7d" integrity sha512-4QPkjJq6Ns3V/RgpEahRk+AGfL0eO6RHHtTWoNNr5mO49G6B5+X6d6THgWEAvTrznU5xYpbAlVKRYcsCgh/Akw== @@ -1813,13 +1813,13 @@ lodash "^4.17.13" to-fast-properties "^2.0.0" -"@cockroachlabs/cluster-ui@^0.2.28": - version "0.2.28" - resolved "https://registry.yarnpkg.com/@cockroachlabs/cluster-ui/-/cluster-ui-0.2.28.tgz#9d4a6467e4abdfb13911f7fc73935dae7a5b188b" - integrity sha512-Yog17595qSxgXeGG7ykpaiXs0JTEhfbdrzI+CNXSub20gi4g2qI8FQ0ksKiSI9k2HoNI2fWBj3c/xkJ9brgCHA== +"@cockroachlabs/cluster-ui@^0.2.30": + version "0.2.30" + resolved "https://registry.yarnpkg.com/@cockroachlabs/cluster-ui/-/cluster-ui-0.2.30.tgz#2a2f96e2be242a08f3c24c9867489c0f4b3d1bb9" + integrity sha512-xf/TEGyUYS/U7CLJxDCgPg4BLeH1oKvpVinAk1FmwULSohVKfPAApKSJ9/jjPuutQ2ESJRSZdAblxUegYWYB9A== dependencies: "@babel/runtime" "^7.12.13" - "@cockroachlabs/crdb-protobuf-client" "^0.0.8" + "@cockroachlabs/crdb-protobuf-client" "^0.0.10-beta.0" "@cockroachlabs/icons" "0.3.0" "@cockroachlabs/ui-components" "0.2.14-alpha.0" "@popperjs/core" "^2.4.0" @@ -1833,13 +1833,13 @@ npm-run-all "^4.1.5" react-helmet "^5.2.0" react-popper "^2.2.3" - react-select "^1.2.1" + react-select "^4.3.0" reselect "^4.0.0" -"@cockroachlabs/crdb-protobuf-client@^0.0.8": - version "0.0.8" - resolved "https://registry.yarnpkg.com/@cockroachlabs/crdb-protobuf-client/-/crdb-protobuf-client-0.0.8.tgz#064f44664091dd23e18167c67cf1713ff7845470" - integrity sha512-JP6miYNX8FvFSTSZ51etCfyxO/mi/TVVyeVIFsnN/CTlbHI6QZODHJIS1OSPvzfBnfXgFwcb1tQmMrK0L9xFEA== +"@cockroachlabs/crdb-protobuf-client@^0.0.10-beta.0": + version "0.0.10-beta.0" + resolved "https://registry.yarnpkg.com/@cockroachlabs/crdb-protobuf-client/-/crdb-protobuf-client-0.0.10-beta.0.tgz#b1bb2283379d8cf9d75adf780ff19374b4485e65" + integrity sha512-FQ29hZTM7CnunHWG88a5pgWXYGy/aAFrW0tn0E0t++QG7BTGkDVEuPdPcoMSCEUcKbFpzYc7AaY050JnU27rAA== "@cockroachlabs/eslint-config@^0.1.11": version "0.1.11" @@ -1873,6 +1873,17 @@ "@emotion/utils" "0.11.3" "@emotion/weak-memoize" "0.2.5" +"@emotion/cache@^11.0.0", "@emotion/cache@^11.1.3": + version "11.1.3" + resolved "https://registry.yarnpkg.com/@emotion/cache/-/cache-11.1.3.tgz#c7683a9484bcd38d5562f2b9947873cf66829afd" + integrity sha512-n4OWinUPJVaP6fXxWZD9OUeQ0lY7DvtmtSuqtRWT0Ofo/sBLCVSgb4/Oa0Q5eFxcwablRKjUXqXtNZVyEwCAuA== + dependencies: + "@emotion/memoize" "^0.7.4" + "@emotion/sheet" "^1.0.0" + "@emotion/utils" "^1.0.0" + "@emotion/weak-memoize" "^0.2.5" + stylis "^4.0.3" + "@emotion/core@^10.0.7": version "10.0.27" resolved "https://registry.yarnpkg.com/@emotion/core/-/core-10.0.27.tgz#7c3f78be681ab2273f3bf11ca3e2edc4a9dd1fdc" @@ -1899,6 +1910,11 @@ resolved "https://registry.yarnpkg.com/@emotion/hash/-/hash-0.7.4.tgz#f14932887422c9056b15a8d222a9074a7dfa2831" integrity sha512-fxfMSBMX3tlIbKUdtGKxqB1fyrH6gVrX39Gsv3y8lRYKUqlgDt3UMqQyGnR1bQMa2B8aGnhLZokZgg8vT0Le+A== +"@emotion/hash@^0.8.0": + version "0.8.0" + resolved "https://registry.yarnpkg.com/@emotion/hash/-/hash-0.8.0.tgz#bbbff68978fefdbe68ccb533bc8cbe1d1afb5413" + integrity sha512-kBJtf7PH6aWwZ6fka3zQ0p6SBYzx4fl1LoZXE2RrnYST9Xljm7WfKJrU4g/Xr3Beg72MLrp1AWNUmuYJTL7Cow== + "@emotion/is-prop-valid@0.8.6": version "0.8.6" resolved "https://registry.yarnpkg.com/@emotion/is-prop-valid/-/is-prop-valid-0.8.6.tgz#4757646f0a58e9dec614c47c838e7147d88c263c" @@ -1911,6 +1927,24 @@ resolved "https://registry.yarnpkg.com/@emotion/memoize/-/memoize-0.7.4.tgz#19bf0f5af19149111c40d98bb0cf82119f5d9eeb" integrity sha512-Ja/Vfqe3HpuzRsG1oBtWTHk2PGZ7GR+2Vz5iYGelAw8dx32K0y7PjVuxK6z1nMpZOqAFsRUPCkK1YjJ56qJlgw== +"@emotion/memoize@^0.7.4": + version "0.7.5" + resolved "https://registry.yarnpkg.com/@emotion/memoize/-/memoize-0.7.5.tgz#2c40f81449a4e554e9fc6396910ed4843ec2be50" + integrity sha512-igX9a37DR2ZPGYtV6suZ6whr8pTFtyHL3K/oLUotxpSVO2ASaprmAe2Dkq7tBo7CRY7MMDrAa9nuQP9/YG8FxQ== + +"@emotion/react@^11.1.1": + version "11.1.5" + resolved "https://registry.yarnpkg.com/@emotion/react/-/react-11.1.5.tgz#15e78f9822894cdc296e6f4e0688bac8120dfe66" + integrity sha512-xfnZ9NJEv9SU9K2sxXM06lzjK245xSeHRpUh67eARBm3PBHjjKIZlfWZ7UQvD0Obvw6ZKjlC79uHrlzFYpOB/Q== + dependencies: + "@babel/runtime" "^7.7.2" + "@emotion/cache" "^11.1.3" + "@emotion/serialize" "^1.0.0" + "@emotion/sheet" "^1.0.1" + "@emotion/utils" "^1.0.0" + "@emotion/weak-memoize" "^0.2.5" + hoist-non-react-statics "^3.3.1" + "@emotion/serialize@^0.11.15": version "0.11.15" resolved "https://registry.yarnpkg.com/@emotion/serialize/-/serialize-0.11.15.tgz#9a0f5873fb458d87d4f23e034413c12ed60a705a" @@ -1922,11 +1956,27 @@ "@emotion/utils" "0.11.3" csstype "^2.5.7" +"@emotion/serialize@^1.0.0": + version "1.0.2" + resolved "https://registry.yarnpkg.com/@emotion/serialize/-/serialize-1.0.2.tgz#77cb21a0571c9f68eb66087754a65fa97bfcd965" + integrity sha512-95MgNJ9+/ajxU7QIAruiOAdYNjxZX7G2mhgrtDWswA21VviYIRP1R5QilZ/bDY42xiKsaktP4egJb3QdYQZi1A== + dependencies: + "@emotion/hash" "^0.8.0" + "@emotion/memoize" "^0.7.4" + "@emotion/unitless" "^0.7.5" + "@emotion/utils" "^1.0.0" + csstype "^3.0.2" + "@emotion/sheet@0.9.4": version "0.9.4" resolved "https://registry.yarnpkg.com/@emotion/sheet/-/sheet-0.9.4.tgz#894374bea39ec30f489bbfc3438192b9774d32e5" integrity sha512-zM9PFmgVSqBw4zL101Q0HrBVTGmpAxFZH/pYx/cjJT5advXguvcgjHFTCaIO3enL/xr89vK2bh0Mfyj9aa0ANA== +"@emotion/sheet@^1.0.0", "@emotion/sheet@^1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@emotion/sheet/-/sheet-1.0.1.tgz#245f54abb02dfd82326e28689f34c27aa9b2a698" + integrity sha512-GbIvVMe4U+Zc+929N1V7nW6YYJtidj31lidSmdYcWozwoBIObXBnaJkKNDjZrLm9Nc0BR+ZyHNaRZxqNZbof5g== + "@emotion/styled-base@^10.0.27": version "10.0.27" resolved "https://registry.yarnpkg.com/@emotion/styled-base/-/styled-base-10.0.27.tgz#d9efa307ae4e938fcc4d0596b40b7e8bc10f7c7c" @@ -1950,7 +2000,7 @@ resolved "https://registry.yarnpkg.com/@emotion/stylis/-/stylis-0.8.5.tgz#deacb389bd6ee77d1e7fcaccce9e16c5c7e78e04" integrity sha512-h6KtPihKFn3T9fuIrwvXXUOwlx3rfUvfZIcP5a6rh8Y7zjE3O06hT5Ss4S/YI1AYhuZ1kjaE/5EaOOI2NqSylQ== -"@emotion/unitless@0.7.5": +"@emotion/unitless@0.7.5", "@emotion/unitless@^0.7.5": version "0.7.5" resolved "https://registry.yarnpkg.com/@emotion/unitless/-/unitless-0.7.5.tgz#77211291c1900a700b8a78cfafda3160d76949ed" integrity sha512-OWORNpfjMsSSUBVrRBVGECkhWcULOAJz9ZW8uK9qgxD+87M7jHRcvh/A96XXNhXTLmKcoYSQtBEX7lHMO7YRwg== @@ -1960,7 +2010,12 @@ resolved "https://registry.yarnpkg.com/@emotion/utils/-/utils-0.11.3.tgz#a759863867befa7e583400d322652a3f44820924" integrity sha512-0o4l6pZC+hI88+bzuaX/6BgOvQVhbt2PfmxauVaYOGgbsAw14wdKyvMCZXnsnsHys94iadcF+RG/wZyx6+ZZBw== -"@emotion/weak-memoize@0.2.5": +"@emotion/utils@^1.0.0": + version "1.0.0" + resolved "https://registry.yarnpkg.com/@emotion/utils/-/utils-1.0.0.tgz#abe06a83160b10570816c913990245813a2fd6af" + integrity sha512-mQC2b3XLDs6QCW+pDQDiyO/EdGZYOygE8s5N5rrzjSI4M3IejPE/JPndCBwRT9z982aqQNi6beWs1UeayrQxxA== + +"@emotion/weak-memoize@0.2.5", "@emotion/weak-memoize@^0.2.5": version "0.2.5" resolved "https://registry.yarnpkg.com/@emotion/weak-memoize/-/weak-memoize-0.2.5.tgz#8eed982e2ee6f7f4e44c253e12962980791efd46" integrity sha512-6U71C2Wp7r5XtFtQzYrW5iKFT67OixrSxjI4MptCHzdSVlgabczzqLe0ZSgnub/5Kp4hSbpDB1tMytZY9pwxxA== @@ -5298,6 +5353,11 @@ csstype@^2.2.0, csstype@^2.5.7: resolved "https://registry.yarnpkg.com/csstype/-/csstype-2.6.8.tgz#0fb6fc2417ffd2816a418c9336da74d7f07db431" integrity sha512-msVS9qTuMT5zwAGCVm4mxfrZ18BNc6Csd0oJAtiFMZ1FAx1CCvy2+5MDmYoix63LM/6NDbNtodCiGYGmFgO0dA== +csstype@^3.0.2: + version "3.0.7" + resolved "https://registry.yarnpkg.com/csstype/-/csstype-3.0.7.tgz#2a5fb75e1015e84dd15692f71e89a1450290950b" + integrity sha512-KxnUB0ZMlnUWCsx2Z8MUsr6qV6ja1w9ArPErJaJaF8a5SOWoHLIszeCTKGRGRgtLgYrs1E8CHkNSP1VZTTPc9g== + custom-event@~1.0.0: version "1.0.1" resolved "https://registry.yarnpkg.com/custom-event/-/custom-event-1.0.1.tgz#5d02a46850adf1b4a317946a3928fccb5bfd0425" @@ -5697,6 +5757,14 @@ dom-converter@^0.2: dependencies: utila "~0.4" +dom-helpers@^5.0.1: + version "5.2.0" + resolved "https://registry.yarnpkg.com/dom-helpers/-/dom-helpers-5.2.0.tgz#57fd054c5f8f34c52a3eeffdb7e7e93cd357d95b" + integrity sha512-Ru5o9+V8CpunKnz5LGgWXkmrH/20cGKwcHwS4m73zIvs54CN9epEmT/HLqFJW3kXpakAFkEdzgy1hzlJe3E4OQ== + dependencies: + "@babel/runtime" "^7.8.7" + csstype "^3.0.2" + dom-matches@>=1.0.1: version "2.0.0" resolved "https://registry.yarnpkg.com/dom-matches/-/dom-matches-2.0.0.tgz#d2728b416a87533980eb089b848d253cf23a758c" @@ -7546,6 +7614,13 @@ hoist-non-react-statics@^3.1.0, hoist-non-react-statics@^3.3.0: dependencies: react-is "^16.7.0" +hoist-non-react-statics@^3.3.1: + version "3.3.2" + resolved "https://registry.yarnpkg.com/hoist-non-react-statics/-/hoist-non-react-statics-3.3.2.tgz#ece0acaf71d62c2969c2ec59feff42a4b1a85b45" + integrity sha512-/gGivxi8JPKWNm/W0jSmzcMPpfpPLc3dY/6GxhX2hQ9iGj3aDfklV4ET7NjKpSinLpJ5vafa9iiGIEZg10SfBw== + dependencies: + react-is "^16.7.0" + homedir-polyfill@^1.0.1: version "1.0.3" resolved "https://registry.yarnpkg.com/homedir-polyfill/-/homedir-polyfill-1.0.3.tgz#743298cef4e5af3e194161fbadcc2151d3a058e8" @@ -9015,6 +9090,11 @@ mem@^4.0.0: mimic-fn "^2.0.0" p-is-promise "^2.0.0" +memoize-one@^5.0.0: + version "5.1.1" + resolved "https://registry.yarnpkg.com/memoize-one/-/memoize-one-5.1.1.tgz#047b6e3199b508eaec03504de71229b8eb1d75c0" + integrity sha512-HKeeBpWvqiVJD57ZUAsJNm71eHTykffzcLZVYWiVfQeI1rJtuEaS7hQiEpWfVVk18donPwJEcFKIkCmPJNOhHA== + memoizerific@^1.11.3: version "1.11.3" resolved "https://registry.yarnpkg.com/memoizerific/-/memoizerific-1.11.3.tgz#7c87a4646444c32d75438570905f2dbd1b1a805a" @@ -11411,6 +11491,13 @@ react-input-autosize@^2.1.2: dependencies: prop-types "^15.5.8" +react-input-autosize@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/react-input-autosize/-/react-input-autosize-3.0.0.tgz#6b5898c790d4478d69420b55441fcc31d5c50a85" + integrity sha512-nL9uS7jEs/zu8sqwFE5MAPx6pPkNAriACQ2rGLlqmKr2sPGtN7TXTyDdQt4lbNXVx7Uzadb40x8qotIuru6Rhg== + dependencies: + prop-types "^15.5.8" + react-inspector@^2.3.0, react-inspector@^2.3.1: version "2.3.1" resolved "https://registry.yarnpkg.com/react-inspector/-/react-inspector-2.3.1.tgz#f0eb7f520669b545b441af9d38ec6d706e5f649c" @@ -11556,6 +11643,19 @@ react-select@^1.2.1: prop-types "^15.5.8" react-input-autosize "^2.1.2" +react-select@^4.3.0: + version "4.3.0" + resolved "https://registry.yarnpkg.com/react-select/-/react-select-4.3.0.tgz#6bde634ae7a378b49f3833c85c126f533483fa2e" + integrity sha512-SBPD1a3TJqE9zoI/jfOLCAoLr/neluaeokjOixr3zZ1vHezkom8K0A9J4QG9IWDqIDE9K/Mv+0y1GjidC2PDtQ== + dependencies: + "@babel/runtime" "^7.12.0" + "@emotion/cache" "^11.0.0" + "@emotion/react" "^11.1.1" + memoize-one "^5.0.0" + prop-types "^15.6.0" + react-input-autosize "^3.0.0" + react-transition-group "^4.3.0" + react-side-effect@^1.1.0: version "1.1.5" resolved "https://registry.yarnpkg.com/react-side-effect/-/react-side-effect-1.1.5.tgz#f26059e50ed9c626d91d661b9f3c8bb38cd0ff2d" @@ -11602,6 +11702,16 @@ react-textarea-autosize@^7.0.4: "@babel/runtime" "^7.1.2" prop-types "^15.6.0" +react-transition-group@^4.3.0: + version "4.4.1" + resolved "https://registry.yarnpkg.com/react-transition-group/-/react-transition-group-4.4.1.tgz#63868f9325a38ea5ee9535d828327f85773345c9" + integrity sha512-Djqr7OQ2aPUiYurhPalTrVy9ddmFCCzwhqQmtN+J3+3DzLO209Fdr70QrN8Z3DsglWql6iY1lDWAfpFiBtuKGw== + dependencies: + "@babel/runtime" "^7.5.5" + dom-helpers "^5.0.1" + loose-envify "^1.4.0" + prop-types "^15.6.2" + react@^0.14.0: version "0.14.9" resolved "https://registry.yarnpkg.com/react/-/react-0.14.9.tgz#9110a6497c49d44ba1c0edd317aec29c2e0d91d1" @@ -13079,6 +13189,11 @@ stylint@^1.5.9: user-home "2.0.0" yargs "4.7.1" +stylis@^4.0.3: + version "4.0.10" + resolved "https://registry.yarnpkg.com/stylis/-/stylis-4.0.10.tgz#446512d1097197ab3f02fb3c258358c3f7a14240" + integrity sha512-m3k+dk7QeJw660eIKRRn3xPF6uuvHs/FFzjX3HQ5ove0qYsiygoAhwn5a3IYKaZPo5LrYD0rfVmtv1gNY1uYwg== + stylus-loader@^3.0.2: version "3.0.2" resolved "https://registry.yarnpkg.com/stylus-loader/-/stylus-loader-3.0.2.tgz#27a706420b05a38e038e7cacb153578d450513c6" From 76ad31bbbed86ad2cfa21d792090ab2dfe4ae6fb Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy <yevgeniy@cockroachlabs.com> Date: Mon, 19 Apr 2021 18:59:45 -0400 Subject: [PATCH 31/37] changefeedccl: Correctly account for memory when closing gzip compressed files. Correctly account for used memory when closing gzip compressed cloud storage file. Release Notes: Bug fix; correctly account for used memory when closing compressed files. --- pkg/ccl/changefeedccl/sink_cloudstorage.go | 12 ++++-- .../changefeedccl/sink_cloudstorage_test.go | 40 +++++++++++++++++++ 2 files changed, 49 insertions(+), 3 deletions(-) diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage.go b/pkg/ccl/changefeedccl/sink_cloudstorage.go index d37b6576359f..7bed4c9f2d24 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage.go @@ -512,8 +512,15 @@ func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSink return nil } - // If the file is written via compression codec, close the codec to ensure it - // has flushed to the underlying buffer. + // Release memory allocated for this file. Note, closing codec + // below may as well write more data to our buffer (and that may cause buffer + // to grow due to reallocation). But we don't account for that additional memory + // because a) we don't know if buffer will be resized (nor by how much), and + // b) if we're out of memory we'd OOMed when trying to close codec anyway. + defer func(delta int) { + s.mem.Shrink(ctx, int64(delta)) + }(file.buf.Cap()) + if file.codec != nil { if err := file.codec.Close(); err != nil { return err @@ -538,7 +545,6 @@ func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSink if err := s.es.WriteFile(ctx, filepath.Join(s.dataFilePartition, filename), bytes.NewReader(file.buf.Bytes())); err != nil { return err } - s.mem.Shrink(ctx, int64(file.buf.Cap())) return nil } diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go index 26e20febd75b..210ef5726d2b 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/span" "github.com/stretchr/testify/require" ) @@ -575,4 +576,43 @@ func TestCloudStorageSink(t *testing.T) { } require.Regexp(t, "memory budget exceeded", err) }) + t.Run(`memory-accounting`, func(t *testing.T) { + before := opts[changefeedbase.OptCompression] + // Compression codecs include buffering that interferes with other tests, + // e.g. the bucketing test that configures very small flush sizes. + defer func() { + opts[changefeedbase.OptCompression] = before + }() + + // A bit of magic constant: we're using bytes.Buffer internally, which + // allocates "small" buffer (64 bytes) initially. We will try to target + // our file size to be less than that value; but we will write + // larger amount of data (thus, hopefully causing multiple bytes.Buffer + // reallocs, plus a file flush). + const targetFileSize = 63 + + rnd, _ := randutil.NewPseudoRand() + for _, compression := range []string{"", "gzip"} { + opts[changefeedbase.OptCompression] = compression + t.Run("compress="+compression, func(t *testing.T) { + t1 := makeTopic(`t1`) + testSpan := roachpb.Span{Key: []byte("a"), EndKey: []byte("b")} + sf := span.MakeFrontier(testSpan) + timestampOracle := &changeAggregatorLowerBoundOracle{sf: sf} + sinkDir := `memory-accounting` + s, err := makeCloudStorageSink( + ctx, `nodelocal://0/`+sinkDir, 1, targetFileSize, + settings, opts, timestampOracle, externalStorageFromURI, user, memAcc, + ) + require.NoError(t, err) + defer func() { require.NoError(t, s.Close()) }() + s.(*cloudStorageSink).sinkID = 7 // Force a deterministic sinkID. + + data := randutil.RandBytes(rnd, 1+rnd.Intn(targetFileSize)) + require.NoError(t, s.EmitRow(ctx, t1, noKey, data, ts(0))) + + require.NoError(t, s.Flush(ctx)) + }) + } + }) } From de7047c379be80df08ca740bd77d9f150002ec90 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten <nvanbenschoten@gmail.com> Date: Mon, 19 Apr 2021 20:34:15 -0400 Subject: [PATCH 32/37] roachtest/tpcc: pass correct warehouse count in multi-region tpccbench This commit fixes an unfortunate bug where we were fooling ourselves into thinking that we supported 3x the number of warehouses in this configuration than we actually do. This is because we were dividing the number of warehouses by the number of regions before passing the count to the `--active-warehouses` flag. However, the load generator already takes into account the number of partitions to split the load across when provided with the `--partitions` flag. So for instance, if tpccbench's line searcher was instructing us to test 3000 warehouses, we would divide this by 3 to arrive at 1000 warehouses, then we would pass this to each workload (`--partitions=3 --active-warehouses=1000`), which would run over 333 warehouses. --- pkg/cmd/roachtest/tpcc.go | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/pkg/cmd/roachtest/tpcc.go b/pkg/cmd/roachtest/tpcc.go index 2b46261c1b55..387cac670c17 100644 --- a/pkg/cmd/roachtest/tpcc.go +++ b/pkg/cmd/roachtest/tpcc.go @@ -436,10 +436,8 @@ func registerTPCC(r *testRegistry) { Distribution: multiRegion, LoadConfig: multiLoadgen, - LoadWarehouses: 5000, - EstimatedMax: 3000, - - MinVersion: "v20.1.0", + LoadWarehouses: 3000, + EstimatedMax: 2000, }) registerTPCCBenchSpec(r, tpccBenchSpec{ Nodes: 9, @@ -449,8 +447,6 @@ func registerTPCC(r *testRegistry) { LoadWarehouses: 2000, EstimatedMax: 900, - - MinVersion: "v20.1.0", }) } @@ -883,7 +879,6 @@ func runTPCCBench(ctx context.Context, t *test, c *cluster, b tpccBenchSpec) { } extraFlags := "" - activeWarehouses := warehouses switch b.LoadConfig { case singleLoadgen: // Nothing. @@ -892,7 +887,6 @@ func runTPCCBench(ctx context.Context, t *test, c *cluster, b tpccBenchSpec) { case multiLoadgen: extraFlags = fmt.Sprintf(` --partitions=%d --partition-affinity=%d`, b.partitions(), groupIdx) - activeWarehouses = warehouses / numLoadGroups default: // Abort the whole test. t.Fatalf("unimplemented LoadConfig %v", b.LoadConfig) @@ -903,10 +897,10 @@ func runTPCCBench(ctx context.Context, t *test, c *cluster, b tpccBenchSpec) { extraFlags += " --method=simple" } t.Status(fmt.Sprintf("running benchmark, warehouses=%d", warehouses)) - histogramsPath := fmt.Sprintf("%s/warehouses=%d/stats.json", perfArtifactsDir, activeWarehouses) + histogramsPath := fmt.Sprintf("%s/warehouses=%d/stats.json", perfArtifactsDir, warehouses) cmd := fmt.Sprintf("./cockroach workload run tpcc --warehouses=%d --active-warehouses=%d "+ "--tolerate-errors --ramp=%s --duration=%s%s --histograms=%s {pgurl%s}", - b.LoadWarehouses, activeWarehouses, rampDur, + b.LoadWarehouses, warehouses, rampDur, loadDur, extraFlags, histogramsPath, sqlGateways) err := c.RunE(ctx, group.loadNodes, cmd) loadDone <- timeutil.Now() @@ -930,7 +924,7 @@ func runTPCCBench(ctx context.Context, t *test, c *cluster, b tpccBenchSpec) { // overload but something that deserves failing the whole test. t.Fatal(err) } - result := tpcc.NewResultWithSnapshots(activeWarehouses, 0, snapshots) + result := tpcc.NewResultWithSnapshots(warehouses, 0, snapshots) resultChan <- result return nil }) From 79922e4fb5774937f66c40ffff4541848cb94f4a Mon Sep 17 00:00:00 2001 From: irfan sharif <irfanmahmoudsharif@gmail.com> Date: Mon, 19 Apr 2021 15:04:35 -0400 Subject: [PATCH 33/37] stop: break deadlock between Stopper.mu and Replica.mu Fixes #63761. As of #61279, it was possible for us to deadlock due to inconsistent lock orderings between Stopper.mu and Replica.mu. We were previously holding onto Stopper.mu while executing all closers, including those that may acquire other locks. Because closers can be defined anywhere (and may consequently grab any in-scope lock), we should order Stopper.mu to come after all other locks in the system. The closer added in #61279 iterated over all non-destroyed replicas, locking Replica.mu to check for the replica's destroy status (in Store.VisitReplicas). This deadlocked with the lease acquisition code path that first grabs an exclusive lock over Replica.mu (see InitOrJoinRequest), and uses the stopper to kick off an async task acquiring the lease. The stopper internally locks Stopper.mu to check whether or not it was already stopped. Release note: None --- pkg/util/stop/stopper.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/util/stop/stopper.go b/pkg/util/stop/stopper.go index 5f0816d703a0..844f19f868f9 100644 --- a/pkg/util/stop/stopper.go +++ b/pkg/util/stop/stopper.go @@ -472,8 +472,9 @@ func (s *Stopper) Stop(ctx context.Context) { s.Quiesce(ctx) - s.mu.Lock() - defer s.mu.Unlock() + // Run the closers without holding s.mu. There's no concern around new + // closers being added; we've marked this stopper as `stopping` above, so + // any attempts to do so will be refused. for _, c := range s.mu.closers { c.Close() } From a73d871218ccc58a1148c8bdf6c93d1a93bfd534 Mon Sep 17 00:00:00 2001 From: Tommy Reilly <treilly@cockroachlabs.com> Date: Fri, 9 Apr 2021 11:31:19 -0400 Subject: [PATCH 34/37] sql: make locality optimized search optimizations work in row engine Implement an unordered depth first input synchronizer that exhausts each input source in turn. Update tablereaders to only call StartScan when Next is called and not in Start. This is required to match the behavior of the vector engine. Enable the regional_by_row tests to be run under two configs, one with vec engine (as before) and a new one for vector off. Release note (sql): Enabled locality optimized search in row engine --- .../logic_test/multi_region_import_export | 2 +- .../testdata/logic_test/regional_by_row | 60 ++--- .../logic_test/regional_by_row_rename_column | 2 +- pkg/sql/distsql_physical_planner.go | 3 - pkg/sql/execinfrapb/data.go | 6 +- pkg/sql/execinfrapb/data.pb.go | 2 +- pkg/sql/execinfrapb/data.proto | 2 +- pkg/sql/logictest/logic.go | 121 +++++----- pkg/sql/rowexec/tablereader.go | 20 +- pkg/sql/rowflow/input_sync.go | 227 ++++++++++++------ pkg/sql/rowflow/input_sync_test.go | 4 +- pkg/sql/rowflow/row_based_flow.go | 17 +- 12 files changed, 270 insertions(+), 196 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_import_export b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_import_export index b806c9416af6..59c39d9747b8 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_import_export +++ b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_import_export @@ -1,4 +1,4 @@ -# LogicTest: multiregion-9node-3region-3azs +# LogicTest: multiregion-9node-3region-3azs multiregion-9node-3region-3azs-vec-off query TTTT colnames SHOW REGIONS diff --git a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row index c5aafb50efc8..8fed70a5f97a 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row +++ b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row @@ -1,4 +1,4 @@ -# LogicTest: multiregion-9node-3region-3azs +# LogicTest: multiregion-9node-3region-3azs multiregion-9node-3region-3azs-vec-off statement ok CREATE DATABASE multi_region_test_db PRIMARY REGION "ca-central-1" REGIONS "ap-southeast-2", "us-east-1" SURVIVE REGION FAILURE @@ -674,10 +674,8 @@ SET locality_optimized_partitioned_index_scan = false # Query with locality optimized search disabled. query T -EXPLAIN SELECT * FROM regional_by_row_table WHERE pk = 1 +SELECT * FROM [EXPLAIN SELECT * FROM regional_by_row_table WHERE pk = 1] OFFSET 2 ---- -distribution: local -vectorized: true · • scan missing stats @@ -703,10 +701,8 @@ SET locality_optimized_partitioned_index_scan = true # Same query with locality optimized search enabled. query T -EXPLAIN (DISTSQL) SELECT * FROM regional_by_row_table WHERE pk = 1 +SELECT * FROM [EXPLAIN (DISTSQL) SELECT * FROM regional_by_row_table WHERE pk = 1] OFFSET 2 ---- -distribution: local -vectorized: true · • union all │ limit: 1 @@ -721,7 +717,7 @@ vectorized: true table: regional_by_row_table@primary spans: [/'ca-central-1'/1 - /'ca-central-1'/1] [/'us-east-1'/1 - /'us-east-1'/1] · -Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJykkVFr2zAUhd_3K8R92oaGLWeDIRh4bC41uElqB1qoTVCsS2rqSK4k04Tg_15iP6QpSSHp4z1X5-g70hbscw0covtp8jcek6__42yW3SbfSBYl0b8Z-U6u0skNMbistBL1fLGZG_0yd2JRI7m7jtKINE_kD2FAQWmJY7FCC_wBGBQUGqNLtFabnbTtD8RyDdynUKmmdTu5oFBqg8C34CpXI3CY7dJTFBKN5wMFiU5UdR97FCRsTLUSZgMUskYoy4mXQ56vf_s5eMzziVCSMKLdIxqgMGkdJyGjYUDDEQ1_0vAXFB0F3bo9k3ViicBZRy_jZpdyhwPzmZzBSc49Xqu0kWhQHqAV3ZEmY_1DN15w2CGpVpUj7CSDf85bpWgbrSy-YzmVXFBAucShkNWtKXFqdNlfM4yT3tcLEq0btsEwxKpf9Z_51sw-Yw4-NI8OzH5XdF9eAwAA__86rhxz +Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJykkdGL1DAQxt_9K4Z5Uom06SlIQKhoFwt1e7YFhWtZcs2wFrtJTVK8Y-n_Lts-nJVb4dbH-Wa-md-XHNH97FFgmWTJhwpewqbIP8NN8u06e59u4fnHtKzKL9kLWA9Y2ndGy353e7-z5tfOy9ue4OunpEhg-AHvgDeQbzZlUkGEDLVRtJUHcihukGPDcLCmJeeMPUnHeSBVdyhChp0eRn-SG4atsYTiiL7zPaHA6nSnIKnIBiEyVORl189rH0WKB9sdpL1HhuUgtRMQ1FjXd2_DGgMehCC1Ag7GfyeLDPPRC4g5iyMWX7H4NYvfYDMxNKN_YHJe7gkFn9hl3PxS7nhhfiJndJbzAW_UxiqypFZozfRIkq15ZYYgWmfIukPngZ9lCJ_yVgW5wWhHf7Gc29wwJLWnJZAzo23p2pp2PrOU-eybBUXOL91oKVI9t-bP_NPM_8cc_dN8tTKHUzM9-x0AAP__wyAi4g== statement ok SET tracing = on,kv,results; SELECT * FROM regional_by_row_table WHERE pk = 1; SET tracing = off @@ -757,10 +753,8 @@ output row: [10 10 11 12 NULL] # scanned in the first child of the limited union all. query T nodeidx=3 USE multi_region_test_db; SET locality_optimized_partitioned_index_scan = true; -EXPLAIN SELECT * FROM regional_by_row_table WHERE pk = 1 +SELECT * FROM [EXPLAIN SELECT * FROM regional_by_row_table WHERE pk = 1] OFFSET 2 ---- -distribution: local -vectorized: true · • union all │ limit: 1 @@ -804,10 +798,8 @@ SET locality_optimized_partitioned_index_scan = false # Query with locality optimized search disabled. query T -EXPLAIN SELECT * FROM child WHERE NOT EXISTS (SELECT * FROM parent WHERE p_id = c_p_id) AND c_id = 10 +SELECT * FROM [EXPLAIN SELECT * FROM child WHERE NOT EXISTS (SELECT * FROM parent WHERE p_id = c_p_id) AND c_id = 10] OFFSET 2 ---- -distribution: full -vectorized: true · • lookup join (anti) │ table: parent@primary @@ -839,10 +831,8 @@ SET locality_optimized_partitioned_index_scan = true # Same query with locality optimized search enabled. query T -EXPLAIN (DISTSQL) SELECT * FROM child WHERE NOT EXISTS (SELECT * FROM parent WHERE p_id = c_p_id) AND c_id = 10 +SELECT * FROM [EXPLAIN (DISTSQL) SELECT * FROM child WHERE NOT EXISTS (SELECT * FROM parent WHERE p_id = c_p_id) AND c_id = 10] OFFSET 2 ---- -distribution: local -vectorized: true · • lookup join (anti) │ table: parent@primary @@ -867,7 +857,7 @@ vectorized: true table: child@primary spans: [/'ca-central-1'/10 - /'ca-central-1'/10] [/'us-east-1'/10 - /'us-east-1'/10] · -Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJy0k2Fr2zwQx98_n-K4N0keNCI7LhRBwGF1mYvndIlhhcUUzzpab4nkyTKkhHz3ETvQOCwh7dg763R_3-9_uttg9WuJAoOH-2gSxtC_CefJ_Es0gHkQBR8T-B9uZ9PPkD8XSwlfPwWzAPrxNIHgYZcI_W5amRlSdp9XPhYSxpA_7j4GA5jEN9DP26DDB8hQaUlxtqIKxTd0MGVYGp1TVWmzC22ahFCuUXCGhSpruwunDHNtCMUGbWGXhAKT7PuSZpRJMkOODCXZrFg2v23I_dIUq8y8IMN5malKwHCBi8X6mi9w6PAhh0xJcEDbZzLIcFpbAb7DfBfTLUNd29falc2eCIWzZe_jcy7l8_dsl_G4J3leMWqljSRDsoOQbv9AHOsPuhy6XdaoWBUWnJMM_C09udOF2rdk1C2TvJQkIApuE5jESQh30zBGhu1wHbQq0vpnXcIPXSjQSkDfH8EYfHc_ar4HY1j3PN4TQvgO5_zKG1zWzNE7jXj_yshuNde960MrDNa9vOPtQnPeW8zNqCq1quhoYk69f8qQ5BO1Y1fp2uR0b3TelGmP00bXBCRVtr1120OomqtmtQ7Fzt-I3bPiUUfMj8Wjs2LvvNg7K746Eqfb_34HAAD__-igvDk= +Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJy0k9GL2kAQxt_7VwzzopYtbmIOjgUh0os0R5pcNdCDMxxpMtyl1d10swEP8X8vJsIZqeJd6Zs7-33O75vMbrD6vUSBcy_wPsfwEaaz6Cs8ePd3wcQPoX_jz-P5t2AAXUH2XCxz-P7Fm3nQD6MYvPudEPpdWZlqkmavKx-LHMaQPe5-DAYwCW-gn7VFiw8SiKbTuReDjQylyilMV1SheEALE4alVhlVldK70qYR-PkaBWdYyLI2u3LCMFOaUGzQFGZJKDBOfyxpRmlOesiRYU4mLZbN3zYZ3FIXq1S_IMN5mcpKwHCBi8X6mi9waPEhh1TmYIEyz6SRYVQbAa7FXBuTLUNVm9felUmfCIW1Ze_jsy7lc_dsl_HYJ3leMWqpdE6a8g5Csv0Lcag-qXJod1mDYlUYsE4y8LfM5FYVcj-SUbdN_FKSgMCbxjAJYx9uIz9Ehu2aHYwqUOpXXcJPVUhQUkDfHcEYXHu_dK4DY1j3HN4TQrgW5_zKGVw2zNE7gzj_K8juka5714dRGKx7WSfbheGct4SbUVUqWdHRxpz6_glDyp-oXbtK1TqjO62ypk17jBpfU8ipMu2t3R582Vw1T-vQbP2L2T5rHnXM_Ng8Omt2zpuds-arI3Oy_fAnAAD__xkswqg= statement ok SET tracing = on,kv,results; SELECT * FROM child WHERE NOT EXISTS (SELECT * FROM parent WHERE p_id = c_p_id) AND c_id = 10; SET tracing = off @@ -903,10 +893,8 @@ Scan /Table/74/1/"\x80"/20/0, /Table/74/1/"\xc0"/20/0 fetched: /parent/primary/'ca-central-1'/20 -> NULL query T -EXPLAIN INSERT INTO child VALUES (1, 1) +SELECT * FROM [EXPLAIN INSERT INTO child VALUES (1, 1)] OFFSET 2 ---- -distribution: local -vectorized: true · • root │ @@ -956,10 +944,8 @@ vectorized: true label: buffer 1 query T -EXPLAIN UPSERT INTO child VALUES (1, 1) +SELECT * FROM [EXPLAIN UPSERT INTO child VALUES (1, 1)] OFFSET 2 ---- -distribution: local -vectorized: true · • root │ @@ -1009,10 +995,8 @@ vectorized: true # We don't yet support locality optimized search for semi join. query T -EXPLAIN DELETE FROM parent WHERE p_id = 1 +SELECT * FROM [EXPLAIN DELETE FROM parent WHERE p_id = 1] OFFSET 2 ---- -distribution: local -vectorized: true · • root │ @@ -1061,10 +1045,8 @@ ALTER TABLE regional_by_row_table ADD CONSTRAINT unique_b_a UNIQUE(b, a) # We should plan uniqueness checks for all unique indexes in # REGIONAL BY ROW tables. query T -EXPLAIN INSERT INTO regional_by_row_table (pk, pk2, a, b) VALUES (1, 1, 1, 1) +SELECT * FROM [EXPLAIN INSERT INTO regional_by_row_table (pk, pk2, a, b) VALUES (1, 1, 1, 1)] OFFSET 2 ---- -distribution: local -vectorized: true · • root │ @@ -1162,10 +1144,8 @@ INSERT INTO regional_by_row_table (crdb_region, pk, pk2, a, b) VALUES ('us-east- # The conflict columns in an upsert should only include the primary key, # not the region column. query T -EXPLAIN UPSERT INTO regional_by_row_table (crdb_region, pk, pk2, a, b) VALUES ('us-east-1', 2, 3, 2, 3) +SELECT * FROM [EXPLAIN UPSERT INTO regional_by_row_table (crdb_region, pk, pk2, a, b) VALUES ('us-east-1', 2, 3, 2, 3)] OFFSET 2 ---- -distribution: local -vectorized: true · • root │ @@ -1253,11 +1233,9 @@ vectorized: true label: buffer 1 query T -EXPLAIN UPSERT INTO regional_by_row_table (crdb_region, pk, pk2, a, b) -VALUES ('us-east-1', 23, 24, 25, 26), ('ca-central-1', 30, 30, 31, 32) +SELECT * FROM [EXPLAIN UPSERT INTO regional_by_row_table (crdb_region, pk, pk2, a, b) +VALUES ('us-east-1', 23, 24, 25, 26), ('ca-central-1', 30, 30, 31, 32)] OFFSET 2 ---- -distribution: local -vectorized: true · • root │ @@ -1600,10 +1578,8 @@ ALTER TABLE regional_by_row_table_as DROP COLUMN crdb_region_col # We do not need uniqueness checks on pk since uniqueness can be inferred # through the functional dependency between pk and the computed region column. query T -EXPLAIN INSERT INTO regional_by_row_table_as (pk, a, b) VALUES (1, 1, 1) +SELECT * FROM [EXPLAIN INSERT INTO regional_by_row_table_as (pk, a, b) VALUES (1, 1, 1)] OFFSET 2 ---- -distribution: local -vectorized: true · • root │ @@ -1646,10 +1622,8 @@ INSERT INTO regional_by_row_table_as (pk, a, b) VALUES (2, 1, 1) # Verify that we plan single-region scans for REGIONAL BY ROW tables with a computed region. query T -EXPLAIN SELECT * FROM regional_by_row_table_as WHERE pk = 10 +SELECT * FROM [EXPLAIN SELECT * FROM regional_by_row_table_as WHERE pk = 10] OFFSET 2 ---- -distribution: local -vectorized: true · • scan missing stats diff --git a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_rename_column b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_rename_column index 12cbfe827aa8..2b92faed3e7d 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_rename_column +++ b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_rename_column @@ -1,4 +1,4 @@ -# LogicTest: multiregion-9node-3region-3azs +# LogicTest: multiregion-9node-3region-3azs multiregion-9node-3region-3azs-vec-off # This test seems to flake (#60717). # It seems to hit the codepath: diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 8813273ca4fc..024fdb5be3fb 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -3408,9 +3408,6 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( // result, the plan will end up with a serial unordered synchronizer, // which has exactly the behavior that we want (in particular, it won't // execute the right child if the limit is reached by the left child). - // TODO(rytaft,yuzefovich): This currently only works with the - // vectorized engine. We should consider adding support for the serial - // unordered synchronizer in the row-based engine (see #61081). p.EnsureSingleStreamPerNode( false, /* forceSerialization */ execinfrapb.PostProcessSpec{Limit: n.hardLimit}, diff --git a/pkg/sql/execinfrapb/data.go b/pkg/sql/execinfrapb/data.go index 4c2dfa6a6262..0d10b3555e0e 100644 --- a/pkg/sql/execinfrapb/data.go +++ b/pkg/sql/execinfrapb/data.go @@ -25,7 +25,7 @@ import ( ) // ConvertToColumnOrdering converts an Ordering type (as defined in data.proto) -// to a sqlbase.ColumnOrdering type. +// to a colinfo.ColumnOrdering type. func ConvertToColumnOrdering(specOrdering Ordering) colinfo.ColumnOrdering { ordering := make(colinfo.ColumnOrdering, len(specOrdering.Columns)) for i, c := range specOrdering.Columns { @@ -39,13 +39,13 @@ func ConvertToColumnOrdering(specOrdering Ordering) colinfo.ColumnOrdering { return ordering } -// ConvertToSpecOrdering converts a sqlbase.ColumnOrdering type +// ConvertToSpecOrdering converts a colinfo.ColumnOrdering type // to an Ordering type (as defined in data.proto). func ConvertToSpecOrdering(columnOrdering colinfo.ColumnOrdering) Ordering { return ConvertToMappedSpecOrdering(columnOrdering, nil) } -// ConvertToMappedSpecOrdering converts a sqlbase.ColumnOrdering type +// ConvertToMappedSpecOrdering converts a colinfo.ColumnOrdering type // to an Ordering type (as defined in data.proto), using the column // indices contained in planToStreamColMap. func ConvertToMappedSpecOrdering( diff --git a/pkg/sql/execinfrapb/data.pb.go b/pkg/sql/execinfrapb/data.pb.go index 0ab4bcdeebe8..03e63988a171 100644 --- a/pkg/sql/execinfrapb/data.pb.go +++ b/pkg/sql/execinfrapb/data.pb.go @@ -289,7 +289,7 @@ func (m *Expression) XXX_DiscardUnknown() { var xxx_messageInfo_Expression proto.InternalMessageInfo // Ordering defines an order - specifically a list of column indices and -// directions. See sqlbase.ColumnOrdering. +// directions. See colinfo.ColumnOrdering. type Ordering struct { Columns []Ordering_Column `protobuf:"bytes,1,rep,name=columns" json:"columns"` } diff --git a/pkg/sql/execinfrapb/data.proto b/pkg/sql/execinfrapb/data.proto index 3fb29d3eeedd..51832404080e 100644 --- a/pkg/sql/execinfrapb/data.proto +++ b/pkg/sql/execinfrapb/data.proto @@ -58,7 +58,7 @@ message Expression { } // Ordering defines an order - specifically a list of column indices and -// directions. See sqlbase.ColumnOrdering. +// directions. See colinfo.ColumnOrdering. message Ordering { option (gogoproto.equal) = true; diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 318cdba188ff..9c274b095cca 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -472,6 +472,63 @@ type testClusterConfig struct { const threeNodeTenantConfigName = "3node-tenant" +var multiregion9node3region3azsLocalities = map[int]roachpb.Locality{ + 1: { + Tiers: []roachpb.Tier{ + {Key: "region", Value: "ap-southeast-2"}, + {Key: "availability-zone", Value: "ap-az1"}, + }, + }, + 2: { + Tiers: []roachpb.Tier{ + {Key: "region", Value: "ap-southeast-2"}, + {Key: "availability-zone", Value: "ap-az2"}, + }, + }, + 3: { + Tiers: []roachpb.Tier{ + {Key: "region", Value: "ap-southeast-2"}, + {Key: "availability-zone", Value: "ap-az3"}, + }, + }, + 4: { + Tiers: []roachpb.Tier{ + {Key: "region", Value: "ca-central-1"}, + {Key: "availability-zone", Value: "ca-az1"}, + }, + }, + 5: { + Tiers: []roachpb.Tier{ + {Key: "region", Value: "ca-central-1"}, + {Key: "availability-zone", Value: "ca-az2"}, + }, + }, + 6: { + Tiers: []roachpb.Tier{ + {Key: "region", Value: "ca-central-1"}, + {Key: "availability-zone", Value: "ca-az3"}, + }, + }, + 7: { + Tiers: []roachpb.Tier{ + {Key: "region", Value: "us-east-1"}, + {Key: "availability-zone", Value: "us-az1"}, + }, + }, + 8: { + Tiers: []roachpb.Tier{ + {Key: "region", Value: "us-east-1"}, + {Key: "availability-zone", Value: "us-az2"}, + }, + }, + 9: { + Tiers: []roachpb.Tier{ + {Key: "region", Value: "us-east-1"}, + {Key: "availability-zone", Value: "us-az3"}, + }, + }, +} + // logicTestConfigs contains all possible cluster configs. A test file can // specify a list of configs they run on in a file-level comment like: // # LogicTest: default distsql @@ -632,62 +689,14 @@ var logicTestConfigs = []testClusterConfig{ name: "multiregion-9node-3region-3azs", numNodes: 9, overrideAutoStats: "false", - localities: map[int]roachpb.Locality{ - 1: { - Tiers: []roachpb.Tier{ - {Key: "region", Value: "ap-southeast-2"}, - {Key: "availability-zone", Value: "ap-az1"}, - }, - }, - 2: { - Tiers: []roachpb.Tier{ - {Key: "region", Value: "ap-southeast-2"}, - {Key: "availability-zone", Value: "ap-az2"}, - }, - }, - 3: { - Tiers: []roachpb.Tier{ - {Key: "region", Value: "ap-southeast-2"}, - {Key: "availability-zone", Value: "ap-az3"}, - }, - }, - 4: { - Tiers: []roachpb.Tier{ - {Key: "region", Value: "ca-central-1"}, - {Key: "availability-zone", Value: "ca-az1"}, - }, - }, - 5: { - Tiers: []roachpb.Tier{ - {Key: "region", Value: "ca-central-1"}, - {Key: "availability-zone", Value: "ca-az2"}, - }, - }, - 6: { - Tiers: []roachpb.Tier{ - {Key: "region", Value: "ca-central-1"}, - {Key: "availability-zone", Value: "ca-az3"}, - }, - }, - 7: { - Tiers: []roachpb.Tier{ - {Key: "region", Value: "us-east-1"}, - {Key: "availability-zone", Value: "us-az1"}, - }, - }, - 8: { - Tiers: []roachpb.Tier{ - {Key: "region", Value: "us-east-1"}, - {Key: "availability-zone", Value: "us-az2"}, - }, - }, - 9: { - Tiers: []roachpb.Tier{ - {Key: "region", Value: "us-east-1"}, - {Key: "availability-zone", Value: "us-az3"}, - }, - }, - }, + localities: multiregion9node3region3azsLocalities, + }, + { + name: "multiregion-9node-3region-3azs-vec-off", + numNodes: 9, + overrideAutoStats: "false", + localities: multiregion9node3region3azsLocalities, + overrideVectorize: "off", }, } diff --git a/pkg/sql/rowexec/tablereader.go b/pkg/sql/rowexec/tablereader.go index 2f78d542e539..c5429f6248bf 100644 --- a/pkg/sql/rowexec/tablereader.go +++ b/pkg/sql/rowexec/tablereader.go @@ -38,6 +38,8 @@ type tableReader struct { limitHint int64 parallelize bool + scanStarted bool + // See TableReaderSpec.MaxTimestampAgeNanos. maxTimestampAge time.Duration @@ -181,8 +183,13 @@ func (tr *tableReader) Start(ctx context.Context) { log.Fatalf(ctx, "tableReader outside of txn") } + // Keep ctx assignment so we remember StartInternal can make a new one. ctx = tr.StartInternal(ctx, tableReaderProcName) + // Appease the linter. + _ = ctx +} +func (tr *tableReader) startScan(ctx context.Context) error { limitBatches := !tr.parallelize log.VEventf(ctx, 1, "starting scan with limitBatches %t", limitBatches) var err error @@ -200,10 +207,8 @@ func (tr *tableReader) Start(ctx context.Context) { tr.EvalCtx.TestingKnobs.ForceProductionBatchSizes, ) } - - if err != nil { - tr.MoveToDraining(err) - } + tr.scanStarted = true + return err } // Release releases this tableReader back to the pool. @@ -232,6 +237,13 @@ func TestingSetScannedRowProgressFrequency(val int64) func() { // Next is part of the RowSource interface. func (tr *tableReader) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) { for tr.State == execinfra.StateRunning { + if !tr.scanStarted { + err := tr.startScan(tr.Ctx) + if err != nil { + tr.MoveToDraining(err) + break + } + } // Check if it is time to emit a progress update. if tr.rowsRead >= tableReaderProgressFrequency { meta := execinfrapb.GetProducerMeta() diff --git a/pkg/sql/rowflow/input_sync.go b/pkg/sql/rowflow/input_sync.go index cc76eac70ecb..ff635a7883f5 100644 --- a/pkg/sql/rowflow/input_sync.go +++ b/pkg/sql/rowflow/input_sync.go @@ -36,44 +36,69 @@ type srcInfo struct { // srcIdx refers to the index of a source inside a []srcInfo array. type srcIdx int -type orderedSynchronizerState int +type serialSynchronizerState int const ( // notInitialized means that the heap has not yet been constructed. A row // needs to be read from each source to build the heap. - notInitialized orderedSynchronizerState = iota - // returningRows is the regular operation mode of the orderedSynchronizer. + notInitialized serialSynchronizerState = iota + // returningRows is the regular mode of operation. // Rows and metadata records are returning to the consumer. returningRows - // draining means the orderedSynchronizer will ignore everything but metadata + // draining means the synchronizer will ignore everything but metadata // records. On the first call to NextRow() while in draining mode, all the // sources are read until exhausted and metadata is accumulated. The state is // then transitioned to drained. draining - // In the drainBuffered mode, all the sources of the orderedSynchronizer have been - // exhausted, and we might have some buffered metadata. Metadata records are - // going to be returned, one by one. + // In the drainBuffered mode, all the sources of the serialOrderedSynchronizer + // have been exhausted, and we might have some buffered metadata. Metadata + // records are going to be returned, one by one. serialUnorderedSynchronizer + // doesn't buffer metadata and doesn't use this. drainBuffered ) -// orderedSynchronizer receives rows from multiple streams and produces a single +// Let setupProcessors peek at sources w/o caring about type of synchronizer. +type serialSynchronizer interface { + getSources() []srcInfo +} + +type serialSynchronizerBase struct { + state serialSynchronizerState + + types []*types.T + sources []srcInfo + rowAlloc rowenc.EncDatumRowAlloc +} + +var _ serialSynchronizer = &serialSynchronizerBase{} + +func (s *serialSynchronizerBase) getSources() []srcInfo { + return s.sources +} + +// Start is part of the RowSource interface. +func (s *serialSynchronizerBase) Start(ctx context.Context) { + for _, src := range s.sources { + src.src.Start(ctx) + } +} + +// OutputTypes is part of the RowSource interface. +func (s *serialSynchronizerBase) OutputTypes() []*types.T { + return s.types +} + +// serialOrderedSynchronizer receives rows from multiple streams and produces a single // stream of rows, ordered according to a set of columns. The rows in each input // stream are assumed to be ordered according to the same set of columns // (intra-stream ordering). -type orderedSynchronizer struct { - // ordering dictates the way in which rows compare. If nil (i.e. - // sqlbase.NoOrdering), then rows are not compared and sources are consumed in - // index order. - ordering colinfo.ColumnOrdering - evalCtx *tree.EvalContext - - sources []srcInfo +type serialOrderedSynchronizer struct { + serialSynchronizerBase - types []*types.T - - // state dictates the operation mode. - state orderedSynchronizerState + // ordering dictates the way in which rows compare. This can't be nil. + ordering colinfo.ColumnOrdering + evalCtx *tree.EvalContext // heap of source indexes. In state notInitialized, heap holds all source // indexes. Once initialized (initHeap is called), heap will be ordered by the // current row from each source and will only contain source indexes of @@ -89,34 +114,22 @@ type orderedSynchronizer struct { // err can be set by the Less function (used by the heap implementation) err error - alloc rowenc.DatumAlloc - rowAlloc rowenc.EncDatumRowAlloc + alloc rowenc.DatumAlloc // metadata is accumulated from all the sources and is passed on as soon as // possible. metadata []*execinfrapb.ProducerMetadata } -var _ execinfra.RowSource = &orderedSynchronizer{} - -// OutputTypes is part of the RowSource interface. -func (s *orderedSynchronizer) OutputTypes() []*types.T { - return s.types -} +var _ execinfra.RowSource = &serialOrderedSynchronizer{} // Len is part of heap.Interface and is only meant to be used internally. -func (s *orderedSynchronizer) Len() int { +func (s *serialOrderedSynchronizer) Len() int { return len(s.heap) } // Less is part of heap.Interface and is only meant to be used internally. -func (s *orderedSynchronizer) Less(i, j int) bool { - // If we're not enforcing any ordering between rows, let's consume sources in - // their index order. - if s.ordering == nil { - return s.heap[i] < s.heap[j] - } - +func (s *serialOrderedSynchronizer) Less(i, j int) bool { si := &s.sources[s.heap[i]] sj := &s.sources[s.heap[j]] cmp, err := si.row.Compare(s.types, &s.alloc, s.ordering, s.evalCtx, sj.row) @@ -128,16 +141,16 @@ func (s *orderedSynchronizer) Less(i, j int) bool { } // Swap is part of heap.Interface and is only meant to be used internally. -func (s *orderedSynchronizer) Swap(i, j int) { +func (s *serialOrderedSynchronizer) Swap(i, j int) { s.heap[i], s.heap[j] = s.heap[j], s.heap[i] } // Push is part of heap.Interface; it's not used as we never insert elements to // the heap (we initialize it with all sources, see initHeap). -func (s *orderedSynchronizer) Push(x interface{}) { panic("unimplemented") } +func (s *serialOrderedSynchronizer) Push(x interface{}) { panic("unimplemented") } // Pop is part of heap.Interface and is only meant to be used internally. -func (s *orderedSynchronizer) Pop() interface{} { +func (s *serialOrderedSynchronizer) Pop() interface{} { s.heap = s.heap[:len(s.heap)-1] return nil } @@ -147,7 +160,7 @@ func (s *orderedSynchronizer) Pop() interface{} { // from it) unless there are no more rows to read from it. // If an error is returned, heap.Init() has not been called, so s.heap is not // an actual heap. In this case, all members of the heap need to be drained. -func (s *orderedSynchronizer) initHeap() error { +func (s *serialOrderedSynchronizer) initHeap() error { // consumeErr is the last error encountered while consuming metadata. var consumeErr error @@ -201,7 +214,9 @@ const ( // not consumed and the error is returned. With the drain mode, metadata records // with error are accumulated like all the others and this method doesn't return // any errors. -func (s *orderedSynchronizer) consumeMetadata(src *srcInfo, mode consumeMetadataOption) error { +func (s *serialOrderedSynchronizer) consumeMetadata( + src *srcInfo, mode consumeMetadataOption, +) error { for { row, meta := src.src.Next() if meta != nil { @@ -232,8 +247,8 @@ func (s *orderedSynchronizer) consumeMetadata(src *srcInfo, mode consumeMetadata // If an error is returned, then either the heap is in a bad state (s.err has // been set), or one of the sources is borked. In either case, advanceRoot() // should not be called again - the caller should update the -// orderedSynchronizer.state accordingly. -func (s *orderedSynchronizer) advanceRoot() error { +// serialOrderedSynchronizer.state accordingly. +func (s *serialOrderedSynchronizer) advanceRoot() error { if s.state != returningRows { return errors.Errorf("advanceRoot() called in unsupported state: %d", s.state) } @@ -270,7 +285,7 @@ func (s *orderedSynchronizer) advanceRoot() error { // drainSources consumes all the rows from the sources. All the data is // discarded, except the metadata records which are accumulated in s.metadata. -func (s *orderedSynchronizer) drainSources() { +func (s *serialOrderedSynchronizer) drainSources() { for _, srcIdx := range s.heap { if err := s.consumeMetadata(&s.sources[srcIdx], drain); err != nil { log.Fatalf(context.TODO(), "unexpected draining error: %s", err) @@ -278,15 +293,8 @@ func (s *orderedSynchronizer) drainSources() { } } -// Start is part of the RowSource interface. -func (s *orderedSynchronizer) Start(ctx context.Context) { - for _, src := range s.sources { - src.src.Start(ctx) - } -} - // Next is part of the RowSource interface. -func (s *orderedSynchronizer) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) { +func (s *serialOrderedSynchronizer) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) { if s.state == notInitialized { if err := s.initHeap(); err != nil { s.ConsumerDone() @@ -328,7 +336,7 @@ func (s *orderedSynchronizer) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerM } // ConsumerDone is part of the RowSource interface. -func (s *orderedSynchronizer) ConsumerDone() { +func (s *serialOrderedSynchronizer) ConsumerDone() { // We're entering draining mode. Only metadata will be forwarded from now on. if s.state != draining { s.consumerStatusChanged(draining, execinfra.RowSource.ConsumerDone) @@ -336,7 +344,7 @@ func (s *orderedSynchronizer) ConsumerDone() { } // ConsumerClosed is part of the RowSource interface. -func (s *orderedSynchronizer) ConsumerClosed() { +func (s *serialOrderedSynchronizer) ConsumerClosed() { // The state shouldn't matter, as no further methods should be called, but // we'll set it to something other than the default. s.consumerStatusChanged(drainBuffered, execinfra.RowSource.ConsumerClosed) @@ -344,8 +352,8 @@ func (s *orderedSynchronizer) ConsumerClosed() { // consumerStatusChanged calls a RowSource method on all the non-exhausted // sources. -func (s *orderedSynchronizer) consumerStatusChanged( - newState orderedSynchronizerState, f func(execinfra.RowSource), +func (s *serialOrderedSynchronizer) consumerStatusChanged( + newState serialSynchronizerState, f func(execinfra.RowSource), ) { if s.state == notInitialized { for i := range s.sources { @@ -363,28 +371,99 @@ func (s *orderedSynchronizer) consumerStatusChanged( s.state = newState } -// makeOrderedSync creates an orderedSynchronizer. ordering dictates how rows -// are to be compared. Use sqlbase.NoOrdering to indicate that the row ordering -// doesn't matter and sources should be consumed in index order (which is useful -// when you intend to fuse the synchronizer and its inputs later; see -// FuseAggresively). -func makeOrderedSync( +// serialUnorderedSynchronizer exhausts its sources one at a time until +// each is drained. It's necessary to have the row engine support locality +// optimized scans. +type serialUnorderedSynchronizer struct { + serialSynchronizerBase + + srcIndex int +} + +var _ execinfra.RowSource = &serialUnorderedSynchronizer{} + +func (u *serialUnorderedSynchronizer) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) { + for u.srcIndex < len(u.sources) { + row, metadata := u.sources[u.srcIndex].src.Next() + + // If we're draining only return metadata. + if u.state == draining && row != nil { + continue + } + + // If we see nil,nil go to next source. + if row == nil && metadata == nil { + u.srcIndex++ + continue + } + + if row != nil { + row = u.rowAlloc.CopyRow(row) + } + + return row, metadata + } + + return nil, nil +} + +// ConsumerDone is part of the RowSource interface. +func (u *serialUnorderedSynchronizer) ConsumerDone() { + if u.state != draining { + for i := range u.sources { + u.sources[i].src.ConsumerDone() + } + u.state = draining + } +} + +// ConsumerClosed is part of the RowSource interface. +func (u *serialUnorderedSynchronizer) ConsumerClosed() { + for i := range u.sources { + u.sources[i].src.ConsumerClosed() + } +} + +// makeSerialSync creates a serialOrderedSynchronizer or a +// serialUnorderedSynchronizer. ordering dictates how rows are to be compared. +// Use colinfo.NoOrdering to indicate that the row ordering doesn't matter and +// sources should be consumed in index order (which is useful when you intend to +// fuse the synchronizer and its inputs later; see FuseAggressively). +func makeSerialSync( ordering colinfo.ColumnOrdering, evalCtx *tree.EvalContext, sources []execinfra.RowSource, ) (execinfra.RowSource, error) { if len(sources) < 2 { - return nil, errors.Errorf("only %d sources for ordered synchronizer", len(sources)) + return nil, errors.Errorf("only %d sources for serial synchronizer", len(sources)) + } + + base := serialSynchronizerBase{ + state: notInitialized, + sources: make([]srcInfo, len(sources)), + types: sources[0].OutputTypes(), } - s := &orderedSynchronizer{ - state: notInitialized, - sources: make([]srcInfo, len(sources)), - types: sources[0].OutputTypes(), - heap: make([]srcIdx, 0, len(sources)), - ordering: ordering, - evalCtx: evalCtx, + + for i := range base.sources { + base.sources[i].src = sources[i] } - for i := range s.sources { - s.sources[i].src = sources[i] - s.heap = append(s.heap, srcIdx(i)) + + var sync execinfra.RowSource + + if len(ordering) > 0 { + os := &serialOrderedSynchronizer{ + serialSynchronizerBase: base, + heap: make([]srcIdx, 0, len(sources)), + ordering: ordering, + evalCtx: evalCtx, + } + for i := range os.sources { + os.heap = append(os.heap, srcIdx(i)) + } + sync = os + } else { + sync = &serialUnorderedSynchronizer{ + serialSynchronizerBase: base, + } } - return s, nil + + return sync, nil } diff --git a/pkg/sql/rowflow/input_sync_test.go b/pkg/sql/rowflow/input_sync_test.go index e89d1a3681d0..dd3bdef1e7f8 100644 --- a/pkg/sql/rowflow/input_sync_test.go +++ b/pkg/sql/rowflow/input_sync_test.go @@ -115,7 +115,7 @@ func TestOrderedSync(t *testing.T) { } evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) defer evalCtx.Stop(context.Background()) - src, err := makeOrderedSync(c.ordering, evalCtx, sources) + src, err := makeSerialSync(c.ordering, evalCtx, sources) if err != nil { t.Fatal(err) } @@ -155,7 +155,7 @@ func TestOrderedSyncDrainBeforeNext(t *testing.T) { ctx := context.Background() evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) defer evalCtx.Stop(ctx) - o, err := makeOrderedSync(colinfo.ColumnOrdering{}, evalCtx, sources) + o, err := makeSerialSync(colinfo.ColumnOrdering{}, evalCtx, sources) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/rowflow/row_based_flow.go b/pkg/sql/rowflow/row_based_flow.go index f6f5cc633b7f..a411cbc45471 100644 --- a/pkg/sql/rowflow/row_based_flow.go +++ b/pkg/sql/rowflow/row_based_flow.go @@ -131,15 +131,18 @@ func (f *rowBasedFlow) setupProcessors( } // ps has an input with multiple streams. This can be either a // multiplexed RowChannel (in case of some unordered synchronizers) - // or an orderedSynchronizer (for other unordered synchronizers or + // or a serialSynchronizer (for other unordered synchronizers or // ordered synchronizers). If it's a multiplexed RowChannel, // then its inputs run in parallel, so there's no fusing with them. - // If it's an orderedSynchronizer, then we look inside it to see if + // If it's a serial synchronizer, then we look inside it to see if // the processor we're trying to fuse feeds into it. - orderedSync, ok := inputSyncs[pIdx][inIdx].(*orderedSynchronizer) + + sync, ok := inputSyncs[pIdx][inIdx].(serialSynchronizer) + if !ok { continue } + // See if we can find a stream attached to the processor we're // trying to fuse. for sIdx, sspec := range in.Streams { @@ -150,8 +153,8 @@ func (f *rowBasedFlow) setupProcessors( if input.ProcessorID != pspec.ProcessorID { continue } - // Fuse the processor with this orderedSynchronizer. - orderedSync.sources[sIdx].src = source + // Fuse the processor with this synchronizer. + sync.getSources()[sIdx].src = source return true } } @@ -304,7 +307,7 @@ func (f *rowBasedFlow) setupInputSyncs( // We have an ordered synchronizer or an unordered one that we really // want to fuse because of the FuseAggressively option. We'll create a // RowChannel for each input for now, but the inputs might be fused with - // the orderedSynchronizer later (in which case the RowChannels will be + // the synchronizer later (in which case the RowChannels will be // dropped). streams := make([]execinfra.RowSource, len(is.Streams)) for i, s := range is.Streams { @@ -320,7 +323,7 @@ func (f *rowBasedFlow) setupInputSyncs( if is.Type == execinfrapb.InputSyncSpec_ORDERED { ordering = execinfrapb.ConvertToColumnOrdering(is.Ordering) } - sync, err = makeOrderedSync(ordering, f.EvalCtx, streams) + sync, err = makeSerialSync(ordering, f.EvalCtx, streams) if err != nil { return nil, err } From 37e589bec2b65a0f3715ed94ae854e851bf229be Mon Sep 17 00:00:00 2001 From: Tobias Grieger <tobias.b.grieger@gmail.com> Date: Tue, 20 Apr 2021 13:55:45 +0200 Subject: [PATCH 35/37] roachtest: remove misplaced skip This wasn't having any effect, as roachtest is never invoked with the race tag. Release note: None --- pkg/cmd/roachtest/election.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/pkg/cmd/roachtest/election.go b/pkg/cmd/roachtest/election.go index 7fcfa15a7b24..28f629c83b59 100644 --- a/pkg/cmd/roachtest/election.go +++ b/pkg/cmd/roachtest/election.go @@ -14,7 +14,6 @@ import ( "context" "time" - "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) @@ -25,8 +24,6 @@ func registerElectionAfterRestart(r *testRegistry) { Skip: "https://github.com/cockroachdb/cockroach/issues/54246", Cluster: makeClusterSpec(3), Run: func(ctx context.Context, t *test, c *cluster) { - skip.UnderRace(t, "race builds make this test exceed its timeout") - t.Status("starting up") c.Put(ctx, cockroach, "./cockroach") c.Start(ctx, t) From 0e73a8a286ab736fc48bc3f7ca60a368823a6312 Mon Sep 17 00:00:00 2001 From: Marius Posta <marius@cockroachlabs.com> Date: Tue, 20 Apr 2021 09:09:17 -0400 Subject: [PATCH 36/37] schemaexpr: fix data race in ProcessColumnSet This commit fixes a data race introduced by my recent changes tracked under #63755, involving the generalized use of catalog.Column instead of descpb.ColumnDescriptor. Fixes #63907 Release note: None --- pkg/sql/catalog/schemaexpr/default_exprs.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pkg/sql/catalog/schemaexpr/default_exprs.go b/pkg/sql/catalog/schemaexpr/default_exprs.go index 1b647ecbc01d..98e89418b11c 100644 --- a/pkg/sql/catalog/schemaexpr/default_exprs.go +++ b/pkg/sql/catalog/schemaexpr/default_exprs.go @@ -91,13 +91,15 @@ func ProcessColumnSet( // Add all public or columns in DELETE_AND_WRITE_ONLY state // that satisfy the condition. + ret := make([]catalog.Column, 0, len(tableDesc.AllColumns())) + ret = append(ret, cols...) for _, col := range tableDesc.WritableColumns() { if inSet(col) { if !colIDSet.Contains(col.GetID()) { colIDSet.Add(col.GetID()) - cols = append(cols, col) + ret = append(ret, col) } } } - return cols + return ret } From f01e15d86dcb564baf8a6149f68af70d679a61d4 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten <nvanbenschoten@gmail.com> Date: Mon, 19 Apr 2021 09:43:33 -0400 Subject: [PATCH 37/37] storage: pool pebbleReadOnly allocations MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit introduces object pooling for `pebbleReadOnly` allocation avoidance. I found this to be important both because it avoids the initial `pebbleReadOnly` allocation, but also because it allows the memory recycling inside of each `pebbleIterator` owned by a `pebbleReadOnly` to work correctly. I found this while running a few microbenchmarks and noticing that the lockTable's calls to `intentInterleavingReader` were resulting in a large number of heap allocations in `(*pebbleReadOnly).NewEngineIterator`. This was because `lowerBoundBuf` and `upperBoundBuf` were always nil and so each append (all 4 of them) in `(*pebbleIterator).init` was causing an allocation. ``` name old time/op new time/op delta KV/Scan/Native/rows=1-16 30.9µs ± 4% 29.9µs ± 6% -3.29% (p=0.000 n=20+20) KV/Scan/Native/rows=100-16 54.2µs ± 4% 52.7µs ± 5% -2.84% (p=0.002 n=20+20) KV/Scan/Native/rows=10-16 34.0µs ± 3% 33.1µs ± 6% -2.64% (p=0.001 n=20+20) KV/Scan/Native/rows=1000-16 253µs ± 5% 255µs ± 5% ~ (p=0.659 n=20+20) KV/Scan/Native/rows=10000-16 2.16ms ± 4% 2.14ms ± 3% ~ (p=0.072 n=20+20) name old alloc/op new alloc/op delta KV/Scan/Native/rows=1-16 8.69kB ± 0% 7.49kB ± 0% -13.79% (p=0.000 n=20+19) KV/Scan/Native/rows=10-16 10.1kB ± 0% 8.9kB ± 0% -11.87% (p=0.000 n=20+18) KV/Scan/Native/rows=100-16 22.7kB ± 0% 21.5kB ± 0% -5.29% (p=0.000 n=17+19) KV/Scan/Native/rows=1000-16 174kB ± 0% 172kB ± 0% -0.66% (p=0.000 n=19+19) KV/Scan/Native/rows=10000-16 1.51MB ± 0% 1.51MB ± 0% -0.05% (p=0.000 n=16+19) name old allocs/op new allocs/op delta KV/Scan/Native/rows=1-16 71.0 ± 0% 62.0 ± 0% -12.68% (p=0.000 n=20+20) KV/Scan/Native/rows=10-16 75.0 ± 0% 66.0 ± 0% -12.00% (p=0.000 n=20+19) KV/Scan/Native/rows=100-16 79.0 ± 0% 70.0 ± 0% -11.39% (p=0.000 n=19+19) KV/Scan/Native/rows=1000-16 87.8 ± 1% 79.0 ± 0% -9.97% (p=0.000 n=20+16) KV/Scan/Native/rows=10000-16 113 ± 2% 103 ± 2% -8.19% (p=0.000 n=17+19) ``` We may want to consider this as a candidate to backport to release-21.1, because the lack of pooling here was even more detrimental with the separated lockTable, which creates a separate EngineIterator. So this may have a small impact on #62078. Release note (performance improvement): A series of heap allocations performed when serving read-only queries are now avoided. --- pkg/storage/pebble.go | 47 ++++++++++++++++++++++++++++++++----------- 1 file changed, 35 insertions(+), 12 deletions(-) diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 200f7be7bc8c..4a500d9d19e0 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -20,6 +20,7 @@ import ( "sort" "strconv" "strings" + "sync" "sync/atomic" "time" @@ -1074,18 +1075,7 @@ func (p *Pebble) NewBatch() Batch { // NewReadOnly implements the Engine interface. func (p *Pebble) NewReadOnly() ReadWriter { - // TODO(sumeer): a sync.Pool for pebbleReadOnly would save on allocations - // for the underlying pebbleIterators. - return &pebbleReadOnly{ - parent: p, - // Defensively set reusable=true. One has to be careful about this since - // an accidental false value would cause these iterators, that are value - // members of pebbleReadOnly, to be put in the pebbleIterPool. - prefixIter: pebbleIterator{reusable: true}, - normalIter: pebbleIterator{reusable: true}, - prefixEngineIter: pebbleIterator{reusable: true}, - normalEngineIter: pebbleIterator{reusable: true}, - } + return newPebbleReadOnly(p) } // NewUnindexedBatch implements the Engine interface. @@ -1273,6 +1263,37 @@ type pebbleReadOnly struct { var _ ReadWriter = &pebbleReadOnly{} +var pebbleReadOnlyPool = sync.Pool{ + New: func() interface{} { + return &pebbleReadOnly{ + // Defensively set reusable=true. One has to be careful about this since + // an accidental false value would cause these iterators, that are value + // members of pebbleReadOnly, to be put in the pebbleIterPool. + prefixIter: pebbleIterator{reusable: true}, + normalIter: pebbleIterator{reusable: true}, + prefixEngineIter: pebbleIterator{reusable: true}, + normalEngineIter: pebbleIterator{reusable: true}, + } + }, +} + +// Instantiates a new pebbleReadOnly. +func newPebbleReadOnly(parent *Pebble) *pebbleReadOnly { + p := pebbleReadOnlyPool.Get().(*pebbleReadOnly) + // When p is a reused pebbleReadOnly from the pool, the iter fields preserve + // the original reusable=true that was set above in pebbleReadOnlyPool.New(), + // and some buffers that are safe to reuse. Everything else has been reset by + // pebbleIterator.destroy(). + *p = pebbleReadOnly{ + parent: parent, + prefixIter: p.prefixIter, + normalIter: p.normalIter, + prefixEngineIter: p.prefixEngineIter, + normalEngineIter: p.normalEngineIter, + } + return p +} + func (p *pebbleReadOnly) Close() { if p.closed { panic("closing an already-closed pebbleReadOnly") @@ -1285,6 +1306,8 @@ func (p *pebbleReadOnly) Close() { p.normalIter.destroy() p.prefixEngineIter.destroy() p.normalEngineIter.destroy() + + pebbleReadOnlyPool.Put(p) } func (p *pebbleReadOnly) Closed() bool {