From 89d381cae738b036bac5a20fb1a9fddacca324b2 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 10:25:10 -0500 Subject: [PATCH 01/13] sql: add Index interface and TableDescriptor methods in catalog Previously, the catalog.TableDescriptor interface and its implementing types would liberally return descpb.IndexDescriptor values, pointers and slices. In an effort to stop manipulating such protos directly, this patch introduces a catalog.Index interface to encapsulate it. In order to enventually propagate this change throughout the code base, this patch marks some existing catalog.TableDescriptor methods as deprecated and introduces new ones to eventually replace them. Partially addresses #57465. Release note: None --- pkg/sql/catalog/BUILD.bazel | 1 + pkg/sql/catalog/descriptor.go | 110 +++++- pkg/sql/catalog/tabledesc/BUILD.bazel | 2 + pkg/sql/catalog/tabledesc/index.go | 468 +++++++++++++++++++++++ pkg/sql/catalog/tabledesc/safe_format.go | 12 +- pkg/sql/catalog/tabledesc/structured.go | 324 +++++----------- pkg/sql/catalog/tabledesc/table.go | 12 +- pkg/sql/catalog/tabledesc/table_desc.go | 284 ++++++++++++-- pkg/sql/crdb_internal.go | 8 +- pkg/sql/delete.go | 3 +- pkg/sql/insert.go | 3 +- pkg/sql/row/partial_index.go | 57 ++- pkg/sql/set_zone_config.go | 11 +- pkg/sql/update.go | 7 +- pkg/sql/upsert.go | 3 +- 15 files changed, 977 insertions(+), 328 deletions(-) create mode 100644 pkg/sql/catalog/tabledesc/index.go diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index 7f26b7ae03db..2412f8841bab 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -14,6 +14,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog", visibility = ["//visibility:public"], deps = [ + "//pkg/geo/geoindex", "//pkg/keys", "//pkg/kv", "//pkg/roachpb", diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index e21607bf954f..445e0d3c35b5 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -13,6 +13,7 @@ package catalog import ( "context" + "github.com/cockroachdb/cockroach/pkg/geo/geoindex" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -20,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -112,22 +112,51 @@ type TableDescriptor interface { GetFormatVersion() descpb.FormatVersion GetPrimaryIndexID() descpb.IndexID - GetPrimaryIndex() *descpb.IndexDescriptor + GetPrimaryIndex() *descpb.IndexDescriptor // deprecated PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span - GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor - ForeachIndex(opts IndexOpts, f func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error) error - AllNonDropIndexes() []*descpb.IndexDescriptor - ForeachNonDropIndex(f func(idxDesc *descpb.IndexDescriptor) error) error + GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor // deprecated + ForeachIndex(opts IndexOpts, f func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error) error // deprecated + AllNonDropIndexes() []*descpb.IndexDescriptor // deprecated + ForeachNonDropIndex(f func(idxDesc *descpb.IndexDescriptor) error) error // deprecated IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span - FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) - FindIndexByName(name string) (_ *descpb.IndexDescriptor, dropped bool, _ error) - FindIndexesWithPartition(name string) []*descpb.IndexDescriptor + FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) // deprecated + FindIndexByName(name string) (_ *descpb.IndexDescriptor, dropped bool, _ error) // deprecated GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) KeysPerRow(id descpb.IndexID) (int, error) - PartialIndexOrds() util.FastIntSet - WritableIndexes() []descpb.IndexDescriptor - DeletableIndexes() []descpb.IndexDescriptor - DeleteOnlyIndexes() []descpb.IndexDescriptor + WritableIndexes() []descpb.IndexDescriptor // deprecated + DeletableIndexes() []descpb.IndexDescriptor // deprecated + DeleteOnlyIndexes() []descpb.IndexDescriptor // deprecated + + PrimaryIndexInterface() Index + AllIndexes() []Index + ActiveIndexes() []Index + NonDropIndexes() []Index + PartialIndexes() []Index + PublicNonPrimaryIndexes() []Index + WritableNonPrimaryIndexes() []Index + DeletableNonPrimaryIndexes() []Index + DeleteOnlyNonPrimaryIndexes() []Index + + ForEachIndex(opts IndexOpts, f func(idx Index) error) error + ForEachActiveIndex(f func(idx Index) error) error + ForEachNonDropIndex(f func(idx Index) error) error + ForEachPartialIndex(f func(idx Index) error) error + ForEachPublicNonPrimaryIndex(f func(idx Index) error) error + ForEachWritableNonPrimaryIndex(f func(idx Index) error) error + ForEachDeletableNonPrimaryIndex(f func(idx Index) error) error + ForEachDeleteOnlyNonPrimaryIndex(f func(idx Index) error) error + + FindIndex(opts IndexOpts, test func(idx Index) bool) Index + FindActiveIndex(test func(idx Index) bool) Index + FindNonDropIndex(test func(idx Index) bool) Index + FindPartialIndex(test func(idx Index) bool) Index + FindPublicNonPrimaryIndex(test func(idx Index) bool) Index + FindWritableNonPrimaryIndex(test func(idx Index) bool) Index + FindDeletableNonPrimaryIndex(test func(idx Index) bool) Index + FindDeleteOnlyNonPrimaryIndex(test func(idx Index) bool) Index + + FindIndexWithID(id descpb.IndexID) (Index, error) + FindIndexWithName(name string) (Index, error) HasPrimaryKey() bool PrimaryKeyString() string @@ -187,6 +216,61 @@ type TableDescriptor interface { WritableColumns() []descpb.ColumnDescriptor } +// Index is an interface around the index descriptor types. +type Index interface { + IndexDesc() *descpb.IndexDescriptor + IndexDescDeepCopy() descpb.IndexDescriptor + + Ordinal() int + Primary() bool + Public() bool + WriteAndDeleteOnly() bool + DeleteOnly() bool + Adding() bool + Dropped() bool + + GetID() descpb.IndexID + GetName() string + IsInterleaved() bool + IsPartial() bool + IsUnique() bool + IsDisabled() bool + IsSharded() bool + IsCreatedExplicitly() bool + GetPredicate() string + GetType() descpb.IndexDescriptor_Type + IsValidOriginIndex(originColIDs descpb.ColumnIDs) bool + IsValidReferencedIndex(referencedColIDs descpb.ColumnIDs) bool + GetGeoConfig() geoindex.Config + GetSharded() descpb.ShardedDescriptor + GetShardColumnName() string + GetVersion() descpb.IndexDescriptorVersion + GetEncodingType() descpb.IndexDescriptorEncodingType + GetPartitioning() descpb.PartitioningDescriptor + FindPartitionByName(name string) descpb.PartitioningDescriptor + PartitionNames() []string + NumInterleaveAncestors() int + GetInterleaveAncestor(ancestorOrdinal int) descpb.InterleaveDescriptor_Ancestor + NumInterleavedBy() int + GetInterleavedBy(interleavedByOrdinal int) descpb.ForeignKeyReference + NumColumns() int + GetColumnID(columnOrdinal int) descpb.ColumnID + GetColumnName(columnOrdinal int) string + GetColumnDirection(columnOrdinal int) descpb.IndexDescriptor_Direction + ContainsColumnID(colID descpb.ColumnID) bool + InvertedColumnID() descpb.ColumnID + InvertedColumnName() string + ForEachColumnID(func(id descpb.ColumnID) error) error + NumStoredColumns() int + GetStoredColumnID(storedColumnOrdinal int) descpb.ColumnID + GetStoredColumnName(storedColumnOrdinal int) string + HasOldStoredColumns() bool + NumExtraColumns() int + GetExtraColumnID(extraColumnOrdinal int) descpb.ColumnID + NumCompositeColumns() int + GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID +} + // TypeDescriptor will eventually be called typedesc.Descriptor. // It is implemented by (Imm|M)utableTypeDescriptor. type TypeDescriptor interface { diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index a107a91d4188..8d8b80d6ea2e 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "tabledesc", srcs = [ + "index.go", "safe_format.go", "structured.go", "table.go", @@ -11,6 +12,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc", visibility = ["//visibility:public"], deps = [ + "//pkg/geo/geoindex", "//pkg/keys", "//pkg/roachpb", "//pkg/settings/cluster", diff --git a/pkg/sql/catalog/tabledesc/index.go b/pkg/sql/catalog/tabledesc/index.go new file mode 100644 index 000000000000..928d389e4be5 --- /dev/null +++ b/pkg/sql/catalog/tabledesc/index.go @@ -0,0 +1,468 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tabledesc + +import ( + "github.com/cockroachdb/cockroach/pkg/geo/geoindex" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" +) + +var _ catalog.Index = (*index)(nil) + +// index implements the catalog.Index interface by wrapping the protobuf index +// descriptor along with some metadata from its parent table descriptor. +type index struct { + desc *descpb.IndexDescriptor + ordinal int + mutationID descpb.MutationID + mutationDirection descpb.DescriptorMutation_Direction + mutationState descpb.DescriptorMutation_State +} + +// IndexDesc returns the underlying protobuf descriptor. +// Ideally, this method should be called as rarely as possible. +func (w index) IndexDesc() *descpb.IndexDescriptor { + return w.desc +} + +// IndexDescDeepCopy returns a deep copy of the underlying protobuf descriptor. +func (w index) IndexDescDeepCopy() descpb.IndexDescriptor { + return *protoutil.Clone(w.desc).(*descpb.IndexDescriptor) +} + +// Ordinal returns the ordinal of the index within the table descriptor. +func (w index) Ordinal() int { + return w.ordinal +} + +// Primary returns true iff the index is the primary index for the table +// descriptor. +func (w index) Primary() bool { + return w.ordinal == 0 +} + +// Public returns true iff the index is active, i.e. readable. +func (w index) Public() bool { + return w.mutationState == descpb.DescriptorMutation_UNKNOWN +} + +// Adding returns true iff the index is an add mutation in the table descriptor. +func (w index) Adding() bool { + return w.mutationDirection == descpb.DescriptorMutation_ADD +} + +// Adding returns true iff the index is a drop mutation in the table descriptor. +func (w index) Dropped() bool { + return w.mutationDirection == descpb.DescriptorMutation_DROP +} + +// WriteAndDeleteOnly returns true iff the index is a mutation in the +// delete-and-write-only state. +func (w index) WriteAndDeleteOnly() bool { + return w.mutationState == descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY +} + +// DeleteOnly returns true iff the index is a mutation in the delete-only state. +func (w index) DeleteOnly() bool { + return w.mutationState == descpb.DescriptorMutation_DELETE_ONLY +} + +// GetID returns the index ID. +func (w index) GetID() descpb.IndexID { + return w.desc.ID +} + +// GetName returns the index name. +func (w index) GetName() string { + return w.desc.Name +} + +// IsInterleaved returns true iff the index is interleaved. +func (w index) IsInterleaved() bool { + return w.desc.IsInterleaved() +} + +// IsPartial returns true iff the index is a partial index. +func (w index) IsPartial() bool { + return w.desc.IsPartial() +} + +// IsUnique returns true iff the index is a unique index. +func (w index) IsUnique() bool { + return w.desc.Unique +} + +// IsDisabled returns true iff the index is disabled. +func (w index) IsDisabled() bool { + return w.desc.Disabled +} + +// IsSharded returns true iff the index is hash sharded. +func (w index) IsSharded() bool { + return w.desc.IsSharded() +} + +// IsCreatedExplicitly returns true iff this index was created explicitly, i.e. +// via 'CREATE INDEX' statement. +func (w index) IsCreatedExplicitly() bool { + return w.desc.CreatedExplicitly +} + +// GetPredicate returns the empty string when the index is not partial, +// otherwise it returns the corresponding expression of the partial index. +// Columns are referred to in the expression by their name. +func (w index) GetPredicate() string { + return w.desc.Predicate +} + +// GetType returns the type of index, inverted or forward. +func (w index) GetType() descpb.IndexDescriptor_Type { + return w.desc.Type +} + +// GetPartitioning returns the partitioning descriptor of the index. +func (w index) GetPartitioning() descpb.PartitioningDescriptor { + return w.desc.Partitioning +} + +// FindPartitionByName searches the index's partitioning descriptor for a +// partition whose name is the input and returns it, or nil if no match is found. +func (w index) FindPartitionByName(name string) descpb.PartitioningDescriptor { + return *w.desc.Partitioning.FindPartitionByName(name) +} + +// PartitionNames returns a slice containing the name of every partition and +// subpartition in an arbitrary order. +func (w index) PartitionNames() []string { + return w.desc.Partitioning.PartitionNames() +} + +// IsValidOriginIndex returns whether the index can serve as an origin index for +// a foreign key constraint with the provided set of originColIDs. +func (w index) IsValidOriginIndex(originColIDs descpb.ColumnIDs) bool { + return w.desc.IsValidOriginIndex(originColIDs) +} + +// IsValidReferencedIndex returns whether the index can serve as a referenced +// index for a foreign key constraint with the provided set of +// referencedColumnIDs. +func (w index) IsValidReferencedIndex(referencedColIDs descpb.ColumnIDs) bool { + return w.desc.IsValidReferencedIndex(referencedColIDs) +} + +// HasOldStoredColumns returns whether the index has stored columns in the old +// format (data encoded the same way as if they were in an implicit column). +func (w index) HasOldStoredColumns() bool { + return w.desc.HasOldStoredColumns() +} + +// InvertedColumnID returns the ColumnID of the inverted column of the inverted +// index. This is always the last column in ColumnIDs. Panics if the index is +// not inverted. +func (w index) InvertedColumnID() descpb.ColumnID { + return w.desc.InvertedColumnID() +} + +// InvertedColumnName returns the name of the inverted column of the inverted +// index. This is always the last column in ColumnNames. Panics if the index is +// not inverted. +func (w index) InvertedColumnName() string { + return w.desc.InvertedColumnName() +} + +// ContainsColumnID returns true if the index descriptor contains the specified +// column ID either in its explicit column IDs, the extra column IDs, or the +// stored column IDs. +func (w index) ContainsColumnID(colID descpb.ColumnID) bool { + return w.desc.ContainsColumnID(colID) +} + +// GetGeoConfig returns the geo config in the index descriptor. +func (w index) GetGeoConfig() geoindex.Config { + return w.desc.GeoConfig +} + +// GetSharded returns the ShardedDescriptor in the index descriptor +func (w index) GetSharded() descpb.ShardedDescriptor { + return w.desc.Sharded +} + +// GetShardColumnName returns the name of the shard column if the index is hash +// sharded, empty string otherwise. +func (w index) GetShardColumnName() string { + return w.desc.Sharded.Name +} + +// GetVersion returns the version of the index descriptor. +func (w index) GetVersion() descpb.IndexDescriptorVersion { + return w.desc.Version +} + +// GetEncodingType returns the encoding type of this index. For backward +// compatibility reasons, this might not match what is stored in +// w.desc.EncodingType. +func (w index) GetEncodingType() descpb.IndexDescriptorEncodingType { + if w.Primary() { + // Primary indexes always use the PrimaryIndexEncoding, regardless of what + // desc.EncodingType indicates. + return descpb.PrimaryIndexEncoding + } + return w.desc.EncodingType +} + +// NumInterleaveAncestors returns the number of interleave ancestors as per the +// index descriptor. +func (w index) NumInterleaveAncestors() int { + return len(w.desc.Interleave.Ancestors) +} + +// GetInterleaveAncestor returns the ancestorOrdinal-th interleave ancestor. +func (w index) GetInterleaveAncestor(ancestorOrdinal int) descpb.InterleaveDescriptor_Ancestor { + return w.desc.Interleave.Ancestors[ancestorOrdinal] +} + +// NumInterleavedBy returns the number of tables/indexes that are interleaved +// into this index. +func (w index) NumInterleavedBy() int { + return len(w.desc.InterleavedBy) +} + +// GetInterleavedBy returns the interleavedByOrdinal-th table/index that is +// interleaved into this index. +func (w index) GetInterleavedBy(interleavedByOrdinal int) descpb.ForeignKeyReference { + return w.desc.InterleavedBy[interleavedByOrdinal] +} + +// NumColumns returns the number of columns as per the index descriptor. +func (w index) NumColumns() int { + return len(w.desc.ColumnIDs) +} + +// GetColumnID returns the ID of the columnOrdinal-th column. +func (w index) GetColumnID(columnOrdinal int) descpb.ColumnID { + return w.desc.ColumnIDs[columnOrdinal] +} + +// GetColumnName returns the name of the columnOrdinal-th column. +func (w index) GetColumnName(columnOrdinal int) string { + return w.desc.ColumnNames[columnOrdinal] +} + +// GetColumnDirection returns the direction of the columnOrdinal-th column. +func (w index) GetColumnDirection(columnOrdinal int) descpb.IndexDescriptor_Direction { + return w.desc.ColumnDirections[columnOrdinal] +} + +// ForEachColumnID applies its argument fn to each of the column IDs in the +// index descriptor. If there is an error, that error is returned immediately. +func (w index) ForEachColumnID(fn func(colID descpb.ColumnID) error) error { + return w.desc.RunOverAllColumns(fn) +} + +// NumStoredColumns returns the number of columns which the index stores in +// addition to the columns which are explicitly part of the index (STORING +// clause). Only used for secondary indexes. +func (w index) NumStoredColumns() int { + return len(w.desc.StoreColumnIDs) +} + +// GetStoredColumnID returns the ID of the storeColumnOrdinal-th store column. +func (w index) GetStoredColumnID(storedColumnOrdinal int) descpb.ColumnID { + return w.desc.StoreColumnIDs[storedColumnOrdinal] +} + +// GetStoredColumnName returns the name of the storeColumnOrdinal-th store column. +func (w index) GetStoredColumnName(storedColumnOrdinal int) string { + return w.desc.StoreColumnNames[storedColumnOrdinal] +} + +// NumExtraColumns returns the number of additional columns referenced by the +// index descriptor. +func (w index) NumExtraColumns() int { + return len(w.desc.ExtraColumnIDs) +} + +// GetExtraColumnID returns the ID of the extraColumnOrdinal-th extra column. +func (w index) GetExtraColumnID(extraColumnOrdinal int) descpb.ColumnID { + return w.desc.ExtraColumnIDs[extraColumnOrdinal] +} + +// NumCompositeColumns returns the number of composite columns referenced by the +// index descriptor. +func (w index) NumCompositeColumns() int { + return len(w.desc.CompositeColumnIDs) +} + +// GetCompositeColumnID returns the ID of the compositeColumnOrdinal-th +// composite column. +func (w index) GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID { + return w.desc.CompositeColumnIDs[compositeColumnOrdinal] +} + +// indexCache contains lazily precomputed slices of catalog.Index interfaces. +// A field value of nil indicates that the slice hasn't been precomputed yet. +type indexCache struct { + all []catalog.Index + active []catalog.Index + nonDrop []catalog.Index + publicNonPrimary []catalog.Index + writableNonPrimary []catalog.Index + deletableNonPrimary []catalog.Index + deleteOnlyNonPrimary []catalog.Index + partial []catalog.Index +} + +// cachedIndexes returns an already-build slice of catalog.Index interfaces if +// it exists, if not it builds it using the provided factory function and args. +// Notice that, as a result, empty slices need to be handled carefully. +func (c *indexCache) cachedIndexes( + cached *[]catalog.Index, + factory func(c *indexCache, desc *wrapper) []catalog.Index, + desc *wrapper, +) []catalog.Index { + if *cached == nil { + *cached = factory(c, desc) + if *cached == nil { + *cached = []catalog.Index{} + } + } + if len(*cached) == 0 { + return nil + } + return *cached +} + +// buildPublicNonPrimary builds a fresh return value for +// desc.PublicNonPrimaryIndexes(). +func buildPublicNonPrimary(_ *indexCache, desc *wrapper) []catalog.Index { + s := make([]catalog.Index, len(desc.Indexes)) + for i := range s { + s[i] = index{desc: &desc.Indexes[i], ordinal: i + 1} + } + return s +} + +func (c *indexCache) publicNonPrimaryIndexes(desc *wrapper) []catalog.Index { + return c.cachedIndexes(&c.publicNonPrimary, buildPublicNonPrimary, desc) +} + +// buildActive builds fresh return value for desc.ActiveIndexes(). +func buildActive(c *indexCache, desc *wrapper) []catalog.Index { + publicNonPrimary := c.publicNonPrimaryIndexes(desc) + s := make([]catalog.Index, 1, 1+len(publicNonPrimary)) + s[0] = index{desc: &desc.PrimaryIndex} + return append(s, publicNonPrimary...) +} + +func (c *indexCache) activeIndexes(desc *wrapper) []catalog.Index { + return c.cachedIndexes(&c.active, buildActive, desc) +} + +// buildAll builds fresh return value for desc.AllIndexes(). +func buildAll(c *indexCache, desc *wrapper) []catalog.Index { + s := make([]catalog.Index, 0, 1+len(desc.Indexes)+len(desc.Mutations)) + s = append(s, c.activeIndexes(desc)...) + for _, m := range desc.Mutations { + if idxDesc := m.GetIndex(); idxDesc != nil { + idx := index{ + desc: idxDesc, + ordinal: len(s), + mutationID: m.MutationID, + mutationState: m.State, + mutationDirection: m.Direction, + } + s = append(s, idx) + } + } + return s +} + +func (c *indexCache) allIndexes(desc *wrapper) []catalog.Index { + return c.cachedIndexes(&c.all, buildAll, desc) +} + +// buildDeletableNonPrimary builds fresh return value for +// desc.DeletableNonPrimaryIndexes(). +func buildDeletableNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { + return c.allIndexes(desc)[1:] +} + +func (c *indexCache) deletableNonPrimaryIndexes(desc *wrapper) []catalog.Index { + return c.cachedIndexes(&c.deletableNonPrimary, buildDeletableNonPrimary, desc) +} + +// buildWritableNonPrimary builds fresh return value for +// desc.WritableNonPrimaryIndexes(). +func buildWritableNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { + deletableNonPrimary := c.deletableNonPrimaryIndexes(desc) + s := make([]catalog.Index, 0, len(deletableNonPrimary)) + for _, idx := range deletableNonPrimary { + if idx.Public() || idx.WriteAndDeleteOnly() { + s = append(s, idx) + } + } + return s +} + +func (c *indexCache) writableNonPrimaryIndexes(desc *wrapper) []catalog.Index { + return c.cachedIndexes(&c.writableNonPrimary, buildWritableNonPrimary, desc) +} + +// buildDeleteOnlyNonPrimary builds fresh return value for +// desc.DeleteOnlyNonPrimaryIndexes(). +func buildDeleteOnlyNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { + deletableNonPublic := c.deletableNonPrimaryIndexes(desc)[len(desc.Indexes):] + s := make([]catalog.Index, 0, len(deletableNonPublic)) + for _, idx := range deletableNonPublic { + if idx.DeleteOnly() { + s = append(s, idx) + } + } + return s +} + +func (c *indexCache) deleteOnlyNonPrimaryIndexes(desc *wrapper) []catalog.Index { + return c.cachedIndexes(&c.deleteOnlyNonPrimary, buildDeleteOnlyNonPrimary, desc) +} + +// buildNonDrop builds fresh return value for desc.NonDropIndexes(). +func buildNonDrop(c *indexCache, desc *wrapper) []catalog.Index { + all := c.allIndexes(desc) + s := make([]catalog.Index, 0, len(all)) + for _, idx := range all { + if !idx.Dropped() && (!idx.Primary() || desc.IsPhysicalTable()) { + s = append(s, idx) + } + } + return s +} + +func (c *indexCache) nonDropIndexes(desc *wrapper) []catalog.Index { + return c.cachedIndexes(&c.nonDrop, buildNonDrop, desc) +} + +// buildPartial builds fresh return value for desc.PartialIndexes(). +func buildPartial(c *indexCache, desc *wrapper) []catalog.Index { + deletableNonPrimary := c.deletableNonPrimaryIndexes(desc) + s := make([]catalog.Index, 0, len(deletableNonPrimary)) + for _, idx := range deletableNonPrimary { + if idx.IsPartial() { + s = append(s, idx) + } + } + return s +} + +func (c *indexCache) partialIndexes(desc *wrapper) []catalog.Index { + return c.cachedIndexes(&c.partial, buildPartial, desc) +} diff --git a/pkg/sql/catalog/tabledesc/safe_format.go b/pkg/sql/catalog/tabledesc/safe_format.go index ee54864a27a6..499205430a54 100644 --- a/pkg/sql/catalog/tabledesc/safe_format.go +++ b/pkg/sql/catalog/tabledesc/safe_format.go @@ -110,11 +110,13 @@ func formatSafeTableIndexes(w *redact.StringBuilder, desc catalog.TableDescripto w.Printf(", PrimaryIndex: %d", desc.GetPrimaryIndexID()) w.Printf(", NextIndexID: %d", desc.TableDesc().NextIndexID) w.Printf(", Indexes: [") - formatSafeIndex(w, desc.GetPrimaryIndex(), nil) - for i := range desc.GetPublicNonPrimaryIndexes() { - w.Printf(", ") - formatSafeIndex(w, &desc.GetPublicNonPrimaryIndexes()[i], nil) - } + _ = desc.ForEachActiveIndex(func(idx catalog.Index) error { + if !idx.Primary() { + w.Printf(", ") + } + formatSafeIndex(w, idx.IndexDesc(), nil) + return nil + }) w.Printf("]") } diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 0bf4321506dc..a14538428026 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -98,31 +98,27 @@ func NewFilledInExistingMutable( skipFKsWithMissingTable bool, tbl *descpb.TableDescriptor, ) (*Mutable, error) { - desc, err := makeFilledInImmutable(ctx, dg, tbl, skipFKsWithMissingTable) + changes, err := maybeFillInDescriptor(ctx, dg, tbl, skipFKsWithMissingTable) if err != nil { return nil, err } - m := &Mutable{wrapper: desc.wrapper} - m.ClusterVersion = *tbl - return m, nil + w := wrapper{TableDescriptor: *tbl, postDeserializationChanges: changes} + return &Mutable{wrapper: w, ClusterVersion: *tbl}, nil } // MakeImmutable returns an Immutable from the given TableDescriptor. func MakeImmutable(tbl descpb.TableDescriptor) Immutable { publicAndNonPublicCols := tbl.Columns - publicAndNonPublicIndexes := tbl.Indexes readableCols := tbl.Columns - desc := Immutable{wrapper: wrapper{TableDescriptor: tbl}} + desc := Immutable{wrapper: wrapper{TableDescriptor: tbl, indexCache: &indexCache{}}} if len(tbl.Mutations) > 0 { publicAndNonPublicCols = make([]descpb.ColumnDescriptor, 0, len(tbl.Columns)+len(tbl.Mutations)) - publicAndNonPublicIndexes = make([]descpb.IndexDescriptor, 0, len(tbl.Indexes)+len(tbl.Mutations)) readableCols = make([]descpb.ColumnDescriptor, 0, len(tbl.Columns)+len(tbl.Mutations)) publicAndNonPublicCols = append(publicAndNonPublicCols, tbl.Columns...) - publicAndNonPublicIndexes = append(publicAndNonPublicIndexes, tbl.Indexes...) readableCols = append(readableCols, tbl.Columns...) // Fill up mutations into the column/index lists by placing the writable columns/indexes @@ -130,10 +126,7 @@ func MakeImmutable(tbl descpb.TableDescriptor) Immutable { for _, m := range tbl.Mutations { switch m.State { case descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY: - if idx := m.GetIndex(); idx != nil { - publicAndNonPublicIndexes = append(publicAndNonPublicIndexes, *idx) - desc.writeOnlyIndexCount++ - } else if col := m.GetColumn(); col != nil { + if col := m.GetColumn(); col != nil { publicAndNonPublicCols = append(publicAndNonPublicCols, *col) desc.writeOnlyColCount++ } @@ -143,9 +136,7 @@ func MakeImmutable(tbl descpb.TableDescriptor) Immutable { for _, m := range tbl.Mutations { switch m.State { case descpb.DescriptorMutation_DELETE_ONLY: - if idx := m.GetIndex(); idx != nil { - publicAndNonPublicIndexes = append(publicAndNonPublicIndexes, *idx) - } else if col := m.GetColumn(); col != nil { + if col := m.GetColumn(); col != nil { publicAndNonPublicCols = append(publicAndNonPublicCols, *col) } } @@ -162,20 +153,12 @@ func MakeImmutable(tbl descpb.TableDescriptor) Immutable { desc.readableColumns = readableCols desc.publicAndNonPublicCols = publicAndNonPublicCols - desc.publicAndNonPublicIndexes = publicAndNonPublicIndexes desc.allChecks = make([]descpb.TableDescriptor_CheckConstraint, len(tbl.Checks)) for i, c := range tbl.Checks { desc.allChecks[i] = *c } - // Track partial index ordinals. - for i := range publicAndNonPublicIndexes { - if publicAndNonPublicIndexes[i].IsPartial() { - desc.partialIndexOrds.Add(i) - } - } - // Remember what columns have user defined types. for i := range desc.publicAndNonPublicCols { typ := desc.publicAndNonPublicCols[i].Type @@ -211,10 +194,12 @@ func NewImmutableWithIsUncommittedVersion( func NewFilledInImmutable( ctx context.Context, dg catalog.DescGetter, tbl *descpb.TableDescriptor, ) (*Immutable, error) { - desc, err := makeFilledInImmutable(ctx, dg, tbl, false /* skipFKsWithMissingTable */) + changes, err := maybeFillInDescriptor(ctx, dg, tbl, false /* skipFKsWithNoMatchingTable */) if err != nil { return nil, err } + desc := MakeImmutable(*tbl) + desc.postDeserializationChanges = changes return &desc, nil } @@ -236,21 +221,6 @@ type PostDeserializationTableDescriptorChanges struct { UpgradedForeignKeyRepresentation bool } -func makeFilledInImmutable( - ctx context.Context, - dg catalog.DescGetter, - tbl *descpb.TableDescriptor, - skipFKsWithMissingTable bool, -) (Immutable, error) { - changes, err := maybeFillInDescriptor(ctx, dg, tbl, skipFKsWithMissingTable) - if err != nil { - return Immutable{}, err - } - desc := MakeImmutable(*tbl) - desc.postDeserializationChanges = changes - return desc, nil -} - // 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( @@ -271,6 +241,7 @@ func (desc *Mutable) SetName(name string) { // GetPrimaryIndex returns a pointer to the primary index of the table // descriptor. +// This method is deprecated, use PrimaryIndexInterface instead. func (desc *wrapper) GetPrimaryIndex() *descpb.IndexDescriptor { return &desc.PrimaryIndex } @@ -300,27 +271,13 @@ func (desc *wrapper) IsPhysicalTable() bool { // FindIndexByID finds an index (active or inactive) with the specified ID. // Must return a pointer to the IndexDescriptor in the TableDescriptor, so that // callers can use returned values to modify the TableDesc. +// This method is deprecated, use FindIndexWithID instead. func (desc *wrapper) FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) { - if desc.PrimaryIndex.ID == id { - return &desc.PrimaryIndex, nil - } - for i := range desc.Indexes { - idx := &desc.Indexes[i] - if idx.ID == id { - return idx, nil - } - } - for _, m := range desc.Mutations { - if idx := m.GetIndex(); idx != nil && idx.ID == id { - return idx, nil - } - } - for _, m := range desc.GCMutations { - if m.IndexID == id { - return nil, ErrIndexGCMutationsList - } + idx, err := desc.FindIndexWithID(id) + if err != nil { + return nil, err } - return nil, fmt.Errorf("index-id \"%d\" does not exist", id) + return idx.IndexDesc(), nil } // KeysPerRow returns the maximum number of keys used to encode a row for the @@ -356,10 +313,11 @@ func (desc *wrapper) AllNonDropColumns() []descpb.ColumnDescriptor { return cols } -// allocateName sets desc.Name to a value that is not EqualName to any +// buildIndexName sets desc.Name to a value that is not EqualName to any // of tableDesc's indexes. allocateName roughly follows PostgreSQL's // convention for automatically-named indexes. -func allocateIndexName(tableDesc *Mutable, idx *descpb.IndexDescriptor) { +func buildIndexName(tableDesc *Mutable, index catalog.Index) string { + idx := index.IndexDesc() segments := make([]string, 0, len(idx.ColumnNames)+2) segments = append(segments, tableDesc.Name) segments = append(segments, idx.ColumnNames...) @@ -371,34 +329,25 @@ func allocateIndexName(tableDesc *Mutable, idx *descpb.IndexDescriptor) { baseName := strings.Join(segments, "_") name := baseName - - exists := func(name string) bool { - _, _, err := tableDesc.FindIndexByName(name) - return err == nil - } - for i := 1; exists(name); i++ { + for i := 1; ; i++ { + foundIndex, _ := tableDesc.FindIndexWithName(name) + if foundIndex == nil { + break + } name = fmt.Sprintf("%s%d", baseName, i) } - idx.Name = name + return name } // AllNonDropIndexes returns all the indexes, including those being added // in the mutations. +// This method is deprecated, use NonDropIndexes instead. func (desc *wrapper) AllNonDropIndexes() []*descpb.IndexDescriptor { - indexes := make([]*descpb.IndexDescriptor, 0, 1+len(desc.Indexes)+len(desc.Mutations)) - if desc.IsPhysicalTable() { - indexes = append(indexes, &desc.PrimaryIndex) - } - for i := range desc.Indexes { - indexes = append(indexes, &desc.Indexes[i]) - } - for _, m := range desc.Mutations { - if idx := m.GetIndex(); idx != nil { - if m.Direction == descpb.DescriptorMutation_ADD { - indexes = append(indexes, idx) - } - } + nonDropIndexes := desc.NonDropIndexes() + indexes := make([]*descpb.IndexDescriptor, len(nonDropIndexes)) + for i, idx := range nonDropIndexes { + indexes[i] = idx.IndexDesc() } return indexes } @@ -529,44 +478,21 @@ func (desc *wrapper) ForeachNonDropColumn(f func(column *descpb.ColumnDescriptor // ForeachNonDropIndex runs a function on all indexes, including those being // added in the mutations. +// This method is deprecated, use ForEachNonDropIndex instead. func (desc *wrapper) ForeachNonDropIndex(f func(*descpb.IndexDescriptor) error) error { - if err := desc.ForeachIndex(catalog.IndexOpts{AddMutations: true}, func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error { - return f(idxDesc) - }); err != nil { - return err - } - return nil + return desc.ForEachNonDropIndex(func(idx catalog.Index) error { + return f(idx.IndexDesc()) + }) } // ForeachIndex runs a function on the set of indexes as specified by opts. +// This method is deprecated, use ForEachIndex instead. func (desc *wrapper) ForeachIndex( opts catalog.IndexOpts, f func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error, ) error { - if desc.IsPhysicalTable() || opts.NonPhysicalPrimaryIndex { - if err := f(&desc.PrimaryIndex, true /* isPrimary */); err != nil { - return err - } - } - for i := range desc.Indexes { - if err := f(&desc.Indexes[i], false /* isPrimary */); err != nil { - return err - } - } - if !opts.AddMutations && !opts.DropMutations { - return nil - } - for _, m := range desc.Mutations { - idx := m.GetIndex() - if idx == nil || - (m.Direction == descpb.DescriptorMutation_ADD && !opts.AddMutations) || - (m.Direction == descpb.DescriptorMutation_DROP && !opts.DropMutations) { - continue - } - if err := f(idx, false /* isPrimary */); err != nil { - return err - } - } - return nil + return desc.ForEachIndex(opts, func(idx catalog.Index) error { + return f(idx.IndexDesc(), idx.Primary()) + }) } // ForeachDependedOnBy runs a function on all indexes, including those being @@ -952,9 +878,9 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } return nil } - doIndex := func(i *descpb.IndexDescriptor) error { + doIndex := func(i catalog.Index) error { if i.IsPartial() { - return f(&i.Predicate) + return f(&i.IndexDesc().Predicate) } return nil } @@ -969,15 +895,14 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } } - // Process indexes. - if err := doIndex(&desc.PrimaryIndex); err != nil { + // Process all indexes. + if err := descI.ForEachIndex(catalog.IndexOpts{ + NonPhysicalPrimaryIndex: true, + DropMutations: true, + AddMutations: true, + }, doIndex); err != nil { return err } - for i := range desc.Indexes { - if err := doIndex(&desc.Indexes[i]); err != nil { - return err - } - } // Process checks. for i := range desc.Checks { @@ -986,18 +911,13 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } } - // Process all mutations. + // Process all non-index mutations. for _, mut := range desc.GetMutations() { if c := mut.GetColumn(); c != nil { if err := doCol(c); err != nil { return err } } - if i := mut.GetIndex(); i != nil { - if err := doIndex(i); err != nil { - return err - } - } if c := mut.GetConstraint(); c != nil && c.ConstraintType == descpb.ConstraintToUpdate_CHECK { if err := doCheck(&c.Check); err != nil { @@ -1176,30 +1096,13 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er desc.NextIndexID = 1 } - // Keep track of unnamed indexes. - anonymousIndexes := make([]*descpb.IndexDescriptor, 0, len(desc.Indexes)+len(desc.Mutations)) - - // Create a slice of modifiable index descriptors. - indexes := make([]*descpb.IndexDescriptor, 0, 1+len(desc.Indexes)+len(desc.Mutations)) - indexes = append(indexes, &desc.PrimaryIndex) - collectIndexes := func(index *descpb.IndexDescriptor) { - if len(index.Name) == 0 { - anonymousIndexes = append(anonymousIndexes, index) - } - indexes = append(indexes, index) - } - for i := range desc.Indexes { - collectIndexes(&desc.Indexes[i]) - } - for _, m := range desc.Mutations { - if index := m.GetIndex(); index != nil { - collectIndexes(index) + // Assign names to unnamed indexes. + _ = desc.ForEachDeletableNonPrimaryIndex(func(idx catalog.Index) error { + if len(idx.GetName()) == 0 { + idx.IndexDesc().Name = buildIndexName(desc, idx) } - } - - for _, index := range anonymousIndexes { - allocateIndexName(desc, index) - } + return nil + }) var compositeColIDs catalog.TableColSet for i := range desc.Columns { @@ -1210,11 +1113,12 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er } // Populate IDs. - for _, index := range indexes { - if index.ID != 0 { + for _, idx := range desc.AllIndexes() { + if idx.GetID() != 0 { // This index has already been populated. Nothing to do. continue } + index := idx.IndexDesc() index.ID = desc.NextIndexID desc.NextIndexID++ @@ -1227,7 +1131,7 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er } } - if index != &desc.PrimaryIndex && index.EncodingType == descpb.SecondaryIndexEncoding { + if !idx.Primary() && index.EncodingType == descpb.SecondaryIndexEncoding { indexHasOldStoredColumns := index.HasOldStoredColumns() // Need to clear ExtraColumnIDs and StoreColumnIDs because they are used // by ContainsColumnID. @@ -2635,18 +2539,6 @@ func (ps partitionInterval) Range() interval.Range { return interval.Range{Start: []byte(ps.start), End: []byte(ps.end)} } -// FindIndexesWithPartition returns all IndexDescriptors (potentially including -// the primary index) which have a partition with the given name. -func (desc *wrapper) FindIndexesWithPartition(name string) []*descpb.IndexDescriptor { - var indexes []*descpb.IndexDescriptor - for _, idx := range desc.AllNonDropIndexes() { - if FindIndexPartitionByName(idx, name) != nil { - indexes = append(indexes, idx) - } - } - return indexes -} - // validatePartitioning validates that any PartitioningDescriptors contained in // table indexes are well-formed. See validatePartitioningDesc for details. func (desc *wrapper) validatePartitioning() error { @@ -2842,27 +2734,19 @@ func (desc *Mutable) RenameColumnDescriptor(column *descpb.ColumnDescriptor, new } } - renameColumnInIndex := func(idx *descpb.IndexDescriptor) { - for i, id := range idx.ColumnIDs { + for _, idx := range desc.AllIndexes() { + idxDesc := idx.IndexDesc() + for i, id := range idxDesc.ColumnIDs { if id == colID { - idx.ColumnNames[i] = newColName + idxDesc.ColumnNames[i] = newColName } } - for i, id := range idx.StoreColumnIDs { + for i, id := range idxDesc.StoreColumnIDs { if id == colID { - idx.StoreColumnNames[i] = newColName + idxDesc.StoreColumnNames[i] = newColName } } } - renameColumnInIndex(&desc.PrimaryIndex) - for i := range desc.Indexes { - renameColumnInIndex(&desc.Indexes[i]) - } - for _, m := range desc.Mutations { - if idx := m.GetIndex(); idx != nil { - renameColumnInIndex(idx) - } - } } // FindActiveColumnsByNames finds all requested columns (in the requested order) @@ -3021,6 +2905,14 @@ func (desc *wrapper) ContainsUserDefinedTypes() bool { return len(desc.GetColumnOrdinalsWithUserDefinedTypes()) > 0 } +// ContainsUserDefinedTypes returns whether or not this table descriptor has +// any columns of user defined types. +// This method is re-implemented for Immutable only for the purpose of calling +// the correct GetColumnOrdinalsWithUserDefinedTypes() method on desc. +func (desc *Immutable) ContainsUserDefinedTypes() bool { + return len(desc.GetColumnOrdinalsWithUserDefinedTypes()) > 0 +} + // GetColumnOrdinalsWithUserDefinedTypes returns a slice of column ordinals // of columns that contain user defined types. func (desc *Immutable) GetColumnOrdinalsWithUserDefinedTypes() []int { @@ -3043,6 +2935,25 @@ func (desc *wrapper) UserDefinedTypeColsHaveSameVersion(otherDesc catalog.TableD return true } +// UserDefinedTypeColsHaveSameVersion returns whether this 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 descriptors +// representing the same table at the same version. +// This method is re-implemented for Immutable only for the purpose of calling +// the correct DeletableColumns() and GetColumnOrdinalsWithUserDefinedTypes() +// methods on desc. +func (desc *Immutable) UserDefinedTypeColsHaveSameVersion(otherDesc catalog.TableDescriptor) bool { + thisCols := desc.DeletableColumns() + otherCols := otherDesc.DeletableColumns() + for _, idx := range desc.GetColumnOrdinalsWithUserDefinedTypes() { + this, other := thisCols[idx].Type, otherCols[idx].Type + if this.TypeMeta.Version != other.TypeMeta.Version { + return false + } + } + return true +} + // FindReadableColumnByID finds the readable column with specified ID. The // column may be undergoing a schema change and is marked nullable regardless // of its configuration. It returns true if the column is undergoing a @@ -3071,26 +2982,15 @@ func (desc *wrapper) FindFamilyByID(id descpb.FamilyID) (*descpb.ColumnFamilyDes // FindIndexByName finds the index with the specified name in the active // list or the mutations list. It returns true if the index is being dropped. +// This method is deprecated, use FindIndexWithName instead. func (desc *wrapper) FindIndexByName( name string, ) (_ *descpb.IndexDescriptor, dropped bool, _ error) { - if desc.IsPhysicalTable() && desc.PrimaryIndex.Name == name { - return &desc.PrimaryIndex, false, nil - } - for i := range desc.Indexes { - idx := &desc.Indexes[i] - if idx.Name == name { - return idx, false, nil - } - } - for _, m := range desc.Mutations { - if idx := m.GetIndex(); idx != nil { - if idx.Name == name { - return idx, m.Direction == descpb.DescriptorMutation_DROP, nil - } - } + idx, err := desc.FindIndexWithName(name) + if err != nil { + return nil, false, err } - return nil, false, fmt.Errorf("index %q does not exist", name) + return idx.IndexDesc(), idx.Dropped(), nil } // NamesForColumnIDs returns the names for the given column ids, or an error @@ -3339,21 +3239,6 @@ func (desc *Mutable) RenameConstraint( } } -// FindActiveIndexByID returns the index with the specified ID, or nil if it -// does not exist. It only searches active indexes. -func (desc *wrapper) FindActiveIndexByID(id descpb.IndexID) *descpb.IndexDescriptor { - if desc.PrimaryIndex.ID == id { - return &desc.PrimaryIndex - } - for i := range desc.Indexes { - idx := &desc.Indexes[i] - if idx.ID == id { - return idx - } - } - return nil -} - // FindIndexByIndexIdx returns an active index with the specified // index's index which has a domain of [0, # of secondary indexes] and whether // the index is a secondary index. @@ -4256,27 +4141,6 @@ func (desc *Immutable) MutationColumns() []descpb.ColumnDescriptor { return desc.publicAndNonPublicCols[len(desc.Columns):] } -// WritableIndexes returns a list of public and write-only mutation indexes. -func (desc *Immutable) WritableIndexes() []descpb.IndexDescriptor { - return desc.publicAndNonPublicIndexes[:len(desc.Indexes)+desc.writeOnlyIndexCount] -} - -// DeletableIndexes returns a list of public and non-public indexes. -func (desc *Immutable) DeletableIndexes() []descpb.IndexDescriptor { - return desc.publicAndNonPublicIndexes -} - -// DeleteOnlyIndexes returns a list of delete-only mutation indexes. -func (desc *Immutable) DeleteOnlyIndexes() []descpb.IndexDescriptor { - return desc.publicAndNonPublicIndexes[len(desc.Indexes)+desc.writeOnlyIndexCount:] -} - -// PartialIndexOrds returns a set containing the ordinal of each partial index -// defined on the table. -func (desc *Immutable) PartialIndexOrds() util.FastIntSet { - return desc.partialIndexOrds -} - // 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 { diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index ee4498e38e1a..fd52c0bc4116 100644 --- a/pkg/sql/catalog/tabledesc/table.go +++ b/pkg/sql/catalog/tabledesc/table.go @@ -344,11 +344,9 @@ func FindFKReferencedIndex( return primaryIndex, nil } // If the PK doesn't match, find the index corresponding to the referenced column. - indexes := referencedTable.GetPublicNonPrimaryIndexes() - for i := range indexes { - idx := &indexes[i] + for _, idx := range referencedTable.PublicNonPrimaryIndexes() { if idx.IsValidReferencedIndex(referencedColIDs) { - return idx, nil + return idx.IndexDesc(), nil } } return nil, pgerror.Newf( @@ -369,11 +367,9 @@ func FindFKOriginIndex( return primaryIndex, nil } // If the PK doesn't match, find the index corresponding to the origin column. - indexes := originTable.GetPublicNonPrimaryIndexes() - for i := range indexes { - idx := &indexes[i] + for _, idx := range originTable.PublicNonPrimaryIndexes() { if idx.IsValidOriginIndex(originColIDs) { - return idx, nil + return idx.IndexDesc(), nil } } return nil, pgerror.Newf( diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index 24a774723813..4465da7a0d57 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -14,8 +14,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" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" ) var _ catalog.TableDescriptor = (*Immutable)(nil) @@ -27,6 +27,12 @@ var _ catalog.TableDescriptor = (*wrapper)(nil) // interface, which is overloaded by Immutable and Mutable. type wrapper struct { descpb.TableDescriptor + + // indexCache, when not nil, points to a struct containing precomputed + // catalog.Index slices. This can therefore only be set when creating an + // Immutable. + indexCache *indexCache + postDeserializationChanges PostDeserializationTableDescriptorChanges } @@ -44,18 +50,6 @@ func (desc *wrapper) GetPostDeserializationChanges() PostDeserializationTableDes return desc.postDeserializationChanges } -// PartialIndexOrds returns a set containing the ordinal of each partial index -// defined on the table. -func (desc *wrapper) PartialIndexOrds() util.FastIntSet { - var partialIndexOrds util.FastIntSet - for i, idx := range desc.DeletableIndexes() { - if idx.IsPartial() { - partialIndexOrds.Add(i) - } - } - return partialIndexOrds -} - // mutationIndexes returns all non-public indexes in the specified state. func (desc *wrapper) mutationIndexes( mutationState descpb.DescriptorMutation_State, @@ -76,11 +70,13 @@ func (desc *wrapper) mutationIndexes( } // DeleteOnlyIndexes returns a list of delete-only mutation indexes. +// This method is deprecated, use DeleteOnlyNonPrimaryIndexes instead. func (desc *wrapper) DeleteOnlyIndexes() []descpb.IndexDescriptor { return desc.mutationIndexes(descpb.DescriptorMutation_DELETE_ONLY) } // WritableIndexes returns a list of public and write-only mutation indexes. +// This method is deprecated, use WritableNonPrimaryIndexes instead. func (desc *wrapper) WritableIndexes() []descpb.IndexDescriptor { if len(desc.Mutations) == 0 { return desc.Indexes @@ -93,7 +89,8 @@ func (desc *wrapper) WritableIndexes() []descpb.IndexDescriptor { return indexes } -// DeletableIndexes implements the catalog.Descriptor interface. +// DeletableIndexes returns a list of deletable indexes. +// This method is deprecated, use DeletableNonPrimaryIndexes instead. func (desc *wrapper) DeletableIndexes() []descpb.IndexDescriptor { if len(desc.Mutations) == 0 { return desc.Indexes @@ -106,7 +103,8 @@ func (desc *wrapper) DeletableIndexes() []descpb.IndexDescriptor { return indexes } -// mutationColumns returns all non-public writable columns in the specified state. +// mutationColumns returns all non-public writable columns in the specified +// state. func (desc *wrapper) mutationColumns( mutationState descpb.DescriptorMutation_State, ) []descpb.ColumnDescriptor { @@ -197,18 +195,10 @@ type Immutable struct { // It is partitioned by the state of the column: public, write-only, delete-only publicAndNonPublicCols []descpb.ColumnDescriptor - // publicAndNonPublicCols is a list of public and non-public indexes. - // It is partitioned by the state of the index: public, write-only, delete-only - publicAndNonPublicIndexes []descpb.IndexDescriptor - - writeOnlyColCount int - writeOnlyIndexCount int + writeOnlyColCount int allChecks []descpb.TableDescriptor_CheckConstraint - // partialIndexOrds contains the ordinal of each partial index. - partialIndexOrds util.FastIntSet - // readableColumns is a list of columns (including those undergoing a schema change) // which can be scanned. Columns in the process of a schema change // are all set to nullable while column backfilling is still in @@ -245,7 +235,9 @@ func (desc *wrapper) GetPrimaryIndexID() descpb.IndexID { return desc.PrimaryIndex.ID } -// GetPublicNonPrimaryIndexes returns the public non-primary indexes of the descriptor. +// GetPublicNonPrimaryIndexes returns the public non-primary indexes of the +// descriptor. +// This method is deprecated, use PublicNonPrimaryIndexes instead. func (desc *wrapper) GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor { return desc.Indexes } @@ -265,7 +257,8 @@ func (desc *wrapper) GetColumnAtIdx(idx int) *descpb.ColumnDescriptor { return &desc.Columns[idx] } -// ReadableColumns implements the catalog.TableDescriptor interface +// ReadableColumns returns a list of columns (including those undergoing a +// schema change) which can be scanned. func (desc *Immutable) ReadableColumns() []descpb.ColumnDescriptor { return desc.readableColumns } @@ -318,3 +311,242 @@ func (desc *Mutable) SetPrimaryIndex(index descpb.IndexDescriptor) { func (desc *Mutable) SetPublicNonPrimaryIndex(indexOrdinal int, index descpb.IndexDescriptor) { desc.Indexes[indexOrdinal-1] = index } + +// PrimaryIndexInterface returns the primary index in the form of a +// catalog.Index interface. +func (desc *wrapper) PrimaryIndexInterface() catalog.Index { + return index{desc: &desc.PrimaryIndex} +} + +// getExistingOrNewIndexCache should be the only place where the indexCache +// field in wrapper is ever read. +func (desc *wrapper) getExistingOrNewIndexCache() *indexCache { + if desc.indexCache == nil { + return &indexCache{} + } + return desc.indexCache +} + +// AllIndexes returns a slice containing all indexes represented in the table +// descriptor, including mutations. +func (desc *wrapper) AllIndexes() []catalog.Index { + return desc.getExistingOrNewIndexCache().allIndexes(desc) +} + +// ActiveIndexes returns a slice of all active (aka public) indexes. +func (desc *wrapper) ActiveIndexes() []catalog.Index { + return desc.getExistingOrNewIndexCache().activeIndexes(desc) +} + +// NonDropIndexes returns a slice of all indexes (including mutations) which are +// not being dropped. +func (desc *wrapper) NonDropIndexes() []catalog.Index { + return desc.getExistingOrNewIndexCache().nonDropIndexes(desc) +} + +// PartialIndexes returns a slice of all partial indexes in the table +// descriptor, including mutations. +func (desc *wrapper) PartialIndexes() []catalog.Index { + return desc.getExistingOrNewIndexCache().partialIndexes(desc) +} + +// PublicNonPrimaryIndexes returns a slice of all active secondary indexes. +func (desc *wrapper) PublicNonPrimaryIndexes() []catalog.Index { + return desc.getExistingOrNewIndexCache().publicNonPrimaryIndexes(desc) +} + +// WritableNonPrimaryIndexes returns a slice of all secondary indexes which +// allow being written to: active + delete-and-write-only. +func (desc *wrapper) WritableNonPrimaryIndexes() []catalog.Index { + return desc.getExistingOrNewIndexCache().writableNonPrimaryIndexes(desc) +} + +// DeletableNonPrimaryIndexes returns a slice of all secondary indexes which +// allow being deleted from: active + delete-and-write-only + delete-only. +func (desc *wrapper) DeletableNonPrimaryIndexes() []catalog.Index { + return desc.getExistingOrNewIndexCache().deletableNonPrimaryIndexes(desc) +} + +// DeletableNonPrimaryIndexes returns a slice of all secondary indexes which +// only allow being deleted from. +func (desc *wrapper) DeleteOnlyNonPrimaryIndexes() []catalog.Index { + return desc.getExistingOrNewIndexCache().deleteOnlyNonPrimaryIndexes(desc) +} + +// ForEachIndex runs f over each index in the table descriptor according to +// filter parameters in opts. +func (desc *wrapper) ForEachIndex(opts catalog.IndexOpts, f func(idx catalog.Index) error) error { + for _, idx := range desc.AllIndexes() { + if !opts.NonPhysicalPrimaryIndex && idx.Primary() && !desc.IsPhysicalTable() { + continue + } + if !opts.AddMutations && idx.Adding() { + continue + } + if !opts.DropMutations && idx.Dropped() { + continue + } + if err := f(idx); err != nil { + return err + } + } + return nil +} + +func forEachIndex(slice []catalog.Index, f func(idx catalog.Index) error) error { + for _, idx := range slice { + if err := f(idx); err != nil { + return err + } + } + return nil +} + +// ForEachActiveIndex is like ForEachIndex over ActiveIndexes(). +func (desc *wrapper) ForEachActiveIndex(f func(idx catalog.Index) error) error { + return forEachIndex(desc.ActiveIndexes(), f) +} + +// ForEachNonDropIndex is like ForEachIndex over NonDropIndexes(). +func (desc *wrapper) ForEachNonDropIndex(f func(idx catalog.Index) error) error { + return forEachIndex(desc.NonDropIndexes(), f) +} + +// ForEachPartialIndex is like ForEachIndex over PartialIndexes(). +func (desc *wrapper) ForEachPartialIndex(f func(idx catalog.Index) error) error { + return forEachIndex(desc.PartialIndexes(), f) +} + +// ForEachPublicNonPrimaryIndex is like ForEachIndex over +// PublicNonPrimaryIndexes(). +func (desc *wrapper) ForEachPublicNonPrimaryIndex(f func(idx catalog.Index) error) error { + return forEachIndex(desc.PublicNonPrimaryIndexes(), f) +} + +// ForEachWritableNonPrimaryIndex is like ForEachIndex over +// WritableNonPrimaryIndexes(). +func (desc *wrapper) ForEachWritableNonPrimaryIndex(f func(idx catalog.Index) error) error { + return forEachIndex(desc.WritableNonPrimaryIndexes(), f) +} + +// ForEachDeletableNonPrimaryIndex is like ForEachIndex over +// DeletableNonPrimaryIndexes(). +func (desc *wrapper) ForEachDeletableNonPrimaryIndex(f func(idx catalog.Index) error) error { + return forEachIndex(desc.DeletableNonPrimaryIndexes(), f) +} + +// ForEachDeleteOnlyNonPrimaryIndex is like ForEachIndex over +// DeleteOnlyNonPrimaryIndexes(). +func (desc *wrapper) ForEachDeleteOnlyNonPrimaryIndex(f func(idx catalog.Index) error) error { + return forEachIndex(desc.DeleteOnlyNonPrimaryIndexes(), f) +} + +// FindIndex returns the first index for which test returns true, nil otherwise, +// according to the parameters in opts just like ForEachIndex. +func (desc *wrapper) FindIndex( + opts catalog.IndexOpts, test func(idx catalog.Index) bool, +) catalog.Index { + for _, idx := range desc.AllIndexes() { + if !opts.NonPhysicalPrimaryIndex && idx.Primary() && !desc.IsPhysicalTable() { + continue + } + if !opts.AddMutations && idx.Adding() { + continue + } + if !opts.DropMutations && idx.Dropped() { + continue + } + if test(idx) { + return idx + } + } + return nil +} + +func findIndex(slice []catalog.Index, test func(idx catalog.Index) bool) catalog.Index { + for _, idx := range slice { + if test(idx) { + return idx + } + } + return nil +} + +// FindActiveIndex returns the first index in ActiveIndex() for which test +// returns true. +func (desc *wrapper) FindActiveIndex(test func(idx catalog.Index) bool) catalog.Index { + return findIndex(desc.ActiveIndexes(), test) +} + +// FindNonDropIndex returns the first index in NonDropIndex() for which test +// returns true. +func (desc *wrapper) FindNonDropIndex(test func(idx catalog.Index) bool) catalog.Index { + return findIndex(desc.NonDropIndexes(), test) +} + +// FindPartialIndex returns the first index in PartialIndex() for which test +// returns true. +func (desc *wrapper) FindPartialIndex(test func(idx catalog.Index) bool) catalog.Index { + return findIndex(desc.PartialIndexes(), test) +} + +// FindPublicNonPrimaryIndex returns the first index in PublicNonPrimaryIndex() +// for which test returns true. +func (desc *wrapper) FindPublicNonPrimaryIndex(test func(idx catalog.Index) bool) catalog.Index { + return findIndex(desc.PublicNonPrimaryIndexes(), test) +} + +// FindWritableNonPrimaryIndex returns the first index in +// WritableNonPrimaryIndex() for which test returns true. +func (desc *wrapper) FindWritableNonPrimaryIndex(test func(idx catalog.Index) bool) catalog.Index { + return findIndex(desc.WritableNonPrimaryIndexes(), test) +} + +// FindDeletableNonPrimaryIndex returns the first index in +// DeletableNonPrimaryIndex() for which test returns true. +func (desc *wrapper) FindDeletableNonPrimaryIndex(test func(idx catalog.Index) bool) catalog.Index { + return findIndex(desc.DeletableNonPrimaryIndexes(), test) +} + +// FindDeleteOnlyNonPrimaryIndex returns the first index in +// DeleteOnlyNonPrimaryIndex() for which test returns true. +func (desc *wrapper) FindDeleteOnlyNonPrimaryIndex( + test func(idx catalog.Index) bool, +) catalog.Index { + return findIndex(desc.DeleteOnlyNonPrimaryIndexes(), test) +} + +// FindIndexWithID returns the first catalog.Index that matches the id +// in the set of all indexes. +func (desc *wrapper) FindIndexWithID(id descpb.IndexID) (catalog.Index, error) { + if idx := desc.FindIndex(catalog.IndexOpts{ + NonPhysicalPrimaryIndex: true, + DropMutations: true, + AddMutations: true, + }, func(idx catalog.Index) bool { + return idx.GetID() == id + }); idx != nil { + return idx, nil + } + for _, m := range desc.GCMutations { + if m.IndexID == id { + return nil, ErrIndexGCMutationsList + } + } + return nil, errors.Errorf("index-id \"%d\" does not exist", id) +} + +// FindIndexWithName returns the first catalog.Index that matches the name in +// the set of all indexes, excluding the primary index of non-physical tables. +func (desc *wrapper) FindIndexWithName(name string) (catalog.Index, error) { + if idx := desc.FindIndex(catalog.IndexOpts{ + NonPhysicalPrimaryIndex: false, + DropMutations: true, + AddMutations: true, + }, func(idx catalog.Index) bool { + return idx.GetName() == name + }); idx != nil { + return idx, nil + } + return nil, errors.Errorf("index %q does not exist", name) +} diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index b803e6cb2f4b..375d9bf235dd 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2831,7 +2831,9 @@ CREATE TABLE crdb_internal.zones ( } for i, s := range subzones { - index := table.FindActiveIndexByID(descpb.IndexID(s.IndexID)) + index := table.FindActiveIndex(func(idx catalog.Index) bool { + return idx.GetID() == descpb.IndexID(s.IndexID) + }) if index == nil { // If we can't find an active index that corresponds to this index // ID then continue, as the index is being dropped, or is already @@ -2840,7 +2842,7 @@ CREATE TABLE crdb_internal.zones ( } if zoneSpecifier != nil { zs := zs - zs.TableOrIndex.Index = tree.UnrestrictedName(index.Name) + zs.TableOrIndex.Index = tree.UnrestrictedName(index.GetName()) zs.Partition = tree.Name(s.PartitionName) zoneSpecifier = &zs } @@ -2856,7 +2858,7 @@ CREATE TABLE crdb_internal.zones ( } else { // We have a partition. Get the parent index partition from the zone and // have it inherit constraints. - if indexSubzone := fullZone.GetSubzone(uint32(index.ID), ""); indexSubzone != nil { + if indexSubzone := fullZone.GetSubzone(uint32(index.GetID()), ""); indexSubzone != nil { subZoneConfig.InheritFromParent(&indexSubzone.Config) } // Inherit remaining fields from the full parent zone. diff --git a/pkg/sql/delete.go b/pkg/sql/delete.go index 26a73ba2537d..e51d96794d83 100644 --- a/pkg/sql/delete.go +++ b/pkg/sql/delete.go @@ -155,8 +155,7 @@ func (d *deleteNode) processSourceRow(params runParams, sourceVals tree.Datums) // satisfy the predicate and therefore do not exist in the partial index. // This set is passed as a argument to tableDeleter.row below. var pm row.PartialIndexUpdateHelper - partialIndexOrds := d.run.td.tableDesc().PartialIndexOrds() - if !partialIndexOrds.Empty() { + if len(d.run.td.tableDesc().PartialIndexes()) > 0 { partialIndexDelVals := sourceVals[d.run.partialIndexDelValsOffset:] err := pm.Init(tree.Datums{}, partialIndexDelVals, d.run.td.tableDesc()) diff --git a/pkg/sql/insert.go b/pkg/sql/insert.go index 47b6521502b0..b4b7dc30e654 100644 --- a/pkg/sql/insert.go +++ b/pkg/sql/insert.go @@ -129,8 +129,7 @@ func (r *insertRun) processSourceRow(params runParams, rowVals tree.Datums) erro // written to when they are partial indexes and the row does not satisfy the // predicate. This set is passed as a parameter to tableInserter.row below. var pm row.PartialIndexUpdateHelper - partialIndexOrds := r.ti.tableDesc().PartialIndexOrds() - if !partialIndexOrds.Empty() { + if len(r.ti.tableDesc().PartialIndexes()) > 0 { partialIndexPutVals := rowVals[len(r.insertCols)+r.checkOrds.Len():] err := pm.Init(partialIndexPutVals, tree.Datums{}, r.ti.tableDesc()) diff --git a/pkg/sql/row/partial_index.go b/pkg/sql/row/partial_index.go index 8b57e193b7de..597046e7e6d1 100644 --- a/pkg/sql/row/partial_index.go +++ b/pkg/sql/row/partial_index.go @@ -44,43 +44,38 @@ func (pm *PartialIndexUpdateHelper) Init( partialIndexPutVals tree.Datums, partialIndexDelVals tree.Datums, tabDesc catalog.TableDescriptor, ) error { colIdx := 0 - partialIndexOrds := tabDesc.PartialIndexOrds() - indexes := tabDesc.DeletableIndexes() - for i, ok := partialIndexOrds.Next(0); ok; i, ok = partialIndexOrds.Next(i + 1) { - index := &indexes[i] - if index.IsPartial() { + for _, idx := range tabDesc.PartialIndexes() { - // Check the boolean partial index put column, if it exists. - if colIdx < len(partialIndexPutVals) { - val, err := tree.GetBool(partialIndexPutVals[colIdx]) - if err != nil { - return err - } - if !val { - // If the value of the column for the index predicate - // expression is false, the row should not be added to the - // partial index. - pm.IgnoreForPut.Add(int(index.ID)) - } + // Check the boolean partial index put column, if it exists. + if colIdx < len(partialIndexPutVals) { + val, err := tree.GetBool(partialIndexPutVals[colIdx]) + if err != nil { + return err } - - // Check the boolean partial index del column, if it exists. - if colIdx < len(partialIndexDelVals) { - val, err := tree.GetBool(partialIndexDelVals[colIdx]) - if err != nil { - return err - } - if !val { - // If the value of the column for the index predicate - // expression is false, the row should not be removed from - // the partial index. - pm.IgnoreForDel.Add(int(index.ID)) - } + if !val { + // If the value of the column for the index predicate + // expression is false, the row should not be added to the + // partial index. + pm.IgnoreForPut.Add(int(idx.GetID())) } + } - colIdx++ + // Check the boolean partial index del column, if it exists. + if colIdx < len(partialIndexDelVals) { + val, err := tree.GetBool(partialIndexDelVals[colIdx]) + if err != nil { + return err + } + if !val { + // If the value of the column for the index predicate + // expression is false, the row should not be removed from + // the partial index. + pm.IgnoreForDel.Add(int(idx.GetID())) + } } + + colIdx++ } return nil diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index e2998034aeae..67b08286ac50 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -332,12 +332,19 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // Backward compatibility for ALTER PARTITION ... OF TABLE. Determine which // index has the specified partition. partitionName := string(n.zoneSpecifier.Partition) - indexes := table.FindIndexesWithPartition(partitionName) + + var indexes []catalog.Index + for _, idx := range table.NonDropIndexes() { + if tabledesc.FindIndexPartitionByName(idx.IndexDesc(), partitionName) != nil { + indexes = append(indexes, idx) + } + } + switch len(indexes) { case 0: return fmt.Errorf("partition %q does not exist on table %q", partitionName, table.GetName()) case 1: - n.zoneSpecifier.TableOrIndex.Index = tree.UnrestrictedName(indexes[0].Name) + n.zoneSpecifier.TableOrIndex.Index = tree.UnrestrictedName(indexes[0].GetName()) default: err := fmt.Errorf( "partition %q exists on multiple indexes of table %q", partitionName, table.GetName()) diff --git a/pkg/sql/update.go b/pkg/sql/update.go index 2a6097e253a6..48ea5cf4a3ae 100644 --- a/pkg/sql/update.go +++ b/pkg/sql/update.go @@ -296,12 +296,11 @@ func (u *updateNode) processSourceRow(params runParams, sourceVals tree.Datums) // Create a set of partial index IDs to not add entries or remove entries // from. var pm row.PartialIndexUpdateHelper - partialIndexOrds := u.run.tu.tableDesc().PartialIndexOrds() - if !partialIndexOrds.Empty() { + if n := len(u.run.tu.tableDesc().PartialIndexes()); n > 0 { partialIndexValOffset := len(u.run.tu.ru.FetchCols) + len(u.run.tu.ru.UpdateCols) + u.run.checkOrds.Len() + u.run.numPassthrough partialIndexVals := sourceVals[partialIndexValOffset:] - partialIndexPutVals := partialIndexVals[:partialIndexOrds.Len()] - partialIndexDelVals := partialIndexVals[partialIndexOrds.Len() : partialIndexOrds.Len()*2] + partialIndexPutVals := partialIndexVals[:n] + partialIndexDelVals := partialIndexVals[n : n*2] err := pm.Init(partialIndexPutVals, partialIndexDelVals, u.run.tu.tableDesc()) if err != nil { diff --git a/pkg/sql/upsert.go b/pkg/sql/upsert.go index cc2741e8ec23..7ebef9e6b3f3 100644 --- a/pkg/sql/upsert.go +++ b/pkg/sql/upsert.go @@ -143,8 +143,7 @@ func (n *upsertNode) processSourceRow(params runParams, rowVals tree.Datums) err // Create a set of partial index IDs to not add or remove entries from. var pm row.PartialIndexUpdateHelper - partialIndexOrds := n.run.tw.tableDesc().PartialIndexOrds() - if !partialIndexOrds.Empty() { + if len(n.run.tw.tableDesc().PartialIndexes()) > 0 { partialIndexValOffset := len(n.run.insertCols) + len(n.run.tw.fetchCols) + len(n.run.tw.updateCols) + n.run.checkOrds.Len() if n.run.tw.canaryOrdinal != -1 { partialIndexValOffset++ From 19a71e291809770a8476f9781b09874d07424aef Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 10:26:26 -0500 Subject: [PATCH 02/13] sql: remove deprecated (Writable|Deletable|DeleteOnly)Indexes methods Previously, these methods would be called on a table descriptor interface (or backing struct) to obtain a slice of descpb.IndexDescriptors. This patch removes these calls, along with the method definitions, in favour of new methods which use the catalog.Index interface type instead. Partially addresses #57465. Release note: None --- pkg/ccl/importccl/import_processor.go | 10 ++--- pkg/sql/catalog/descriptor.go | 3 -- pkg/sql/catalog/tabledesc/table_desc.go | 53 ------------------------- pkg/sql/opt_catalog.go | 20 +++++----- pkg/sql/row/deleter.go | 17 +++++--- pkg/sql/row/inserter.go | 8 +++- pkg/sql/row/updater.go | 26 ++++++------ 7 files changed, 46 insertions(+), 91 deletions(-) diff --git a/pkg/ccl/importccl/import_processor.go b/pkg/ccl/importccl/import_processor.go index 4ebe56aa50e3..5bb40aa3fe75 100644 --- a/pkg/ccl/importccl/import_processor.go +++ b/pkg/ccl/importccl/import_processor.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -190,11 +191,10 @@ func makeInputConverter( } if singleTable != nil { - indexes := singleTable.DeletableIndexes() - for _, idx := range indexes { - if idx.IsPartial() { - return nil, unimplemented.NewWithIssue(50225, "cannot import into table with partial indexes") - } + if idx := singleTable.FindDeletableNonPrimaryIndex(func(idx catalog.Index) bool { + return idx.IsPartial() + }); idx != nil { + return nil, unimplemented.NewWithIssue(50225, "cannot import into table with partial indexes") } // If we're using a format like CSV where data columns are not "named", and diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 445e0d3c35b5..4099119d2127 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -123,9 +123,6 @@ type TableDescriptor interface { FindIndexByName(name string) (_ *descpb.IndexDescriptor, dropped bool, _ error) // deprecated GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) KeysPerRow(id descpb.IndexID) (int, error) - WritableIndexes() []descpb.IndexDescriptor // deprecated - DeletableIndexes() []descpb.IndexDescriptor // deprecated - DeleteOnlyIndexes() []descpb.IndexDescriptor // deprecated PrimaryIndexInterface() Index AllIndexes() []Index diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index 4465da7a0d57..4bad6d4f0d28 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -50,59 +50,6 @@ func (desc *wrapper) GetPostDeserializationChanges() PostDeserializationTableDes return desc.postDeserializationChanges } -// mutationIndexes returns all non-public indexes in the specified state. -func (desc *wrapper) mutationIndexes( - mutationState descpb.DescriptorMutation_State, -) []descpb.IndexDescriptor { - if len(desc.Mutations) == 0 { - return nil - } - indexes := make([]descpb.IndexDescriptor, 0, len(desc.Mutations)) - for _, m := range desc.Mutations { - if m.State != mutationState { - continue - } - if idx := m.GetIndex(); idx != nil { - indexes = append(indexes, *idx) - } - } - return indexes -} - -// DeleteOnlyIndexes returns a list of delete-only mutation indexes. -// This method is deprecated, use DeleteOnlyNonPrimaryIndexes instead. -func (desc *wrapper) DeleteOnlyIndexes() []descpb.IndexDescriptor { - return desc.mutationIndexes(descpb.DescriptorMutation_DELETE_ONLY) -} - -// WritableIndexes returns a list of public and write-only mutation indexes. -// This method is deprecated, use WritableNonPrimaryIndexes instead. -func (desc *wrapper) WritableIndexes() []descpb.IndexDescriptor { - if len(desc.Mutations) == 0 { - return desc.Indexes - } - indexes := make([]descpb.IndexDescriptor, 0, len(desc.Indexes)+len(desc.Mutations)) - // Add all public indexes. - indexes = append(indexes, desc.Indexes...) - // Add all non-public writable indexes. - indexes = append(indexes, desc.mutationIndexes(descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY)...) - return indexes -} - -// DeletableIndexes returns a list of deletable indexes. -// This method is deprecated, use DeletableNonPrimaryIndexes instead. -func (desc *wrapper) DeletableIndexes() []descpb.IndexDescriptor { - if len(desc.Mutations) == 0 { - return desc.Indexes - } - indexes := make([]descpb.IndexDescriptor, 0, len(desc.Indexes)+len(desc.Mutations)) - // Add all writable indexes. - indexes = append(indexes, desc.WritableIndexes()...) - // Add all delete-only indexes. - indexes = append(indexes, desc.DeleteOnlyIndexes()...) - return indexes -} - // mutationColumns returns all non-public writable columns in the specified // state. func (desc *wrapper) mutationColumns( diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 823bc26db9ad..2c906387a545 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -629,9 +629,9 @@ func newOptTable( colDescs := ot.desc.DeletableColumns() numCols := len(colDescs) + len(colinfo.AllSystemColumnDescs) // One for each inverted index virtual column. - secondaryIndexes := ot.desc.DeletableIndexes() - for i := range secondaryIndexes { - if secondaryIndexes[i].Type == descpb.IndexDescriptor_INVERTED { + secondaryIndexes := ot.desc.DeletableNonPrimaryIndexes() + for _, index := range secondaryIndexes { + if index.GetType() == descpb.IndexDescriptor_INVERTED { numCols++ } } @@ -721,7 +721,7 @@ func newOptTable( if i == 0 { idxDesc = desc.GetPrimaryIndex() } else { - idxDesc = &secondaryIndexes[i-1] + idxDesc = secondaryIndexes[i-1].IndexDesc() } // If there is a subzone that applies to the entire index, use that, @@ -989,19 +989,19 @@ func (ot *optTable) getColDesc(i int) *descpb.ColumnDescriptor { // IndexCount is part of the cat.Table interface. func (ot *optTable) IndexCount() int { // Primary index is always present, so count is always >= 1. - return 1 + len(ot.desc.GetPublicNonPrimaryIndexes()) + return len(ot.desc.ActiveIndexes()) } // WritableIndexCount is part of the cat.Table interface. func (ot *optTable) WritableIndexCount() int { // Primary index is always present, so count is always >= 1. - return 1 + len(ot.desc.WritableIndexes()) + return 1 + len(ot.desc.WritableNonPrimaryIndexes()) } // DeletableIndexCount is part of the cat.Table interface. func (ot *optTable) DeletableIndexCount() int { // Primary index is always present, so count is always >= 1. - return 1 + len(ot.desc.DeletableIndexes()) + return len(ot.desc.AllIndexes()) } // Index is part of the cat.Table interface. @@ -1812,19 +1812,19 @@ func (ot *optVirtualTable) getColDesc(i int) *descpb.ColumnDescriptor { // IndexCount is part of the cat.Table interface. func (ot *optVirtualTable) IndexCount() int { // Primary index is always present, so count is always >= 1. - return 1 + len(ot.desc.GetPublicNonPrimaryIndexes()) + return len(ot.desc.ActiveIndexes()) } // WritableIndexCount is part of the cat.Table interface. func (ot *optVirtualTable) WritableIndexCount() int { // Primary index is always present, so count is always >= 1. - return 1 + len(ot.desc.WritableIndexes()) + return 1 + len(ot.desc.WritableNonPrimaryIndexes()) } // DeletableIndexCount is part of the cat.Table interface. func (ot *optVirtualTable) DeletableIndexCount() int { // Primary index is always present, so count is always >= 1. - return 1 + len(ot.desc.DeletableIndexes()) + return len(ot.desc.AllIndexes()) } // Index is part of the cat.Table interface. diff --git a/pkg/sql/row/deleter.go b/pkg/sql/row/deleter.go index 039872d08202..8394957d3d1c 100644 --- a/pkg/sql/row/deleter.go +++ b/pkg/sql/row/deleter.go @@ -44,7 +44,11 @@ type Deleter struct { func MakeDeleter( codec keys.SQLCodec, tableDesc *tabledesc.Immutable, requestedCols []descpb.ColumnDescriptor, ) Deleter { - indexes := tableDesc.DeletableIndexes() + indexes := tableDesc.DeletableNonPrimaryIndexes() + indexDescs := make([]descpb.IndexDescriptor, len(indexes)) + for i, index := range indexes { + indexDescs[i] = *index.IndexDesc() + } var fetchCols []descpb.ColumnDescriptor var fetchColIDtoRowIndex catalog.TableColMap @@ -63,19 +67,22 @@ func MakeDeleter( } return nil } - for _, colID := range tableDesc.GetPrimaryIndex().ColumnIDs { + for j := 0; j < tableDesc.PrimaryIndexInterface().NumColumns(); j++ { + colID := tableDesc.PrimaryIndexInterface().GetColumnID(j) if err := maybeAddCol(colID); err != nil { return Deleter{} } } for _, index := range indexes { - for _, colID := range index.ColumnIDs { + for j := 0; j < index.NumColumns(); j++ { + colID := index.GetColumnID(j) if err := maybeAddCol(colID); err != nil { return Deleter{} } } // The extra columns are needed to fix #14601. - for _, colID := range index.ExtraColumnIDs { + for j := 0; j < index.NumExtraColumns(); j++ { + colID := index.GetExtraColumnID(j) if err := maybeAddCol(colID); err != nil { return Deleter{} } @@ -84,7 +91,7 @@ func MakeDeleter( } rd := Deleter{ - Helper: newRowHelper(codec, tableDesc, indexes), + Helper: newRowHelper(codec, tableDesc, indexDescs), FetchCols: fetchCols, FetchColIDtoRowIndex: fetchColIDtoRowIndex, } diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index 695d49a3cefa..c627f3087a1f 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -50,8 +50,14 @@ func MakeInserter( insertCols []descpb.ColumnDescriptor, 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, tableDesc.WritableIndexes()), + Helper: newRowHelper(codec, tableDesc, writableIndexDescs), InsertCols: insertCols, InsertColIDtoRowIndex: ColIDtoRowIndexFromCols(insertCols), marshaled: make([]roachpb.Value, len(insertCols)), diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index ebb451dce3d9..124dde45ba1e 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -112,7 +112,7 @@ func MakeUpdater( // needsUpdate returns true if the given index may need to be updated for // the current UPDATE mutation. - needsUpdate := func(index descpb.IndexDescriptor) bool { + needsUpdate := func(index catalog.Index) bool { // If the UPDATE is set to only update columns and not secondary // indexes, return false. if updateType == UpdaterOnlyColumns { @@ -133,7 +133,7 @@ func MakeUpdater( if index.IsPartial() { return true } - return index.RunOverAllColumns(func(id descpb.ColumnID) error { + return index.ForEachColumnID(func(id descpb.ColumnID) error { if _, ok := updateColIDtoRowIndex.Get(id); ok { return returnTruePseudoError } @@ -141,22 +141,20 @@ func MakeUpdater( }) != nil } - writableIndexes := tableDesc.WritableIndexes() - includeIndexes := make([]descpb.IndexDescriptor, 0, len(writableIndexes)) - for _, index := range writableIndexes { - if needsUpdate(index) { - includeIndexes = append(includeIndexes, index) - } - } - + includeIndexes := make([]descpb.IndexDescriptor, 0, len(tableDesc.WritableNonPrimaryIndexes())) var deleteOnlyIndexes []descpb.IndexDescriptor - for _, idx := range tableDesc.DeleteOnlyIndexes() { - if needsUpdate(idx) { + for _, index := range tableDesc.DeletableNonPrimaryIndexes() { + if !needsUpdate(index) { + continue + } + if !index.DeleteOnly() { + includeIndexes = append(includeIndexes, *index.IndexDesc()) + } else { if deleteOnlyIndexes == nil { // Allocate at most once. - deleteOnlyIndexes = make([]descpb.IndexDescriptor, 0, len(tableDesc.DeleteOnlyIndexes())) + deleteOnlyIndexes = make([]descpb.IndexDescriptor, 0, len(tableDesc.DeleteOnlyNonPrimaryIndexes())) } - deleteOnlyIndexes = append(deleteOnlyIndexes, idx) + deleteOnlyIndexes = append(deleteOnlyIndexes, *index.IndexDesc()) } } From e50fdd717102b86ef1483da93eb16ba9f454283f Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 10:26:33 -0500 Subject: [PATCH 03/13] sql: remove deprecated AllNonDropIndexes method Previously, this method would be called on a table descriptor interface (or backing struct) to obtain a slice of descpb.IndexDescriptor pointers. This patch removes these calls, along with the method definitions, in favour of new methods which use the catalog.Index interface type instead. Partially addresses #57465. Release note: None --- pkg/ccl/backupccl/restore_job.go | 12 +++--- pkg/ccl/storageccl/key_rewriter.go | 6 +-- pkg/sql/alter_column_type.go | 11 ++++-- pkg/sql/alter_table.go | 17 +++++---- pkg/sql/catalog/descriptor.go | 1 - pkg/sql/catalog/tabledesc/structured.go | 48 ++++++++--------------- pkg/sql/catalog/tabledesc/table.go | 4 +- pkg/sql/create_table.go | 51 +++++++++++++------------ pkg/sql/drop_index.go | 12 ++---- pkg/sql/drop_table.go | 14 ++++--- pkg/sql/rename_column.go | 4 +- pkg/sql/revert.go | 8 ++-- pkg/sql/set_zone_config.go | 6 +-- pkg/sql/show_fingerprints.go | 8 +++- pkg/sql/truncate.go | 5 ++- 15 files changed, 103 insertions(+), 104 deletions(-) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 105db55cbd7a..1c805ce45207 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -833,11 +833,11 @@ func spansForAllRestoreTableIndexes( added := make(map[tableAndIndex]bool, len(tables)) sstIntervalTree := interval.NewTree(interval.ExclusiveOverlapper) for _, table := range tables { - for _, index := range table.AllNonDropIndexes() { - if err := sstIntervalTree.Insert(intervalSpan(table.IndexSpan(codec, index.ID)), false); err != nil { + for _, index := range table.NonDropIndexes() { + if err := sstIntervalTree.Insert(intervalSpan(table.IndexSpan(codec, index.GetID())), false); err != nil { panic(errors.NewAssertionErrorWithWrappedErrf(err, "IndexSpan")) } - added[tableAndIndex{tableID: table.GetID(), indexID: index.ID}] = true + added[tableAndIndex{tableID: table.GetID(), indexID: index.GetID()}] = true } } // If there are desc revisions, ensure that we also add any index spans @@ -853,10 +853,10 @@ func spansForAllRestoreTableIndexes( rawTbl := descpb.TableFromDescriptor(rev.Desc, hlc.Timestamp{}) if rawTbl != nil && rawTbl.State != descpb.DescriptorState_DROP { tbl := tabledesc.NewImmutable(*rawTbl) - for _, idx := range tbl.AllNonDropIndexes() { - key := tableAndIndex{tableID: tbl.ID, indexID: idx.ID} + for _, idx := range tbl.NonDropIndexes() { + key := tableAndIndex{tableID: tbl.ID, indexID: idx.GetID()} if !added[key] { - if err := sstIntervalTree.Insert(intervalSpan(tbl.IndexSpan(codec, idx.ID)), false); err != nil { + if err := sstIntervalTree.Insert(intervalSpan(tbl.IndexSpan(codec, idx.GetID())), false); err != nil { panic(errors.NewAssertionErrorWithWrappedErrf(err, "IndexSpan")) } added[key] = true diff --git a/pkg/ccl/storageccl/key_rewriter.go b/pkg/ccl/storageccl/key_rewriter.go index a44734e71763..7b00c5f05117 100644 --- a/pkg/ccl/storageccl/key_rewriter.go +++ b/pkg/ccl/storageccl/key_rewriter.go @@ -105,9 +105,9 @@ func MakeKeyRewriter(descs map[descpb.ID]*tabledesc.Immutable) (*KeyRewriter, er // The PrefixEnd() of index 1 is the same as the prefix of index 2, so use a // map to avoid duplicating entries. - for _, index := range desc.AllNonDropIndexes() { - oldPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(oldID, index.ID)) - newPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(desc.ID, index.ID)) + for _, index := range desc.NonDropIndexes() { + oldPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(oldID, index.GetID())) + newPrefix := roachpb.Key(makeKeyRewriterPrefixIgnoringInterleaved(desc.ID, index.GetID())) if !seenPrefixes[string(oldPrefix)] { seenPrefixes[string(oldPrefix)] = true prefixes.rewrites = append(prefixes.rewrites, prefixRewrite{ diff --git a/pkg/sql/alter_column_type.go b/pkg/sql/alter_column_type.go index b134a4e0d80c..5e9d1e84b69b 100644 --- a/pkg/sql/alter_column_type.go +++ b/pkg/sql/alter_column_type.go @@ -220,9 +220,14 @@ func alterColumnTypeGeneral( // Disallow ALTER COLUMN TYPE general for columns that are // part of indexes. - for _, idx := range tableDesc.AllNonDropIndexes() { - for _, id := range append(idx.ColumnIDs, idx.ExtraColumnIDs...) { - if col.ID == id { + for _, idx := range tableDesc.NonDropIndexes() { + for i := 0; i < idx.NumColumns(); i++ { + if idx.GetColumnID(i) == col.ID { + return colInIndexNotSupportedErr + } + } + for i := 0; i < idx.NumExtraColumns(); i++ { + if idx.GetExtraColumnID(i) == col.ID { return colInIndexNotSupportedErr } } diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index fd2f2cc7a672..2d4e487a9214 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -441,7 +441,7 @@ func (n *alterTableNode) startExec(params runParams) error { "column %q is referenced by the primary key", colToDrop.Name) } var idxNamesToDelete []string - for _, idx := range n.tableDesc.AllNonDropIndexes() { + for _, idx := range n.tableDesc.NonDropIndexes() { // We automatically drop indexes that reference the column // being dropped. @@ -450,14 +450,15 @@ func (n *alterTableNode) startExec(params runParams) error { containsThisColumn := false // Analyze the index. - for _, id := range idx.ColumnIDs { - if id == colToDrop.ID { + for j := 0; j < idx.NumColumns(); j++ { + if idx.GetColumnID(j) == colToDrop.ID { containsThisColumn = true break } } if !containsThisColumn { - for _, id := range idx.ExtraColumnIDs { + for j := 0; j < idx.NumExtraColumns(); j++ { + id := idx.GetExtraColumnID(j) if n.tableDesc.GetPrimaryIndex().ContainsColumnID(id) { // All secondary indices necessary contain the PK // columns, too. (See the comments on the definition of @@ -476,8 +477,8 @@ func (n *alterTableNode) startExec(params runParams) error { // The loop above this comment is for the old STORING encoding. The // loop below is for the new encoding (where the STORING columns are // always in the value part of a KV). - for _, id := range idx.StoreColumnIDs { - if id == colToDrop.ID { + for j := 0; j < idx.NumStoredColumns(); j++ { + if idx.GetStoredColumnID(j) == colToDrop.ID { containsThisColumn = true break } @@ -487,7 +488,7 @@ func (n *alterTableNode) startExec(params runParams) error { // If the column being dropped is referenced in the partial // index predicate, then the index should be dropped. if !containsThisColumn && idx.IsPartial() { - expr, err := parser.ParseExpr(idx.Predicate) + expr, err := parser.ParseExpr(idx.GetPredicate()) if err != nil { return err } @@ -504,7 +505,7 @@ func (n *alterTableNode) startExec(params runParams) error { // Perform the DROP. if containsThisColumn { - idxNamesToDelete = append(idxNamesToDelete, idx.Name) + idxNamesToDelete = append(idxNamesToDelete, idx.GetName()) } } diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 4099119d2127..98d81c5c88ac 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -116,7 +116,6 @@ type TableDescriptor interface { PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor // deprecated ForeachIndex(opts IndexOpts, f func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error) error // deprecated - AllNonDropIndexes() []*descpb.IndexDescriptor // deprecated ForeachNonDropIndex(f func(idxDesc *descpb.IndexDescriptor) error) error // deprecated IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) // deprecated diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index a14538428026..73ef018eead6 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -340,18 +340,6 @@ func buildIndexName(tableDesc *Mutable, index catalog.Index) string { return name } -// AllNonDropIndexes returns all the indexes, including those being added -// in the mutations. -// This method is deprecated, use NonDropIndexes instead. -func (desc *wrapper) AllNonDropIndexes() []*descpb.IndexDescriptor { - nonDropIndexes := desc.NonDropIndexes() - indexes := make([]*descpb.IndexDescriptor, len(nonDropIndexes)) - for i, idx := range nonDropIndexes { - indexes[i] = idx.IndexDesc() - } - return indexes -} - // AllActiveAndInactiveChecks returns all check constraints, including both // "active" ones on the table descriptor which are being enforced for all // writes, and "inactive" ones queued in the mutations list. @@ -1621,7 +1609,8 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des backref.Name, desc.Name, originTable.GetName()) } - for _, index := range desc.AllNonDropIndexes() { + for _, indexI := range desc.NonDropIndexes() { + index := indexI.IndexDesc() // Check interleaves. if len(index.Interleave.Ancestors) > 0 { // Only check the most recent ancestor, the rest of them don't point @@ -1792,10 +1781,10 @@ func ValidateTableLocalityConfig( // ValidateIndexNameIsUnique validates that the index name does not exist. func (desc *wrapper) ValidateIndexNameIsUnique(indexName string) error { - for _, index := range desc.AllNonDropIndexes() { - if indexName == index.Name { - return sqlerrors.NewRelationAlreadyExistsError(indexName) - } + if desc.FindNonDropIndex(func(idx catalog.Index) bool { + return idx.GetName() == indexName + }) != nil { + return sqlerrors.NewRelationAlreadyExistsError(indexName) } return nil } @@ -2230,7 +2219,8 @@ func (desc *wrapper) validateTableIndexes(columnNames map[string]descpb.ColumnID indexNames := map[string]struct{}{} indexIDs := map[descpb.IndexID]string{} - for _, index := range desc.AllNonDropIndexes() { + for _, indexI := range desc.NonDropIndexes() { + index := indexI.IndexDesc() if err := catalog.ValidateName(index.Name, "index"); err != nil { return err } @@ -3291,12 +3281,9 @@ func (desc *wrapper) FindFKByName(name string) (*descpb.ForeignKeyConstraint, er // IsInterleaved returns true if any part of this this table is interleaved with // another table's data. func (desc *wrapper) IsInterleaved() bool { - for _, index := range desc.AllNonDropIndexes() { - if index.IsInterleaved() { - return true - } - } - return false + return nil != desc.FindNonDropIndex(func(idx catalog.Index) bool { + return idx.IsInterleaved() + }) } // IsPrimaryIndexDefaultRowID returns whether or not the table's primary @@ -4059,8 +4046,8 @@ func (desc *wrapper) GetFamilyOfColumn( // subpartition in an arbitrary order. func (desc *wrapper) PartitionNames() []string { var names []string - for _, index := range desc.AllNonDropIndexes() { - names = append(names, index.Partitioning.PartitionNames()...) + for _, index := range desc.NonDropIndexes() { + names = append(names, index.PartitionNames()...) } return names } @@ -4144,12 +4131,9 @@ func (desc *Immutable) MutationColumns() []descpb.ColumnDescriptor { // 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 { - for _, idx := range desc.AllNonDropIndexes() { - if idx.Sharded.IsSharded && idx.Sharded.Name == col.Name { - return true - } - } - return false + return nil != desc.FindNonDropIndex(func(idx catalog.Index) bool { + return idx.IsSharded() && idx.GetShardColumnName() == col.Name + }) } // TableDesc implements the TableDescriptor interface. diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index fd52c0bc4116..e3007e79f8bc 100644 --- a/pkg/sql/catalog/tabledesc/table.go +++ b/pkg/sql/catalog/tabledesc/table.go @@ -211,8 +211,8 @@ func (desc *wrapper) collectConstraintInfo( info := make(map[string]descpb.ConstraintDetail) // Indexes provide PK and Unique constraints that are enforced by an index. - indexes := desc.AllNonDropIndexes() - for _, index := range indexes { + for _, indexI := range desc.NonDropIndexes() { + index := indexI.IndexDesc() if index.ID == desc.PrimaryIndex.ID { if _, ok := info[index.Name]; ok { return nil, pgerror.Newf(pgcode.DuplicateObject, diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 23f5dcd21ac9..7c2c857cfcf8 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -350,9 +350,9 @@ func (n *createTableNode) startExec(params runParams) error { } } - for _, index := range desc.AllNonDropIndexes() { - if len(index.Interleave.Ancestors) > 0 { - if err := params.p.finalizeInterleave(params.ctx, desc, index); err != nil { + for _, index := range desc.NonDropIndexes() { + if index.NumInterleaveAncestors() > 0 { + if err := params.p.finalizeInterleave(params.ctx, desc, index.IndexDesc()); err != nil { return err } } @@ -1682,14 +1682,14 @@ func NewTableDesc( // Assign any implicitly added shard columns to the column family of the first column // in their corresponding set of index columns. - for _, index := range desc.AllNonDropIndexes() { - if index.IsSharded() && !columnsInExplicitFamilies[index.Sharded.Name] { + for _, index := range desc.NonDropIndexes() { + if index.IsSharded() && !columnsInExplicitFamilies[index.GetShardColumnName()] { // Ensure that the shard column wasn't explicitly assigned a column family // during table creation (this will happen when a create statement is // "roundtripped", for example). - family := tabledesc.GetColumnFamilyForShard(&desc, index.Sharded.ColumnNames) + family := tabledesc.GetColumnFamilyForShard(&desc, index.GetSharded().ColumnNames) if family != "" { - if err := desc.AddColumnToFamilyMaybeCreate(index.Sharded.Name, family, false, false); err != nil { + if err := desc.AddColumnToFamilyMaybeCreate(index.GetShardColumnName(), family, false, false); err != nil { return nil, err } } @@ -2134,47 +2134,50 @@ func replaceLikeTableOpts(n *tree.CreateTable, params runParams) (tree.TableDefs } } if opts.Has(tree.LikeTableOptIndexes) { - for _, idx := range td.AllNonDropIndexes() { + for _, idx := range td.NonDropIndexes() { indexDef := tree.IndexTableDef{ - Name: tree.Name(idx.Name), - Inverted: idx.Type == descpb.IndexDescriptor_INVERTED, - Storing: make(tree.NameList, 0, len(idx.StoreColumnNames)), - Columns: make(tree.IndexElemList, 0, len(idx.ColumnNames)), + Name: tree.Name(idx.GetName()), + Inverted: idx.GetType() == descpb.IndexDescriptor_INVERTED, + Storing: make(tree.NameList, 0, idx.NumStoredColumns()), + Columns: make(tree.IndexElemList, 0, idx.NumColumns()), } - columnNames := idx.ColumnNames + numColumns := idx.NumColumns() if idx.IsSharded() { indexDef.Sharded = &tree.ShardedIndexDef{ - ShardBuckets: tree.NewDInt(tree.DInt(idx.Sharded.ShardBuckets)), + ShardBuckets: tree.NewDInt(tree.DInt(idx.GetSharded().ShardBuckets)), } - columnNames = idx.Sharded.ColumnNames + numColumns = len(idx.GetSharded().ColumnNames) } - for i, name := range columnNames { + for j := 0; j < numColumns; j++ { + name := idx.GetColumnName(j) + if idx.IsSharded() { + name = idx.GetSharded().ColumnNames[j] + } elem := tree.IndexElem{ Column: tree.Name(name), Direction: tree.Ascending, } - if idx.ColumnDirections[i] == descpb.IndexDescriptor_DESC { + if idx.GetColumnDirection(j) == descpb.IndexDescriptor_DESC { elem.Direction = tree.Descending } indexDef.Columns = append(indexDef.Columns, elem) } - for _, name := range idx.StoreColumnNames { - indexDef.Storing = append(indexDef.Storing, tree.Name(name)) + for j := 0; j < idx.NumStoredColumns(); j++ { + indexDef.Storing = append(indexDef.Storing, tree.Name(idx.GetStoredColumnName(j))) } var def tree.TableDef = &indexDef - if idx.Unique { - isPK := idx.ID == td.GetPrimaryIndexID() - if isPK && td.IsPrimaryIndexDefaultRowID() { + if idx.IsUnique() { + if idx.Primary() && td.IsPrimaryIndexDefaultRowID() { continue } def = &tree.UniqueConstraintTableDef{ IndexTableDef: indexDef, - PrimaryKey: isPK, + PrimaryKey: idx.Primary(), } } if idx.IsPartial() { - indexDef.Predicate, err = parser.ParseExpr(idx.Predicate) + indexDef.Predicate, err = parser.ParseExpr(idx.GetPredicate()) if err != nil { return nil, err } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index b3e93f340706..5a8ea252eae2 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "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/catalog/tabledesc" @@ -201,14 +202,9 @@ func (n *dropIndexNode) maybeDropShardColumn( if dropped { return nil } - shouldDropShardColumn := true - for _, otherIdx := range tableDesc.AllNonDropIndexes() { - if otherIdx.ContainsColumnID(shardColDesc.ID) { - shouldDropShardColumn = false - break - } - } - if !shouldDropShardColumn { + if tableDesc.FindNonDropIndex(func(otherIdx catalog.Index) bool { + return otherIdx.ContainsColumnID(shardColDesc.ID) + }) != nil { return nil } return n.dropShardColumnAndConstraint(params, tableDesc, shardColDesc) diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index 2aafe6d4c145..d5450fa05de3 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -84,8 +84,9 @@ func (p *planner) DropTable(ctx context.Context, n *tree.DropTable) (planNode, e } } } - for _, idx := range droppedDesc.AllNonDropIndexes() { - for _, ref := range idx.InterleavedBy { + for _, idx := range droppedDesc.NonDropIndexes() { + for i := 0; i < idx.NumInterleavedBy(); i++ { + ref := idx.GetInterleavedBy(i) if _, ok := td[ref.Table]; !ok { if err := p.canRemoveInterleave(ctx, droppedDesc.Name, ref, n.DropBehavior); err != nil { return nil, err @@ -300,13 +301,14 @@ func (p *planner) dropTableImpl( tableDesc.InboundFKs = nil // Remove interleave relationships. - for _, idx := range tableDesc.AllNonDropIndexes() { - if len(idx.Interleave.Ancestors) > 0 { - if err := p.removeInterleaveBackReference(ctx, tableDesc, idx); err != nil { + for _, idx := range tableDesc.NonDropIndexes() { + if idx.NumInterleaveAncestors() > 0 { + if err := p.removeInterleaveBackReference(ctx, tableDesc, idx.IndexDesc()); err != nil { return droppedViews, err } } - for _, ref := range idx.InterleavedBy { + for i := 0; i < idx.NumInterleavedBy(); i++ { + ref := idx.GetInterleavedBy(i) if err := p.removeInterleave(ctx, ref); err != nil { return droppedViews, err } diff --git a/pkg/sql/rename_column.go b/pkg/sql/rename_column.go index f50bd0b9605e..ae31effd7962 100644 --- a/pkg/sql/rename_column.go +++ b/pkg/sql/rename_column.go @@ -216,8 +216,8 @@ func (p *planner) renameColumn( // Keep the shardedDesc name in sync with the column name. shardedDesc.Name = string(newName) } - for _, idx := range tableDesc.AllNonDropIndexes() { - maybeUpdateShardedDesc(&idx.Sharded) + for _, idx := range tableDesc.NonDropIndexes() { + maybeUpdateShardedDesc(&idx.IndexDesc().Sharded) } // Rename the column in the indexes. diff --git a/pkg/sql/revert.go b/pkg/sql/revert.go index 714c1b10ddbe..8c11a550c58b 100644 --- a/pkg/sql/revert.go +++ b/pkg/sql/revert.go @@ -54,13 +54,15 @@ func RevertTables( if !tables[i].IsPhysicalTable() { return errors.Errorf("cannot revert virtual table %s", tables[i].Name) } - for _, idx := range tables[i].AllNonDropIndexes() { - for _, parent := range idx.Interleave.Ancestors { + for _, idx := range tables[i].NonDropIndexes() { + for j := 0; j < idx.NumInterleaveAncestors(); j++ { + parent := idx.GetInterleaveAncestor(j) if !reverting[parent.TableID] { return errors.New("cannot revert table without reverting all interleaved tables and indexes") } } - for _, child := range idx.InterleavedBy { + for j := 0; j < idx.NumInterleavedBy(); j++ { + child := idx.GetInterleavedBy(j) if !reverting[child.Table] { return errors.New("cannot revert table without reverting all interleaved tables and indexes") } diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index 67b08286ac50..bf8fe6322feb 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -360,10 +360,10 @@ func (n *setZoneConfigNode) startExec(params runParams) error { var specifiers []tree.ZoneSpecifier if n.zoneSpecifier.TargetsPartition() && n.allIndexes { sqltelemetry.IncrementPartitioningCounter(sqltelemetry.AlterAllPartitions) - for _, idx := range table.AllNonDropIndexes() { - if p := tabledesc.FindIndexPartitionByName(idx, string(n.zoneSpecifier.Partition)); p != nil { + for _, idx := range table.NonDropIndexes() { + if p := tabledesc.FindIndexPartitionByName(idx.IndexDesc(), string(n.zoneSpecifier.Partition)); p != nil { zs := n.zoneSpecifier - zs.TableOrIndex.Index = tree.UnrestrictedName(idx.Name) + zs.TableOrIndex.Index = tree.UnrestrictedName(idx.GetName()) specifiers = append(specifiers, zs) } } diff --git a/pkg/sql/show_fingerprints.go b/pkg/sql/show_fingerprints.go index 6e6084769e93..3472fe97afc3 100644 --- a/pkg/sql/show_fingerprints.go +++ b/pkg/sql/show_fingerprints.go @@ -65,9 +65,15 @@ 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: tableDesc.AllNonDropIndexes(), + indexes: indexDescs, }, nil } diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 43a711530bd1..ba053a8130e8 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -132,8 +132,9 @@ func (t *truncateNode) startExec(params runParams) error { return err } } - for _, idx := range tableDesc.AllNonDropIndexes() { - for _, ref := range idx.InterleavedBy { + for _, idx := range tableDesc.NonDropIndexes() { + for i := 0; i < idx.NumInterleavedBy(); i++ { + ref := idx.GetInterleavedBy(i) if err := maybeEnqueue(ref.Table, "interleaved by"); err != nil { return err } From 216e638141cc41e2caadbd0d472e0a174bb8db5d Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 10:26:36 -0500 Subject: [PATCH 04/13] sql: remove deprecated ForeachNonDropIndex method Previously, this method would be called on a table descriptor interface (or backing struct) to apply a function on the *descpb.IndexDescriptor of each of the table's non-drop indexes. This patch removes these calls, along with the method definition, in favour of new methods which use the catalog.Index interface type instead. Partially addresses #57465. Release note: None --- pkg/ccl/backupccl/backup_planning.go | 6 ++--- pkg/ccl/backupccl/restore_planning.go | 3 ++- pkg/sql/alter_table.go | 15 ++++------- pkg/sql/catalog/descriptor.go | 1 - pkg/sql/catalog/tabledesc/structured.go | 33 +++++++------------------ pkg/sql/create_table.go | 11 +++++---- pkg/sql/truncate.go | 3 ++- 7 files changed, 27 insertions(+), 45 deletions(-) diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 668bf5dbf0fd..0b5345fc14e3 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -182,13 +182,13 @@ func getLogicallyMergedTableSpans( checkForKVInBounds func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error), ) ([]roachpb.Span, error) { var nonDropIndexIDs []descpb.IndexID - if err := table.ForeachNonDropIndex(func(idxDesc *descpb.IndexDescriptor) error { - key := tableAndIndex{tableID: table.GetID(), indexID: idxDesc.ID} + if err := table.ForEachNonDropIndex(func(idx catalog.Index) error { + key := tableAndIndex{tableID: table.GetID(), indexID: idx.GetID()} if added[key] { return nil } added[key] = true - nonDropIndexIDs = append(nonDropIndexIDs, idxDesc.ID) + nonDropIndexIDs = append(nonDropIndexIDs, idx.GetID()) return nil }); err != nil { return nil, err diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 11180465d8dd..3e13127e297c 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -1021,7 +1021,8 @@ func RewriteTableDescs( return err } - if err := table.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { + if err := table.ForEachNonDropIndex(func(indexI catalog.Index) error { + index := indexI.IndexDesc() // Verify that for any interleaved index being restored, the interleave // parent is also being restored. Otherwise, the interleave entries in the // restored IndexDescriptors won't have anything to point to. diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 2d4e487a9214..ab372819d5ce 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -814,16 +814,11 @@ func (n *alterTableNode) startExec(params runParams) error { // new name exists. This is what postgres does. switch details.Kind { case descpb.ConstraintTypeUnique, descpb.ConstraintTypePK: - if err := n.tableDesc.ForeachNonDropIndex(func( - descriptor *descpb.IndexDescriptor, - ) error { - if descriptor.Name == string(t.NewName) { - return pgerror.Newf(pgcode.DuplicateRelation, - "relation %v already exists", t.NewName) - } - return nil - }); err != nil { - return err + if n.tableDesc.FindNonDropIndex(func(idx catalog.Index) bool { + return idx.GetName() == string(t.NewName) + }) != nil { + return pgerror.Newf(pgcode.DuplicateRelation, + "relation %v already exists", t.NewName) } } diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 98d81c5c88ac..8a4a1d79d3c4 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -116,7 +116,6 @@ type TableDescriptor interface { PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor // deprecated ForeachIndex(opts IndexOpts, f func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error) error // deprecated - ForeachNonDropIndex(f func(idxDesc *descpb.IndexDescriptor) error) error // deprecated IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) // deprecated FindIndexByName(name string) (_ *descpb.IndexDescriptor, dropped bool, _ error) // deprecated diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 73ef018eead6..7870cdccca76 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -464,15 +464,6 @@ func (desc *wrapper) ForeachNonDropColumn(f func(column *descpb.ColumnDescriptor return nil } -// ForeachNonDropIndex runs a function on all indexes, including those being -// added in the mutations. -// This method is deprecated, use ForEachNonDropIndex instead. -func (desc *wrapper) ForeachNonDropIndex(f func(*descpb.IndexDescriptor) error) error { - return desc.ForEachNonDropIndex(func(idx catalog.Index) error { - return f(idx.IndexDesc()) - }) -} - // ForeachIndex runs a function on the set of indexes as specified by opts. // This method is deprecated, use ForEachIndex instead. func (desc *wrapper) ForeachIndex( @@ -2535,7 +2526,8 @@ func (desc *wrapper) validatePartitioning() error { partitionNames := make(map[string]string) a := &rowenc.DatumAlloc{} - return desc.ForeachNonDropIndex(func(idxDesc *descpb.IndexDescriptor) error { + return desc.ForEachNonDropIndex(func(idx catalog.Index) error { + idxDesc := idx.IndexDesc() return desc.validatePartitioningDescriptor( a, idxDesc, &idxDesc.Partitioning, 0 /* colOffset */, partitionNames, ) @@ -3927,12 +3919,8 @@ func (desc *wrapper) InvalidateFKConstraints() { // being added in the mutations. func (desc *wrapper) AllIndexSpans(codec keys.SQLCodec) roachpb.Spans { var spans roachpb.Spans - err := desc.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { - spans = append(spans, desc.IndexSpan(codec, index.ID)) - return nil - }) - if err != nil { - panic(err) + for _, index := range desc.NonDropIndexes() { + spans = append(spans, desc.IndexSpan(codec, index.GetID())) } return spans } @@ -4078,16 +4066,13 @@ func (desc *wrapper) FindAllReferences() (map[descpb.ID]struct{}, error) { fk := &desc.InboundFKs[i] refs[fk.OriginTableID] = struct{}{} } - if err := desc.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { - for _, a := range index.Interleave.Ancestors { - refs[a.TableID] = struct{}{} + for _, index := range desc.NonDropIndexes() { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + refs[index.GetInterleaveAncestor(i).TableID] = struct{}{} } - for _, c := range index.InterleavedBy { - refs[c.Table] = struct{}{} + for i := 0; i < index.NumInterleavedBy(); i++ { + refs[index.GetInterleavedBy(i).Table] = struct{}{} } - return nil - }); err != nil { - return nil, err } for _, c := range desc.AllNonDropColumns() { diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 7c2c857cfcf8..5c296f02ced8 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1883,16 +1883,17 @@ func NewTableDesc( } // Record the types of indexes that the table has. - if err := desc.ForeachNonDropIndex(func(idx *descpb.IndexDescriptor) error { + if err := desc.ForEachNonDropIndex(func(idx catalog.Index) error { if idx.IsSharded() { telemetry.Inc(sqltelemetry.HashShardedIndexCounter) } - if idx.Type == descpb.IndexDescriptor_INVERTED { + if idx.GetType() == descpb.IndexDescriptor_INVERTED { telemetry.Inc(sqltelemetry.InvertedIndexCounter) - if !geoindex.IsEmptyConfig(&idx.GeoConfig) { - if geoindex.IsGeographyConfig(&idx.GeoConfig) { + geoConfig := idx.GetGeoConfig() + if !geoindex.IsEmptyConfig(&geoConfig) { + if geoindex.IsGeographyConfig(&geoConfig) { telemetry.Inc(sqltelemetry.GeographyInvertedIndexCounter) - } else if geoindex.IsGeometryConfig(&idx.GeoConfig) { + } else if geoindex.IsGeometryConfig(&geoConfig) { telemetry.Inc(sqltelemetry.GeometryInvertedIndexCounter) } } diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index ba053a8130e8..9e8e38c91457 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -382,7 +382,8 @@ func (p *planner) reassignInterleaveIndexReferences( ) error { for _, table := range tables { changed := false - if err := table.ForeachNonDropIndex(func(index *descpb.IndexDescriptor) error { + if err := table.ForEachNonDropIndex(func(indexI catalog.Index) error { + index := indexI.IndexDesc() for j, a := range index.Interleave.Ancestors { if a.TableID == truncatedID { index.Interleave.Ancestors[j].IndexID = indexIDMapping[index.Interleave.Ancestors[j].IndexID] From 4c0b2c11814c76b1d3de23cfdf8d95602b136fa2 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 10:26:39 -0500 Subject: [PATCH 05/13] sql: remove deprecated ForeachIndex method Previously, this method would be called on a table descriptor interface (or backing struct) to apply a function on the *descpb.IndexDescriptor of each of the table's indexes. This patch removes these calls, along with the method definition, in favour of new methods which use the catalog.Index interface type instead. Partially addresses #57465. Release note: None --- pkg/ccl/backupccl/targets.go | 10 +- pkg/ccl/partitionccl/partition.go | 7 +- pkg/sql/catalog/descriptor.go | 5 +- pkg/sql/catalog/tabledesc/structured.go | 26 ++---- pkg/sql/crdb_internal.go | 61 ++++++------ pkg/sql/information_schema.go | 20 ++-- pkg/sql/partition_utils.go | 12 +-- pkg/sql/pg_catalog.go | 119 ++++++++++++------------ 8 files changed, 122 insertions(+), 138 deletions(-) diff --git a/pkg/ccl/backupccl/targets.go b/pkg/ccl/backupccl/targets.go index 9ad845ec076c..93b8ec72761b 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -668,17 +668,19 @@ func ensureInterleavesIncluded(tables []catalog.TableDescriptor) error { } for _, table := range tables { - if err := table.ForeachIndex(catalog.IndexOpts{ + if err := table.ForEachIndex(catalog.IndexOpts{ AddMutations: true, - }, func(index *descpb.IndexDescriptor, _ bool) error { - for _, a := range index.Interleave.Ancestors { + }, func(index catalog.Index) error { + for i := 0; i < index.NumInterleaveAncestors(); i++ { + a := index.GetInterleaveAncestor(i) if !inBackup[a.TableID] { return errors.Errorf( "cannot backup table %q without interleave parent (ID %d)", table.GetName(), a.TableID, ) } } - for _, c := range index.InterleavedBy { + for i := 0; i < index.NumInterleavedBy(); i++ { + c := index.GetInterleavedBy(i) if !inBackup[c.Table] { return errors.Errorf( "cannot backup table %q without interleave child table (ID %d)", table.GetName(), c.Table, diff --git a/pkg/ccl/partitionccl/partition.go b/pkg/ccl/partitionccl/partition.go index a9119a9b7169..e61cd183e1d5 100644 --- a/pkg/ccl/partitionccl/partition.go +++ b/pkg/ccl/partitionccl/partition.go @@ -275,12 +275,11 @@ func selectPartitionExprs( a := &rowenc.DatumAlloc{} var prefixDatums []tree.Datum - if err := tableDesc.ForeachIndex(catalog.IndexOpts{ + if err := tableDesc.ForEachIndex(catalog.IndexOpts{ AddMutations: true, - }, func(idxDesc *descpb.IndexDescriptor, _ bool) error { - genExpr := true + }, func(idx catalog.Index) error { return selectPartitionExprsByName( - a, evalCtx, tableDesc, idxDesc, &idxDesc.Partitioning, prefixDatums, exprsByPartName, genExpr) + a, evalCtx, tableDesc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, prefixDatums, exprsByPartName, true /* genExpr */) }); err != nil { return nil, err } diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 8a4a1d79d3c4..c40086ebe85d 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -26,7 +26,7 @@ import ( "github.com/cockroachdb/redact" ) -// IndexOpts configures the behavior of TableDescriptor.ForeachIndex. +// IndexOpts configures the behavior of TableDescriptor.ForEachIndex. type IndexOpts struct { // NonPhysicalPrimaryIndex should be included. NonPhysicalPrimaryIndex bool @@ -114,8 +114,7 @@ type TableDescriptor interface { GetPrimaryIndexID() descpb.IndexID GetPrimaryIndex() *descpb.IndexDescriptor // deprecated PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span - GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor // deprecated - ForeachIndex(opts IndexOpts, f func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error) error // deprecated + GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor // deprecated IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) // deprecated FindIndexByName(name string) (_ *descpb.IndexDescriptor, dropped bool, _ error) // deprecated diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 7870cdccca76..d85cd6178d39 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -464,16 +464,6 @@ func (desc *wrapper) ForeachNonDropColumn(f func(column *descpb.ColumnDescriptor return nil } -// ForeachIndex runs a function on the set of indexes as specified by opts. -// This method is deprecated, use ForEachIndex instead. -func (desc *wrapper) ForeachIndex( - opts catalog.IndexOpts, f func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error, -) error { - return desc.ForEachIndex(opts, func(idx catalog.Index) error { - return f(idx.IndexDesc(), idx.Primary()) - }) -} - // ForeachDependedOnBy runs a function on all indexes, including those being // added in the mutations. func (desc *wrapper) ForeachDependedOnBy( @@ -1465,15 +1455,13 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des // un-upgraded foreign key references on the other table. This logic // somewhat parallels the logic in maybeUpgradeForeignKeyRepOnIndex. unupgradedFKsPresent := false - if err := referencedTable.ForeachIndex(catalog.IndexOpts{}, func( - referencedIdx *descpb.IndexDescriptor, isPrimary bool, - ) error { + if err := referencedTable.ForEachIndex(catalog.IndexOpts{}, func(referencedIdx catalog.Index) error { if found { // TODO (lucy): If we ever revisit the tabledesc.Immutable methods, add // a way to break out of the index loop. return nil } - if len(referencedIdx.ReferencedBy) > 0 { + if len(referencedIdx.IndexDesc().ReferencedBy) > 0 { unupgradedFKsPresent = true } else { return nil @@ -1485,8 +1473,8 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des } // Now check the backreferences. Backreferences in ReferencedBy only had // Index and Table populated. - for i := range referencedIdx.ReferencedBy { - backref := &referencedIdx.ReferencedBy[i] + for i := range referencedIdx.IndexDesc().ReferencedBy { + backref := &referencedIdx.IndexDesc().ReferencedBy[i] if backref.Table != desc.ID { continue } @@ -1546,15 +1534,13 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des // un-upgraded foreign key references on the other table. This logic // somewhat parallels the logic in maybeUpgradeForeignKeyRepOnIndex. unupgradedFKsPresent := false - if err := originTable.ForeachIndex(catalog.IndexOpts{}, func( - originIdx *descpb.IndexDescriptor, isPrimary bool, - ) error { + if err := originTable.ForEachIndex(catalog.IndexOpts{}, func(originIdx catalog.Index) error { if found { // TODO (lucy): If we ever revisit the tabledesc.Immutable methods, add // a way to break out of the index loop. return nil } - fk := originIdx.ForeignKey + fk := originIdx.IndexDesc().ForeignKey if fk.IsSet() { unupgradedFKsPresent = true } else { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 375d9bf235dd..b7d28f5087e6 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1823,12 +1823,8 @@ CREATE TABLE crdb_internal.create_statements ( if createNofk == "" { createNofk = stmt } - hasPartitions := false - _ = table.ForeachIndex(catalog.IndexOpts{}, func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error { - if idxDesc.Partitioning.NumColumns != 0 { - hasPartitions = true - } - return nil + hasPartitions := nil != table.FindIndex(catalog.IndexOpts{}, func(idx catalog.Index) bool { + return idx.GetPartitioning().NumColumns != 0 }) return addRow( dbDescID, @@ -2056,22 +2052,22 @@ CREATE TABLE crdb_internal.table_indexes ( tableName := tree.NewDString(table.GetName()) // We report the primary index of non-physical tables here. These // indexes are not reported as a part of ForeachIndex. - return table.ForeachIndex(catalog.IndexOpts{ + return table.ForEachIndex(catalog.IndexOpts{ NonPhysicalPrimaryIndex: true, - }, func(idx *descpb.IndexDescriptor, isPrimary bool) error { + }, func(idx catalog.Index) error { row = row[:0] idxType := secondary - if isPrimary { + if idx.Primary() { idxType = primary } row = append(row, tableID, tableName, - tree.NewDInt(tree.DInt(idx.ID)), - tree.NewDString(idx.Name), + tree.NewDInt(tree.DInt(idx.GetID())), + tree.NewDString(idx.GetName()), idxType, - tree.MakeDBool(tree.DBool(idx.Unique)), - tree.MakeDBool(idx.Type == descpb.IndexDescriptor_INVERTED), + tree.MakeDBool(tree.DBool(idx.IsUnique())), + tree.MakeDBool(idx.GetType() == descpb.IndexDescriptor_INVERTED), ) return pusher.pushRow(row...) }) @@ -2116,7 +2112,8 @@ CREATE TABLE crdb_internal.index_columns ( parentName := parent.GetName() tableName := tree.NewDString(table.GetName()) - reportIndex := func(idx *descpb.IndexDescriptor) error { + reportIndex := func(idxI catalog.Index) error { + idx := idxI.IndexDesc() idxID := tree.NewDInt(tree.DInt(idx.ID)) idxName := tree.NewDString(idx.Name) @@ -2184,11 +2181,7 @@ CREATE TABLE crdb_internal.index_columns ( return nil } - return table.ForeachIndex(catalog.IndexOpts{ - NonPhysicalPrimaryIndex: true, - }, func(idxDesc *descpb.IndexDescriptor, _ bool) error { - return reportIndex(idxDesc) - }) + return table.ForEachIndex(catalog.IndexOpts{NonPhysicalPrimaryIndex: true}, reportIndex) }) }, } @@ -2220,17 +2213,19 @@ CREATE TABLE crdb_internal.backward_dependencies ( viewDep := tree.NewDString("view") sequenceDep := tree.NewDString("sequence") interleaveDep := tree.NewDString("interleave") + return forEachTableDescAllWithTableLookup(ctx, p, dbContext, hideVirtual, func( db *dbdesc.Immutable, _ string, table catalog.TableDescriptor, tableLookup tableLookupFn, ) error { tableID := tree.NewDInt(tree.DInt(table.GetID())) tableName := tree.NewDString(table.GetName()) - reportIdxDeps := func(idx *descpb.IndexDescriptor) error { - for _, interleaveParent := range idx.Interleave.Ancestors { + reportIdxDeps := func(idx catalog.Index) error { + for i := 0; i < idx.NumInterleaveAncestors(); i++ { + interleaveParent := idx.GetInterleaveAncestor(i) if err := addRow( tableID, tableName, - tree.NewDInt(tree.DInt(idx.ID)), + tree.NewDInt(tree.DInt(idx.GetID())), tree.DNull, tree.NewDInt(tree.DInt(interleaveParent.TableID)), interleaveDep, @@ -2268,10 +2263,7 @@ CREATE TABLE crdb_internal.backward_dependencies ( } // Record the backward references of the primary index. - if err := table.ForeachIndex(catalog.IndexOpts{}, - func(idxDesc *descpb.IndexDescriptor, _ bool) error { - return reportIdxDeps(idxDesc) - }); err != nil { + if err := table.ForEachIndex(catalog.IndexOpts{}, reportIdxDeps); err != nil { return err } @@ -2367,11 +2359,12 @@ CREATE TABLE crdb_internal.forward_dependencies ( tableID := tree.NewDInt(tree.DInt(table.GetID())) tableName := tree.NewDString(table.GetName()) - reportIdxDeps := func(idx *descpb.IndexDescriptor) error { - for _, interleaveRef := range idx.InterleavedBy { + reportIdxDeps := func(idx catalog.Index) error { + for i := 0; i < idx.NumInterleavedBy(); i++ { + interleaveRef := idx.GetInterleavedBy(i) if err := addRow( tableID, tableName, - tree.NewDInt(tree.DInt(idx.ID)), + tree.NewDInt(tree.DInt(idx.GetID())), tree.NewDInt(tree.DInt(interleaveRef.Table)), interleaveDep, tree.NewDInt(tree.DInt(interleaveRef.Index)), @@ -2399,9 +2392,7 @@ CREATE TABLE crdb_internal.forward_dependencies ( } // Record the backward references of the primary index. - if err := table.ForeachIndex(catalog.IndexOpts{}, func(idxDesc *descpb.IndexDescriptor, isPrimary bool) error { - return reportIdxDeps(idxDesc) - }); err != nil { + if err := table.ForEachIndex(catalog.IndexOpts{}, reportIdxDeps); err != nil { return err } reportDependedOnBy := func( @@ -3415,10 +3406,10 @@ CREATE TABLE crdb_internal.partitions ( worker := func(pusher rowPusher) error { return forEachTableDescAll(ctx, p, dbContext, hideVirtual, /* virtual tables have no partitions*/ func(db *dbdesc.Immutable, _ string, table catalog.TableDescriptor) error { - return table.ForeachIndex(catalog.IndexOpts{ + return table.ForEachIndex(catalog.IndexOpts{ AddMutations: true, - }, func(index *descpb.IndexDescriptor, _ bool) error { - return addPartitioningRows(ctx, p, dbName, table, index, &index.Partitioning, + }, func(index catalog.Index) error { + return addPartitioningRows(ctx, p, dbName, table, index.IndexDesc(), &index.IndexDesc().Partitioning, tree.DNull /* parentName */, 0 /* colOffset */, pusher.pushRow) }) }) diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 1aabaf87d425..454dc658dc6f 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -1297,7 +1297,7 @@ CREATE TABLE information_schema.statistics ( ) } - return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { + return table.ForEachIndex(catalog.IndexOpts{}, func(index catalog.Index) error { // Columns in the primary key that aren't in index.ColumnNames or // index.StoreColumnNames are implicit columns in the index. var implicitCols map[string]struct{} @@ -1305,10 +1305,10 @@ CREATE TABLE information_schema.statistics ( if index.HasOldStoredColumns() { // Old STORING format: implicit columns are extra columns minus stored // columns. - hasImplicitCols = len(index.ExtraColumnIDs) > len(index.StoreColumnNames) + hasImplicitCols = index.NumExtraColumns() > index.NumStoredColumns() } else { // New STORING format: implicit columns are extra columns. - hasImplicitCols = len(index.ExtraColumnIDs) > 0 + hasImplicitCols = index.NumExtraColumns() > 0 } if hasImplicitCols { implicitCols = make(map[string]struct{}) @@ -1318,18 +1318,20 @@ CREATE TABLE information_schema.statistics ( } sequence := 1 - for i, col := range index.ColumnNames { + for i := 0; i < index.NumColumns(); i++ { + col := index.GetColumnName(i) // We add a row for each column of index. - dir := dStringForIndexDirection(index.ColumnDirections[i]) - if err := appendRow(index, col, sequence, dir, false, false); err != nil { + dir := dStringForIndexDirection(index.GetColumnDirection(i)) + if err := appendRow(index.IndexDesc(), col, sequence, dir, false, false); err != nil { return err } sequence++ delete(implicitCols, col) } - for _, col := range index.StoreColumnNames { + 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, col, sequence, + if err := appendRow(index.IndexDesc(), col, sequence, indexDirectionNA, true, false); err != nil { return err } @@ -1346,7 +1348,7 @@ CREATE TABLE information_schema.statistics ( for _, col := range table.GetPrimaryIndex().ColumnNames { if _, isImplicit := implicitCols[col]; isImplicit { // We add a row for each implicit column of index. - if err := appendRow(index, col, sequence, + if err := appendRow(index.IndexDesc(), col, sequence, indexDirectionAsc, false, true); err != nil { return err } diff --git a/pkg/sql/partition_utils.go b/pkg/sql/partition_utils.go index 87f21cb282cb..3e74449a6ac1 100644 --- a/pkg/sql/partition_utils.go +++ b/pkg/sql/partition_utils.go @@ -100,23 +100,23 @@ func GenerateSubzoneSpans( var indexCovering covering.Covering var partitionCoverings []covering.Covering - if err := tableDesc.ForeachIndex(catalog.IndexOpts{ + if err := tableDesc.ForEachIndex(catalog.IndexOpts{ AddMutations: true, - }, func(idxDesc *descpb.IndexDescriptor, _ bool) error { - _, indexSubzoneExists := subzoneIndexByIndexID[idxDesc.ID] + }, func(idx catalog.Index) error { + _, indexSubzoneExists := subzoneIndexByIndexID[idx.GetID()] if indexSubzoneExists { - idxSpan := tableDesc.IndexSpan(codec, idxDesc.ID) + idxSpan := tableDesc.IndexSpan(codec, idx.GetID()) // Each index starts with a unique prefix, so (from a precedence // perspective) it's safe to append them all together. indexCovering = append(indexCovering, covering.Range{ Start: idxSpan.Key, End: idxSpan.EndKey, - Payload: zonepb.Subzone{IndexID: uint32(idxDesc.ID)}, + Payload: zonepb.Subzone{IndexID: uint32(idx.GetID())}, }) } var emptyPrefix []tree.Datum indexPartitionCoverings, err := indexCoveringsForPartitioning( - a, codec, tableDesc, idxDesc, &idxDesc.Partitioning, subzoneIndexByPartition, emptyPrefix) + a, codec, tableDesc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, subzoneIndexByPartition, emptyPrefix) if err != nil { return err } diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index d2a098285593..741106caeb0a 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -445,9 +445,10 @@ https://www.postgresql.org/docs/12/catalog-pg-attribute.html`, // Columns for each index. columnIdxMap := table.ColumnIdxMap() - return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { - for _, colID := range index.ColumnIDs { - idxID := h.IndexOid(table.GetID(), index.ID) + return table.ForEachIndex(catalog.IndexOpts{}, func(index catalog.Index) error { + for i := 0; i < index.NumColumns(); i++ { + colID := index.GetColumnID(i) + idxID := h.IndexOid(table.GetID(), index.GetID()) column := table.GetColumnAtIdx(columnIdxMap.GetDefault(colID)) if err := addColumn(column, idxID, column.GetPGAttributeNum()); err != nil { return err @@ -624,31 +625,31 @@ https://www.postgresql.org/docs/9.5/catalog-pg-class.html`, } // Indexes. - return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { + return table.ForEachIndex(catalog.IndexOpts{}, func(index catalog.Index) error { indexType := forwardIndexOid - if index.Type == descpb.IndexDescriptor_INVERTED { + if index.GetType() == descpb.IndexDescriptor_INVERTED { indexType = invertedIndexOid } return addRow( - h.IndexOid(table.GetID(), index.ID), // oid - tree.NewDName(index.Name), // relname - namespaceOid, // relnamespace - oidZero, // reltype - oidZero, // reloftype - getOwnerOID(table), // relowner - indexType, // relam - oidZero, // relfilenode - oidZero, // reltablespace - tree.DNull, // relpages - tree.DNull, // reltuples - zeroVal, // relallvisible - oidZero, // reltoastrelid - tree.DBoolFalse, // relhasindex - tree.DBoolFalse, // relisshared - relPersistencePermanent, // relPersistence - tree.DBoolFalse, // relistemp - relKindIndex, // relkind - tree.NewDInt(tree.DInt(len(index.ColumnNames))), // relnatts + h.IndexOid(table.GetID(), index.GetID()), // oid + tree.NewDName(index.GetName()), // relname + namespaceOid, // relnamespace + oidZero, // reltype + oidZero, // reloftype + getOwnerOID(table), // relowner + indexType, // relam + oidZero, // relfilenode + oidZero, // reltablespace + tree.DNull, // relpages + tree.DNull, // reltuples + zeroVal, // relallvisible + oidZero, // reltoastrelid + tree.DBoolFalse, // relhasindex + tree.DBoolFalse, // relisshared + relPersistencePermanent, // relPersistence + tree.DBoolFalse, // relistemp + relKindIndex, // relkind + tree.NewDInt(tree.DInt(index.NumColumns())), // relnatts zeroVal, // relchecks tree.DBoolFalse, // relhasoids tree.DBoolFalse, // relhaspkey @@ -1441,11 +1442,15 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, return forEachTableDesc(ctx, p, dbContext, hideVirtual, /* virtual tables do not have indexes */ func(db *dbdesc.Immutable, scName string, table catalog.TableDescriptor) error { tableOid := tableOid(table.GetID()) - return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, isPrimary bool) error { + return table.ForEachIndex(catalog.IndexOpts{}, func(index catalog.Index) error { isMutation, isWriteOnly := - table.GetIndexMutationCapabilities(index.ID) + table.GetIndexMutationCapabilities(index.GetID()) isReady := isMutation && isWriteOnly - indkey, err := colIDArrayToVector(index.ColumnIDs) + columnIDs := make([]descpb.ColumnID, index.NumColumns()) + for i := range columnIDs { + columnIDs[i] = index.GetColumnID(i) + } + indkey, err := colIDArrayToVector(columnIDs) if err != nil { return err } @@ -1455,7 +1460,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, // is ASC/DESC and if nulls appear first/last. collationOids := tree.NewDArray(types.Oid) indoption := tree.NewDArray(types.Int) - for i, columnID := range index.ColumnIDs { + for i, columnID := range columnIDs { col, err := table.FindColumnByID(columnID) if err != nil { return err @@ -1466,7 +1471,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, // Currently, nulls always appear first if the order is ascending, // and always appear last if the order is descending. var thisIndOption tree.DInt - if index.ColumnDirections[i] == descpb.IndexDescriptor_ASC { + if index.GetColumnDirection(i) == descpb.IndexDescriptor_ASC { thisIndOption = indoptionNullsFirst } else { thisIndOption = indoptionDesc @@ -1479,30 +1484,30 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, indoptionIntVector := tree.NewDIntVectorFromDArray(indoption) // TODO(bram): #27763 indclass still needs to be populated but it // requires pg_catalog.pg_opclass first. - indclass, err := makeZeroedOidVector(len(index.ColumnIDs)) + indclass, err := makeZeroedOidVector(index.NumColumns()) if err != nil { return err } return addRow( - h.IndexOid(table.GetID(), index.ID), // indexrelid - tableOid, // indrelid - tree.NewDInt(tree.DInt(len(index.ColumnNames))), // indnatts - tree.MakeDBool(tree.DBool(index.Unique)), // indisunique - tree.MakeDBool(tree.DBool(isPrimary)), // indisprimary - tree.DBoolFalse, // indisexclusion - tree.MakeDBool(tree.DBool(index.Unique)), // indimmediate - tree.DBoolFalse, // indisclustered - tree.MakeDBool(tree.DBool(!isMutation)), // indisvalid - tree.DBoolFalse, // indcheckxmin - tree.MakeDBool(tree.DBool(isReady)), // indisready - tree.DBoolTrue, // indislive - tree.DBoolFalse, // indisreplident - indkey, // indkey - collationOidVector, // indcollation - indclass, // indclass - indoptionIntVector, // indoption - tree.DNull, // indexprs - tree.DNull, // indpred + h.IndexOid(table.GetID(), index.GetID()), // indexrelid + tableOid, // indrelid + tree.NewDInt(tree.DInt(index.NumColumns())), // indnatts + tree.MakeDBool(tree.DBool(index.IsUnique())), // indisunique + tree.MakeDBool(tree.DBool(index.Primary())), // indisprimary + tree.DBoolFalse, // indisexclusion + tree.MakeDBool(tree.DBool(index.IsUnique())), // indimmediate + tree.DBoolFalse, // indisclustered + tree.MakeDBool(tree.DBool(!isMutation)), // indisvalid + tree.DBoolFalse, // indcheckxmin + tree.MakeDBool(tree.DBool(isReady)), // indisready + tree.DBoolTrue, // indislive + tree.DBoolFalse, // indisreplident + indkey, // indkey + collationOidVector, // indcollation + indclass, // indclass + indoptionIntVector, // indoption + tree.DNull, // indexprs + tree.DNull, // indpred ) }) }) @@ -1519,18 +1524,18 @@ https://www.postgresql.org/docs/9.5/view-pg-indexes.html`, func(db *dbdesc.Immutable, scName string, table catalog.TableDescriptor, tableLookup tableLookupFn) error { scNameName := tree.NewDName(scName) tblName := tree.NewDName(table.GetName()) - return table.ForeachIndex(catalog.IndexOpts{}, func(index *descpb.IndexDescriptor, _ bool) error { - def, err := indexDefFromDescriptor(ctx, p, db, table, index, tableLookup) + return table.ForEachIndex(catalog.IndexOpts{}, func(index catalog.Index) error { + def, err := indexDefFromDescriptor(ctx, p, db, table, index.IndexDesc(), tableLookup) if err != nil { return err } return addRow( - h.IndexOid(table.GetID(), index.ID), // oid - scNameName, // schemaname - tblName, // tablename - tree.NewDName(index.Name), // indexname - tree.DNull, // tablespace - tree.NewDString(def), // indexdef + h.IndexOid(table.GetID(), index.GetID()), // oid + scNameName, // schemaname + tblName, // tablename + tree.NewDName(index.GetName()), // indexname + tree.DNull, // tablespace + tree.NewDString(def), // indexdef ) }) }) From 3d14dade69923be76645436f51e655f2ef9d3b7b Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 10:26:41 -0500 Subject: [PATCH 06/13] sql: remove deprecated FindIndexByID method Previously, this method would be called on a table descriptor interface (or backing struct) to return a target *descpb.IndexDescriptor. This patch removes these calls, along with the method definition, in favour of new methods which use the catalog.Index interface type instead. Partially addresses #57465. Release note: None --- pkg/ccl/backupccl/backup_planning.go | 4 +- pkg/ccl/partitionccl/zone_test.go | 7 +-- pkg/ccl/storageccl/key_rewriter.go | 12 ++--- pkg/sql/alter_index.go | 4 +- pkg/sql/backfill.go | 13 +++--- pkg/sql/catalog/descriptor.go | 1 - pkg/sql/catalog/tabledesc/structured.go | 61 ++++++++++--------------- pkg/sql/create_table.go | 4 +- pkg/sql/drop_table.go | 7 +-- pkg/sql/row/errors.go | 35 ++++++++------ pkg/sql/rowenc/index_encoding.go | 4 +- pkg/sql/rowenc/index_encoding_test.go | 3 +- pkg/sql/schema_changer.go | 19 ++++---- pkg/sql/sem/builtins/builtins.go | 15 +++--- pkg/sql/show_create_clauses.go | 4 +- 15 files changed, 97 insertions(+), 96 deletions(-) diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 0b5345fc14e3..0e3fab9a86cf 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -225,11 +225,11 @@ func getLogicallyMergedTableSpans( lhsSpan := table.IndexSpan(codec, lhsIndexID) rhsSpan := table.IndexSpan(codec, rhsIndexID) - lhsIndex, err := table.FindIndexByID(lhsIndexID) + lhsIndex, err := table.FindIndexWithID(lhsIndexID) if err != nil { return nil, err } - rhsIndex, err := table.FindIndexByID(rhsIndexID) + rhsIndex, err := table.FindIndexWithID(rhsIndexID) if err != nil { return nil, err } diff --git a/pkg/ccl/partitionccl/zone_test.go b/pkg/ccl/partitionccl/zone_test.go index 2e3190f3f8e1..86c9b1e19182 100644 --- a/pkg/ccl/partitionccl/zone_test.go +++ b/pkg/ccl/partitionccl/zone_test.go @@ -301,13 +301,14 @@ func TestGenerateSubzoneSpans(t *testing.T) { var actual []string for _, span := range spans { subzone := test.parsed.subzones[span.SubzoneIndex] - idxDesc, err := test.parsed.tableDesc.FindIndexByID(descpb.IndexID(subzone.IndexID)) + idx, err := test.parsed.tableDesc.FindIndexWithID(descpb.IndexID(subzone.IndexID)) if err != nil { t.Fatalf("could not find index with ID %d: %+v", subzone.IndexID, err) } directions := []encoding.Direction{encoding.Ascending /* index ID */} - for _, cd := range idxDesc.ColumnDirections { + for i := 0; i < idx.NumColumns(); i++ { + cd := idx.GetColumnDirection(i) ed, err := cd.ToEncodingDirection() if err != nil { t.Fatal(err) @@ -319,7 +320,7 @@ func TestGenerateSubzoneSpans(t *testing.T) { if len(subzone.PartitionName) > 0 { subzoneShort = "." + subzone.PartitionName } else { - subzoneShort = "@" + idxDesc.Name + subzoneShort = "@" + idx.GetName() } // Verify that we're always doing the space savings when we can. diff --git a/pkg/ccl/storageccl/key_rewriter.go b/pkg/ccl/storageccl/key_rewriter.go index 7b00c5f05117..c12b906fbd9e 100644 --- a/pkg/ccl/storageccl/key_rewriter.go +++ b/pkg/ccl/storageccl/key_rewriter.go @@ -190,22 +190,22 @@ func (kr *KeyRewriter) RewriteKey(key []byte, isFromSpan bool) ([]byte, bool, er // If there isn't any more data, we are at some split boundary. return key, true, nil } - idx, err := desc.FindIndexByID(descpb.IndexID(indexID)) + idx, err := desc.FindIndexWithID(descpb.IndexID(indexID)) if err != nil { return nil, false, err } - if len(idx.InterleavedBy) == 0 { + if idx.NumInterleavedBy() == 0 { // Not interleaved. return key, true, nil } // We do not support interleaved secondary indexes. - if idx.ID != desc.GetPrimaryIndexID() { + if !idx.Primary() { return nil, false, errors.New("restoring interleaved secondary indexes not supported") } - colIDs, _ := idx.FullColumnIDs() + colIDs, _ := idx.IndexDesc().FullColumnIDs() var skipCols int - for _, ancestor := range idx.Interleave.Ancestors { - skipCols += int(ancestor.SharedPrefixLen) + for i := 0; i < idx.NumInterleaveAncestors(); i++ { + skipCols += int(idx.GetInterleaveAncestor(i).SharedPrefixLen) } for i := 0; i < len(colIDs)-skipCols; i++ { n, err := encoding.PeekLength(k) diff --git a/pkg/sql/alter_index.go b/pkg/sql/alter_index.go index 73db4001448a..e6e04c7a0d1c 100644 --- a/pkg/sql/alter_index.go +++ b/pkg/sql/alter_index.go @@ -48,11 +48,11 @@ func (p *planner) AlterIndex(ctx context.Context, n *tree.AlterIndex) (planNode, // 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. - indexDesc, err = tableDesc.FindIndexByID(indexDesc.ID) + index, err := tableDesc.FindIndexWithID(indexDesc.ID) if err != nil { return nil, err } - return &alterIndexNode{n: n, tableDesc: tableDesc, indexDesc: indexDesc}, nil + return &alterIndexNode{n: n, tableDesc: tableDesc, indexDesc: index.IndexDesc()}, nil } // ReadingOwnWrites implements the planNodeReadingOwnWrites interface. diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index a15711fd4004..08401ad1eedf 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -1648,27 +1648,28 @@ func runSchemaChangesInTxn( // write the modified table descriptors explicitly. for _, idxID := range append( []descpb.IndexID{pkSwap.OldPrimaryIndexId}, pkSwap.OldIndexes...) { - oldIndex, err := tableDesc.FindIndexByID(idxID) + oldIndex, err := tableDesc.FindIndexWithID(idxID) if err != nil { return err } - if len(oldIndex.Interleave.Ancestors) != 0 { - ancestorInfo := oldIndex.Interleave.Ancestors[len(oldIndex.Interleave.Ancestors)-1] + if oldIndex.NumInterleaveAncestors() != 0 { + ancestorInfo := oldIndex.GetInterleaveAncestor(oldIndex.NumInterleaveAncestors() - 1) ancestor, err := planner.Descriptors().GetMutableTableVersionByID(ctx, ancestorInfo.TableID, planner.txn) if err != nil { return err } - ancestorIdx, err := ancestor.FindIndexByID(ancestorInfo.IndexID) + ancestorIdxI, err := ancestor.FindIndexWithID(ancestorInfo.IndexID) if err != nil { return err } + ancestorIdx := ancestorIdxI.IndexDesc() foundAncestor := false for k, ref := range ancestorIdx.InterleavedBy { - if ref.Table == tableDesc.ID && ref.Index == oldIndex.ID { + if ref.Table == tableDesc.ID && ref.Index == oldIndex.GetID() { if foundAncestor { return errors.AssertionFailedf( "ancestor entry in %s for %s@%s found more than once", - ancestor.Name, tableDesc.Name, oldIndex.Name) + ancestor.Name, tableDesc.Name, oldIndex.GetName()) } ancestorIdx.InterleavedBy = append( ancestorIdx.InterleavedBy[:k], ancestorIdx.InterleavedBy[k+1:]...) diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index c40086ebe85d..2964621e99e2 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -116,7 +116,6 @@ type TableDescriptor interface { PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor // deprecated IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span - FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) // deprecated FindIndexByName(name string) (_ *descpb.IndexDescriptor, dropped bool, _ error) // deprecated GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) KeysPerRow(id descpb.IndexID) (int, error) diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index d85cd6178d39..27ada1abf10c 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -268,18 +268,6 @@ func (desc *wrapper) IsPhysicalTable() bool { return desc.IsSequence() || (desc.IsTable() && !desc.IsVirtualTable()) || desc.MaterializedView() } -// FindIndexByID finds an index (active or inactive) with the specified ID. -// Must return a pointer to the IndexDescriptor in the TableDescriptor, so that -// callers can use returned values to modify the TableDesc. -// This method is deprecated, use FindIndexWithID instead. -func (desc *wrapper) FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) { - idx, err := desc.FindIndexWithID(id) - if err != nil { - return nil, err - } - return idx.IndexDesc(), nil -} - // KeysPerRow returns the maximum number of keys used to encode a row for the // given index. If a secondary index doesn't store any columns, then it only // has one k/v pair, but if it stores some columns, it can return up to one @@ -288,11 +276,11 @@ func (desc *wrapper) KeysPerRow(indexID descpb.IndexID) (int, error) { if desc.PrimaryIndex.ID == indexID { return len(desc.Families), nil } - idx, err := desc.FindIndexByID(indexID) + idx, err := desc.FindIndexWithID(indexID) if err != nil { return 0, err } - if len(idx.StoreColumnIDs) == 0 { + if idx.NumStoredColumns() == 0 { return 1, nil } return len(desc.Families), nil @@ -649,7 +637,7 @@ func maybeUpgradeForeignKeyRepOnIndex( } } if tbl, ok := otherUnupgradedTables[ref.Table]; ok { - referencedIndex, err := tbl.FindIndexByID(ref.Index) + referencedIndex, err := tbl.FindIndexWithID(ref.Index) if err != nil { return false, err } @@ -658,7 +646,7 @@ func maybeUpgradeForeignKeyRepOnIndex( OriginTableID: desc.ID, OriginColumnIDs: idx.ColumnIDs[:numCols], ReferencedTableID: ref.Table, - ReferencedColumnIDs: referencedIndex.ColumnIDs[:numCols], + ReferencedColumnIDs: referencedIndex.IndexDesc().ColumnIDs[:numCols], Name: ref.Name, Validity: ref.Validity, OnDelete: ref.OnDelete, @@ -687,10 +675,11 @@ func maybeUpgradeForeignKeyRepOnIndex( } if otherTable, ok := otherUnupgradedTables[ref.Table]; ok { - originIndex, err := otherTable.FindIndexByID(ref.Index) + originIndexI, err := otherTable.FindIndexWithID(ref.Index) if err != nil { return false, err } + originIndex := originIndexI.IndexDesc() // There are two cases. Either the other table is old (not upgraded yet), // or it's new (already upgraded). var inFK descpb.ForeignKeyConstraint @@ -1423,12 +1412,12 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des return nil, nil, errors.Wrapf(err, "missing table=%d index=%d", errors.Safe(tableID), errors.Safe(indexID)) } - targetIndex, err := targetTable.FindIndexByID(indexID) + targetIndex, err := targetTable.FindIndexWithID(indexID) if err != nil { return nil, nil, errors.Wrapf(err, "missing table=%s index=%d", targetTable.GetName(), errors.Safe(indexID)) } - return targetTable, targetIndex, nil + return targetTable, targetIndex.IndexDesc(), nil } // Check foreign keys. @@ -1483,7 +1472,7 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des // that it shouldn't be possible for this index to not exist. See // planner.MaybeUpgradeDependentOldForeignKeyVersionTables, which is // called from the drop index implementation.) - originalOriginIndex, err := desc.FindIndexByID(backref.Index) + originalOriginIndex, err := desc.FindIndexWithID(backref.Index) if err != nil { return errors.AssertionFailedf( "missing index %d on %q from pre-19.2 foreign key "+ @@ -1560,7 +1549,7 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des // that it shouldn't be possible for this index to not exist. See // planner.MaybeUpgradeDependentOldForeignKeyVersionTables, which is // called from the drop index implementation.) - originalReferencedIndex, err := desc.FindIndexByID(fk.Index) + originalReferencedIndex, err := desc.FindIndexWithID(fk.Index) if err != nil { return errors.AssertionFailedf( "missing index %d on %q from pre-19.2 foreign key forward reference %q on %q", @@ -1622,23 +1611,23 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des "invalid interleave backreference table=%d index=%d", backref.Table, backref.Index) } - targetIndex, err := targetTable.FindIndexByID(backref.Index) + targetIndex, err := targetTable.FindIndexWithID(backref.Index) if err != nil { return errors.Wrapf(err, "invalid interleave backreference table=%s index=%d", targetTable.GetName(), backref.Index) } - if len(targetIndex.Interleave.Ancestors) == 0 { + if targetIndex.NumInterleaveAncestors() == 0 { return errors.AssertionFailedf( "broken interleave backward reference from %q@%q to %q@%q", - desc.Name, index.Name, targetTable.GetName(), targetIndex.Name) + desc.Name, index.Name, targetTable.GetName(), targetIndex.GetName()) } // The last ancestor is required to be a backreference. - ancestor := targetIndex.Interleave.Ancestors[len(targetIndex.Interleave.Ancestors)-1] + ancestor := targetIndex.GetInterleaveAncestor(targetIndex.NumInterleaveAncestors() - 1) if ancestor.TableID != desc.ID || ancestor.IndexID != index.ID { return errors.AssertionFailedf( "broken interleave backward reference from %q@%q to %q@%q", - desc.Name, index.Name, targetTable.GetName(), targetIndex.Name) + desc.Name, index.Name, targetTable.GetName(), targetIndex.GetName()) } } } @@ -3389,25 +3378,25 @@ func (desc *Mutable) MakeMutationComplete(m descpb.DescriptorMutation) error { // Promote the new primary index into the primary index position on the descriptor, // and remove it from the secondary indexes list. - newIndex, err := desc.FindIndexByID(args.NewPrimaryIndexId) + newIndex, err := desc.FindIndexWithID(args.NewPrimaryIndexId) if err != nil { return err } - if args.NewPrimaryIndexName == "" { - newIndex.Name = PrimaryKeyIndexName - } else { - newIndex.Name = args.NewPrimaryIndexName - } { - primaryIndex := *protoutil.Clone(newIndex).(*descpb.IndexDescriptor) + primaryIndex := newIndex.IndexDescDeepCopy() + if args.NewPrimaryIndexName == "" { + primaryIndex.Name = PrimaryKeyIndexName + } else { + primaryIndex.Name = args.NewPrimaryIndexName + } // The primary index "implicitly" stores all columns in the table. // Explicitly including them in the stored columns list is incorrect. primaryIndex.StoreColumnNames, primaryIndex.StoreColumnIDs = nil, nil desc.SetPrimaryIndex(primaryIndex) } - idx, err := getIndexIdxByID(newIndex.ID) + idx, err := getIndexIdxByID(newIndex.GetID()) if err != nil { return err } @@ -3419,7 +3408,7 @@ func (desc *Mutable) MakeMutationComplete(m descpb.DescriptorMutation) error { newID := args.NewIndexes[j] // All our new indexes have been inserted into the table descriptor by now, since the primary key swap // is the last mutation processed in a group of mutations under the same mutation ID. - newIndex, err := desc.FindIndexByID(newID) + newIndex, err := desc.FindIndexWithID(newID) if err != nil { return err } @@ -3432,7 +3421,7 @@ func (desc *Mutable) MakeMutationComplete(m descpb.DescriptorMutation) error { return err } oldIndexCopy := protoutil.Clone(oldIndex).(*descpb.IndexDescriptor) - newIndex.Name = oldIndexCopy.Name + newIndex.IndexDesc().Name = oldIndexCopy.Name // Splice out old index from the indexes list. desc.RemovePublicNonPrimaryIndex(oldIndexIdx) // Add a drop mutation for the old index. The code that calls this function will schedule diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 5c296f02ced8..747257902378 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1118,11 +1118,11 @@ func (p *planner) finalizeInterleave( return err } } - ancestorIndex, err := ancestorTable.FindIndexByID(ancestor.IndexID) + ancestorIndex, err := ancestorTable.FindIndexWithID(ancestor.IndexID) if err != nil { return err } - ancestorIndex.InterleavedBy = append(ancestorIndex.InterleavedBy, + ancestorIndex.IndexDesc().InterleavedBy = append(ancestorIndex.IndexDesc().InterleavedBy, descpb.ForeignKeyReference{Table: desc.ID, Index: index.ID}) if err := p.writeSchemaChange( diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index d5450fa05de3..609884ae1a85 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -258,11 +258,11 @@ func (p *planner) removeInterleave(ctx context.Context, ref descpb.ForeignKeyRef // The referenced table is being dropped. No need to modify it further. return nil } - idx, err := table.FindIndexByID(ref.Index) + idx, err := table.FindIndexWithID(ref.Index) if err != nil { return err } - idx.Interleave.Ancestors = nil + idx.IndexDesc().Interleave.Ancestors = nil // No job description, since this is presumably part of some larger schema change. return p.writeSchemaChange(ctx, table, descpb.InvalidMutationID, "") } @@ -644,10 +644,11 @@ func (p *planner) removeInterleaveBackReference( // The referenced table is being dropped. No need to modify it further. return nil } - targetIdx, err := t.FindIndexByID(ancestor.IndexID) + targetIdxI, err := t.FindIndexWithID(ancestor.IndexID) if err != nil { return err } + targetIdx := targetIdxI.IndexDesc() foundAncestor := false for k, ref := range targetIdx.InterleavedBy { if ref.Table == tableDesc.ID && ref.Index == idx.ID { diff --git a/pkg/sql/row/errors.go b/pkg/sql/row/errors.go index 4303f1caebc3..fbe3ff3a2f22 100644 --- a/pkg/sql/row/errors.go +++ b/pkg/sql/row/errors.go @@ -154,25 +154,32 @@ func DecodeRowInfo( if err != nil { return nil, nil, nil, err } - index, err := tableDesc.FindIndexByID(indexID) + index, err := tableDesc.FindIndexWithID(indexID) if err != nil { return nil, nil, nil, err } var rf Fetcher - colIDs := index.ColumnIDs - if allColumns { - if index.ID == tableDesc.GetPrimaryIndexID() { - publicColumns := tableDesc.GetPublicColumns() - colIDs = make([]descpb.ColumnID, len(publicColumns)) - for i := range publicColumns { - colIDs[i] = publicColumns[i].ID - } - } else { - colIDs, _ = index.FullColumnIDs() - colIDs = append(colIDs, index.StoreColumnIDs...) + var colIDs []descpb.ColumnID + if !allColumns { + colIDs = make([]descpb.ColumnID, index.NumColumns()) + for i := range colIDs { + colIDs[i] = index.GetColumnID(i) } + } else if index.Primary() { + publicColumns := tableDesc.GetPublicColumns() + colIDs = make([]descpb.ColumnID, len(publicColumns)) + for i := range publicColumns { + colIDs[i] = publicColumns[i].ID + } + } else { + colIDs = make([]descpb.ColumnID, 0, index.NumColumns()+index.NumExtraColumns()+index.NumStoredColumns()) + _ = index.ForEachColumnID(func(id descpb.ColumnID) error { + colIDs = append(colIDs, id) + return nil + }) } + var valNeededForCol util.FastIntSet valNeededForCol.AddRange(0, len(colIDs)-1) @@ -189,7 +196,7 @@ func DecodeRowInfo( tableArgs := FetcherTableArgs{ Desc: tableDesc, - Index: index, + Index: index.IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: indexID != tableDesc.GetPrimaryIndexID(), Cols: cols, @@ -232,7 +239,7 @@ func DecodeRowInfo( } values[i] = datums[i].String() } - return index, names, values, nil + return index.IndexDesc(), names, values, nil } func (f *singleKVFetcher) close(context.Context) {} diff --git a/pkg/sql/rowenc/index_encoding.go b/pkg/sql/rowenc/index_encoding.go index 60a394389c1a..e37db2b86bf4 100644 --- a/pkg/sql/rowenc/index_encoding.go +++ b/pkg/sql/rowenc/index_encoding.go @@ -41,8 +41,8 @@ import ( func MakeIndexKeyPrefix( codec keys.SQLCodec, desc catalog.TableDescriptor, indexID descpb.IndexID, ) []byte { - if i, err := desc.FindIndexByID(indexID); err == nil && len(i.Interleave.Ancestors) > 0 { - ancestor := &i.Interleave.Ancestors[0] + if i, err := desc.FindIndexWithID(indexID); err == nil && i.NumInterleaveAncestors() > 0 { + ancestor := i.GetInterleaveAncestor(0) return codec.IndexPrefix(uint32(ancestor.TableID), uint32(ancestor.IndexID)) } return codec.IndexPrefix(uint32(desc.GetID()), uint32(indexID)) diff --git a/pkg/sql/rowenc/index_encoding_test.go b/pkg/sql/rowenc/index_encoding_test.go index 8189a66e7e18..9310878129eb 100644 --- a/pkg/sql/rowenc/index_encoding_test.go +++ b/pkg/sql/rowenc/index_encoding_test.go @@ -1241,10 +1241,11 @@ func ExtractIndexKey( return entry.Key, nil } - index, err := tableDesc.FindIndexByID(indexID) + indexI, 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) diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 49463d69be91..ccabfd4985cd 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1070,12 +1070,12 @@ func (sc *SchemaChanger) done(ctx context.Context) error { // If any old indexes (including the old primary index) being rewritten are interleaved // children, we will have to update their parents as well. for _, idxID := range append([]descpb.IndexID{swap.OldPrimaryIndexId}, swap.OldIndexes...) { - oldIndex, err := scTable.FindIndexByID(idxID) + oldIndex, err := scTable.FindIndexWithID(idxID) if err != nil { return err } - if len(oldIndex.Interleave.Ancestors) != 0 { - ancestor := oldIndex.Interleave.Ancestors[len(oldIndex.Interleave.Ancestors)-1] + if oldIndex.NumInterleaveAncestors() != 0 { + ancestor := oldIndex.GetInterleaveAncestor(oldIndex.NumInterleaveAncestors() - 1) if ancestor.TableID != scTable.ID { interleaveParents[ancestor.TableID] = struct{}{} } @@ -1222,27 +1222,28 @@ func (sc *SchemaChanger) done(ctx context.Context) error { // corresponding piece in runSchemaChangesInTxn. for _, idxID := range append( []descpb.IndexID{pkSwap.OldPrimaryIndexId}, pkSwap.OldIndexes...) { - oldIndex, err := scTable.FindIndexByID(idxID) + oldIndex, err := scTable.FindIndexWithID(idxID) if err != nil { return err } - if len(oldIndex.Interleave.Ancestors) != 0 { - ancestorInfo := oldIndex.Interleave.Ancestors[len(oldIndex.Interleave.Ancestors)-1] + if oldIndex.NumInterleaveAncestors() != 0 { + ancestorInfo := oldIndex.GetInterleaveAncestor(oldIndex.NumInterleaveAncestors() - 1) ancestor, err := descsCol.GetMutableTableVersionByID(ctx, ancestorInfo.TableID, txn) if err != nil { return err } - ancestorIdx, err := ancestor.FindIndexByID(ancestorInfo.IndexID) + ancestorIdxI, err := ancestor.FindIndexWithID(ancestorInfo.IndexID) if err != nil { return err } + ancestorIdx := ancestorIdxI.IndexDesc() foundAncestor := false for k, ref := range ancestorIdx.InterleavedBy { - if ref.Table == scTable.ID && ref.Index == oldIndex.ID { + if ref.Table == scTable.ID && ref.Index == oldIndex.GetID() { if foundAncestor { return errors.AssertionFailedf( "ancestor entry in %s for %s@%s found more than once", - ancestor.Name, scTable.Name, oldIndex.Name) + ancestor.Name, scTable.Name, oldIndex.GetName()) } ancestorIdx.InterleavedBy = append( ancestorIdx.InterleavedBy[:k], ancestorIdx.InterleavedBy[k+1:]...) diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 2395dc5e9629..54d02c7a444e 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -3673,10 +3673,11 @@ may increase either contention or retry errors, or both.`, if err != nil { return nil, err } - indexDesc, err := tableDesc.FindIndexByID(descpb.IndexID(indexID)) + index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) 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 @@ -4136,14 +4137,14 @@ may increase either contention or retry errors, or both.`, if err != nil { return nil, err } - indexDesc, err := tableDesc.FindIndexByID(descpb.IndexID(indexID)) + index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) if err != nil { return nil, err } - if indexDesc.GeoConfig.S2Geography == nil { + 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, indexDesc) + keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.IndexDesc()) if err != nil { return nil, err } @@ -4170,14 +4171,14 @@ may increase either contention or retry errors, or both.`, if err != nil { return nil, err } - indexDesc, err := tableDesc.FindIndexByID(descpb.IndexID(indexID)) + index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) if err != nil { return nil, err } - if indexDesc.GeoConfig.S2Geometry == nil { + 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, indexDesc) + keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.IndexDesc()) if err != nil { return nil, err } diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index 4044d4bed0e5..340e5a86709e 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -136,7 +136,7 @@ func showComments( } for _, indexComment := range tc.indexes { - idx, err := table.FindIndexByID(descpb.IndexID(indexComment.subID)) + idx, err := table.FindIndexWithID(descpb.IndexID(indexComment.subID)) if err != nil { return err } @@ -145,7 +145,7 @@ func showComments( f.FormatNode(&tree.CommentOnIndex{ Index: tree.TableIndexName{ Table: *tn, - Index: tree.UnrestrictedName(idx.Name), + Index: tree.UnrestrictedName(idx.GetName()), }, Comment: &indexComment.comment, }) From afa4e4300762d67fdda81ef6f3020308fe9353d3 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 10:26:44 -0500 Subject: [PATCH 07/13] sql: remove deprecated FindIndexByName method Previously, this method would be called on a table descriptor interface (or backing struct) to return a target *descpb.IndexDescriptor. This patch removes these calls, along with the method definition, in favour of new methods which use the catalog.Index interface type instead. Partially addresses #57465. Release note: None --- pkg/ccl/partitionccl/drop_test.go | 6 +++--- pkg/ccl/partitionccl/partition_test.go | 20 ++++++++++---------- pkg/sql/alter_primary_key.go | 2 +- pkg/sql/alter_table.go | 4 ++-- pkg/sql/catalog/descriptor.go | 1 - pkg/sql/catalog/tabledesc/structured.go | 13 ------------- pkg/sql/create_index.go | 4 ++-- pkg/sql/descriptor_mutation_test.go | 12 +++--------- pkg/sql/drop_index.go | 11 ++++++----- pkg/sql/drop_test.go | 20 ++++++++++---------- pkg/sql/rename_index.go | 6 +++--- pkg/sql/resolver.go | 4 ++-- pkg/sql/schema_changer_test.go | 4 ++-- pkg/sql/span_builder_test.go | 4 ++-- pkg/sql/tests/hash_sharded_test.go | 12 ++++++------ pkg/sql/zone_config.go | 12 ++++++------ 16 files changed, 58 insertions(+), 77 deletions(-) diff --git a/pkg/ccl/partitionccl/drop_test.go b/pkg/ccl/partitionccl/drop_test.go index 19fa3028bb72..4fc714521c88 100644 --- a/pkg/ccl/partitionccl/drop_test.go +++ b/pkg/ccl/partitionccl/drop_test.go @@ -70,11 +70,11 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) { PARTITION p2 VALUES IN (2) )`) tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - indexDesc, _, err := tableDesc.FindIndexByName("i") + index, err := tableDesc.FindIndexWithName("i") if err != nil { t.Fatal(err) } - indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, indexDesc.ID) + indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, index.GetID()) tests.CheckKeyCount(t, kvDB, indexSpan, numRows) // Set zone configs on the primary index, secondary index, and one partition @@ -117,7 +117,7 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) { } } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - if _, _, err := tableDesc.FindIndexByName("i"); err == nil { + if _, err := tableDesc.FindIndexWithName("i"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } close(asyncNotification) diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index c63d395e1d97..a48ccc3fb810 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -177,21 +177,21 @@ func (pt *partitioningTest) parse() error { if !strings.HasPrefix(indexName, "@") { panic(errors.Errorf("unsupported config: %s", c)) } - idxDesc, _, err := pt.parsed.tableDesc.FindIndexByName(indexName[1:]) + idx, err := pt.parsed.tableDesc.FindIndexWithName(indexName[1:]) if err != nil { return errors.Wrapf(err, "could not find index %s", indexName) } - subzone.IndexID = uint32(idxDesc.ID) + subzone.IndexID = uint32(idx.GetID()) if len(constraints) > 0 { if subzone.PartitionName == "" { fmt.Fprintf(&zoneConfigStmts, `ALTER INDEX %s@%s CONFIGURE ZONE USING constraints = '[%s]';`, - pt.parsed.tableName, idxDesc.Name, constraints, + pt.parsed.tableName, idx.GetName(), constraints, ) } else { fmt.Fprintf(&zoneConfigStmts, `ALTER PARTITION %s OF INDEX %s@%s CONFIGURE ZONE USING constraints = '[%s]';`, - subzone.PartitionName, pt.parsed.tableName, idxDesc.Name, constraints, + subzone.PartitionName, pt.parsed.tableName, idx.GetName(), constraints, ) } } @@ -1326,23 +1326,23 @@ func TestRepartitioning(t *testing.T) { } sqlDB.Exec(t, fmt.Sprintf("ALTER TABLE %s RENAME TO %s", test.old.parsed.tableName, test.new.parsed.tableName)) - testIndex, _, err := test.new.parsed.tableDesc.FindIndexByName(test.index) + testIndex, err := test.new.parsed.tableDesc.FindIndexWithName(test.index) if err != nil { t.Fatalf("%+v", err) } var repartition bytes.Buffer - if testIndex.ID == test.new.parsed.tableDesc.GetPrimaryIndexID() { + if testIndex.GetID() == test.new.parsed.tableDesc.GetPrimaryIndexID() { fmt.Fprintf(&repartition, `ALTER TABLE %s `, test.new.parsed.tableName) } else { - fmt.Fprintf(&repartition, `ALTER INDEX %s@%s `, test.new.parsed.tableName, testIndex.Name) + fmt.Fprintf(&repartition, `ALTER INDEX %s@%s `, test.new.parsed.tableName, testIndex.GetName()) } - if testIndex.Partitioning.NumColumns == 0 { + if testIndex.GetPartitioning().NumColumns == 0 { repartition.WriteString(`PARTITION BY NOTHING`) } else { if err := sql.ShowCreatePartitioning( - &rowenc.DatumAlloc{}, keys.SystemSQLCodec, test.new.parsed.tableDesc, testIndex, - &testIndex.Partitioning, &repartition, 0 /* indent */, 0, /* colOffset */ + &rowenc.DatumAlloc{}, keys.SystemSQLCodec, test.new.parsed.tableDesc, testIndex.IndexDesc(), + &testIndex.IndexDesc().Partitioning, &repartition, 0 /* indent */, 0, /* colOffset */ ); err != nil { t.Fatalf("%+v", err) } diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index 70fd900f7f55..4d262b7d5e16 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -120,7 +120,7 @@ func (p *planner) AlterPrimaryKey( } nameExists := func(name string) bool { - _, _, err := tableDesc.FindIndexByName(name) + _, err := tableDesc.FindIndexWithName(name) return err == nil } diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index ab372819d5ce..49458aa3fb4c 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -228,9 +228,9 @@ func (n *alterTableNode) startExec(params runParams) error { } idx.Partitioning = partitioning } - _, dropped, err := n.tableDesc.FindIndexByName(string(d.Name)) + foundIndex, err := n.tableDesc.FindIndexWithName(string(d.Name)) if err == nil { - if dropped { + if foundIndex.Dropped() { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "index %q being dropped, try again later", d.Name) } diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 2964621e99e2..9c33f0b43f9f 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -116,7 +116,6 @@ type TableDescriptor interface { PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor // deprecated IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span - FindIndexByName(name string) (_ *descpb.IndexDescriptor, dropped bool, _ error) // deprecated GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) KeysPerRow(id descpb.IndexID) (int, error) diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 27ada1abf10c..ccdd8c312137 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -2937,19 +2937,6 @@ func (desc *wrapper) FindFamilyByID(id descpb.FamilyID) (*descpb.ColumnFamilyDes return nil, fmt.Errorf("family-id \"%d\" does not exist", id) } -// FindIndexByName finds the index with the specified name in the active -// list or the mutations list. It returns true if the index is being dropped. -// This method is deprecated, use FindIndexWithName instead. -func (desc *wrapper) FindIndexByName( - name string, -) (_ *descpb.IndexDescriptor, dropped bool, _ error) { - idx, err := desc.FindIndexWithName(name) - if err != nil { - return nil, false, err - } - return idx.IndexDesc(), idx.Dropped(), nil -} - // NamesForColumnIDs returns the names for the given column ids, or an error // if one or more column ids was missing. Note - this allocates! It's not for // hot path code. diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index f57500b1e8a2..337c014a6d75 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -371,9 +371,9 @@ func maybeCreateAndAddShardCol( func (n *createIndexNode) startExec(params runParams) error { telemetry.Inc(sqltelemetry.SchemaChangeCreateCounter("index")) - _, dropped, err := n.tableDesc.FindIndexByName(string(n.n.Name)) + foundIndex, err := n.tableDesc.FindIndexWithName(string(n.n.Name)) if err == nil { - if dropped { + if foundIndex.Dropped() { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "index %q being dropped, try again later", string(n.n.Name)) } diff --git a/pkg/sql/descriptor_mutation_test.go b/pkg/sql/descriptor_mutation_test.go index 45585f5acbf2..5aa0c07c854e 100644 --- a/pkg/sql/descriptor_mutation_test.go +++ b/pkg/sql/descriptor_mutation_test.go @@ -480,20 +480,14 @@ func (mt mutationTest) writeIndexMutation( ctx context.Context, index string, m descpb.DescriptorMutation, ) { tableDesc := mt.tableDesc - idx, _, err := tableDesc.FindIndexByName(index) + idx, err := tableDesc.FindIndexWithName(index) if err != nil { mt.Fatal(err) } // The rewrite below potentially invalidates the original object with an overwrite. // Clarify what's going on. - idxCopy := *idx - for i, index := range tableDesc.GetPublicNonPrimaryIndexes() { - if idxCopy.ID == index.ID { - tableDesc.RemovePublicNonPrimaryIndex(i + 1) - break - } - } - + idxCopy := *idx.IndexDesc() + tableDesc.RemovePublicNonPrimaryIndex(idx.Ordinal()) m.Descriptor_ = &descpb.DescriptorMutation_Index{Index: &idxCopy} mt.writeMutation(ctx, m) } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 5a8ea252eae2..a599ea90895e 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -118,12 +118,12 @@ func (n *dropIndexNode) startExec(params runParams) error { // If we couldn't find the index by name, this is either a legitimate error or // this statement contains an 'IF EXISTS' qualifier. Both of these cases are // handled by `dropIndexByName()` below so we just ignore the error here. - idxDesc, dropped, _ := tableDesc.FindIndexByName(string(index.idxName)) + idx, _ := tableDesc.FindIndexWithName(string(index.idxName)) var shardColName string // If we're dropping a sharded index, record the name of its shard column to // potentially drop it if no other index refers to it. - if idxDesc != nil && idxDesc.IsSharded() && !dropped { - shardColName = idxDesc.Sharded.Name + if idx != nil && idx.IsSharded() && !idx.Dropped() { + shardColName = idx.GetShardColumnName() } if err := params.p.dropIndexByName( @@ -241,7 +241,7 @@ func (p *planner) dropIndexByName( constraintBehavior dropIndexConstraintBehavior, jobDesc string, ) error { - idx, dropped, err := tableDesc.FindIndexByName(string(idxName)) + idxI, 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. @@ -252,10 +252,11 @@ func (p *planner) dropIndexByName( // Index does not exist, but we want it to: error out. return pgerror.WithCandidateCode(err, pgcode.UndefinedObject) } - if dropped { + if idxI.Dropped() { return nil } + idx := idxI.IndexDesc() if idx.Unique && behavior != tree.DropCascade && constraintBehavior != ignoreIdxConstraint && !idx.CreatedExplicitly { return errors.WithHint( pgerror.Newf(pgcode.DependentObjectsStillExist, diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index 7b7249be34a4..7ee23c0358da 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -425,18 +425,18 @@ func TestDropIndex(t *testing.T) { } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(keys.SystemSQLCodec), 3*numRows) - idx, _, err := tableDesc.FindIndexByName("foo") + idx, err := tableDesc.FindIndexWithName("foo") if err != nil { t.Fatal(err) } - indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, idx.ID) + indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, idx.GetID()) tests.CheckKeyCount(t, kvDB, indexSpan, numRows) if _, err := sqlDB.Exec(`DROP INDEX t.kv@foo`); err != nil { t.Fatal(err) } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - if _, _, err := tableDesc.FindIndexByName("foo"); err == nil { + if _, err := tableDesc.FindIndexWithName("foo"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } // Index data hasn't been deleted. @@ -462,11 +462,11 @@ func TestDropIndex(t *testing.T) { } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - newIdx, _, err := tableDesc.FindIndexByName("foo") + newIdx, err := tableDesc.FindIndexWithName("foo") if err != nil { t.Fatal(err) } - newIdxSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, newIdx.ID) + newIdxSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, newIdx.GetID()) tests.CheckKeyCount(t, kvDB, newIdxSpan, numRows) tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(keys.SystemSQLCodec), 4*numRows) @@ -525,11 +525,11 @@ func TestDropIndexWithZoneConfigOSS(t *testing.T) { t.Fatal(err) } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - indexDesc, _, err := tableDesc.FindIndexByName("foo") + index, err := tableDesc.FindIndexWithName("foo") if err != nil { t.Fatal(err) } - indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, indexDesc.ID) + indexSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, index.GetID()) tests.CheckKeyCount(t, kvDB, indexSpan, numRows) // Hack in zone configs for the primary and secondary indexes. (You need a CCL @@ -539,7 +539,7 @@ func TestDropIndexWithZoneConfigOSS(t *testing.T) { zoneConfig := zonepb.ZoneConfig{ Subzones: []zonepb.Subzone{ {IndexID: uint32(tableDesc.GetPrimaryIndexID()), Config: s.(*server.TestServer).Cfg.DefaultZoneConfig}, - {IndexID: uint32(indexDesc.ID), Config: s.(*server.TestServer).Cfg.DefaultZoneConfig}, + {IndexID: uint32(index.GetID()), Config: s.(*server.TestServer).Cfg.DefaultZoneConfig}, }, } zoneConfigBytes, err := protoutil.Marshal(&zoneConfig) @@ -564,7 +564,7 @@ func TestDropIndexWithZoneConfigOSS(t *testing.T) { // declares column families. tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") - if _, _, err := tableDesc.FindIndexByName("foo"); err == nil { + if _, err := tableDesc.FindIndexWithName("foo"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } } @@ -597,7 +597,7 @@ func TestDropIndexInterleaved(t *testing.T) { // Ensure that index is not active. tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "intlv") - if _, _, err := tableDesc.FindIndexByName("intlv_idx"); err == nil { + if _, err := tableDesc.FindIndexWithName("intlv_idx"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } } diff --git a/pkg/sql/rename_index.go b/pkg/sql/rename_index.go index baff4d82f67d..e4d1de5993cd 100644 --- a/pkg/sql/rename_index.go +++ b/pkg/sql/rename_index.go @@ -52,7 +52,7 @@ func (p *planner) RenameIndex(ctx context.Context, n *tree.RenameIndex) (planNod return newZeroNode(nil /* columns */), nil } - idx, _, err := tableDesc.FindIndexByName(string(n.Index.Index)) + idx, err := tableDesc.FindIndexWithName(string(n.Index.Index)) if err != nil { if n.IfExists { // Noop. @@ -66,7 +66,7 @@ func (p *planner) RenameIndex(ctx context.Context, n *tree.RenameIndex) (planNod return nil, err } - return &renameIndexNode{n: n, idx: idx, tableDesc: tableDesc}, nil + return &renameIndexNode{n: n, idx: idx.IndexDesc(), tableDesc: tableDesc}, nil } // ReadingOwnWrites implements the planNodeReadingOwnWrites interface. @@ -98,7 +98,7 @@ func (n *renameIndexNode) startExec(params runParams) error { return nil } - if _, _, err := tableDesc.FindIndexByName(string(n.n.NewName)); err == nil { + if _, err := tableDesc.FindIndexWithName(string(n.n.NewName)); err == nil { return pgerror.Newf(pgcode.DuplicateRelation, "index name %q already exists", string(n.n.NewName)) } diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index 164d0ce09bb7..c8a2238c5d31 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -461,8 +461,8 @@ func findTableContainingIndex( continue } - _, dropped, err := tableDesc.FindIndexByName(string(idxName)) - if err != nil || dropped { + idx, err := tableDesc.FindIndexWithName(string(idxName)) + if err != nil || idx.Dropped() { // err is nil if the index does not exist on the table. continue } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index affeb9f7960f..9e0f8945cd04 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -6293,13 +6293,13 @@ CREATE INDEX i ON t.test (a) WHERE b > 2 } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - indexDesc, _, err := tableDesc.FindIndexByName("i") + index, err := tableDesc.FindIndexWithName("i") if err != nil { t.Fatalf("unexpected error: %s", err) } // Collect all the keys in the partial index. - span := tableDesc.IndexSpan(keys.SystemSQLCodec, indexDesc.ID) + span := tableDesc.IndexSpan(keys.SystemSQLCodec, index.GetID()) keys, err := kvDB.Scan(ctx, span.Key, span.EndKey, 0) if err != nil { t.Fatalf("unexpected error: %s", err) diff --git a/pkg/sql/span_builder_test.go b/pkg/sql/span_builder_test.go index cb68ac354417..cddec253abe1 100644 --- a/pkg/sql/span_builder_test.go +++ b/pkg/sql/span_builder_test.go @@ -103,11 +103,11 @@ func TestSpanBuilderCanSplitSpan(t *testing.T) { t.Fatal(err) } desc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "t") - idx, _, err := desc.FindIndexByName(tc.index) + idx, err := desc.FindIndexWithName(tc.index) if err != nil { t.Fatal(err) } - builder := span.MakeBuilder(evalCtx, execCfg.Codec, desc, idx) + builder := span.MakeBuilder(evalCtx, execCfg.Codec, desc, idx.IndexDesc()) if res := builder.CanSplitSpanIntoSeparateFamilies( 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/tests/hash_sharded_test.go b/pkg/sql/tests/hash_sharded_test.go index aee5ac1fcbe1..588edd51347d 100644 --- a/pkg/sql/tests/hash_sharded_test.go +++ b/pkg/sql/tests/hash_sharded_test.go @@ -30,11 +30,11 @@ import ( func getShardColumnID( t *testing.T, tableDesc *tabledesc.Immutable, shardedIndexName string, ) descpb.ColumnID { - idx, _, err := tableDesc.FindIndexByName(shardedIndexName) + idx, err := tableDesc.FindIndexWithName(shardedIndexName) if err != nil { t.Fatal(err) } - shardCol, _, err := tableDesc.FindColumnByName(tree.Name(idx.Sharded.Name)) + shardCol, _, err := tableDesc.FindColumnByName(tree.Name(idx.GetShardColumnName())) if err != nil { t.Fatal(err) } @@ -49,7 +49,7 @@ func getShardColumnID( func verifyTableDescriptorState( t *testing.T, tableDesc *tabledesc.Immutable, shardedIndexName string, ) { - idx, _, err := tableDesc.FindIndexByName(shardedIndexName) + idx, err := tableDesc.FindIndexWithName(shardedIndexName) if err != nil { t.Fatal(err) } @@ -73,7 +73,7 @@ func verifyTableDescriptorState( if !foundCheckConstraint { t.Fatalf(`Could not find hidden check constraint for shard column`) } - if idx.ColumnIDs[0] != shardColID { + if idx.GetColumnID(0) != shardColID { t.Fatalf(`Expected shard column to be the first column in the set of index columns`) } } @@ -113,12 +113,12 @@ func TestBasicHashShardedIndexes(t *testing.T) { // Ensure that secondary indexes on table `kv` have the shard column in their // `ExtraColumnIDs` field so they can reconstruct the sharded primary key. - fooDesc, _, err := tableDesc.FindIndexByName("foo") + foo, err := tableDesc.FindIndexWithName("foo") if err != nil { t.Fatal(err) } foundShardColumn := false - for _, colID := range fooDesc.ExtraColumnIDs { + for _, colID := range foo.IndexDesc().ExtraColumnIDs { if colID == shardColID { foundShardColumn = true break diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index 014e4faeec69..dd6fe8345f1c 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -315,13 +315,13 @@ func resolveSubzone( } indexName := string(zs.TableOrIndex.Index) - var index *descpb.IndexDescriptor + var index catalog.Index if indexName == "" { - index = table.GetPrimaryIndex() - indexName = index.Name + index = table.PrimaryIndexInterface() + indexName = index.GetName() } else { var err error - index, _, err = table.FindIndexByName(indexName) + index, err = table.FindIndexWithName(indexName) if err != nil { return nil, "", err } @@ -329,12 +329,12 @@ func resolveSubzone( partitionName := string(zs.Partition) if partitionName != "" { - if partitioning := tabledesc.FindIndexPartitionByName(index, partitionName); partitioning == nil { + if partitioning := tabledesc.FindIndexPartitionByName(index.IndexDesc(), partitionName); partitioning == nil { return nil, "", fmt.Errorf("partition %q does not exist on index %q", partitionName, indexName) } } - return index, partitionName, nil + return index.IndexDesc(), partitionName, nil } func deleteRemovedPartitionZoneConfigs( From 4e3ebcdb130af025d7a1f3d8485ace26258741cb Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 11:11:28 -0500 Subject: [PATCH 08/13] sql: remove deprecated GetPublicNonPrimaryIndexes method Previously, this method would be called on a table descriptor interface (or backing struct) to obtain a slice of descpb.IndexDescriptors. This patch removes these calls, along with the method definition, in favour of new methods which use the catalog.Index interface type instead. Partially addresses #57465. Release note: None --- pkg/sql/alter_primary_key.go | 7 +- pkg/sql/catalog/descriptor.go | 1 - pkg/sql/catalog/tabledesc/table_desc.go | 7 - pkg/sql/crdb_internal.go | 22 ++- pkg/sql/create_stats.go | 19 ++- pkg/sql/create_table.go | 9 +- pkg/sql/descriptor_mutation_test.go | 6 +- pkg/sql/distsql_physical_planner.go | 11 +- pkg/sql/distsql_plan_stats.go | 6 +- pkg/sql/drop_index.go | 98 +++++------ pkg/sql/flowinfra/cluster_test.go | 2 +- pkg/sql/opt_catalog.go | 13 +- pkg/sql/partition_test.go | 4 +- pkg/sql/pgwire/testdata/pgtest/notice | 2 +- pkg/sql/physicalplan/span_resolver_test.go | 2 +- pkg/sql/refresh_materialized_view.go | 6 +- pkg/sql/rename_column.go | 10 +- pkg/sql/row/fetcher.go | 7 +- pkg/sql/row/fetcher_test.go | 28 +-- pkg/sql/row/row_converter.go | 2 +- pkg/sql/rowenc/client_index_encoding_test.go | 171 +++++++++---------- pkg/sql/rowenc/index_encoding.go | 10 +- pkg/sql/rowenc/index_encoding_test.go | 10 +- pkg/sql/rowexec/BUILD.bazel | 1 + pkg/sql/rowexec/joinreader_test.go | 17 +- pkg/sql/rowexec/scrub_tablereader.go | 9 +- pkg/sql/rowexec/tablereader_test.go | 2 +- pkg/sql/rowexec/zigzagjoiner.go | 6 +- pkg/sql/scan.go | 29 +--- pkg/sql/schema_changer.go | 4 +- pkg/sql/schema_changer_test.go | 37 ++-- pkg/sql/scrub.go | 17 +- pkg/sql/scrub_test.go | 40 ++--- pkg/sql/show_create.go | 17 +- pkg/sql/table_ref_test.go | 2 +- pkg/sql/table_test.go | 13 +- pkg/sql/truncate.go | 36 ++-- pkg/sql/virtual_schema.go | 7 +- 38 files changed, 319 insertions(+), 371 deletions(-) diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index 4d262b7d5e16..30aee385f00b 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -261,10 +261,9 @@ func (p *planner) AlterPrimaryKey( return shouldRewrite || !idx.Unique || idx.Type == descpb.IndexDescriptor_INVERTED } var indexesToRewrite []*descpb.IndexDescriptor - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - idx := &tableDesc.GetPublicNonPrimaryIndexes()[i] - if idx.ID != newPrimaryIndexDesc.ID && shouldRewriteIndex(idx) { - indexesToRewrite = append(indexesToRewrite, idx) + for _, idx := range tableDesc.PublicNonPrimaryIndexes() { + if idx.GetID() != newPrimaryIndexDesc.ID && shouldRewriteIndex(idx.IndexDesc()) { + indexesToRewrite = append(indexesToRewrite, idx.IndexDesc()) } } diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 9c33f0b43f9f..59b9338be046 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -114,7 +114,6 @@ type TableDescriptor interface { GetPrimaryIndexID() descpb.IndexID GetPrimaryIndex() *descpb.IndexDescriptor // deprecated PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span - GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor // deprecated IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) KeysPerRow(id descpb.IndexID) (int, error) diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index 4bad6d4f0d28..70f6893ce004 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -182,13 +182,6 @@ func (desc *wrapper) GetPrimaryIndexID() descpb.IndexID { return desc.PrimaryIndex.ID } -// GetPublicNonPrimaryIndexes returns the public non-primary indexes of the -// descriptor. -// This method is deprecated, use PublicNonPrimaryIndexes instead. -func (desc *wrapper) GetPublicNonPrimaryIndexes() []descpb.IndexDescriptor { - return desc.Indexes -} - // IsTemporary returns true if this is a temporary table. func (desc *wrapper) IsTemporary() bool { return desc.GetTemporary() diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index b7d28f5087e6..ebb6313a1961 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1807,7 +1807,7 @@ CREATE TABLE crdb_internal.create_statements ( if err != nil { return err } - if err := showAlterStatementWithInterleave(ctx, &name, contextName, lookup, table.GetPublicNonPrimaryIndexes(), table, alterStmts, + if err := showAlterStatementWithInterleave(ctx, &name, contextName, lookup, table.PublicNonPrimaryIndexes(), table, alterStmts, validateStmts, &p.semaCtx); err != nil { return err } @@ -1847,7 +1847,7 @@ func showAlterStatementWithInterleave( tn *tree.TableName, contextName string, lCtx simpleSchemaResolver, - allIdx []descpb.IndexDescriptor, + allIdx []catalog.Index, table catalog.TableDescriptor, alterStmts *tree.DArray, validateStmts *tree.DArray, @@ -1888,14 +1888,12 @@ func showAlterStatementWithInterleave( return err } - for i := range allIdx { - idx := &allIdx[i] + for _, idx := range allIdx { // Create CREATE INDEX commands for INTERLEAVE tables. These commands // are included in the ALTER TABLE statements. - if len(idx.Interleave.Ancestors) > 0 { + if idx.NumInterleaveAncestors() > 0 { f := tree.NewFmtCtx(tree.FmtSimple) - intl := idx.Interleave - parentTableID := intl.Ancestors[len(intl.Ancestors)-1].TableID + parentTableID := idx.GetInterleaveAncestor(idx.NumInterleaveAncestors() - 1).TableID var err error var parentName tree.TableName if lCtx != nil { @@ -1921,11 +1919,11 @@ func showAlterStatementWithInterleave( tableName.ExplicitSchema = false } var sharedPrefixLen int - for _, ancestor := range intl.Ancestors { - sharedPrefixLen += int(ancestor.SharedPrefixLen) + for i := 0; i < idx.NumInterleaveAncestors(); i++ { + sharedPrefixLen += int(idx.GetInterleaveAncestor(i).SharedPrefixLen) } // Write the CREATE INDEX statements. - if err := showCreateIndexWithInterleave(ctx, f, table, idx, tableName, parentName, sharedPrefixLen, semaCtx); err != nil { + if err := showCreateIndexWithInterleave(ctx, f, table, idx.IndexDesc(), tableName, parentName, sharedPrefixLen, semaCtx); err != nil { return err } if err := alterStmts.Append(tree.NewDString(f.CloseAndGetString())); err != nil { @@ -2504,8 +2502,8 @@ CREATE TABLE crdb_internal.ranges_no_leases ( parents[id] = uint32(desc.ParentID) tableNames[id] = desc.GetName() indexNames[id] = make(map[uint32]string) - for _, idx := range desc.GetPublicNonPrimaryIndexes() { - indexNames[id][uint32(idx.ID)] = idx.Name + for _, idx := range desc.PublicNonPrimaryIndexes() { + indexNames[id][uint32(idx.GetID())] = idx.GetName() } case *dbdesc.Immutable: dbNames[id] = desc.GetName() diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index f727c46f2cdf..acea824d6180 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -303,7 +303,7 @@ const maxNonIndexCols = 100 func createStatsDefaultColumns( desc *tabledesc.Immutable, multiColEnabled bool, ) ([]jobspb.CreateStatsDetails_ColStat, error) { - colStats := make([]jobspb.CreateStatsDetails_ColStat, 0, len(desc.GetPublicNonPrimaryIndexes())+1) + colStats := make([]jobspb.CreateStatsDetails_ColStat, 0, len(desc.ActiveIndexes())) requestedStats := make(map[string]struct{}) @@ -372,19 +372,22 @@ func createStatsDefaultColumns( } // Add column stats for each secondary index. - for i := range desc.GetPublicNonPrimaryIndexes() { - isInverted := desc.GetPublicNonPrimaryIndexes()[i].Type == descpb.IndexDescriptor_INVERTED + for _, idx := range desc.PublicNonPrimaryIndexes() { + isInverted := idx.GetType() == descpb.IndexDescriptor_INVERTED - for j := range desc.GetPublicNonPrimaryIndexes()[i].ColumnIDs { + for j := 0; j < idx.NumColumns(); j++ { // Generate stats for each indexed column. - addIndexColumnStatsIfNotExists(desc.GetPublicNonPrimaryIndexes()[i].ColumnIDs[j], isInverted) + addIndexColumnStatsIfNotExists(idx.GetColumnID(j), isInverted) // Only collect multi-column stats if enabled. if j == 0 || !multiColEnabled { continue } - colIDs := desc.GetPublicNonPrimaryIndexes()[i].ColumnIDs[: j+1 : j+1] + colIDs := make([]descpb.ColumnID, j+1) + for k := 0; k <= j; k++ { + colIDs[k] = idx.GetColumnID(k) + } // Check for existing stats and remember the requested stats. if !trackStatsIfNotExists(colIDs) { @@ -399,8 +402,8 @@ func createStatsDefaultColumns( } // Add columns referenced in partial index predicate expressions. - if desc.GetPublicNonPrimaryIndexes()[i].IsPartial() { - expr, err := parser.ParseExpr(desc.GetPublicNonPrimaryIndexes()[i].Predicate) + if idx.IsPartial() { + expr, err := parser.ParseExpr(idx.GetPredicate()) if err != nil { return nil, err } diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 747257902378..3089afa737d1 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -977,8 +977,8 @@ func addIndexForFK( if err := tbl.AllocateIDs(ctx); err != nil { return 0, err } - added := tbl.GetPublicNonPrimaryIndexes()[len(tbl.GetPublicNonPrimaryIndexes())-1] - return added.ID, nil + added := tbl.PublicNonPrimaryIndexes()[len(tbl.PublicNonPrimaryIndexes())-1] + return added.GetID(), nil } // TODO (lucy): In the EmptyTable case, we add an index mutation, making this @@ -1700,10 +1700,9 @@ func NewTableDesc( return nil, err } - for i := range desc.GetPublicNonPrimaryIndexes() { - idx := &desc.GetPublicNonPrimaryIndexes()[i] + for _, idx := range desc.PublicNonPrimaryIndexes() { // Increment the counter if this index could be storing data across multiple column families. - if len(idx.StoreColumnNames) > 1 && len(desc.Families) > 1 { + if idx.NumStoredColumns() > 1 && len(desc.Families) > 1 { telemetry.Inc(sqltelemetry.SecondaryIndexColumnFamiliesCounter) } } diff --git a/pkg/sql/descriptor_mutation_test.go b/pkg/sql/descriptor_mutation_test.go index 5aa0c07c854e..d90650f22a2f 100644 --- a/pkg/sql/descriptor_mutation_test.go +++ b/pkg/sql/descriptor_mutation_test.go @@ -642,9 +642,9 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR, INDEX foo (v)); // Check that a mutation can only be inserted with an explicit mutation state. tableDesc = mTest.tableDesc - indexIdx := len(tableDesc.GetPublicNonPrimaryIndexes()) - tableDesc.Mutations = []descpb.DescriptorMutation{{Descriptor_: &descpb.DescriptorMutation_Index{Index: &tableDesc.GetPublicNonPrimaryIndexes()[indexIdx-1]}}} - tableDesc.RemovePublicNonPrimaryIndex(indexIdx) + index := tableDesc.PublicNonPrimaryIndexes()[len(tableDesc.PublicNonPrimaryIndexes())-1] + tableDesc.Mutations = []descpb.DescriptorMutation{{Descriptor_: &descpb.DescriptorMutation_Index{Index: index.IndexDesc()}}} + tableDesc.RemovePublicNonPrimaryIndex(index.Ordinal()) if err := tableDesc.ValidateTable(ctx); !testutils.IsError(err, "mutation in state UNKNOWN, direction NONE, index foo, id 2") { t.Fatal(err) } diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 603f80d8d44b..e9ecc462845a 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -960,14 +960,9 @@ func (dsp *DistSQLPlanner) nodeVersionIsCompatible(nodeID roachpb.NodeID) bool { } func getIndexIdx(index *descpb.IndexDescriptor, desc *tabledesc.Immutable) (uint32, error) { - if index.ID == desc.GetPrimaryIndexID() { - return 0, nil - } - for i := range desc.GetPublicNonPrimaryIndexes() { - if index.ID == desc.GetPublicNonPrimaryIndexes()[i].ID { - // IndexIdx is 1 based (0 means primary index). - return uint32(i + 1), nil - } + foundIndex, _ := desc.FindIndexWithID(index.ID) + if foundIndex != nil && foundIndex.Public() { + return uint32(foundIndex.Ordinal()), nil } return 0, errors.Errorf("invalid index %v (table %s)", index, desc.Name) } diff --git a/pkg/sql/distsql_plan_stats.go b/pkg/sql/distsql_plan_stats.go index dbbdce5166fe..2420beb0808b 100644 --- a/pkg/sql/distsql_plan_stats.go +++ b/pkg/sql/distsql_plan_stats.go @@ -130,9 +130,9 @@ func (dsp *DistSQLPlanner) createStatsPlan( // TODO(mjibson): allow multiple inverted indexes on the same column (i.e., // with different configurations). See #50655. col := s.columns[0] - for _, indexDesc := range desc.GetPublicNonPrimaryIndexes() { - if indexDesc.Type == descpb.IndexDescriptor_INVERTED && indexDesc.ColumnIDs[0] == col { - spec.Index = &indexDesc + for _, index := range desc.PublicNonPrimaryIndexes() { + if index.GetType() == descpb.IndexDescriptor_INVERTED && index.GetColumnID(0) == col { + spec.Index = index.IndexDesc() break } } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index a599ea90895e..1ee00a927004 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -312,12 +312,11 @@ 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, 0, len(tableDesc.GetPublicNonPrimaryIndexes())+1) + remainingIndexes := make([]*descpb.IndexDescriptor, 0, len(tableDesc.ActiveIndexes())) remainingIndexes = append(remainingIndexes, tableDesc.GetPrimaryIndex()) - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - index := &tableDesc.GetPublicNonPrimaryIndexes()[i] - if index.ID != idx.ID { - remainingIndexes = append(remainingIndexes, index) + for _, index := range tableDesc.PublicNonPrimaryIndexes() { + if index.GetID() != idx.ID { + remainingIndexes = append(remainingIndexes, index.IndexDesc()) } } @@ -457,51 +456,11 @@ func (p *planner) dropIndexByName( ) } - found := false - for i, idxEntry := range tableDesc.GetPublicNonPrimaryIndexes() { - if idxEntry.ID == idx.ID { - // Unsplit all manually split ranges in the index so they can be - // automatically merged by the merge queue. Gate this on being the - // system tenant because secondary tenants aren't allowed to scan - // the meta ranges directly. - if p.ExecCfg().Codec.ForSystemTenant() { - span := tableDesc.IndexSpan(p.ExecCfg().Codec, idxEntry.ID) - ranges, err := kvclient.ScanMetaKVs(ctx, p.txn, span) - if err != nil { - return err - } - for _, r := range ranges { - var desc roachpb.RangeDescriptor - if err := r.ValueProto(&desc); err != nil { - return err - } - // We have to explicitly check that the range descriptor's start key - // lies within the span of the index since ScanMetaKVs returns all - // intersecting spans. - if !desc.GetStickyBit().IsEmpty() && span.Key.Compare(desc.StartKey.AsRawKey()) <= 0 { - // Swallow "key is not the start of a range" errors because it would - // mean that the sticky bit was removed and merged concurrently. DROP - // INDEX should not fail because of this. - if err := p.ExecCfg().DB.AdminUnsplit(ctx, desc.StartKey); err != nil && !strings.Contains(err.Error(), "is not the start of a range") { - return err - } - } - } - } + foundIndex := tableDesc.FindPublicNonPrimaryIndex(func(idxEntry catalog.Index) bool { + return idxEntry.GetID() == idx.ID + }) - // the idx we picked up with FindIndexByID at the top may not - // contain the same field any more due to other schema changes - // intervening since the initial lookup. So we send the recent - // copy idxEntry for drop instead. - if err := tableDesc.AddIndexMutation(&idxEntry, descpb.DescriptorMutation_DROP); err != nil { - return err - } - tableDesc.RemovePublicNonPrimaryIndex(i + 1) - found = true - break - } - } - if !found { + if foundIndex == nil { return pgerror.Newf( pgcode.ObjectNotInPrerequisiteState, "index %q in the middle of being added, try again later", @@ -509,6 +468,47 @@ func (p *planner) dropIndexByName( ) } + idxEntry := *foundIndex.IndexDesc() + idxOrdinal := foundIndex.Ordinal() + + // Unsplit all manually split ranges in the index so they can be + // automatically merged by the merge queue. Gate this on being the + // system tenant because secondary tenants aren't allowed to scan + // the meta ranges directly. + if p.ExecCfg().Codec.ForSystemTenant() { + span := tableDesc.IndexSpan(p.ExecCfg().Codec, idxEntry.ID) + ranges, err := kvclient.ScanMetaKVs(ctx, p.txn, span) + if err != nil { + return err + } + for _, r := range ranges { + var desc roachpb.RangeDescriptor + if err := r.ValueProto(&desc); err != nil { + return err + } + // We have to explicitly check that the range descriptor's start key + // lies within the span of the index since ScanMetaKVs returns all + // intersecting spans. + if !desc.GetStickyBit().IsEmpty() && span.Key.Compare(desc.StartKey.AsRawKey()) <= 0 { + // Swallow "key is not the start of a range" errors because it would + // mean that the sticky bit was removed and merged concurrently. DROP + // INDEX should not fail because of this. + if err := p.ExecCfg().DB.AdminUnsplit(ctx, desc.StartKey); err != nil && !strings.Contains(err.Error(), "is not the start of a range") { + return err + } + } + } + } + + // the idx we picked up with FindIndexByID at the top may not + // contain the same field any more due to other schema changes + // intervening since the initial lookup. So we send the recent + // copy idxEntry for drop instead. + if err := tableDesc.AddIndexMutation(&idxEntry, descpb.DescriptorMutation_DROP); err != nil { + return err + } + tableDesc.RemovePublicNonPrimaryIndex(idxOrdinal) + if err := p.removeIndexComment(ctx, tableDesc.ID, idx.ID); err != nil { return err } diff --git a/pkg/sql/flowinfra/cluster_test.go b/pkg/sql/flowinfra/cluster_test.go index f001207c7b7d..99587f2fd52c 100644 --- a/pkg/sql/flowinfra/cluster_test.go +++ b/pkg/sql/flowinfra/cluster_test.go @@ -71,7 +71,7 @@ func TestClusterFlow(t *testing.T) { desc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") makeIndexSpan := func(start, end int) execinfrapb.TableReaderSpan { var span roachpb.Span - prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPublicNonPrimaryIndexes()[0].ID)) + prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.PublicNonPrimaryIndexes()[0].GetID())) span.Key = append(prefix, encoding.EncodeVarintAscending(nil, int64(start))...) span.EndKey = append(span.EndKey, prefix...) span.EndKey = append(span.EndKey, encoding.EncodeVarintAscending(nil, int64(end))...) diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 2c906387a545..0ead46194773 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -1717,7 +1717,7 @@ func newOptVirtualTable( // Build the indexes (add 1 to account for lack of primary index in // indexes slice). - ot.indexes = make([]optVirtualIndex, 1+len(ot.desc.GetPublicNonPrimaryIndexes())) + ot.indexes = make([]optVirtualIndex, len(ot.desc.ActiveIndexes())) // Set up the primary index. ot.indexes[0] = optVirtualIndex{ tab: ot, @@ -1730,17 +1730,16 @@ func newOptVirtualTable( }, } - for i := range ot.desc.GetPublicNonPrimaryIndexes() { - idxDesc := &ot.desc.GetPublicNonPrimaryIndexes()[i] - if len(idxDesc.ColumnIDs) > 1 { + for _, idx := range ot.desc.PublicNonPrimaryIndexes() { + if idx.NumColumns() > 1 { panic(errors.AssertionFailedf("virtual indexes with more than 1 col not supported")) } // Add 1, since the 0th index will the primary that we added above. - ot.indexes[i+1] = optVirtualIndex{ + ot.indexes[idx.Ordinal()] = optVirtualIndex{ tab: ot, - desc: idxDesc, - indexOrdinal: i + 1, + desc: idx.IndexDesc(), + indexOrdinal: idx.Ordinal(), // The virtual indexes don't return the bogus PK key? numCols: ot.ColumnCount(), } diff --git a/pkg/sql/partition_test.go b/pkg/sql/partition_test.go index f00219c80e4d..b620bacb121f 100644 --- a/pkg/sql/partition_test.go +++ b/pkg/sql/partition_test.go @@ -63,7 +63,7 @@ func TestRemovePartitioningOSS(t *testing.T) { } { - secondaryIndex := tableDesc.GetPublicNonPrimaryIndexes()[0] + secondaryIndex := *tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc() secondaryIndex.Partitioning = descpb.PartitioningDescriptor{ NumColumns: 1, Range: []descpb.PartitioningDescriptor_Range{{ @@ -109,7 +109,7 @@ func TestRemovePartitioningOSS(t *testing.T) { Config: s.(*server.TestServer).Cfg.DefaultZoneConfig, }, { - IndexID: uint32(tableDesc.GetPublicNonPrimaryIndexes()[0].ID), + IndexID: uint32(tableDesc.PublicNonPrimaryIndexes()[0].GetID()), PartitionName: "p2", Config: s.(*server.TestServer).Cfg.DefaultZoneConfig, }, diff --git a/pkg/sql/pgwire/testdata/pgtest/notice b/pkg/sql/pgwire/testdata/pgtest/notice index 342927ff44b1..648ad7f6a688 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","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":531,"Routine":"dropIndexByName","UnknownFields":null} +{"Severity":"NOTICE","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":528,"Routine":"dropIndexByName","UnknownFields":null} {"Type":"CommandComplete","CommandTag":"DROP INDEX"} until noncrdb_only diff --git a/pkg/sql/physicalplan/span_resolver_test.go b/pkg/sql/physicalplan/span_resolver_test.go index 69d38ad76a2e..90ff68eb6204 100644 --- a/pkg/sql/physicalplan/span_resolver_test.go +++ b/pkg/sql/physicalplan/span_resolver_test.go @@ -169,7 +169,7 @@ func populateCache(db *gosql.DB, expectedNumRows int) error { func splitRangeAtVal( ts *server.TestServer, tableDesc *tabledesc.Immutable, pk int, ) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) { - if len(tableDesc.GetPublicNonPrimaryIndexes()) != 0 { + if len(tableDesc.PublicNonPrimaryIndexes()) != 0 { return roachpb.RangeDescriptor{}, roachpb.RangeDescriptor{}, errors.AssertionFailedf("expected table with just a PK, got: %+v", tableDesc) } diff --git a/pkg/sql/refresh_materialized_view.go b/pkg/sql/refresh_materialized_view.go index 692ddde82121..8df3efb73927 100644 --- a/pkg/sql/refresh_materialized_view.go +++ b/pkg/sql/refresh_materialized_view.go @@ -69,9 +69,9 @@ func (n *refreshMaterializedViewNode) startExec(params runParams) error { // Prepare the new set of indexes by cloning all existing indexes on the view. newPrimaryIndex := protoutil.Clone(n.desc.GetPrimaryIndex()).(*descpb.IndexDescriptor) - newIndexes := make([]descpb.IndexDescriptor, len(n.desc.GetPublicNonPrimaryIndexes())) - for i := range n.desc.GetPublicNonPrimaryIndexes() { - newIndexes[i] = *protoutil.Clone(&n.desc.GetPublicNonPrimaryIndexes()[i]).(*descpb.IndexDescriptor) + newIndexes := make([]descpb.IndexDescriptor, len(n.desc.PublicNonPrimaryIndexes())) + for i, idx := range n.desc.PublicNonPrimaryIndexes() { + newIndexes[i] = idx.IndexDescDeepCopy() } // Reset and allocate new IDs for the new indexes. diff --git a/pkg/sql/rename_column.go b/pkg/sql/rename_column.go index ae31effd7962..0d39b3a240ba 100644 --- a/pkg/sql/rename_column.go +++ b/pkg/sql/rename_column.go @@ -178,13 +178,15 @@ func (p *planner) renameColumn( } // Rename the column in partial index predicates. - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - if index := &tableDesc.GetPublicNonPrimaryIndexes()[i]; index.IsPartial() { - newExpr, err := schemaexpr.RenameColumn(index.Predicate, *oldName, *newName) + for _, index := range tableDesc.PublicNonPrimaryIndexes() { + if index.IsPartial() { + newExpr, err := schemaexpr.RenameColumn(index.GetPredicate(), *oldName, *newName) if err != nil { return false, err } - index.Predicate = newExpr + indexDesc := *index.IndexDesc() + indexDesc.Predicate = newExpr + tableDesc.SetPublicNonPrimaryIndex(index.Ordinal(), indexDesc) } } diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index f196d5fd99cd..57cea21ff34d 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -1418,7 +1418,12 @@ func (rf *Fetcher) checkPrimaryIndexDatumEncodings(ctx context.Context) error { return nil }) - rh := rowHelper{TableDesc: table.desc, Indexes: table.desc.GetPublicNonPrimaryIndexes()} + 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} return table.desc.ForeachFamily(func(family *descpb.ColumnFamilyDescriptor) error { var lastColID descpb.ColumnID diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index a911085d33e8..c5ab2da47d8b 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -49,22 +49,13 @@ func makeFetcherArgs(entries []initFetcherArgs) []FetcherTableArgs { fetcherArgs := make([]FetcherTableArgs, len(entries)) for i, entry := range entries { - var index *descpb.IndexDescriptor - var isSecondaryIndex bool - - if entry.indexIdx > 0 { - index = &entry.tableDesc.GetPublicNonPrimaryIndexes()[entry.indexIdx-1] - isSecondaryIndex = true - } else { - index = entry.tableDesc.GetPrimaryIndex() - } - + index := entry.tableDesc.ActiveIndexes()[entry.indexIdx] fetcherArgs[i] = FetcherTableArgs{ Spans: entry.spans, Desc: entry.tableDesc, - Index: index, + Index: index.IndexDesc(), ColIdxMap: entry.tableDesc.ColumnIdxMap(), - IsSecondaryIndex: isSecondaryIndex, + IsSecondaryIndex: !index.Primary(), Cols: entry.tableDesc.Columns, ValNeededForCol: entry.valNeededForCol, } @@ -655,7 +646,7 @@ func TestNextRowSecondaryIndex(t *testing.T) { if err := rf.StartScan( context.Background(), kv.NewTxn(ctx, kvDB, 0), - roachpb.Spans{tableDesc.IndexSpan(keys.SystemSQLCodec, tableDesc.GetPublicNonPrimaryIndexes()[0].ID)}, + roachpb.Spans{tableDesc.IndexSpan(keys.SystemSQLCodec, tableDesc.PublicNonPrimaryIndexes()[0].GetID())}, false, /*limitBatches*/ 0, /*limitHint*/ false, /*traceKV*/ @@ -677,10 +668,10 @@ func TestNextRowSecondaryIndex(t *testing.T) { count++ - if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPublicNonPrimaryIndexes()[0].ID { + if desc.GetID() != tableDesc.ID || index.ID != tableDesc.PublicNonPrimaryIndexes()[0].GetID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.GetPublicNonPrimaryIndexes()[0].Name, + tableDesc.Name, tableDesc.PublicNonPrimaryIndexes()[0].GetName(), desc.GetName(), index.Name, ) } @@ -987,12 +978,7 @@ func TestNextRowInterleaved(t *testing.T) { idLookups := make(map[uint64]*fetcherEntryArgs, len(entries)) for i, entry := range entries { tableDesc := catalogkv.TestingGetImmutableTableDescriptor(kvDB, keys.SystemSQLCodec, sqlutils.TestDB, entry.tableName) - var indexID descpb.IndexID - if entry.indexIdx == 0 { - indexID = tableDesc.GetPrimaryIndexID() - } else { - indexID = tableDesc.GetPublicNonPrimaryIndexes()[entry.indexIdx-1].ID - } + indexID := tableDesc.ActiveIndexes()[entry.indexIdx].GetID() idLookups[idLookupKey(tableDesc.ID, indexID)] = entry // We take every entry's index span (primary or diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index 1096a1cf3e1b..f23d2b7729b2 100644 --- a/pkg/sql/row/row_converter.go +++ b/pkg/sql/row/row_converter.go @@ -414,7 +414,7 @@ func NewDatumRowConverter( return nil, errors.New("unexpected hidden column") } - padding := 2 * (len(tableDesc.GetPublicNonPrimaryIndexes()) + len(tableDesc.Families)) + padding := 2 * (len(tableDesc.PublicNonPrimaryIndexes()) + len(tableDesc.Families)) c.BatchCap = kvDatumRowConverterBatchSize + padding c.KvBatch.KVs = make([]roachpb.KeyValue, 0, c.BatchCap) diff --git a/pkg/sql/rowenc/client_index_encoding_test.go b/pkg/sql/rowenc/client_index_encoding_test.go index 7a1e207f6408..755878279ab1 100644 --- a/pkg/sql/rowenc/client_index_encoding_test.go +++ b/pkg/sql/rowenc/client_index_encoding_test.go @@ -24,7 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "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/rowenc" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -57,14 +56,14 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { child := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, sqlutils.TestDB, "child1") grandchild := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, sqlutils.TestDB, "grandchild1") - parentDescIdx := parent.GetPublicNonPrimaryIndexes()[0] - childDescIdx := child.GetPublicNonPrimaryIndexes()[0] - childNonUniqueIdx := child.GetPublicNonPrimaryIndexes()[1] - childUniqueIdx := child.GetPublicNonPrimaryIndexes()[2] - grandchildDescIdx := grandchild.GetPublicNonPrimaryIndexes()[0] + parentDescIdx := parent.PublicNonPrimaryIndexes()[0] + childDescIdx := child.PublicNonPrimaryIndexes()[0] + childNonUniqueIdx := child.PublicNonPrimaryIndexes()[1] + childUniqueIdx := child.PublicNonPrimaryIndexes()[2] + grandchildDescIdx := grandchild.PublicNonPrimaryIndexes()[0] testCases := []struct { - index *descpb.IndexDescriptor + index catalog.Index // See ShortToLongKeyFmt for how to represent a key. input string expected string @@ -75,29 +74,29 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // See encodeStartConstraintAscending. { - index: parent.GetPrimaryIndex(), + index: parent.PrimaryIndexInterface(), input: "/NOTNULLASC", expected: "/NOTNULLASC", }, { - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NOTNULLASC", }, { - index: grandchild.GetPrimaryIndex(), + index: grandchild.PrimaryIndexInterface(), input: "/1/#/2/3/#/NOTNULLASC", expected: "/1/#/2/3/#/NOTNULLASC", }, { - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/NOTNULLASC", expected: "/1/#/NOTNULLASC", }, { - index: grandchild.GetPrimaryIndex(), + index: grandchild.PrimaryIndexInterface(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NOTNULLASC", }, @@ -108,23 +107,23 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // See encodeStartConstraintDescending. { - index: &parentDescIdx, + index: parentDescIdx, input: "/NULLDESC", expected: "/NULLDESC", }, { - index: &childDescIdx, + index: childDescIdx, input: "/1/#/2/NULLDESC", expected: "/1/#/2/NULLDESC", }, { - index: &grandchildDescIdx, + index: grandchildDescIdx, input: "/1/#/2/3/#/NULLDESC", expected: "/1/#/2/3/#/NULLDESC", }, { - index: &childDescIdx, + index: childDescIdx, input: "/1/#/NULLDESC", expected: "/1/#/NULLDESC", }, @@ -132,19 +131,19 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // Keys that belong to the given index (neither parent nor // children keys) do not need to be tightened. { - index: parent.GetPrimaryIndex(), + index: parent.PrimaryIndexInterface(), input: "/1", expected: "/1", }, { - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/3", expected: "/1/#/2/3", }, // Parent keys wrt child index is not tightened. { - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1", expected: "/1", }, @@ -152,24 +151,24 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // Children keys wrt to parent index is tightened (pushed // forwards) to the next parent key. { - index: parent.GetPrimaryIndex(), + index: parent.PrimaryIndexInterface(), input: "/1/#/2/3", expected: "/2", }, { - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/3/#/4", expected: "/1/#/2/4", }, // Key with len > 1 tokens. { - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/12345678901234/#/1234/1234567890/#/123/1234567", expected: "/12345678901234/#/1234/1234567891", }, { - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/12345678901234/#/d1403.2594/shelloworld/#/123/1234567", expected: "/12345678901234/#/d1403.2594/shelloworld/PrefixEnd", }, @@ -178,22 +177,22 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // We should expect two extra columns (in addition to the // two index columns). { - index: &childNonUniqueIdx, + index: childNonUniqueIdx, input: "/2/3", expected: "/2/3", }, { - index: &childNonUniqueIdx, + index: childNonUniqueIdx, input: "/2/3/4", expected: "/2/3/4", }, { - index: &childNonUniqueIdx, + index: childNonUniqueIdx, input: "/2/3/4/5", expected: "/2/3/4/5", }, { - index: &childNonUniqueIdx, + index: childNonUniqueIdx, input: "/2/3/4/5/#/10", expected: "/2/3/4/6", }, @@ -201,27 +200,27 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // Unique indexes only include implicit columns if they have // a NULL value. { - index: &childUniqueIdx, + index: childUniqueIdx, input: "/2/3", expected: "/2/3", }, { - index: &childUniqueIdx, + index: childUniqueIdx, input: "/2/3/4", expected: "/2/4", }, { - index: &childUniqueIdx, + index: childUniqueIdx, input: "/2/NULLASC/4", expected: "/2/NULLASC/4", }, { - index: &childUniqueIdx, + index: childUniqueIdx, input: "/2/NULLASC/4/5", expected: "/2/NULLASC/4/5", }, { - index: &childUniqueIdx, + index: childUniqueIdx, input: "/2/NULLASC/4/5/#/6", expected: "/2/NULLASC/4/6", }, @@ -231,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, actual) + actual, err := rowenc.AdjustStartKeyForInterleave(codec, tc.index.IndexDesc(), actual) if err != nil { t.Fatal(err) } @@ -269,15 +268,15 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { child := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, sqlutils.TestDB, "child1") grandchild := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, sqlutils.TestDB, "grandchild1") - parentDescIdx := parent.GetPublicNonPrimaryIndexes()[0] - childDescIdx := child.GetPublicNonPrimaryIndexes()[0] - childNonUniqueIdx := child.GetPublicNonPrimaryIndexes()[1] - childUniqueIdx := child.GetPublicNonPrimaryIndexes()[2] - grandchildDescIdx := grandchild.GetPublicNonPrimaryIndexes()[0] + parentDescIdx := parent.PublicNonPrimaryIndexes()[0] + childDescIdx := child.PublicNonPrimaryIndexes()[0] + childNonUniqueIdx := child.PublicNonPrimaryIndexes()[1] + childUniqueIdx := child.PublicNonPrimaryIndexes()[2] + grandchildDescIdx := grandchild.PublicNonPrimaryIndexes()[0] testCases := []struct { table catalog.TableDescriptor - index *descpb.IndexDescriptor + index catalog.Index // See ShortToLongKeyFmt for how to represent a key. input string // If the end key is assumed to be inclusive when passed to @@ -292,21 +291,21 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.GetPrimaryIndex(), + index: parent.PrimaryIndexInterface(), input: "/NOTNULLASC", expected: "/NULLASC/#", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NULLASC/#", }, { table: grandchild, - index: grandchild.GetPrimaryIndex(), + index: grandchild.PrimaryIndexInterface(), input: "/1/#/2/3/#/NOTNULLASC", expected: "/1/#/2/3/#/NULLASC/#", }, @@ -315,7 +314,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // partial primary key columns. { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/NOTNULLASC", expected: "/1/#/NOTNULLASC", }, @@ -323,7 +322,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // No change since key belongs to an ancestor. { table: grandchild, - index: grandchild.GetPrimaryIndex(), + index: grandchild.PrimaryIndexInterface(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NOTNULLASC", }, @@ -339,25 +338,25 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: &parentDescIdx, + index: parentDescIdx, input: "/NOTNULLDESC", expected: "/NOTNULLDESC", }, { table: child, - index: &childDescIdx, + index: childDescIdx, input: "/1/#/2/NOTNULLDESC", expected: "/1/#/2/NOTNULLDESC", }, { table: grandchild, - index: &grandchildDescIdx, + index: grandchildDescIdx, input: "/1/#/2/3/#/NOTNULLDESC", expected: "/1/#/2/3/#/NOTNULLDESC", }, { table: grandchild, - index: &grandchildDescIdx, + index: grandchildDescIdx, input: "/1/#/2/NOTNULLDESC", expected: "/1/#/2/NOTNULLDESC", }, @@ -368,7 +367,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.GetPrimaryIndex(), + index: parent.PrimaryIndexInterface(), input: "/NULLASC", inclusive: true, expected: "/NULLASC/#", @@ -376,7 +375,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/NULLASC", inclusive: true, expected: "/1/#/2/NULLASC/#", @@ -388,13 +387,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.GetPrimaryIndex(), + index: parent.PrimaryIndexInterface(), input: "/1", expected: "/0/#", }, { table: parent, - index: parent.GetPrimaryIndex(), + index: parent.PrimaryIndexInterface(), input: "/1", inclusive: true, expected: "/1/#", @@ -402,13 +401,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/3", expected: "/1/#/2/2/#", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/3", inclusive: true, expected: "/1/#/2/3/#", @@ -418,19 +417,19 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.GetPrimaryIndex(), + index: parent.PrimaryIndexInterface(), input: "/1/#", expected: "/1/#", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#", expected: "/1/#", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/2/#", expected: "/1/#/2/2/#", }, @@ -440,13 +439,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.GetPrimaryIndex(), + index: parent.PrimaryIndexInterface(), input: "/1/#/2/3", expected: "/1/#", }, { table: parent, - index: parent.GetPrimaryIndex(), + index: parent.PrimaryIndexInterface(), input: "/1/#/2/3", inclusive: true, expected: "/1/#", @@ -454,13 +453,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/3/#/4", expected: "/1/#/2/3/#", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/3/#/4", inclusive: true, expected: "/1/#/2/3/#", @@ -470,13 +469,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1", expected: "/1", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1", inclusive: true, expected: "/2", @@ -487,13 +486,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2", expected: "/1/#/2", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2", inclusive: true, expected: "/1/#/3", @@ -504,26 +503,26 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: &childDescIdx, + index: childDescIdx, input: "/1/#/2/3", expected: "/1/#/2/3", }, { table: child, - index: &childDescIdx, + index: childDescIdx, input: "/1/#/2/3", inclusive: true, expected: "/1/#/2/4", }, { table: child, - index: &childDescIdx, + index: childDescIdx, input: "/1/#/2", expected: "/1/#/2", }, { table: child, - index: &childDescIdx, + index: childDescIdx, input: "/1/#/2", inclusive: true, expected: "/1/#/3", @@ -532,7 +531,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // Key with len > 1 tokens. { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/12345678901234/#/12345/12345678901234/#/123/1234567", expected: "/12345678901234/#/12345/12345678901234/#", }, @@ -542,19 +541,19 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // two index columns). { table: child, - index: &childNonUniqueIdx, + index: childNonUniqueIdx, input: "/2/3", expected: "/2/3", }, { table: child, - index: &childNonUniqueIdx, + index: childNonUniqueIdx, input: "/2/3/4", expected: "/2/3/4", }, { table: child, - index: &childNonUniqueIdx, + index: childNonUniqueIdx, input: "/2/3/4/5", expected: "/2/3/4/5", }, @@ -562,14 +561,14 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // interleaved rows. { table: child, - index: &childNonUniqueIdx, + index: childNonUniqueIdx, input: "/2/3/4/5/#/10", expected: "/2/3/4/5/#/10", }, { table: child, - index: &childUniqueIdx, + index: childUniqueIdx, input: "/2/3", expected: "/2/3", }, @@ -577,19 +576,19 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // interleaved rows. { table: child, - index: &childUniqueIdx, + index: childUniqueIdx, input: "/2/3/4", expected: "/2/3/4", }, { table: child, - index: &childUniqueIdx, + index: childUniqueIdx, input: "/2/NULLASC/4", expected: "/2/NULLASC/4", }, { table: child, - index: &childUniqueIdx, + index: childUniqueIdx, input: "/2/NULLASC/4/5", expected: "/2/NULLASC/4/5", }, @@ -597,7 +596,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // interleaved rows. { table: child, - index: &childUniqueIdx, + index: childUniqueIdx, input: "/2/NULLASC/4/5/#/6", expected: "/2/NULLASC/4/5/#/6", }, @@ -607,20 +606,20 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // decimal value. { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/d3.4567", expected: "/1/#/2/d3.4567", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/d3.4567", inclusive: true, expected: "/1/#/2/d3.4567/#", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/d3.4567/#/8", expected: "/1/#/2/d3.4567/#", }, @@ -630,20 +629,20 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // bytes value. { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/shelloworld", expected: "/1/#/2/shelloworld", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/shelloworld", inclusive: true, expected: "/1/#/2/shelloworld/#", }, { table: child, - index: child.GetPrimaryIndex(), + index: child.PrimaryIndexInterface(), input: "/1/#/2/shelloworld/#/3", expected: "/1/#/2/shelloworld/#", }, @@ -653,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, actual, tc.inclusive) + actual, err := rowenc.AdjustEndKeyForInterleave(codec, tc.table, tc.index.IndexDesc(), actual, tc.inclusive) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/rowenc/index_encoding.go b/pkg/sql/rowenc/index_encoding.go index e37db2b86bf4..83a4857bbfa8 100644 --- a/pkg/sql/rowenc/index_encoding.go +++ b/pkg/sql/rowenc/index_encoding.go @@ -546,7 +546,7 @@ func DecodeIndexKeyPrefix( // TODO(dan): This whole operation is n^2 because of the interleaves // bookkeeping. We could improve it to n with a prefix tree of components. - interleaves := append([]descpb.IndexDescriptor{*desc.GetPrimaryIndex()}, desc.GetPublicNonPrimaryIndexes()...) + interleaves := desc.ActiveIndexes() for component := 0; ; component++ { var tableID descpb.ID @@ -561,9 +561,9 @@ func DecodeIndexKeyPrefix( } for i := len(interleaves) - 1; i >= 0; i-- { - if len(interleaves[i].Interleave.Ancestors) <= component || - interleaves[i].Interleave.Ancestors[component].TableID != tableID || - interleaves[i].Interleave.Ancestors[component].IndexID != indexID { + if interleaves[i].NumInterleaveAncestors() <= component || + interleaves[i].GetInterleaveAncestor(component).TableID != tableID || + interleaves[i].GetInterleaveAncestor(component).IndexID != indexID { // This component, and thus this interleave, doesn't match what was // decoded, remove it. @@ -578,7 +578,7 @@ func DecodeIndexKeyPrefix( // Anything left has the same SharedPrefixLen at index `component`, so just // use the first one. - for i := uint32(0); i < interleaves[0].Interleave.Ancestors[component].SharedPrefixLen; i++ { + for i := uint32(0); i < interleaves[0].GetInterleaveAncestor(component).SharedPrefixLen; i++ { l, err := encoding.PeekLength(key) if err != nil { return 0, nil, err diff --git a/pkg/sql/rowenc/index_encoding_test.go b/pkg/sql/rowenc/index_encoding_test.go index 9310878129eb..1f56d12717a4 100644 --- a/pkg/sql/rowenc/index_encoding_test.go +++ b/pkg/sql/rowenc/index_encoding_test.go @@ -233,7 +233,7 @@ func TestIndexKey(t *testing.T) { primaryIndexKV := kv.KeyValue{Key: primaryKey, Value: &primaryValue} secondaryIndexEntry, err := EncodeSecondaryIndex( - codec, tableDesc, &tableDesc.GetPublicNonPrimaryIndexes()[0], colMap, testValues, true /* includeEmpty */) + codec, tableDesc, tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), colMap, testValues, true /* includeEmpty */) if len(secondaryIndexEntry) != 1 { t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndexEntry), secondaryIndexEntry) } @@ -276,7 +276,7 @@ func TestIndexKey(t *testing.T) { } checkEntry(tableDesc.GetPrimaryIndex(), primaryIndexKV) - checkEntry(&tableDesc.GetPublicNonPrimaryIndexes()[0], secondaryIndexKV) + checkEntry(tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), secondaryIndexKV) } } @@ -377,8 +377,8 @@ func TestInvertedIndexKey(t *testing.T) { indexKeyTest{50, nil, nil, primaryValues, secondaryValues, }) - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - tableDesc.GetPublicNonPrimaryIndexes()[i].Version = version + for _, idx := range tableDesc.PublicNonPrimaryIndexes() { + idx.IndexDesc().Version = version } testValues := append(primaryValues, secondaryValues...) @@ -386,7 +386,7 @@ func TestInvertedIndexKey(t *testing.T) { codec := keys.SystemSQLCodec secondaryIndexEntries, err := EncodeSecondaryIndex( - codec, tableDesc, &tableDesc.GetPublicNonPrimaryIndexes()[0], colMap, testValues, true /* includeEmpty */) + codec, tableDesc, tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), colMap, testValues, true /* includeEmpty */) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index d82f7fa2e548..8f0d1cdef2ca 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -140,6 +140,7 @@ go_test( "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/backfill", + "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/colinfo", diff --git a/pkg/sql/rowexec/joinreader_test.go b/pkg/sql/rowexec/joinreader_test.go index d295cb3185d9..8ddbfde95fe8 100644 --- a/pkg/sql/rowexec/joinreader_test.go +++ b/pkg/sql/rowexec/joinreader_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "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/catalog/tabledesc" @@ -1273,18 +1274,14 @@ func BenchmarkJoinReader(b *testing.B) { // Get the table descriptor and find the index that will provide us with // the expected match ratio. tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", tableName) - indexIdx := uint32(0) - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - require.Equal(b, 1, len(tableDesc.GetPublicNonPrimaryIndexes()[i].ColumnNames), "all indexes created in this benchmark should only contain one column") - if tableDesc.GetPublicNonPrimaryIndexes()[i].ColumnNames[0] == columnDef.name { - // Found indexIdx. - indexIdx = uint32(i + 1) - break - } - } - if indexIdx == 0 { + foundIndex := tableDesc.FindPublicNonPrimaryIndex(func(idx catalog.Index) bool { + require.Equal(b, 1, idx.NumColumns(), "all indexes created in this benchmark should only contain one column") + return idx.GetColumnName(0) == columnDef.name + }) + if foundIndex == nil { b.Fatalf("failed to find secondary index for column %s", columnDef.name) } + indexIdx := uint32(foundIndex.Ordinal()) input := newRowGeneratingSource(rowenc.OneIntCol, sqlutils.ToRowFn(func(rowIdx int) tree.Datum { // Convert to 0-based. return tree.NewDInt(tree.DInt(rowIdx - 1)) diff --git a/pkg/sql/rowexec/scrub_tablereader.go b/pkg/sql/rowexec/scrub_tablereader.go index c0cd6011f7db..f5a3fd3be4f7 100644 --- a/pkg/sql/rowexec/scrub_tablereader.go +++ b/pkg/sql/rowexec/scrub_tablereader.go @@ -149,12 +149,7 @@ func (tr *scrubTableReader) generateScrubErrorRow( row rowenc.EncDatumRow, scrubErr *scrub.Error, ) (rowenc.EncDatumRow, error) { details := make(map[string]interface{}) - var index *descpb.IndexDescriptor - if tr.indexIdx == 0 { - index = tr.tableDesc.GetPrimaryIndex() - } else { - index = &tr.tableDesc.GetPublicNonPrimaryIndexes()[tr.indexIdx-1] - } + index := tr.tableDesc.ActiveIndexes()[tr.indexIdx] // Collect all the row values into JSON rowDetails := make(map[string]interface{}) for i, colIdx := range tr.fetcherResultToColIdx { @@ -163,7 +158,7 @@ func (tr *scrubTableReader) generateScrubErrorRow( rowDetails[col.Name] = row[i].String(col.Type) } details["row_data"] = rowDetails - details["index_name"] = index.Name + details["index_name"] = index.GetName() details["error_message"] = scrub.UnwrapScrubError(error(scrubErr)).Error() detailsJSON, err := tree.MakeDJSON(details) diff --git a/pkg/sql/rowexec/tablereader_test.go b/pkg/sql/rowexec/tablereader_test.go index d7fa4ae09c08..af472b0dfaa1 100644 --- a/pkg/sql/rowexec/tablereader_test.go +++ b/pkg/sql/rowexec/tablereader_test.go @@ -72,7 +72,7 @@ func TestTableReader(t *testing.T) { makeIndexSpan := func(start, end int) execinfrapb.TableReaderSpan { var span roachpb.Span - prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, td, td.GetPublicNonPrimaryIndexes()[0].ID)) + prefix := roachpb.Key(rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, td, td.PublicNonPrimaryIndexes()[0].GetID())) span.Key = append(prefix, encoding.EncodeVarintAscending(nil, int64(start))...) span.EndKey = append(span.EndKey, prefix...) span.EndKey = append(span.EndKey, encoding.EncodeVarintAscending(nil, int64(end))...) diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index 2eee1170d716..4520fe8bab8e 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -419,11 +419,7 @@ func (z *zigzagJoiner) setupInfo( info.table = &tables[side] info.eqColumns = spec.EqColumns[side].Columns indexOrdinal := spec.IndexOrdinals[side] - if indexOrdinal == 0 { - info.index = info.table.GetPrimaryIndex() - } else { - info.index = &info.table.GetPublicNonPrimaryIndexes()[indexOrdinal-1] - } + info.index = info.table.ActiveIndexes()[indexOrdinal].IndexDesc() var columnIDs []descpb.ColumnID columnIDs, info.indexDirs = info.index.FullColumnIDs() diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 0d66d4d1b34e..4651a48b08a5 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -230,35 +230,18 @@ func (n *scanNode) initTable( func (n *scanNode) lookupSpecifiedIndex(indexFlags *tree.IndexFlags) error { if indexFlags.Index != "" { // Search index by name. - indexName := string(indexFlags.Index) - if indexName == n.desc.GetPrimaryIndex().Name { - n.specifiedIndex = n.desc.GetPrimaryIndex() - } else { - for i := range n.desc.GetPublicNonPrimaryIndexes() { - if indexName == n.desc.GetPublicNonPrimaryIndexes()[i].Name { - n.specifiedIndex = &n.desc.GetPublicNonPrimaryIndexes()[i] - break - } - } - } - if n.specifiedIndex == nil { + foundIndex, _ := n.desc.FindIndexWithName(string(indexFlags.Index)) + if foundIndex == nil || !foundIndex.Public() { return errors.Errorf("index %q not found", tree.ErrString(&indexFlags.Index)) } + n.specifiedIndex = foundIndex.IndexDesc() } else if indexFlags.IndexID != 0 { // Search index by ID. - if n.desc.GetPrimaryIndexID() == descpb.IndexID(indexFlags.IndexID) { - n.specifiedIndex = n.desc.GetPrimaryIndex() - } else { - for i := range n.desc.GetPublicNonPrimaryIndexes() { - if n.desc.GetPublicNonPrimaryIndexes()[i].ID == descpb.IndexID(indexFlags.IndexID) { - n.specifiedIndex = &n.desc.GetPublicNonPrimaryIndexes()[i] - break - } - } - } - if n.specifiedIndex == nil { + 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() } return nil } diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index ccabfd4985cd..8d0599a2eb1f 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1181,8 +1181,8 @@ func (sc *SchemaChanger) done(ctx context.Context) error { return err } childJobs = append(childJobs, pkJob) - for i := range scTable.GetPublicNonPrimaryIndexes() { - idxJob, err := sc.createIndexGCJob(ctx, &scTable.GetPublicNonPrimaryIndexes()[i], txn, desc) + for _, idx := range scTable.PublicNonPrimaryIndexes() { + idxJob, err := sc.createIndexGCJob(ctx, idx.IndexDesc(), txn, desc) if err != nil { return err } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 9e0f8945cd04..f2f3c07e5a43 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -143,7 +143,7 @@ INSERT INTO t.test VALUES ('a', 'b'), ('c', 'd'); // Check that RunStateMachineBeforeBackfill functions properly. expectedVersion = tableDesc.Version // Make a copy of the index for use in a mutation. - index := protoutil.Clone(&tableDesc.GetPublicNonPrimaryIndexes()[0]).(*descpb.IndexDescriptor) + index := tableDesc.PublicNonPrimaryIndexes()[0].IndexDescDeepCopy() index.Name = "bar" index.ID = tableDesc.NextIndexID tableDesc.NextIndexID++ @@ -152,7 +152,7 @@ INSERT INTO t.test VALUES ('a', 'b'), ('c', 'd'); &execCfg, cluster.MakeTestingClusterSettings(), ) tableDesc.Mutations = append(tableDesc.Mutations, descpb.DescriptorMutation{ - Descriptor_: &descpb.DescriptorMutation_Index{Index: index}, + Descriptor_: &descpb.DescriptorMutation_Index{Index: &index}, Direction: descpb.DescriptorMutation_ADD, State: descpb.DescriptorMutation_DELETE_ONLY, MutationID: tableDesc.NextMutationID, @@ -251,7 +251,7 @@ CREATE INDEX foo ON t.test (v) for r := retry.Start(retryOpts); r.Next(); { tableDesc = catalogkv.TestingGetMutableExistingTableDescriptor( kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetPublicNonPrimaryIndexes()) == 1 { + if len(tableDesc.PublicNonPrimaryIndexes()) == 1 { break } } @@ -278,7 +278,7 @@ CREATE INDEX foo ON t.test (v) // Ensure that the version gets incremented. tableDesc = catalogkv.TestingGetMutableExistingTableDescriptor( kvDB, keys.SystemSQLCodec, "t", "test") - name := tableDesc.GetPublicNonPrimaryIndexes()[0].Name + name := tableDesc.PublicNonPrimaryIndexes()[0].GetName() if name != "ufo" { t.Fatalf("bad index name %s", name) } @@ -298,7 +298,7 @@ CREATE INDEX foo ON t.test (v) for r := retry.Start(retryOpts); r.Next(); { tableDesc = catalogkv.TestingGetMutableExistingTableDescriptor( kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetPublicNonPrimaryIndexes()) == count+1 { + if len(tableDesc.PublicNonPrimaryIndexes()) == count+1 { break } } @@ -2218,8 +2218,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT UNIQUE DEFAULT 23 CREATE FAMILY F3 // The index is not regenerated. tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetPublicNonPrimaryIndexes()) > 0 { - t.Fatalf("indexes %+v", tableDesc.GetPublicNonPrimaryIndexes()) + if len(tableDesc.PublicNonPrimaryIndexes()) > 0 { + t.Fatalf("indexes %+v", tableDesc.PublicNonPrimaryIndexes()) } // Unfortunately this is the same failure present when an index drop @@ -3663,16 +3663,17 @@ CREATE TABLE d.t ( kvDB, keys.SystemSQLCodec, "d", "t") // Verify that this descriptor uses the new STORING encoding. Overwrite it // with one that uses the old encoding. - for i, index := range tableDesc.GetPublicNonPrimaryIndexes() { - if len(index.ExtraColumnIDs) != 1 { + for _, index := range tableDesc.PublicNonPrimaryIndexes() { + if index.NumExtraColumns() != 1 { t.Fatalf("ExtraColumnIDs not set properly: %s", tableDesc) } - if len(index.StoreColumnIDs) != 1 { + if index.NumStoredColumns() != 1 { t.Fatalf("StoreColumnIDs not set properly: %s", tableDesc) } - index.ExtraColumnIDs = append(index.ExtraColumnIDs, index.StoreColumnIDs...) - index.StoreColumnIDs = nil - tableDesc.SetPublicNonPrimaryIndex(i+1, index) + newIndexDesc := *index.IndexDesc() + newIndexDesc.ExtraColumnIDs = append(newIndexDesc.ExtraColumnIDs, newIndexDesc.StoreColumnIDs...) + newIndexDesc.StoreColumnIDs = nil + tableDesc.SetPublicNonPrimaryIndex(index.Ordinal(), newIndexDesc) } if err := kvDB.Put( context.Background(), @@ -5108,8 +5109,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetPublicNonPrimaryIndexes()) > 0 || len(tableDesc.Mutations) > 0 { - t.Fatalf("descriptor broken %d, %d", len(tableDesc.GetPublicNonPrimaryIndexes()), len(tableDesc.Mutations)) + if len(tableDesc.PublicNonPrimaryIndexes()) > 0 || len(tableDesc.Mutations) > 0 { + t.Fatalf("descriptor broken %d, %d", len(tableDesc.PublicNonPrimaryIndexes()), len(tableDesc.Mutations)) } } @@ -5181,8 +5182,8 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v JSON); } tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetPublicNonPrimaryIndexes()) > 0 || len(tableDesc.Mutations) > 0 { - t.Fatalf("descriptor broken %d, %d", len(tableDesc.GetPublicNonPrimaryIndexes()), len(tableDesc.Mutations)) + if len(tableDesc.PublicNonPrimaryIndexes()) > 0 || len(tableDesc.Mutations) > 0 { + t.Fatalf("descriptor broken %d, %d", len(tableDesc.PublicNonPrimaryIndexes()), len(tableDesc.Mutations)) } } @@ -5888,7 +5889,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT8); // Wait until indexes are created. for r := retry.Start(retryOpts); r.Next(); { tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - if len(tableDesc.GetPublicNonPrimaryIndexes()) == 1 { + if len(tableDesc.PublicNonPrimaryIndexes()) == 1 { break } } diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index ea7688858a58..59d1dcdbacda 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -345,9 +345,8 @@ func pairwiseOp(left []string, right []string, op string) []string { func createPhysicalCheckOperations( tableDesc *tabledesc.Immutable, tableName *tree.TableName, ) (checks []checkOperation) { - checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, tableDesc.GetPrimaryIndex())) - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, &tableDesc.GetPublicNonPrimaryIndexes()[i])) + for _, idx := range tableDesc.ActiveIndexes() { + checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, idx.IndexDesc())) } return checks } @@ -367,11 +366,11 @@ func createIndexCheckOperations( if indexNames == nil { // Populate results with all secondary indexes of the // table. - for i := range tableDesc.GetPublicNonPrimaryIndexes() { + for _, idx := range tableDesc.PublicNonPrimaryIndexes() { results = append(results, newIndexCheckOperation( tableName, tableDesc, - &tableDesc.GetPublicNonPrimaryIndexes()[i], + idx.IndexDesc(), asOf, )) } @@ -383,15 +382,15 @@ func createIndexCheckOperations( for _, idxName := range indexNames { names[idxName.String()] = struct{}{} } - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - if _, ok := names[tableDesc.GetPublicNonPrimaryIndexes()[i].Name]; ok { + for _, idx := range tableDesc.PublicNonPrimaryIndexes() { + if _, ok := names[idx.GetName()]; ok { results = append(results, newIndexCheckOperation( tableName, tableDesc, - &tableDesc.GetPublicNonPrimaryIndexes()[i], + idx.IndexDesc(), asOf, )) - delete(names, tableDesc.GetPublicNonPrimaryIndexes()[i].Name) + delete(names, idx.GetName()) } } if len(names) > 0 { diff --git a/pkg/sql/scrub_test.go b/pkg/sql/scrub_test.go index cc1d74200b01..6b851e88f2e1 100644 --- a/pkg/sql/scrub_test.go +++ b/pkg/sql/scrub_test.go @@ -60,7 +60,7 @@ INSERT INTO t."tEst" VALUES (10, 20); // Construct datums for our row values (k, v). values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(20)} tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "tEst") - secondaryIndex := &tableDesc.GetPublicNonPrimaryIndexes()[0] + secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap colIDtoRowIndex.Set(tableDesc.Columns[0].ID, 0) @@ -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, colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -129,7 +129,7 @@ CREATE INDEX secondary ON t.test (v); } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - secondaryIndexDesc := &tableDesc.GetPublicNonPrimaryIndexes()[0] + secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap colIDtoRowIndex.Set(tableDesc.Columns[0].ID, 0) @@ -137,18 +137,18 @@ 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)} - secondaryIndex, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndexDesc, colIDtoRowIndex, values, true /* includeEmpty */) + secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } - if len(secondaryIndex) != 1 { - t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndex), secondaryIndex) + if len(secondaryIndexKey) != 1 { + t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndexKey), secondaryIndexKey) } // Put the new secondary k/v into the database. - if err := kvDB.Put(context.Background(), secondaryIndex[0].Key, &secondaryIndex[0].Value); err != nil { + if err := kvDB.Put(context.Background(), secondaryIndexKey[0].Key, &secondaryIndexKey[0].Value); err != nil { t.Fatalf("unexpected error: %s", err) } @@ -223,7 +223,7 @@ INSERT INTO t.test VALUES (10, 20, 1337); } tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - secondaryIndexDesc := &tableDesc.GetPublicNonPrimaryIndexes()[0] + secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap colIDtoRowIndex.Set(tableDesc.Columns[0].ID, 0) @@ -232,30 +232,30 @@ 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)} - secondaryIndex, err := rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndexDesc, colIDtoRowIndex, values, true /* includeEmpty */) + secondaryIndexKey, err := rowenc.EncodeSecondaryIndex( + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) - if len(secondaryIndex) != 1 { - t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndex), secondaryIndex) + if len(secondaryIndexKey) != 1 { + t.Fatalf("expected 1 index entry, got %d. got %#v", len(secondaryIndexKey), secondaryIndexKey) } if err != nil { t.Fatalf("unexpected error: %s", err) } // Delete the existing secondary k/v. - if err := kvDB.Del(context.Background(), secondaryIndex[0].Key); err != nil { + if err := kvDB.Del(context.Background(), secondaryIndexKey[0].Key); err != nil { t.Fatalf("unexpected error: %s", err) } // Generate a secondary index k/v that has a different value. values = []tree.Datum{tree.NewDInt(10), tree.NewDInt(20), tree.NewDInt(314)} - secondaryIndex, err = rowenc.EncodeSecondaryIndex( - keys.SystemSQLCodec, tableDesc, secondaryIndexDesc, colIDtoRowIndex, values, true /* includeEmpty */) + secondaryIndexKey, err = rowenc.EncodeSecondaryIndex( + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } // Put the incorrect secondary k/v. - if err := kvDB.Put(context.Background(), secondaryIndex[0].Key, &secondaryIndex[0].Value); err != nil { + if err := kvDB.Put(context.Background(), secondaryIndexKey[0].Key, &secondaryIndexKey[0].Value); err != nil { t.Fatalf("unexpected error: %s", err) } @@ -445,7 +445,7 @@ func TestScrubFKConstraintFKMissing(t *testing.T) { // Construct datums for the child row values (child_id, parent_id). values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(314)} - secondaryIndex := &tableDesc.GetPublicNonPrimaryIndexes()[0] + secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap colIDtoRowIndex.Set(tableDesc.Columns[0].ID, 0) @@ -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, colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), 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, colIDtoRowIndex, values, true /* includeEmpty */) + keys.SystemSQLCodec, tableDesc, secondaryIndex.IndexDesc(), colIDtoRowIndex, values, true /* includeEmpty */) if err != nil { t.Fatalf("unexpected error: %s", err) } diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index 58b6794b2b42..4917b73ebfaa 100644 --- a/pkg/sql/show_create.go +++ b/pkg/sql/show_create.go @@ -141,16 +141,15 @@ func ShowCreateTable( } } allIdx := append( - append([]descpb.IndexDescriptor{}, desc.GetPublicNonPrimaryIndexes()...), - *desc.GetPrimaryIndex()) - for i := range allIdx { - idx := &allIdx[i] + append([]catalog.Index{}, desc.PublicNonPrimaryIndexes()...), + desc.PrimaryIndexInterface()) + for _, idx := range allIdx { // Only add indexes to the create_statement column, and not to the // create_nofks column if they are not associated with an INTERLEAVE // statement. // Initialize to false if Interleave has no ancestors, indicating that the // index is not interleaved at all. - includeInterleaveClause := len(idx.Interleave.Ancestors) == 0 + includeInterleaveClause := idx.NumInterleaveAncestors() == 0 if displayOptions.FKDisplayMode != OmitFKClausesFromCreate { // The caller is instructing us to not omit FK clauses from inside the CREATE. // (i.e. the caller does not want them as separate DDL.) @@ -158,10 +157,10 @@ func ShowCreateTable( // clauses as well. includeInterleaveClause = true } - if idx.ID != desc.GetPrimaryIndex().ID && includeInterleaveClause { + if !idx.Primary() && includeInterleaveClause { // Showing the primary index is handled above. f.WriteString(",\n\t") - idxStr, err := catformat.IndexForDisplay(ctx, desc, &descpb.AnonymousTable, idx, &p.RunParams(ctx).p.semaCtx) + idxStr, err := catformat.IndexForDisplay(ctx, desc, &descpb.AnonymousTable, idx.IndexDesc(), &p.RunParams(ctx).p.semaCtx) if err != nil { return "", err } @@ -172,12 +171,12 @@ func ShowCreateTable( // Add interleave or Foreign Key indexes only to the create_table columns, // and not the create_nofks column. if includeInterleaveClause { - if err := showCreateInterleave(idx, &f.Buffer, dbPrefix, lCtx); err != nil { + if err := showCreateInterleave(idx.IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { return "", err } } if err := ShowCreatePartitioning( - a, p.ExecCfg().Codec, desc, idx, &idx.Partitioning, &f.Buffer, 1 /* indent */, 0, /* colOffset */ + a, p.ExecCfg().Codec, desc, idx.IndexDesc(), &idx.IndexDesc().Partitioning, &f.Buffer, 1 /* indent */, 0, /* colOffset */ ); err != nil { return "", err } diff --git a/pkg/sql/table_ref_test.go b/pkg/sql/table_ref_test.go index a3dd0c2ee320..def879d5350d 100644 --- a/pkg/sql/table_ref_test.go +++ b/pkg/sql/table_ref_test.go @@ -61,7 +61,7 @@ CREATE INDEX bc ON test.t(b, c); } } pkID := tableDesc.GetPrimaryIndexID() - secID := tableDesc.GetPublicNonPrimaryIndexes()[0].ID + secID := tableDesc.PublicNonPrimaryIndexes()[0].GetID() // Retrieve the numeric descriptors. tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "hidden") diff --git a/pkg/sql/table_test.go b/pkg/sql/table_test.go index af8e676e69fc..c504131e2cb3 100644 --- a/pkg/sql/table_test.go +++ b/pkg/sql/table_test.go @@ -302,11 +302,16 @@ func TestMakeTableDescIndexes(t *testing.T) { if err != nil { t.Fatalf("%d (%s): %v", i, d.sql, err) } - if !reflect.DeepEqual(d.primary, *schema.GetPrimaryIndex()) { - t.Fatalf("%d (%s): primary mismatch: expected %+v, but got %+v", i, d.sql, d.primary, schema.GetPrimaryIndex()) + activeIndexDescs := make([]descpb.IndexDescriptor, len(schema.ActiveIndexes())) + for i, index := range schema.ActiveIndexes() { + activeIndexDescs[i] = *index.IndexDesc() } - if !reflect.DeepEqual(d.indexes, append([]descpb.IndexDescriptor{}, schema.GetPublicNonPrimaryIndexes()...)) { - t.Fatalf("%d (%s): index mismatch: expected %+v, but got %+v", i, d.sql, d.indexes, schema.GetPublicNonPrimaryIndexes()) + + if !reflect.DeepEqual(d.primary, activeIndexDescs[0]) { + t.Fatalf("%d (%s): primary mismatch: expected %+v, but got %+v", i, d.sql, d.primary, activeIndexDescs[0]) + } + if !reflect.DeepEqual(d.indexes, activeIndexDescs[1:]) { + t.Fatalf("%d (%s): index mismatch: expected %+v, but got %+v", i, d.sql, d.indexes, activeIndexDescs[1:]) } } diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 9e8e38c91457..4de6cf76eab8 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -194,24 +194,19 @@ func (p *planner) truncateTable( return err } - // Collect all of the old indexes. - oldIndexes := make([]descpb.IndexDescriptor, len(tableDesc.GetPublicNonPrimaryIndexes())+1) - oldIndexes[0] = *protoutil.Clone(tableDesc.GetPrimaryIndex()).(*descpb.IndexDescriptor) - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - oldIndexes[i+1] = *protoutil.Clone(&tableDesc.GetPublicNonPrimaryIndexes()[i]).(*descpb.IndexDescriptor) - } - - // Reset all of the index IDs. - { - primaryIndex := *tableDesc.GetPrimaryIndex() - primaryIndex.ID = descpb.IndexID(0) - tableDesc.SetPrimaryIndex(primaryIndex) + // Collect all of the old indexes and reset all of the index IDs. + oldIndexes := make([]descpb.IndexDescriptor, len(tableDesc.ActiveIndexes())) + for _, idx := range tableDesc.ActiveIndexes() { + oldIndexes[idx.Ordinal()] = idx.IndexDescDeepCopy() + newIndex := *idx.IndexDesc() + newIndex.ID = descpb.IndexID(0) + if idx.Primary() { + tableDesc.SetPrimaryIndex(newIndex) + } else { + tableDesc.SetPublicNonPrimaryIndex(idx.Ordinal(), newIndex) + } } - for i, index := range tableDesc.GetPublicNonPrimaryIndexes() { - index.ID = descpb.IndexID(0) - tableDesc.SetPublicNonPrimaryIndex(i+1, index) - } // Create new ID's for all of the indexes in the table. if err := tableDesc.AllocateIDs(ctx); err != nil { return err @@ -219,9 +214,8 @@ func (p *planner) truncateTable( // Construct a mapping from old index ID's to new index ID's. indexIDMapping := make(map[descpb.IndexID]descpb.IndexID, len(oldIndexes)) - indexIDMapping[oldIndexes[0].ID] = tableDesc.GetPrimaryIndexID() - for i := range tableDesc.GetPublicNonPrimaryIndexes() { - indexIDMapping[oldIndexes[i+1].ID] = tableDesc.GetPublicNonPrimaryIndexes()[i].ID + for _, idx := range tableDesc.ActiveIndexes() { + indexIDMapping[oldIndexes[idx.Ordinal()].ID] = idx.GetID() } // Resolve all outstanding mutations. Make all new schema elements @@ -289,9 +283,9 @@ func (p *planner) truncateTable( for i := range oldIndexIDs { oldIndexIDs[i] = oldIndexes[i+1].ID } - newIndexIDs := make([]descpb.IndexID, len(tableDesc.GetPublicNonPrimaryIndexes())) + newIndexIDs := make([]descpb.IndexID, len(tableDesc.PublicNonPrimaryIndexes())) for i := range newIndexIDs { - newIndexIDs[i] = tableDesc.GetPublicNonPrimaryIndexes()[i].ID + newIndexIDs[i] = tableDesc.PublicNonPrimaryIndexes()[i].GetID() } swapInfo := &descpb.PrimaryKeySwap{ OldPrimaryIndexId: oldIndexes[0].ID, diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index 7f1614c738f5..1ea8ff00f0d0 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -176,10 +176,11 @@ func (t virtualSchemaTable) initVirtualTableDesc( if err != nil { return mutDesc.TableDescriptor, err } - for i, idx := range mutDesc.GetPublicNonPrimaryIndexes() { - if len(idx.ColumnIDs) > 1 { + for _, index := range mutDesc.PublicNonPrimaryIndexes() { + if index.NumColumns() > 1 { panic("we don't know how to deal with virtual composite indexes yet") } + idx := *index.IndexDesc() // All indexes of virtual tables automatically STORE all other columns in // the table. idx.StoreColumnIDs = make([]descpb.ColumnID, len(mutDesc.Columns)-len(idx.ColumnIDs)) @@ -198,7 +199,7 @@ func (t virtualSchemaTable) initVirtualTableDesc( idx.StoreColumnNames[outputIdx] = mutDesc.Columns[j].Name outputIdx++ } - mutDesc.SetPublicNonPrimaryIndex(i+1, idx) + mutDesc.SetPublicNonPrimaryIndex(index.Ordinal(), idx) } return mutDesc.TableDescriptor, nil } From 20766f2d28e5d47e0b3d9c5e30eebb7936e99e54 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 11:11:39 -0500 Subject: [PATCH 09/13] sql: remove deprecated GetPrimaryIndex method Previously, this method would be called on a table descriptor interface (or backing struct) to obtain the primary index descriptor in the form of a *descpb.IndexDescriptor. This patch removes these calls, along with the method definition, in favour of new methods which use the catalog.Index interface type instead. Partially addresses #57465. Release note: None --- pkg/ccl/changefeedccl/avro_test.go | 2 +- pkg/ccl/changefeedccl/encoder.go | 8 ++-- pkg/ccl/changefeedccl/rowfetcher_cache.go | 4 +- pkg/server/settingsworker.go | 2 +- pkg/sql/alter_primary_key.go | 18 ++++----- pkg/sql/alter_table.go | 24 ++++++------ pkg/sql/backfill/backfill.go | 4 +- pkg/sql/catalog/descriptor.go | 1 - pkg/sql/catalog/tabledesc/structured.go | 7 ---- pkg/sql/catalog/tabledesc/table.go | 8 ++-- pkg/sql/check.go | 6 ++- pkg/sql/create_index.go | 2 +- pkg/sql/create_stats.go | 9 +++-- pkg/sql/create_table.go | 38 ++++++++++--------- pkg/sql/delete_range.go | 4 +- pkg/sql/drop_index.go | 4 +- pkg/sql/information_schema.go | 6 ++- pkg/sql/opt_catalog.go | 4 +- pkg/sql/opt_exec_factory.go | 6 +-- pkg/sql/partition_test.go | 2 +- .../physicalplan/fake_span_resolver_test.go | 2 +- pkg/sql/refresh_materialized_view.go | 5 +-- pkg/sql/row/fetcher_mvcc_test.go | 2 +- pkg/sql/row/fetcher_test.go | 4 +- pkg/sql/row/helper.go | 9 +++-- pkg/sql/row/inserter.go | 7 ++-- pkg/sql/row/updater.go | 3 +- pkg/sql/rowenc/index_encoding_test.go | 17 +++++---- pkg/sql/rowenc/testutils.go | 14 +++---- pkg/sql/rowexec/zigzagjoiner.go | 4 +- pkg/sql/scan.go | 2 +- pkg/sql/schema_changer.go | 2 +- pkg/sql/schema_changer_test.go | 2 +- pkg/sql/scrub.go | 5 ++- pkg/sql/scrub_fk.go | 11 ++++-- pkg/sql/scrub_index.go | 3 +- pkg/sql/scrub_test.go | 10 ++--- pkg/sql/show_create.go | 10 ++--- pkg/sql/tablewriter_delete.go | 4 +- 39 files changed, 145 insertions(+), 130 deletions(-) diff --git a/pkg/ccl/changefeedccl/avro_test.go b/pkg/ccl/changefeedccl/avro_test.go index 1fb6cdb3e690..d5e49f0edf11 100644 --- a/pkg/ccl/changefeedccl/avro_test.go +++ b/pkg/ccl/changefeedccl/avro_test.go @@ -285,7 +285,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()) + indexSchema, err := indexToAvroSchema(tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc()) require.NoError(t, err) require.Equal(t, `{"type":"record","name":"_u2603_","fields":[`+ diff --git a/pkg/ccl/changefeedccl/encoder.go b/pkg/ccl/changefeedccl/encoder.go index d4028a232997..afd75d2e8254 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -142,8 +142,10 @@ func (e *jsonEncoder) EncodeKey(_ context.Context, row encodeRow) ([]byte, error func (e *jsonEncoder) encodeKeyRaw(row encodeRow) ([]interface{}, error) { colIdxByID := row.tableDesc.ColumnIdxMap() - jsonEntries := make([]interface{}, len(row.tableDesc.GetPrimaryIndex().ColumnIDs)) - for i, colID := range row.tableDesc.GetPrimaryIndex().ColumnIDs { + primaryIndex := row.tableDesc.PrimaryIndexInterface() + jsonEntries := make([]interface{}, primaryIndex.NumColumns()) + for i := 0; i < primaryIndex.NumColumns(); i++ { + colID := primaryIndex.GetColumnID(i) idx, ok := colIdxByID.Get(colID) if !ok { return nil, errors.Errorf(`unknown column id: %d`, colID) @@ -342,7 +344,7 @@ func (e *confluentAvroEncoder) EncodeKey(ctx context.Context, row encodeRow) ([] registered, ok := e.keyCache[cacheKey] if !ok { var err error - registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.GetPrimaryIndex()) + registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.PrimaryIndexInterface().IndexDesc()) if err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/rowfetcher_cache.go b/pkg/ccl/changefeedccl/rowfetcher_cache.go index 4d1146e2f388..56472572daf6 100644 --- a/pkg/ccl/changefeedccl/rowfetcher_cache.go +++ b/pkg/ccl/changefeedccl/rowfetcher_cache.go @@ -121,7 +121,7 @@ func (c *rowFetcherCache) TableDescForKey( } // Skip over the column data. - for ; skippedCols < len(tableDesc.GetPrimaryIndex().ColumnIDs); skippedCols++ { + for ; skippedCols < tableDesc.PrimaryIndexInterface().NumColumns(); skippedCols++ { l, err := encoding.PeekLength(remaining) if err != nil { return nil, err @@ -173,7 +173,7 @@ func (c *rowFetcherCache) RowFetcherForTableDesc( row.FetcherTableArgs{ Spans: tableDesc.AllIndexSpans(c.codec), Desc: tableDesc, - Index: tableDesc.GetPrimaryIndex(), + Index: tableDesc.PrimaryIndexInterface().IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, Cols: tableDesc.Columns, diff --git a/pkg/server/settingsworker.go b/pkg/server/settingsworker.go index 17ed42db5dac..968a0d57e4c8 100644 --- a/pkg/server/settingsworker.go +++ b/pkg/server/settingsworker.go @@ -50,7 +50,7 @@ func processSystemConfigKVs( { types := []*types.T{tbl.Columns[0].Type} nameRow := make([]rowenc.EncDatum, 1) - _, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.GetPrimaryIndex(), types, nameRow, nil, kv.Key) + _, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.PrimaryIndexInterface().IndexDesc(), types, nameRow, nil, kv.Key) if err != nil { return errors.Wrap(err, "failed to decode key") } diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index 30aee385f00b..3a1ddc516716 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -91,19 +91,19 @@ func (p *planner) AlterPrimaryKey( } // Disable primary key changes on tables that are interleaved parents. - if len(tableDesc.GetPrimaryIndex().InterleavedBy) != 0 { + if tableDesc.PrimaryIndexInterface().NumInterleavedBy() != 0 { var sb strings.Builder sb.WriteString("[") comma := ", " - for i := range tableDesc.GetPrimaryIndex().InterleavedBy { - interleave := &tableDesc.GetPrimaryIndex().InterleavedBy[i] + for i := 0; i < tableDesc.PrimaryIndexInterface().NumInterleavedBy(); i++ { + interleaveTableID := tableDesc.PrimaryIndexInterface().GetInterleavedBy(i).Table if i != 0 { sb.WriteString(comma) } childTable, err := p.Descriptors().GetTableVersionByID( ctx, p.Txn(), - interleave.Table, + interleaveTableID, tree.ObjectLookupFlags{}, ) if err != nil { @@ -222,7 +222,7 @@ func (p *planner) AlterPrimaryKey( // Create a new index that indexes everything the old primary index // does, but doesn't store anything. if shouldCopyPrimaryKey(tableDesc, newPrimaryIndexDesc) { - oldPrimaryIndexCopy := protoutil.Clone(tableDesc.GetPrimaryIndex()).(*descpb.IndexDescriptor) + oldPrimaryIndexCopy := tableDesc.PrimaryIndexInterface().IndexDescDeepCopy() // Clear the name of the index so that it gets generated by AllocateIDs. oldPrimaryIndexCopy.Name = "" oldPrimaryIndexCopy.StoreColumnIDs = nil @@ -230,7 +230,7 @@ func (p *planner) AlterPrimaryKey( // Make the copy of the old primary index not-interleaved. This decision // can be revisited based on user experience. oldPrimaryIndexCopy.Interleave = descpb.InterleaveDescriptor{} - if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, oldPrimaryIndexCopy, newPrimaryIndexDesc); err != nil { + if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, &oldPrimaryIndexCopy, newPrimaryIndexDesc); err != nil { return err } } @@ -314,7 +314,7 @@ func (p *planner) AlterPrimaryKey( // Mark the primary key of the table as valid. { - primaryIndex := *tableDesc.GetPrimaryIndex() + primaryIndex := *tableDesc.PrimaryIndexInterface().IndexDesc() primaryIndex.Disabled = false tableDesc.SetPrimaryIndex(primaryIndex) } @@ -344,7 +344,7 @@ func (p *planner) AlterPrimaryKey( // * The new primary key isn't the same hash sharded old primary key with a // different bucket count. func shouldCopyPrimaryKey(desc *tabledesc.Mutable, newPK *descpb.IndexDescriptor) bool { - oldPK := desc.GetPrimaryIndex() + oldPK := desc.PrimaryIndexInterface() if !desc.HasPrimaryKey() { return false } @@ -354,7 +354,7 @@ func shouldCopyPrimaryKey(desc *tabledesc.Mutable, newPK *descpb.IndexDescriptor // The first column in the columnIDs is the shard column, which will be different. // Slice it out to see what the actual index columns are. if oldPK.IsSharded() && newPK.IsSharded() && - descpb.ColumnIDs(oldPK.ColumnIDs[1:]).Equals(newPK.ColumnIDs[1:]) { + descpb.ColumnIDs(oldPK.IndexDesc().ColumnIDs[1:]).Equals(newPK.ColumnIDs[1:]) { return false } diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 49458aa3fb4c..8874f181cfb9 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -436,7 +436,7 @@ func (n *alterTableNode) startExec(params runParams) error { return err } - if n.tableDesc.GetPrimaryIndex().ContainsColumnID(colToDrop.ID) { + if n.tableDesc.PrimaryIndexInterface().ContainsColumnID(colToDrop.ID) { return pgerror.Newf(pgcode.InvalidColumnReference, "column %q is referenced by the primary key", colToDrop.Name) } @@ -459,7 +459,7 @@ func (n *alterTableNode) startExec(params runParams) error { if !containsThisColumn { for j := 0; j < idx.NumExtraColumns(); j++ { id := idx.GetExtraColumnID(j) - if n.tableDesc.GetPrimaryIndex().ContainsColumnID(id) { + if n.tableDesc.PrimaryIndexInterface().ContainsColumnID(id) { // All secondary indices necessary contain the PK // columns, too. (See the comments on the definition of // IndexDescriptor). The presence of a PK column in the @@ -740,26 +740,28 @@ func (n *alterTableNode) startExec(params runParams) error { descriptorChanged = true case *tree.AlterTablePartitionBy: - partitioning, err := CreatePartitioning( + primaryIndex := n.tableDesc.PrimaryIndexInterface() + newPartitioning, err := CreatePartitioning( params.ctx, params.p.ExecCfg().Settings, params.EvalContext(), - n.tableDesc, n.tableDesc.GetPrimaryIndex(), t.PartitionBy) + n.tableDesc, primaryIndex.IndexDesc(), t.PartitionBy) if err != nil { return err } - descriptorChanged = descriptorChanged || !n.tableDesc.GetPrimaryIndex().Partitioning.Equal(&partitioning) + oldPartitioning := primaryIndex.GetPartitioning() + descriptorChanged = descriptorChanged || !oldPartitioning.Equal(&newPartitioning) err = deleteRemovedPartitionZoneConfigs( params.ctx, params.p.txn, - n.tableDesc, n.tableDesc.GetPrimaryIndex(), &n.tableDesc.GetPrimaryIndex().Partitioning, - &partitioning, params.extendedEvalCtx.ExecCfg, + n.tableDesc, primaryIndex.IndexDesc(), &oldPartitioning, + &newPartitioning, params.extendedEvalCtx.ExecCfg, ) if err != nil { return err } { - primaryIndex := *n.tableDesc.GetPrimaryIndex() - primaryIndex.Partitioning = partitioning - n.tableDesc.SetPrimaryIndex(primaryIndex) + newPrimaryIndex := *primaryIndex.IndexDesc() + newPrimaryIndex.Partitioning = newPartitioning + n.tableDesc.SetPrimaryIndex(newPrimaryIndex) } case *tree.AlterTableSetAudit: @@ -1033,7 +1035,7 @@ func applyColumnMutation( } // Prevent a column in a primary key from becoming non-null. - if tableDesc.GetPrimaryIndex().ContainsColumnID(col.ID) { + if tableDesc.PrimaryIndexInterface().ContainsColumnID(col.ID) { return pgerror.Newf(pgcode.InvalidTableDefinition, `column "%s" is in a primary index`, col.Name) } diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 030cab6a678a..c8b5a12a51f4 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -124,7 +124,7 @@ func (cb *ColumnBackfiller) init( tableArgs := row.FetcherTableArgs{ Desc: desc, - Index: desc.GetPrimaryIndex(), + Index: desc.PrimaryIndexInterface().IndexDesc(), ColIdxMap: desc.ColumnIdxMap(), Cols: desc.Columns, ValNeededForCol: valNeededForCol, @@ -602,7 +602,7 @@ func (ib *IndexBackfiller) init( tableArgs := row.FetcherTableArgs{ Desc: desc, - Index: desc.GetPrimaryIndex(), + Index: desc.PrimaryIndexInterface().IndexDesc(), ColIdxMap: ib.colIdxMap, Cols: ib.cols, ValNeededForCol: valNeededForCol, diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 59b9338be046..884e1fc8957a 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -112,7 +112,6 @@ type TableDescriptor interface { GetFormatVersion() descpb.FormatVersion GetPrimaryIndexID() descpb.IndexID - GetPrimaryIndex() *descpb.IndexDescriptor // deprecated PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index ccdd8c312137..2b9019ddbea6 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -239,13 +239,6 @@ func (desc *Mutable) SetName(name string) { desc.Name = name } -// GetPrimaryIndex returns a pointer to the primary index of the table -// descriptor. -// This method is deprecated, use PrimaryIndexInterface instead. -func (desc *wrapper) GetPrimaryIndex() *descpb.IndexDescriptor { - return &desc.PrimaryIndex -} - // GetParentSchemaID returns the ParentSchemaID if the descriptor has // one. If the descriptor was created before the field was added, then the // descriptor belongs to a table under the `public` physical schema. The static diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index e3007e79f8bc..d82fd5ba480e 100644 --- a/pkg/sql/catalog/tabledesc/table.go +++ b/pkg/sql/catalog/tabledesc/table.go @@ -339,9 +339,9 @@ func FindFKReferencedIndex( ) (*descpb.IndexDescriptor, error) { // Search for a unique index on the referenced table that matches our foreign // key columns. - primaryIndex := referencedTable.GetPrimaryIndex() + primaryIndex := referencedTable.PrimaryIndexInterface() if primaryIndex.IsValidReferencedIndex(referencedColIDs) { - return primaryIndex, nil + return primaryIndex.IndexDesc(), nil } // If the PK doesn't match, find the index corresponding to the referenced column. for _, idx := range referencedTable.PublicNonPrimaryIndexes() { @@ -363,8 +363,8 @@ func FindFKOriginIndex( ) (*descpb.IndexDescriptor, error) { // Search for an index on the origin table that matches our foreign // key columns. - if primaryIndex := originTable.GetPrimaryIndex(); primaryIndex.IsValidOriginIndex(originColIDs) { - return primaryIndex, nil + if primaryIndex := originTable.PrimaryIndexInterface(); primaryIndex.IsValidOriginIndex(originColIDs) { + return primaryIndex.IndexDesc(), nil } // If the PK doesn't match, find the index corresponding to the origin column. for _, idx := range originTable.PublicNonPrimaryIndexes() { diff --git a/pkg/sql/check.go b/pkg/sql/check.go index bfed97e5bac2..13cf97dc9f3c 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -96,7 +96,8 @@ func matchFullUnacceptableKeyQuery( srcNotNullExistsClause[i] = fmt.Sprintf("%s IS NOT NULL", srcCols[i]) } - for _, id := range srcTbl.GetPrimaryIndex().ColumnIDs { + for i := 0; i < srcTbl.PrimaryIndexInterface().NumColumns(); i++ { + id := srcTbl.PrimaryIndexInterface().GetColumnID(i) alreadyPresent := false for _, otherID := range fk.OriginColumnIDs { if id == otherID { @@ -159,7 +160,8 @@ func nonMatchingRowQuery( return "", nil, err } // Get primary key columns not included in the FK - for _, pkColID := range srcTbl.GetPrimaryIndex().ColumnIDs { + for i := 0; i < srcTbl.PrimaryIndexInterface().NumColumns(); i++ { + pkColID := srcTbl.PrimaryIndexInterface().GetColumnID(i) found := false for _, id := range fk.OriginColumnIDs { if pkColID == id { diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index 337c014a6d75..9089b94f6254 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -391,7 +391,7 @@ func (n *createIndexNode) startExec(params runParams) error { // Warn against creating a non-partitioned index on a partitioned table, // which is undesirable in most cases. - if n.n.PartitionBy == nil && n.tableDesc.GetPrimaryIndex().Partitioning.NumColumns > 0 { + if n.n.PartitionBy == nil && n.tableDesc.PrimaryIndexInterface().GetPartitioning().NumColumns > 0 { params.p.BufferClientNotice( params.ctx, errors.WithHint( diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index acea824d6180..fefc08be11f8 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -350,16 +350,19 @@ func createStatsDefaultColumns( } // Add column stats for the primary key. - for i := range desc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < desc.PrimaryIndexInterface().NumColumns(); i++ { // Generate stats for each column in the primary key. - addIndexColumnStatsIfNotExists(desc.GetPrimaryIndex().ColumnIDs[i], false /* isInverted */) + addIndexColumnStatsIfNotExists(desc.PrimaryIndexInterface().GetColumnID(i), false /* isInverted */) // Only collect multi-column stats if enabled. if i == 0 || !multiColEnabled { continue } - colIDs := desc.GetPrimaryIndex().ColumnIDs[: i+1 : i+1] + colIDs := make([]descpb.ColumnID, i+1) + for j := 0; j <= i; j++ { + colIDs[j] = desc.PrimaryIndexInterface().GetColumnID(j) + } // Remember the requested stats so we don't request duplicates. trackStatsIfNotExists(colIDs) diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 3089afa737d1..70b7dd50917b 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -785,9 +785,9 @@ func ResolveFK( referencedColNames := d.ToCols // If no columns are specified, attempt to default to PK. if len(referencedColNames) == 0 { - referencedColNames = make(tree.NameList, len(target.GetPrimaryIndex().ColumnNames)) - for i, n := range target.GetPrimaryIndex().ColumnNames { - referencedColNames[i] = tree.Name(n) + referencedColNames = make(tree.NameList, target.PrimaryIndexInterface().NumColumns()) + for i := range referencedColNames { + referencedColNames[i] = tree.Name(target.PrimaryIndexInterface().GetColumnName(i)) } } @@ -1025,7 +1025,7 @@ func addInterleave( if err != nil { return err } - parentIndex := parentTable.GetPrimaryIndex() + parentIndex := parentTable.PrimaryIndexInterface() // typeOfIndex is used to give more informative error messages. var typeOfIndex string @@ -1035,12 +1035,12 @@ func addInterleave( typeOfIndex = "index" } - if len(interleave.Fields) != len(parentIndex.ColumnIDs) { + if len(interleave.Fields) != parentIndex.NumColumns() { return pgerror.Newf( pgcode.InvalidSchemaDefinition, "declared interleaved columns (%s) must match the parent's primary index (%s)", &interleave.Fields, - strings.Join(parentIndex.ColumnNames, ", "), + strings.Join(parentIndex.IndexDesc().ColumnNames, ", "), ) } if len(interleave.Fields) > len(index.ColumnIDs) { @@ -1053,7 +1053,8 @@ func addInterleave( ) } - for i, targetColID := range parentIndex.ColumnIDs { + for i := 0; i < parentIndex.NumColumns(); i++ { + targetColID := parentIndex.GetColumnID(i) targetCol, err := parentTable.FindColumnByID(targetColID) if err != nil { return err @@ -1071,22 +1072,25 @@ func addInterleave( strings.Join(index.ColumnNames, ", "), ) } - if !col.Type.Identical(targetCol.Type) || index.ColumnDirections[i] != parentIndex.ColumnDirections[i] { + if !col.Type.Identical(targetCol.Type) || index.ColumnDirections[i] != parentIndex.GetColumnDirection(i) { return pgerror.Newf( pgcode.InvalidSchemaDefinition, "declared interleaved columns (%s) must match type and sort direction of the parent's primary index (%s)", &interleave.Fields, - strings.Join(parentIndex.ColumnNames, ", "), + strings.Join(parentIndex.IndexDesc().ColumnNames, ", "), ) } } - ancestorPrefix := append( - []descpb.InterleaveDescriptor_Ancestor(nil), parentIndex.Interleave.Ancestors...) + ancestorPrefix := make([]descpb.InterleaveDescriptor_Ancestor, parentIndex.NumInterleaveAncestors()) + for i := range ancestorPrefix { + ancestorPrefix[i] = parentIndex.GetInterleaveAncestor(i) + } + intl := descpb.InterleaveDescriptor_Ancestor{ TableID: parentTable.ID, - IndexID: parentIndex.ID, - SharedPrefixLen: uint32(len(parentIndex.ColumnIDs)), + IndexID: parentIndex.GetID(), + SharedPrefixLen: uint32(parentIndex.NumColumns()), } for _, ancestor := range ancestorPrefix { intl.SharedPrefixLen -= ancestor.SharedPrefixLen @@ -1648,7 +1652,7 @@ func NewTableDesc( } // If explicit primary keys are required, error out since a primary key was not supplied. - if len(desc.GetPrimaryIndex().ColumnNames) == 0 && desc.IsPhysicalTable() && evalCtx != nil && + if desc.PrimaryIndexInterface().NumColumns() == 0 && desc.IsPhysicalTable() && evalCtx != nil && evalCtx.SessionData != nil && evalCtx.SessionData.RequireExplicitPrimaryKeys { return nil, errors.Errorf( "no primary key specified for table %s (require_explicit_primary_keys = true)", desc.Name) @@ -1708,19 +1712,19 @@ func NewTableDesc( } if n.Interleave != nil { - if err := addInterleave(ctx, txn, vt, &desc, desc.GetPrimaryIndex(), n.Interleave); err != nil { + if err := addInterleave(ctx, txn, vt, &desc, desc.PrimaryIndexInterface().IndexDesc(), n.Interleave); err != nil { return nil, err } } if n.PartitionBy != nil { partitioning, err := CreatePartitioning( - ctx, st, evalCtx, &desc, desc.GetPrimaryIndex(), n.PartitionBy) + ctx, st, evalCtx, &desc, desc.PrimaryIndexInterface().IndexDesc(), n.PartitionBy) if err != nil { return nil, err } { - newPrimaryIndex := *desc.GetPrimaryIndex() + newPrimaryIndex := *desc.PrimaryIndexInterface().IndexDesc() newPrimaryIndex.Partitioning = partitioning desc.SetPrimaryIndex(newPrimaryIndex) } diff --git a/pkg/sql/delete_range.go b/pkg/sql/delete_range.go index 6885ec9de161..55608262e27b 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(), + Index: d.desc.PrimaryIndexInterface().IndexDesc(), Spans: d.spans, } for i, interleaved := range d.interleavedDesc { allTables[i+1] = row.FetcherTableArgs{ Desc: interleaved, - Index: interleaved.GetPrimaryIndex(), + Index: interleaved.PrimaryIndexInterface().IndexDesc(), Spans: d.spans, } } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 1ee00a927004..4d7a9f4f666e 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -312,8 +312,8 @@ 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, 0, len(tableDesc.ActiveIndexes())) - remainingIndexes = append(remainingIndexes, tableDesc.GetPrimaryIndex()) + remainingIndexes := make([]*descpb.IndexDescriptor, 1, len(tableDesc.ActiveIndexes())) + remainingIndexes[0] = tableDesc.PrimaryIndexInterface().IndexDesc() for _, index := range tableDesc.PublicNonPrimaryIndexes() { if index.GetID() != idx.ID { remainingIndexes = append(remainingIndexes, index.IndexDesc()) diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 454dc658dc6f..9e803063f9ce 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -1312,7 +1312,8 @@ CREATE TABLE information_schema.statistics ( } if hasImplicitCols { implicitCols = make(map[string]struct{}) - for _, col := range table.GetPrimaryIndex().ColumnNames { + for i := 0; i < table.PrimaryIndexInterface().NumColumns(); i++ { + col := table.PrimaryIndexInterface().GetColumnName(i) implicitCols[col] = struct{}{} } } @@ -1345,7 +1346,8 @@ CREATE TABLE information_schema.statistics ( // // Note that simply iterating over implicitCols map // produces non-deterministic output. - for _, col := range table.GetPrimaryIndex().ColumnNames { + for i := 0; i < table.PrimaryIndexInterface().NumColumns(); i++ { + col := table.PrimaryIndexInterface().GetColumnName(i) if _, isImplicit := implicitCols[col]; isImplicit { // We add a row for each implicit column of index. if err := appendRow(index.IndexDesc(), col, sequence, diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 0ead46194773..2254813ae704 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -719,7 +719,7 @@ func newOptTable( for i := range ot.indexes { var idxDesc *descpb.IndexDescriptor if i == 0 { - idxDesc = desc.GetPrimaryIndex() + idxDesc = desc.PrimaryIndexInterface().IndexDesc() } else { idxDesc = secondaryIndexes[i-1].IndexDesc() } @@ -1121,7 +1121,7 @@ func (oi *optIndex) init( oi.zone = zone oi.indexOrdinal = indexOrdinal oi.invertedVirtualColOrd = invertedVirtualColOrd - if desc == tab.desc.GetPrimaryIndex() { + if desc == tab.desc.PrimaryIndexInterface().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. diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index a3ce1f1e12f0..3590c656e8c5 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -547,8 +547,8 @@ func (ef *execFactory) ConstructIndexJoin( return nil, err } - primaryIndex := tabDesc.GetPrimaryIndex() - tableScan.index = primaryIndex + primaryIndex := tabDesc.PrimaryIndexInterface() + tableScan.index = primaryIndex.IndexDesc() tableScan.disableBatchLimit() n := &indexJoinNode{ @@ -1562,7 +1562,7 @@ func (ef *execFactory) ConstructDeleteRange( autoCommit bool, ) (exec.Node, error) { tabDesc := table.(*optTable).desc - indexDesc := tabDesc.GetPrimaryIndex() + indexDesc := tabDesc.PrimaryIndexInterface().IndexDesc() sb := span.MakeBuilder(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, indexDesc) if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { diff --git a/pkg/sql/partition_test.go b/pkg/sql/partition_test.go index b620bacb121f..0859d27ce203 100644 --- a/pkg/sql/partition_test.go +++ b/pkg/sql/partition_test.go @@ -50,7 +50,7 @@ func TestRemovePartitioningOSS(t *testing.T) { // Hack in partitions. Doing this properly requires a CCL binary. { - primaryIndex := *tableDesc.GetPrimaryIndex() + primaryIndex := *tableDesc.PrimaryIndexInterface().IndexDesc() primaryIndex.Partitioning = descpb.PartitioningDescriptor{ NumColumns: 1, Range: []descpb.PartitioningDescriptor_Range{{ diff --git a/pkg/sql/physicalplan/fake_span_resolver_test.go b/pkg/sql/physicalplan/fake_span_resolver_test.go index 016902f142eb..a52c69688973 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()) + primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.PrimaryIndexInterface().IndexDesc()) span := tableDesc.PrimaryIndexSpan(keys.SystemSQLCodec) diff --git a/pkg/sql/refresh_materialized_view.go b/pkg/sql/refresh_materialized_view.go index 8df3efb73927..1b8f52128abe 100644 --- a/pkg/sql/refresh_materialized_view.go +++ b/pkg/sql/refresh_materialized_view.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" ) type refreshMaterializedViewNode struct { @@ -68,7 +67,7 @@ func (n *refreshMaterializedViewNode) startExec(params runParams) error { } // Prepare the new set of indexes by cloning all existing indexes on the view. - newPrimaryIndex := protoutil.Clone(n.desc.GetPrimaryIndex()).(*descpb.IndexDescriptor) + newPrimaryIndex := n.desc.PrimaryIndexInterface().IndexDescDeepCopy() newIndexes := make([]descpb.IndexDescriptor, len(n.desc.PublicNonPrimaryIndexes())) for i, idx := range n.desc.PublicNonPrimaryIndexes() { newIndexes[i] = idx.IndexDescDeepCopy() @@ -87,7 +86,7 @@ func (n *refreshMaterializedViewNode) startExec(params runParams) error { // Queue the refresh mutation. n.desc.AddMaterializedViewRefreshMutation(&descpb.MaterializedViewRefresh{ - NewPrimaryIndex: *newPrimaryIndex, + NewPrimaryIndex: newPrimaryIndex, NewIndexes: newIndexes, AsOf: params.p.Txn().ReadTimestamp(), ShouldBackfill: n.n.RefreshDataOption != tree.RefreshDataClear, diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index cd932a7eb0a9..eedfccc22575 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -101,7 +101,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { args = append(args, row.FetcherTableArgs{ Spans: desc.AllIndexSpans(keys.SystemSQLCodec), Desc: desc, - Index: desc.GetPrimaryIndex(), + Index: desc.PrimaryIndexInterface().IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, Cols: desc.Columns, diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index c5ab2da47d8b..896ec181429d 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -194,7 +194,7 @@ func TestNextRowSingle(t *testing.T) { if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPrimaryIndexID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.GetPrimaryIndex().Name, + tableDesc.Name, tableDesc.PrimaryIndexInterface().GetName(), desc.GetName(), index.Name, ) } @@ -314,7 +314,7 @@ func TestNextRowBatchLimiting(t *testing.T) { if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPrimaryIndexID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.GetPrimaryIndex().Name, + tableDesc.Name, tableDesc.PrimaryIndexInterface().GetName(), desc.GetName(), index.Name, ) } diff --git a/pkg/sql/row/helper.go b/pkg/sql/row/helper.go index b9a40fc5dc9a..95f431b97be9 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -49,7 +49,7 @@ func newRowHelper( // Pre-compute the encoding directions of the index key values for // pretty-printing in traces. - rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.GetPrimaryIndex()) + rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.PrimaryIndexInterface().IndexDesc()) rh.secIndexValDirs = make([][]encoding.Direction, len(rh.Indexes)) for i := range rh.Indexes { @@ -89,7 +89,7 @@ func (rh *rowHelper) encodePrimaryIndex( rh.TableDesc.GetPrimaryIndexID()) } primaryIndexKey, _, err = rowenc.EncodeIndexKey( - rh.TableDesc, rh.TableDesc.GetPrimaryIndex(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) + rh.TableDesc, rh.TableDesc.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) return primaryIndexKey, err } @@ -137,8 +137,9 @@ func (rh *rowHelper) encodeSecondaryIndexes( // #6233). Once it is, use the shared one. func (rh *rowHelper) skipColumnInPK(colID descpb.ColumnID, value tree.Datum) (bool, error) { if rh.primaryIndexCols.Empty() { - for _, colID := range rh.TableDesc.GetPrimaryIndex().ColumnIDs { - rh.primaryIndexCols.Add(colID) + for i := 0; i < rh.TableDesc.PrimaryIndexInterface().NumColumns(); i++ { + pkColID := rh.TableDesc.PrimaryIndexInterface().GetColumnID(i) + rh.primaryIndexCols.Add(pkColID) } } if !rh.primaryIndexCols.Contains(colID) { diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index c627f3087a1f..acaf3b549518 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -63,9 +63,10 @@ func MakeInserter( marshaled: make([]roachpb.Value, len(insertCols)), } - for i, col := range tableDesc.GetPrimaryIndex().ColumnIDs { - if _, ok := ri.InsertColIDtoRowIndex.Get(col); !ok { - return Inserter{}, fmt.Errorf("missing %q primary key column", tableDesc.GetPrimaryIndex().ColumnNames[i]) + for i := 0; i < tableDesc.PrimaryIndexInterface().NumColumns(); i++ { + colID := tableDesc.PrimaryIndexInterface().GetColumnID(i) + if _, ok := ri.InsertColIDtoRowIndex.Get(colID); !ok { + return Inserter{}, fmt.Errorf("missing %q primary key column", tableDesc.PrimaryIndexInterface().GetColumnName(i)) } } diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index 124dde45ba1e..fc62788e3f4c 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -98,7 +98,8 @@ func MakeUpdater( updateColIDtoRowIndex := ColIDtoRowIndexFromCols(updateCols) var primaryIndexCols catalog.TableColSet - for _, colID := range tableDesc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < tableDesc.PrimaryIndexInterface().NumColumns(); i++ { + colID := tableDesc.PrimaryIndexInterface().GetColumnID(i) primaryIndexCols.Add(colID) } diff --git a/pkg/sql/rowenc/index_encoding_test.go b/pkg/sql/rowenc/index_encoding_test.go index 1f56d12717a4..dc664f9f06ab 100644 --- a/pkg/sql/rowenc/index_encoding_test.go +++ b/pkg/sql/rowenc/index_encoding_test.go @@ -225,7 +225,7 @@ func TestIndexKey(t *testing.T) { codec := keys.SystemSQLCodec primaryKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc, tableDesc.GetPrimaryIndexID()) - primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.GetPrimaryIndex(), colMap, testValues, primaryKeyPrefix) + primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colMap, testValues, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -275,7 +275,7 @@ func TestIndexKey(t *testing.T) { } } - checkEntry(tableDesc.GetPrimaryIndex(), primaryIndexKV) + checkEntry(tableDesc.PrimaryIndexInterface().IndexDesc(), primaryIndexKV) checkEntry(tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), secondaryIndexKV) } } @@ -941,7 +941,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(), colMap, tc.table.values, primaryKeyPrefix) + desc, desc.PrimaryIndexInterface().IndexDesc(), colMap, tc.table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -962,7 +962,8 @@ func TestIndexKeyEquivSignature(t *testing.T) { // Column values should be at the beginning of the // remaining bytes of the key. - colVals, null, err := EncodeColumns(desc.GetPrimaryIndex().ColumnIDs, desc.GetPrimaryIndex().ColumnDirections, colMap, tc.table.values, nil /*key*/) + pkIndexDesc := desc.PrimaryIndexInterface().IndexDesc() + colVals, null, err := EncodeColumns(pkIndexDesc.ColumnIDs, pkIndexDesc.ColumnDirections, colMap, tc.table.values, nil /*key*/) if err != nil { t.Fatal(err) } @@ -997,7 +998,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.TableDescriptor, desc.GetPrimaryIndex()) + equivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.PrimaryIndexInterface().IndexDesc()) if err != nil { t.Fatal(err) } @@ -1082,13 +1083,13 @@ func TestEquivSignature(t *testing.T) { desc, colMap := makeTableDescForTest(table.indexKeyArgs) primaryKeyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPrimaryIndexID()) primaryKey, _, err := EncodeIndexKey( - desc, desc.GetPrimaryIndex(), colMap, table.values, primaryKeyPrefix) + desc, desc.PrimaryIndexInterface().IndexDesc(), colMap, table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } // Extract out the table's equivalence signature. - tempEquivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex()) + tempEquivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.PrimaryIndexInterface().IndexDesc()) if err != nil { t.Fatal(err) } @@ -1322,6 +1323,6 @@ func ExtractIndexKey( decodedValues[len(values)+i] = value.Datum } indexKey, _, err := EncodeIndexKey( - tableDesc, tableDesc.GetPrimaryIndex(), colMap, decodedValues, indexKeyPrefix) + tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colMap, decodedValues, indexKeyPrefix) return indexKey, err } diff --git a/pkg/sql/rowenc/testutils.go b/pkg/sql/rowenc/testutils.go index 46e49737aaa9..c3f72944f33c 100644 --- a/pkg/sql/rowenc/testutils.go +++ b/pkg/sql/rowenc/testutils.go @@ -999,9 +999,9 @@ func RandEncDatumRowsOfTypes(rng *rand.Rand, numRows int, types []*types.T) EncD func TestingMakePrimaryIndexKey( desc catalog.TableDescriptor, vals ...interface{}, ) (roachpb.Key, error) { - index := desc.GetPrimaryIndex() - if len(vals) > len(index.ColumnIDs) { - return nil, errors.Errorf("got %d values, PK has %d columns", len(vals), len(index.ColumnIDs)) + index := desc.PrimaryIndexInterface() + if len(vals) > index.NumColumns() { + return nil, errors.Errorf("got %d values, PK has %d columns", len(vals), index.NumColumns()) } datums := make([]tree.Datum, len(vals)) for i, v := range vals { @@ -1018,7 +1018,7 @@ func TestingMakePrimaryIndexKey( return nil, errors.Errorf("unexpected value type %T", v) } // Check that the value type matches. - colID := index.ColumnIDs[i] + colID := index.GetColumnID(i) var done bool if err := desc.ForeachPublicColumn(func(c *descpb.ColumnDescriptor) error { if !done && c.ID == colID { @@ -1037,11 +1037,11 @@ func TestingMakePrimaryIndexKey( // MakeIndexKeyPrefix. var colIDToRowIndex catalog.TableColMap for i := range vals { - colIDToRowIndex.Set(index.ColumnIDs[i], i) + colIDToRowIndex.Set(index.GetColumnID(i), i) } - keyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, index.ID) - key, _, err := EncodeIndexKey(desc, index, colIDToRowIndex, datums, keyPrefix) + keyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, index.GetID()) + key, _, err := EncodeIndexKey(desc, index.IndexDesc(), colIDToRowIndex, datums, keyPrefix) if err != nil { return nil, err } diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index 4520fe8bab8e..f6f315cee8b3 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -647,8 +647,8 @@ func (zi *zigzagJoinerInfo) eqOrdering() (colinfo.ColumnOrdering, error) { if err != nil { return nil, err } - } else if idx := findColumnID(zi.table.GetPrimaryIndex().ColumnIDs, colID); idx != -1 { - direction, err = zi.table.GetPrimaryIndex().ColumnDirections[idx].ToEncodingDirection() + } else if idx := findColumnID(zi.table.PrimaryIndexInterface().IndexDesc().ColumnIDs, colID); idx != -1 { + direction, err = zi.table.PrimaryIndexInterface().GetColumnDirection(idx).ToEncodingDirection() if err != nil { return nil, err } diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 4651a48b08a5..8964e74849f5 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -315,7 +315,7 @@ func initColsForScan( // Initializes the column structures. func (n *scanNode) initDescDefaults(colCfg scanColumnsConfig) error { n.colCfg = colCfg - n.index = n.desc.GetPrimaryIndex() + n.index = n.desc.PrimaryIndexInterface().IndexDesc() var err error n.cols, err = initColsForScan(n.desc, n.colCfg) diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 8d0599a2eb1f..5a8197674235 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1176,7 +1176,7 @@ func (sc *SchemaChanger) done(ctx context.Context) error { // existing indexes on the table. if mutation.Direction == descpb.DescriptorMutation_ADD { desc := fmt.Sprintf("REFRESH MATERIALIZED VIEW %q cleanup", scTable.Name) - pkJob, err := sc.createIndexGCJob(ctx, scTable.GetPrimaryIndex(), txn, desc) + pkJob, err := sc.createIndexGCJob(ctx, scTable.PrimaryIndexInterface().IndexDesc(), txn, desc) if err != nil { return err } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index f2f3c07e5a43..9771eecc7a3b 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -3078,7 +3078,7 @@ CREATE TABLE t.test (k INT NOT NULL, v INT); if len(tableDesc.Mutations) != 0 { return errors.Errorf("expected 0 mutations after cancellation, found %d", len(tableDesc.Mutations)) } - if len(tableDesc.GetPrimaryIndex().ColumnNames) != 1 || tableDesc.GetPrimaryIndex().ColumnNames[0] != "rowid" { + if tableDesc.PrimaryIndexInterface().NumColumns() != 1 || tableDesc.PrimaryIndexInterface().GetColumnName(0) != "rowid" { return errors.Errorf("expected primary key change to not succeed after cancellation") } return nil diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 59d1dcdbacda..1b7fbb64fddb 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -286,7 +286,8 @@ func (n *scrubNode) startScrubTable( func getPrimaryColIdxs( tableDesc *tabledesc.Immutable, columns []*descpb.ColumnDescriptor, ) (primaryColIdxs []int, err error) { - for i, colID := range tableDesc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < tableDesc.PrimaryIndexInterface().NumColumns(); i++ { + colID := tableDesc.PrimaryIndexInterface().GetColumnID(i) rowIdx := -1 for idx, col := range columns { if col.ID == colID { @@ -298,7 +299,7 @@ func getPrimaryColIdxs( return nil, errors.Errorf( "could not find primary index column in projection: columnID=%d columnName=%s", colID, - tableDesc.GetPrimaryIndex().ColumnNames[i]) + tableDesc.PrimaryIndexInterface().GetColumnName(i)) } primaryColIdxs = append(primaryColIdxs, rowIdx) } diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index d60d25e5b578..b81fbd9c69af 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -114,7 +114,8 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { // Get primary key columns not included in the FK. var colIDs []descpb.ColumnID colIDs = append(colIDs, o.constraint.FK.OriginColumnIDs...) - for _, pkColID := range o.tableDesc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { + pkColID := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) found := false for _, id := range o.constraint.FK.OriginColumnIDs { if pkColID == id { @@ -147,8 +148,9 @@ func (o *sqlForeignKeyCheckOperation) Next(params runParams) (tree.Datums, error // Collect the primary index values for generating the primary key // pretty string. - primaryKeyDatums := make(tree.Datums, 0, len(o.tableDesc.GetPrimaryIndex().ColumnIDs)) - for _, id := range o.tableDesc.GetPrimaryIndex().ColumnIDs { + primaryKeyDatums := make(tree.Datums, 0, o.tableDesc.PrimaryIndexInterface().NumColumns()) + for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { + id := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) idx := o.colIDToRowIdx.GetDefault(id) primaryKeyDatums = append(primaryKeyDatums, row[idx]) } @@ -163,7 +165,8 @@ func (o *sqlForeignKeyCheckOperation) Next(params runParams) (tree.Datums, error } rowDetails[col.Name] = row[idx].String() } - for _, id := range o.tableDesc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { + id := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) found := false for _, fkID := range o.constraint.FK.OriginColumnIDs { if id == fkID { diff --git a/pkg/sql/scrub_index.go b/pkg/sql/scrub_index.go index 90bf988216c7..ece75437b5ed 100644 --- a/pkg/sql/scrub_index.go +++ b/pkg/sql/scrub_index.go @@ -83,7 +83,8 @@ func (o *indexCheckOperation) Start(params runParams) error { var pkColumns, otherColumns []*descpb.ColumnDescriptor - for _, colID := range o.tableDesc.GetPrimaryIndex().ColumnIDs { + for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { + colID := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) col := &o.tableDesc.Columns[colToIdx.GetDefault(colID)] pkColumns = append(pkColumns, col) colToIdx.Set(colID, -1) diff --git a/pkg/sql/scrub_test.go b/pkg/sql/scrub_test.go index 6b851e88f2e1..4ff368838957 100644 --- a/pkg/sql/scrub_test.go +++ b/pkg/sql/scrub_test.go @@ -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(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) 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(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), 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(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), 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(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), 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(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index 4917b73ebfaa..c5193f14e5d9 100644 --- a/pkg/sql/show_create.go +++ b/pkg/sql/show_create.go @@ -96,16 +96,16 @@ func ShowCreateTable( return "", err } f.WriteString(colstr) - if desc.IsPhysicalTable() && desc.GetPrimaryIndex().ColumnIDs[0] == col.ID { + if desc.IsPhysicalTable() && desc.PrimaryIndexInterface().GetColumnID(0) == col.ID { // Only set primaryKeyIsOnVisibleColumn to true if the primary key // is on a visible column (not rowid). primaryKeyIsOnVisibleColumn = true } } if primaryKeyIsOnVisibleColumn || - (desc.IsPhysicalTable() && desc.GetPrimaryIndex().IsSharded()) { + (desc.IsPhysicalTable() && desc.PrimaryIndexInterface().IsSharded()) { f.WriteString(",\n\tCONSTRAINT ") - formatQuoteNames(&f.Buffer, desc.GetPrimaryIndex().Name) + formatQuoteNames(&f.Buffer, desc.PrimaryIndexInterface().GetName()) f.WriteString(" ") f.WriteString(desc.PrimaryKeyString()) } @@ -189,11 +189,11 @@ func ShowCreateTable( return "", err } - if err := showCreateInterleave(desc.GetPrimaryIndex(), &f.Buffer, dbPrefix, lCtx); err != nil { + if err := showCreateInterleave(desc.PrimaryIndexInterface().IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { return "", err } if err := ShowCreatePartitioning( - a, p.ExecCfg().Codec, desc, desc.GetPrimaryIndex(), &desc.GetPrimaryIndex().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ + a, p.ExecCfg().Codec, desc, desc.PrimaryIndexInterface().IndexDesc(), &desc.PrimaryIndexInterface().IndexDesc().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ ); err != nil { return "", err } diff --git a/pkg/sql/tablewriter_delete.go b/pkg/sql/tablewriter_delete.go index e081bbf6dace..ac0a84c0d86b 100644 --- a/pkg/sql/tablewriter_delete.go +++ b/pkg/sql/tablewriter_delete.go @@ -123,7 +123,7 @@ func (td *tableDeleter) deleteAllRowsScan( var rf row.Fetcher tableArgs := row.FetcherTableArgs{ Desc: td.tableDesc(), - Index: td.tableDesc().GetPrimaryIndex(), + Index: td.tableDesc().PrimaryIndexInterface().IndexDesc(), ColIdxMap: td.rd.FetchColIDtoRowIndex, Cols: td.rd.FetchCols, ValNeededForCol: valNeededForCol, @@ -254,7 +254,7 @@ func (td *tableDeleter) deleteIndexScan( var rf row.Fetcher tableArgs := row.FetcherTableArgs{ Desc: td.tableDesc(), - Index: td.tableDesc().GetPrimaryIndex(), + Index: td.tableDesc().PrimaryIndexInterface().IndexDesc(), ColIdxMap: td.rd.FetchColIDtoRowIndex, Cols: td.rd.FetchCols, ValNeededForCol: valNeededForCol, From 71b65849fbd3f30c6d30a3b6e1c6282af05c3269 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 4 Jan 2021 11:11:42 -0500 Subject: [PATCH 10/13] sql: rename PrimaryIndexInterface method to GetPrimaryIndex Previously, the name PrimaryIndexInterface had been chosen because a deprecated GetPrimaryIndex() method already existed and because PrimaryIndex is an existing field in descpb.IndexDescriptor which is embedded in the structs implementing the catalog.TableDescriptor interface. This patch renames the method to GetPrimaryIndex: although its return type differs it has the same meaning as the recently-removed method of the same name. Release note: None --- pkg/ccl/changefeedccl/avro_test.go | 2 +- pkg/ccl/changefeedccl/encoder.go | 4 +- pkg/ccl/changefeedccl/rowfetcher_cache.go | 4 +- pkg/server/settingsworker.go | 2 +- pkg/sql/alter_primary_key.go | 12 +-- pkg/sql/alter_table.go | 8 +- pkg/sql/backfill/backfill.go | 4 +- pkg/sql/catalog/descriptor.go | 2 +- pkg/sql/catalog/tabledesc/table.go | 4 +- pkg/sql/catalog/tabledesc/table_desc.go | 6 +- pkg/sql/check.go | 8 +- pkg/sql/create_index.go | 2 +- pkg/sql/create_stats.go | 6 +- pkg/sql/create_table.go | 14 ++-- pkg/sql/delete_range.go | 4 +- pkg/sql/drop_index.go | 2 +- pkg/sql/information_schema.go | 8 +- pkg/sql/opt_catalog.go | 4 +- pkg/sql/opt_exec_factory.go | 4 +- pkg/sql/partition_test.go | 2 +- .../physicalplan/fake_span_resolver_test.go | 2 +- pkg/sql/refresh_materialized_view.go | 2 +- pkg/sql/row/deleter.go | 4 +- pkg/sql/row/fetcher_mvcc_test.go | 2 +- pkg/sql/row/fetcher_test.go | 4 +- pkg/sql/row/helper.go | 8 +- pkg/sql/row/inserter.go | 6 +- pkg/sql/row/updater.go | 4 +- pkg/sql/rowenc/client_index_encoding_test.go | 82 +++++++++---------- pkg/sql/rowenc/index_encoding_test.go | 16 ++-- pkg/sql/rowenc/testutils.go | 2 +- pkg/sql/rowexec/zigzagjoiner.go | 4 +- pkg/sql/scan.go | 2 +- pkg/sql/schema_changer.go | 2 +- pkg/sql/schema_changer_test.go | 2 +- pkg/sql/scrub.go | 6 +- pkg/sql/scrub_fk.go | 14 ++-- pkg/sql/scrub_index.go | 4 +- pkg/sql/scrub_test.go | 10 +-- pkg/sql/show_create.go | 12 +-- pkg/sql/tablewriter_delete.go | 4 +- pkg/sql/zone_config.go | 2 +- 42 files changed, 148 insertions(+), 148 deletions(-) diff --git a/pkg/ccl/changefeedccl/avro_test.go b/pkg/ccl/changefeedccl/avro_test.go index d5e49f0edf11..612f2e6853aa 100644 --- a/pkg/ccl/changefeedccl/avro_test.go +++ b/pkg/ccl/changefeedccl/avro_test.go @@ -285,7 +285,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.PrimaryIndexInterface().IndexDesc()) + indexSchema, err := indexToAvroSchema(tableDesc, tableDesc.GetPrimaryIndex().IndexDesc()) require.NoError(t, err) require.Equal(t, `{"type":"record","name":"_u2603_","fields":[`+ diff --git a/pkg/ccl/changefeedccl/encoder.go b/pkg/ccl/changefeedccl/encoder.go index afd75d2e8254..e49f13a3a8d5 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -142,7 +142,7 @@ func (e *jsonEncoder) EncodeKey(_ context.Context, row encodeRow) ([]byte, error func (e *jsonEncoder) encodeKeyRaw(row encodeRow) ([]interface{}, error) { colIdxByID := row.tableDesc.ColumnIdxMap() - primaryIndex := row.tableDesc.PrimaryIndexInterface() + primaryIndex := row.tableDesc.GetPrimaryIndex() jsonEntries := make([]interface{}, primaryIndex.NumColumns()) for i := 0; i < primaryIndex.NumColumns(); i++ { colID := primaryIndex.GetColumnID(i) @@ -344,7 +344,7 @@ func (e *confluentAvroEncoder) EncodeKey(ctx context.Context, row encodeRow) ([] registered, ok := e.keyCache[cacheKey] if !ok { var err error - registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.PrimaryIndexInterface().IndexDesc()) + registered.schema, err = indexToAvroSchema(row.tableDesc, row.tableDesc.GetPrimaryIndex().IndexDesc()) if err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/rowfetcher_cache.go b/pkg/ccl/changefeedccl/rowfetcher_cache.go index 56472572daf6..15ec48f390e1 100644 --- a/pkg/ccl/changefeedccl/rowfetcher_cache.go +++ b/pkg/ccl/changefeedccl/rowfetcher_cache.go @@ -121,7 +121,7 @@ func (c *rowFetcherCache) TableDescForKey( } // Skip over the column data. - for ; skippedCols < tableDesc.PrimaryIndexInterface().NumColumns(); skippedCols++ { + for ; skippedCols < tableDesc.GetPrimaryIndex().NumColumns(); skippedCols++ { l, err := encoding.PeekLength(remaining) if err != nil { return nil, err @@ -173,7 +173,7 @@ func (c *rowFetcherCache) RowFetcherForTableDesc( row.FetcherTableArgs{ Spans: tableDesc.AllIndexSpans(c.codec), Desc: tableDesc, - Index: tableDesc.PrimaryIndexInterface().IndexDesc(), + Index: tableDesc.GetPrimaryIndex().IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, Cols: tableDesc.Columns, diff --git a/pkg/server/settingsworker.go b/pkg/server/settingsworker.go index 968a0d57e4c8..9d099ee31457 100644 --- a/pkg/server/settingsworker.go +++ b/pkg/server/settingsworker.go @@ -50,7 +50,7 @@ func processSystemConfigKVs( { types := []*types.T{tbl.Columns[0].Type} nameRow := make([]rowenc.EncDatum, 1) - _, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.PrimaryIndexInterface().IndexDesc(), types, nameRow, nil, kv.Key) + _, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.GetPrimaryIndex().IndexDesc(), types, nameRow, nil, kv.Key) if err != nil { return errors.Wrap(err, "failed to decode key") } diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index 3a1ddc516716..9f4a2777418d 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -91,12 +91,12 @@ func (p *planner) AlterPrimaryKey( } // Disable primary key changes on tables that are interleaved parents. - if tableDesc.PrimaryIndexInterface().NumInterleavedBy() != 0 { + if tableDesc.GetPrimaryIndex().NumInterleavedBy() != 0 { var sb strings.Builder sb.WriteString("[") comma := ", " - for i := 0; i < tableDesc.PrimaryIndexInterface().NumInterleavedBy(); i++ { - interleaveTableID := tableDesc.PrimaryIndexInterface().GetInterleavedBy(i).Table + for i := 0; i < tableDesc.GetPrimaryIndex().NumInterleavedBy(); i++ { + interleaveTableID := tableDesc.GetPrimaryIndex().GetInterleavedBy(i).Table if i != 0 { sb.WriteString(comma) } @@ -222,7 +222,7 @@ func (p *planner) AlterPrimaryKey( // Create a new index that indexes everything the old primary index // does, but doesn't store anything. if shouldCopyPrimaryKey(tableDesc, newPrimaryIndexDesc) { - oldPrimaryIndexCopy := tableDesc.PrimaryIndexInterface().IndexDescDeepCopy() + oldPrimaryIndexCopy := tableDesc.GetPrimaryIndex().IndexDescDeepCopy() // Clear the name of the index so that it gets generated by AllocateIDs. oldPrimaryIndexCopy.Name = "" oldPrimaryIndexCopy.StoreColumnIDs = nil @@ -314,7 +314,7 @@ func (p *planner) AlterPrimaryKey( // Mark the primary key of the table as valid. { - primaryIndex := *tableDesc.PrimaryIndexInterface().IndexDesc() + primaryIndex := *tableDesc.GetPrimaryIndex().IndexDesc() primaryIndex.Disabled = false tableDesc.SetPrimaryIndex(primaryIndex) } @@ -344,7 +344,7 @@ func (p *planner) AlterPrimaryKey( // * The new primary key isn't the same hash sharded old primary key with a // different bucket count. func shouldCopyPrimaryKey(desc *tabledesc.Mutable, newPK *descpb.IndexDescriptor) bool { - oldPK := desc.PrimaryIndexInterface() + oldPK := desc.GetPrimaryIndex() if !desc.HasPrimaryKey() { return false } diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 8874f181cfb9..46fed26abea7 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -436,7 +436,7 @@ func (n *alterTableNode) startExec(params runParams) error { return err } - if n.tableDesc.PrimaryIndexInterface().ContainsColumnID(colToDrop.ID) { + if n.tableDesc.GetPrimaryIndex().ContainsColumnID(colToDrop.ID) { return pgerror.Newf(pgcode.InvalidColumnReference, "column %q is referenced by the primary key", colToDrop.Name) } @@ -459,7 +459,7 @@ func (n *alterTableNode) startExec(params runParams) error { if !containsThisColumn { for j := 0; j < idx.NumExtraColumns(); j++ { id := idx.GetExtraColumnID(j) - if n.tableDesc.PrimaryIndexInterface().ContainsColumnID(id) { + if n.tableDesc.GetPrimaryIndex().ContainsColumnID(id) { // All secondary indices necessary contain the PK // columns, too. (See the comments on the definition of // IndexDescriptor). The presence of a PK column in the @@ -740,7 +740,7 @@ func (n *alterTableNode) startExec(params runParams) error { descriptorChanged = true case *tree.AlterTablePartitionBy: - primaryIndex := n.tableDesc.PrimaryIndexInterface() + primaryIndex := n.tableDesc.GetPrimaryIndex() newPartitioning, err := CreatePartitioning( params.ctx, params.p.ExecCfg().Settings, params.EvalContext(), @@ -1035,7 +1035,7 @@ func applyColumnMutation( } // Prevent a column in a primary key from becoming non-null. - if tableDesc.PrimaryIndexInterface().ContainsColumnID(col.ID) { + if tableDesc.GetPrimaryIndex().ContainsColumnID(col.ID) { return pgerror.Newf(pgcode.InvalidTableDefinition, `column "%s" is in a primary index`, col.Name) } diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index c8b5a12a51f4..4812f6a2b016 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -124,7 +124,7 @@ func (cb *ColumnBackfiller) init( tableArgs := row.FetcherTableArgs{ Desc: desc, - Index: desc.PrimaryIndexInterface().IndexDesc(), + Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: desc.ColumnIdxMap(), Cols: desc.Columns, ValNeededForCol: valNeededForCol, @@ -602,7 +602,7 @@ func (ib *IndexBackfiller) init( tableArgs := row.FetcherTableArgs{ Desc: desc, - Index: desc.PrimaryIndexInterface().IndexDesc(), + Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: ib.colIdxMap, Cols: ib.cols, ValNeededForCol: valNeededForCol, diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 884e1fc8957a..e83724143271 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -117,7 +117,7 @@ type TableDescriptor interface { GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) KeysPerRow(id descpb.IndexID) (int, error) - PrimaryIndexInterface() Index + GetPrimaryIndex() Index AllIndexes() []Index ActiveIndexes() []Index NonDropIndexes() []Index diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index d82fd5ba480e..2deaa3bdf681 100644 --- a/pkg/sql/catalog/tabledesc/table.go +++ b/pkg/sql/catalog/tabledesc/table.go @@ -339,7 +339,7 @@ func FindFKReferencedIndex( ) (*descpb.IndexDescriptor, error) { // Search for a unique index on the referenced table that matches our foreign // key columns. - primaryIndex := referencedTable.PrimaryIndexInterface() + primaryIndex := referencedTable.GetPrimaryIndex() if primaryIndex.IsValidReferencedIndex(referencedColIDs) { return primaryIndex.IndexDesc(), nil } @@ -363,7 +363,7 @@ func FindFKOriginIndex( ) (*descpb.IndexDescriptor, error) { // Search for an index on the origin table that matches our foreign // key columns. - if primaryIndex := originTable.PrimaryIndexInterface(); primaryIndex.IsValidOriginIndex(originColIDs) { + if primaryIndex := originTable.GetPrimaryIndex(); primaryIndex.IsValidOriginIndex(originColIDs) { return primaryIndex.IndexDesc(), nil } // If the PK doesn't match, find the index corresponding to the origin column. diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index 70f6893ce004..bd4fe92153f9 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -252,9 +252,9 @@ func (desc *Mutable) SetPublicNonPrimaryIndex(indexOrdinal int, index descpb.Ind desc.Indexes[indexOrdinal-1] = index } -// PrimaryIndexInterface returns the primary index in the form of a -// catalog.Index interface. -func (desc *wrapper) PrimaryIndexInterface() catalog.Index { +// GetPrimaryIndex returns the primary index in the form of a catalog.Index +// interface. +func (desc *wrapper) GetPrimaryIndex() catalog.Index { return index{desc: &desc.PrimaryIndex} } diff --git a/pkg/sql/check.go b/pkg/sql/check.go index 13cf97dc9f3c..a2541655aa78 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -96,8 +96,8 @@ func matchFullUnacceptableKeyQuery( srcNotNullExistsClause[i] = fmt.Sprintf("%s IS NOT NULL", srcCols[i]) } - for i := 0; i < srcTbl.PrimaryIndexInterface().NumColumns(); i++ { - id := srcTbl.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < srcTbl.GetPrimaryIndex().NumColumns(); i++ { + id := srcTbl.GetPrimaryIndex().GetColumnID(i) alreadyPresent := false for _, otherID := range fk.OriginColumnIDs { if id == otherID { @@ -160,8 +160,8 @@ func nonMatchingRowQuery( return "", nil, err } // Get primary key columns not included in the FK - for i := 0; i < srcTbl.PrimaryIndexInterface().NumColumns(); i++ { - pkColID := srcTbl.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < srcTbl.GetPrimaryIndex().NumColumns(); i++ { + pkColID := srcTbl.GetPrimaryIndex().GetColumnID(i) found := false for _, id := range fk.OriginColumnIDs { if pkColID == id { diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index 9089b94f6254..1a649bea41c1 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -391,7 +391,7 @@ func (n *createIndexNode) startExec(params runParams) error { // Warn against creating a non-partitioned index on a partitioned table, // which is undesirable in most cases. - if n.n.PartitionBy == nil && n.tableDesc.PrimaryIndexInterface().GetPartitioning().NumColumns > 0 { + if n.n.PartitionBy == nil && n.tableDesc.GetPrimaryIndex().GetPartitioning().NumColumns > 0 { params.p.BufferClientNotice( params.ctx, errors.WithHint( diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index fefc08be11f8..4f94b3e6fe8d 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -350,9 +350,9 @@ func createStatsDefaultColumns( } // Add column stats for the primary key. - for i := 0; i < desc.PrimaryIndexInterface().NumColumns(); i++ { + for i := 0; i < desc.GetPrimaryIndex().NumColumns(); i++ { // Generate stats for each column in the primary key. - addIndexColumnStatsIfNotExists(desc.PrimaryIndexInterface().GetColumnID(i), false /* isInverted */) + addIndexColumnStatsIfNotExists(desc.GetPrimaryIndex().GetColumnID(i), false /* isInverted */) // Only collect multi-column stats if enabled. if i == 0 || !multiColEnabled { @@ -361,7 +361,7 @@ func createStatsDefaultColumns( colIDs := make([]descpb.ColumnID, i+1) for j := 0; j <= i; j++ { - colIDs[j] = desc.PrimaryIndexInterface().GetColumnID(j) + colIDs[j] = desc.GetPrimaryIndex().GetColumnID(j) } // Remember the requested stats so we don't request duplicates. diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 70b7dd50917b..75e158efce9a 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -785,9 +785,9 @@ func ResolveFK( referencedColNames := d.ToCols // If no columns are specified, attempt to default to PK. if len(referencedColNames) == 0 { - referencedColNames = make(tree.NameList, target.PrimaryIndexInterface().NumColumns()) + referencedColNames = make(tree.NameList, target.GetPrimaryIndex().NumColumns()) for i := range referencedColNames { - referencedColNames[i] = tree.Name(target.PrimaryIndexInterface().GetColumnName(i)) + referencedColNames[i] = tree.Name(target.GetPrimaryIndex().GetColumnName(i)) } } @@ -1025,7 +1025,7 @@ func addInterleave( if err != nil { return err } - parentIndex := parentTable.PrimaryIndexInterface() + parentIndex := parentTable.GetPrimaryIndex() // typeOfIndex is used to give more informative error messages. var typeOfIndex string @@ -1652,7 +1652,7 @@ func NewTableDesc( } // If explicit primary keys are required, error out since a primary key was not supplied. - if desc.PrimaryIndexInterface().NumColumns() == 0 && desc.IsPhysicalTable() && evalCtx != nil && + if desc.GetPrimaryIndex().NumColumns() == 0 && desc.IsPhysicalTable() && evalCtx != nil && evalCtx.SessionData != nil && evalCtx.SessionData.RequireExplicitPrimaryKeys { return nil, errors.Errorf( "no primary key specified for table %s (require_explicit_primary_keys = true)", desc.Name) @@ -1712,19 +1712,19 @@ func NewTableDesc( } if n.Interleave != nil { - if err := addInterleave(ctx, txn, vt, &desc, desc.PrimaryIndexInterface().IndexDesc(), n.Interleave); err != nil { + if err := addInterleave(ctx, txn, vt, &desc, desc.GetPrimaryIndex().IndexDesc(), n.Interleave); err != nil { return nil, err } } if n.PartitionBy != nil { partitioning, err := CreatePartitioning( - ctx, st, evalCtx, &desc, desc.PrimaryIndexInterface().IndexDesc(), n.PartitionBy) + ctx, st, evalCtx, &desc, desc.GetPrimaryIndex().IndexDesc(), n.PartitionBy) if err != nil { return nil, err } { - newPrimaryIndex := *desc.PrimaryIndexInterface().IndexDesc() + newPrimaryIndex := *desc.GetPrimaryIndex().IndexDesc() newPrimaryIndex.Partitioning = partitioning desc.SetPrimaryIndex(newPrimaryIndex) } diff --git a/pkg/sql/delete_range.go b/pkg/sql/delete_range.go index 55608262e27b..2535d66f7e90 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.PrimaryIndexInterface().IndexDesc(), + Index: d.desc.GetPrimaryIndex().IndexDesc(), Spans: d.spans, } for i, interleaved := range d.interleavedDesc { allTables[i+1] = row.FetcherTableArgs{ Desc: interleaved, - Index: interleaved.PrimaryIndexInterface().IndexDesc(), + Index: interleaved.GetPrimaryIndex().IndexDesc(), Spans: d.spans, } } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 4d7a9f4f666e..5627069b7dbf 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -313,7 +313,7 @@ func (p *planner) dropIndexByName( // 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.PrimaryIndexInterface().IndexDesc() + remainingIndexes[0] = tableDesc.GetPrimaryIndex().IndexDesc() for _, index := range tableDesc.PublicNonPrimaryIndexes() { if index.GetID() != idx.ID { remainingIndexes = append(remainingIndexes, index.IndexDesc()) diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 9e803063f9ce..ccaaa4b6f489 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -1312,8 +1312,8 @@ CREATE TABLE information_schema.statistics ( } if hasImplicitCols { implicitCols = make(map[string]struct{}) - for i := 0; i < table.PrimaryIndexInterface().NumColumns(); i++ { - col := table.PrimaryIndexInterface().GetColumnName(i) + for i := 0; i < table.GetPrimaryIndex().NumColumns(); i++ { + col := table.GetPrimaryIndex().GetColumnName(i) implicitCols[col] = struct{}{} } } @@ -1346,8 +1346,8 @@ CREATE TABLE information_schema.statistics ( // // Note that simply iterating over implicitCols map // produces non-deterministic output. - for i := 0; i < table.PrimaryIndexInterface().NumColumns(); i++ { - col := table.PrimaryIndexInterface().GetColumnName(i) + for i := 0; i < table.GetPrimaryIndex().NumColumns(); i++ { + 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, diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 2254813ae704..2536937821ce 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -719,7 +719,7 @@ func newOptTable( for i := range ot.indexes { var idxDesc *descpb.IndexDescriptor if i == 0 { - idxDesc = desc.PrimaryIndexInterface().IndexDesc() + idxDesc = desc.GetPrimaryIndex().IndexDesc() } else { idxDesc = secondaryIndexes[i-1].IndexDesc() } @@ -1121,7 +1121,7 @@ func (oi *optIndex) init( oi.zone = zone oi.indexOrdinal = indexOrdinal oi.invertedVirtualColOrd = invertedVirtualColOrd - if desc == tab.desc.PrimaryIndexInterface().IndexDesc() { + if desc == 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. diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 3590c656e8c5..e8d50074d62d 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -547,7 +547,7 @@ func (ef *execFactory) ConstructIndexJoin( return nil, err } - primaryIndex := tabDesc.PrimaryIndexInterface() + primaryIndex := tabDesc.GetPrimaryIndex() tableScan.index = primaryIndex.IndexDesc() tableScan.disableBatchLimit() @@ -1562,7 +1562,7 @@ func (ef *execFactory) ConstructDeleteRange( autoCommit bool, ) (exec.Node, error) { tabDesc := table.(*optTable).desc - indexDesc := tabDesc.PrimaryIndexInterface().IndexDesc() + indexDesc := tabDesc.GetPrimaryIndex().IndexDesc() sb := span.MakeBuilder(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, indexDesc) if err := ef.planner.maybeSetSystemConfig(tabDesc.GetID()); err != nil { diff --git a/pkg/sql/partition_test.go b/pkg/sql/partition_test.go index 0859d27ce203..da6a3d7b6a1d 100644 --- a/pkg/sql/partition_test.go +++ b/pkg/sql/partition_test.go @@ -50,7 +50,7 @@ func TestRemovePartitioningOSS(t *testing.T) { // Hack in partitions. Doing this properly requires a CCL binary. { - primaryIndex := *tableDesc.PrimaryIndexInterface().IndexDesc() + primaryIndex := *tableDesc.GetPrimaryIndex().IndexDesc() primaryIndex.Partitioning = descpb.PartitioningDescriptor{ NumColumns: 1, Range: []descpb.PartitioningDescriptor_Range{{ diff --git a/pkg/sql/physicalplan/fake_span_resolver_test.go b/pkg/sql/physicalplan/fake_span_resolver_test.go index a52c69688973..8f90519a72f5 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.PrimaryIndexInterface().IndexDesc()) + primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.GetPrimaryIndex().IndexDesc()) span := tableDesc.PrimaryIndexSpan(keys.SystemSQLCodec) diff --git a/pkg/sql/refresh_materialized_view.go b/pkg/sql/refresh_materialized_view.go index 1b8f52128abe..b0f3b55e459c 100644 --- a/pkg/sql/refresh_materialized_view.go +++ b/pkg/sql/refresh_materialized_view.go @@ -67,7 +67,7 @@ func (n *refreshMaterializedViewNode) startExec(params runParams) error { } // Prepare the new set of indexes by cloning all existing indexes on the view. - newPrimaryIndex := n.desc.PrimaryIndexInterface().IndexDescDeepCopy() + newPrimaryIndex := n.desc.GetPrimaryIndex().IndexDescDeepCopy() newIndexes := make([]descpb.IndexDescriptor, len(n.desc.PublicNonPrimaryIndexes())) for i, idx := range n.desc.PublicNonPrimaryIndexes() { newIndexes[i] = idx.IndexDescDeepCopy() diff --git a/pkg/sql/row/deleter.go b/pkg/sql/row/deleter.go index 8394957d3d1c..4ebe2e29232b 100644 --- a/pkg/sql/row/deleter.go +++ b/pkg/sql/row/deleter.go @@ -67,8 +67,8 @@ func MakeDeleter( } return nil } - for j := 0; j < tableDesc.PrimaryIndexInterface().NumColumns(); j++ { - colID := tableDesc.PrimaryIndexInterface().GetColumnID(j) + for j := 0; j < tableDesc.GetPrimaryIndex().NumColumns(); j++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(j) if err := maybeAddCol(colID); err != nil { return Deleter{} } diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index eedfccc22575..5ff5d707f7ec 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -101,7 +101,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { args = append(args, row.FetcherTableArgs{ Spans: desc.AllIndexSpans(keys.SystemSQLCodec), Desc: desc, - Index: desc.PrimaryIndexInterface().IndexDesc(), + Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, Cols: desc.Columns, diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index 896ec181429d..86245eb097ae 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -194,7 +194,7 @@ func TestNextRowSingle(t *testing.T) { if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPrimaryIndexID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.PrimaryIndexInterface().GetName(), + tableDesc.Name, tableDesc.GetPrimaryIndex().GetName(), desc.GetName(), index.Name, ) } @@ -314,7 +314,7 @@ func TestNextRowBatchLimiting(t *testing.T) { if desc.GetID() != tableDesc.ID || index.ID != tableDesc.GetPrimaryIndexID() { t.Fatalf( "unexpected row retrieved from fetcher.\nnexpected: table %s - index %s\nactual: table %s - index %s", - tableDesc.Name, tableDesc.PrimaryIndexInterface().GetName(), + tableDesc.Name, tableDesc.GetPrimaryIndex().GetName(), desc.GetName(), index.Name, ) } diff --git a/pkg/sql/row/helper.go b/pkg/sql/row/helper.go index 95f431b97be9..8a83c055455b 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -49,7 +49,7 @@ func newRowHelper( // Pre-compute the encoding directions of the index key values for // pretty-printing in traces. - rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.PrimaryIndexInterface().IndexDesc()) + rh.primIndexValDirs = catalogkeys.IndexKeyValDirs(rh.TableDesc.GetPrimaryIndex().IndexDesc()) rh.secIndexValDirs = make([][]encoding.Direction, len(rh.Indexes)) for i := range rh.Indexes { @@ -89,7 +89,7 @@ func (rh *rowHelper) encodePrimaryIndex( rh.TableDesc.GetPrimaryIndexID()) } primaryIndexKey, _, err = rowenc.EncodeIndexKey( - rh.TableDesc, rh.TableDesc.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) + rh.TableDesc, rh.TableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, rh.primaryIndexKeyPrefix) return primaryIndexKey, err } @@ -137,8 +137,8 @@ func (rh *rowHelper) encodeSecondaryIndexes( // #6233). Once it is, use the shared one. func (rh *rowHelper) skipColumnInPK(colID descpb.ColumnID, value tree.Datum) (bool, error) { if rh.primaryIndexCols.Empty() { - for i := 0; i < rh.TableDesc.PrimaryIndexInterface().NumColumns(); i++ { - pkColID := rh.TableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < rh.TableDesc.GetPrimaryIndex().NumColumns(); i++ { + pkColID := rh.TableDesc.GetPrimaryIndex().GetColumnID(i) rh.primaryIndexCols.Add(pkColID) } } diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index acaf3b549518..00a9cf11e5f1 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -63,10 +63,10 @@ func MakeInserter( marshaled: make([]roachpb.Value, len(insertCols)), } - for i := 0; i < tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - colID := tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(i) if _, ok := ri.InsertColIDtoRowIndex.Get(colID); !ok { - return Inserter{}, fmt.Errorf("missing %q primary key column", tableDesc.PrimaryIndexInterface().GetColumnName(i)) + return Inserter{}, fmt.Errorf("missing %q primary key column", tableDesc.GetPrimaryIndex().GetColumnName(i)) } } diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index fc62788e3f4c..f37b59ffd6c3 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -98,8 +98,8 @@ func MakeUpdater( updateColIDtoRowIndex := ColIDtoRowIndexFromCols(updateCols) var primaryIndexCols catalog.TableColSet - for i := 0; i < tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - colID := tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(i) primaryIndexCols.Add(colID) } diff --git a/pkg/sql/rowenc/client_index_encoding_test.go b/pkg/sql/rowenc/client_index_encoding_test.go index 755878279ab1..81baed783953 100644 --- a/pkg/sql/rowenc/client_index_encoding_test.go +++ b/pkg/sql/rowenc/client_index_encoding_test.go @@ -74,29 +74,29 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // See encodeStartConstraintAscending. { - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/NOTNULLASC", expected: "/NOTNULLASC", }, { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NOTNULLASC", }, { - index: grandchild.PrimaryIndexInterface(), + index: grandchild.GetPrimaryIndex(), input: "/1/#/2/3/#/NOTNULLASC", expected: "/1/#/2/3/#/NOTNULLASC", }, { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/NOTNULLASC", expected: "/1/#/NOTNULLASC", }, { - index: grandchild.PrimaryIndexInterface(), + index: grandchild.GetPrimaryIndex(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NOTNULLASC", }, @@ -131,19 +131,19 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // Keys that belong to the given index (neither parent nor // children keys) do not need to be tightened. { - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1", expected: "/1", }, { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3", expected: "/1/#/2/3", }, // Parent keys wrt child index is not tightened. { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1", expected: "/1", }, @@ -151,24 +151,24 @@ func TestAdjustStartKeyForInterleave(t *testing.T) { // Children keys wrt to parent index is tightened (pushed // forwards) to the next parent key. { - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1/#/2/3", expected: "/2", }, { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3/#/4", expected: "/1/#/2/4", }, // Key with len > 1 tokens. { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/12345678901234/#/1234/1234567890/#/123/1234567", expected: "/12345678901234/#/1234/1234567891", }, { - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/12345678901234/#/d1403.2594/shelloworld/#/123/1234567", expected: "/12345678901234/#/d1403.2594/shelloworld/PrefixEnd", }, @@ -291,21 +291,21 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/NOTNULLASC", expected: "/NULLASC/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NULLASC/#", }, { table: grandchild, - index: grandchild.PrimaryIndexInterface(), + index: grandchild.GetPrimaryIndex(), input: "/1/#/2/3/#/NOTNULLASC", expected: "/1/#/2/3/#/NULLASC/#", }, @@ -314,7 +314,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // partial primary key columns. { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/NOTNULLASC", expected: "/1/#/NOTNULLASC", }, @@ -322,7 +322,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // No change since key belongs to an ancestor. { table: grandchild, - index: grandchild.PrimaryIndexInterface(), + index: grandchild.GetPrimaryIndex(), input: "/1/#/2/NOTNULLASC", expected: "/1/#/2/NOTNULLASC", }, @@ -367,7 +367,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/NULLASC", inclusive: true, expected: "/NULLASC/#", @@ -375,7 +375,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/NULLASC", inclusive: true, expected: "/1/#/2/NULLASC/#", @@ -387,13 +387,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1", expected: "/0/#", }, { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1", inclusive: true, expected: "/1/#", @@ -401,13 +401,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3", expected: "/1/#/2/2/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3", inclusive: true, expected: "/1/#/2/3/#", @@ -417,19 +417,19 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1/#", expected: "/1/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#", expected: "/1/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/2/#", expected: "/1/#/2/2/#", }, @@ -439,13 +439,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1/#/2/3", expected: "/1/#", }, { table: parent, - index: parent.PrimaryIndexInterface(), + index: parent.GetPrimaryIndex(), input: "/1/#/2/3", inclusive: true, expected: "/1/#", @@ -453,13 +453,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3/#/4", expected: "/1/#/2/3/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/3/#/4", inclusive: true, expected: "/1/#/2/3/#", @@ -469,13 +469,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1", expected: "/1", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1", inclusive: true, expected: "/2", @@ -486,13 +486,13 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2", expected: "/1/#/2", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2", inclusive: true, expected: "/1/#/3", @@ -531,7 +531,7 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // Key with len > 1 tokens. { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/12345678901234/#/12345/12345678901234/#/123/1234567", expected: "/12345678901234/#/12345/12345678901234/#", }, @@ -606,20 +606,20 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // decimal value. { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/d3.4567", expected: "/1/#/2/d3.4567", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/d3.4567", inclusive: true, expected: "/1/#/2/d3.4567/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/d3.4567/#/8", expected: "/1/#/2/d3.4567/#", }, @@ -629,20 +629,20 @@ func TestAdjustEndKeyForInterleave(t *testing.T) { // bytes value. { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/shelloworld", expected: "/1/#/2/shelloworld", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/shelloworld", inclusive: true, expected: "/1/#/2/shelloworld/#", }, { table: child, - index: child.PrimaryIndexInterface(), + index: child.GetPrimaryIndex(), input: "/1/#/2/shelloworld/#/3", expected: "/1/#/2/shelloworld/#", }, diff --git a/pkg/sql/rowenc/index_encoding_test.go b/pkg/sql/rowenc/index_encoding_test.go index dc664f9f06ab..2fd77c186f69 100644 --- a/pkg/sql/rowenc/index_encoding_test.go +++ b/pkg/sql/rowenc/index_encoding_test.go @@ -225,7 +225,7 @@ func TestIndexKey(t *testing.T) { codec := keys.SystemSQLCodec primaryKeyPrefix := MakeIndexKeyPrefix(codec, tableDesc, tableDesc.GetPrimaryIndexID()) - primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colMap, testValues, primaryKeyPrefix) + primaryKey, _, err := EncodeIndexKey(tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colMap, testValues, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -275,7 +275,7 @@ func TestIndexKey(t *testing.T) { } } - checkEntry(tableDesc.PrimaryIndexInterface().IndexDesc(), primaryIndexKV) + checkEntry(tableDesc.GetPrimaryIndex().IndexDesc(), primaryIndexKV) checkEntry(tableDesc.PublicNonPrimaryIndexes()[0].IndexDesc(), secondaryIndexKV) } } @@ -941,7 +941,7 @@ func TestIndexKeyEquivSignature(t *testing.T) { desc, colMap := makeTableDescForTest(tc.table.indexKeyArgs) primaryKeyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPrimaryIndexID()) primaryKey, _, err := EncodeIndexKey( - desc, desc.PrimaryIndexInterface().IndexDesc(), colMap, tc.table.values, primaryKeyPrefix) + desc, desc.GetPrimaryIndex().IndexDesc(), colMap, tc.table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } @@ -962,7 +962,7 @@ func TestIndexKeyEquivSignature(t *testing.T) { // Column values should be at the beginning of the // remaining bytes of the key. - pkIndexDesc := desc.PrimaryIndexInterface().IndexDesc() + pkIndexDesc := desc.GetPrimaryIndex().IndexDesc() colVals, null, err := EncodeColumns(pkIndexDesc.ColumnIDs, pkIndexDesc.ColumnDirections, colMap, tc.table.values, nil /*key*/) if err != nil { t.Fatal(err) @@ -998,7 +998,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.TableDescriptor, desc.PrimaryIndexInterface().IndexDesc()) + equivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex().IndexDesc()) if err != nil { t.Fatal(err) } @@ -1083,13 +1083,13 @@ func TestEquivSignature(t *testing.T) { desc, colMap := makeTableDescForTest(table.indexKeyArgs) primaryKeyPrefix := MakeIndexKeyPrefix(keys.SystemSQLCodec, desc, desc.GetPrimaryIndexID()) primaryKey, _, err := EncodeIndexKey( - desc, desc.PrimaryIndexInterface().IndexDesc(), colMap, table.values, primaryKeyPrefix) + desc, desc.GetPrimaryIndex().IndexDesc(), colMap, table.values, primaryKeyPrefix) if err != nil { t.Fatal(err) } // Extract out the table's equivalence signature. - tempEquivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.PrimaryIndexInterface().IndexDesc()) + tempEquivSigs, err := TableEquivSignatures(&desc.TableDescriptor, desc.GetPrimaryIndex().IndexDesc()) if err != nil { t.Fatal(err) } @@ -1323,6 +1323,6 @@ func ExtractIndexKey( decodedValues[len(values)+i] = value.Datum } indexKey, _, err := EncodeIndexKey( - tableDesc, tableDesc.PrimaryIndexInterface().IndexDesc(), colMap, decodedValues, indexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colMap, decodedValues, indexKeyPrefix) return indexKey, err } diff --git a/pkg/sql/rowenc/testutils.go b/pkg/sql/rowenc/testutils.go index c3f72944f33c..4b2a721b2666 100644 --- a/pkg/sql/rowenc/testutils.go +++ b/pkg/sql/rowenc/testutils.go @@ -999,7 +999,7 @@ func RandEncDatumRowsOfTypes(rng *rand.Rand, numRows int, types []*types.T) EncD func TestingMakePrimaryIndexKey( desc catalog.TableDescriptor, vals ...interface{}, ) (roachpb.Key, error) { - index := desc.PrimaryIndexInterface() + index := desc.GetPrimaryIndex() if len(vals) > index.NumColumns() { return nil, errors.Errorf("got %d values, PK has %d columns", len(vals), index.NumColumns()) } diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index f6f315cee8b3..c723dbe664c9 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -647,8 +647,8 @@ func (zi *zigzagJoinerInfo) eqOrdering() (colinfo.ColumnOrdering, error) { if err != nil { return nil, err } - } else if idx := findColumnID(zi.table.PrimaryIndexInterface().IndexDesc().ColumnIDs, colID); idx != -1 { - direction, err = zi.table.PrimaryIndexInterface().GetColumnDirection(idx).ToEncodingDirection() + } else if idx := findColumnID(zi.table.GetPrimaryIndex().IndexDesc().ColumnIDs, 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 8964e74849f5..435c318eb1db 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -315,7 +315,7 @@ func initColsForScan( // Initializes the column structures. func (n *scanNode) initDescDefaults(colCfg scanColumnsConfig) error { n.colCfg = colCfg - n.index = n.desc.PrimaryIndexInterface().IndexDesc() + n.index = n.desc.GetPrimaryIndex().IndexDesc() var err error n.cols, err = initColsForScan(n.desc, n.colCfg) diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 5a8197674235..efb6083fa104 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1176,7 +1176,7 @@ func (sc *SchemaChanger) done(ctx context.Context) error { // existing indexes on the table. if mutation.Direction == descpb.DescriptorMutation_ADD { desc := fmt.Sprintf("REFRESH MATERIALIZED VIEW %q cleanup", scTable.Name) - pkJob, err := sc.createIndexGCJob(ctx, scTable.PrimaryIndexInterface().IndexDesc(), txn, desc) + pkJob, err := sc.createIndexGCJob(ctx, scTable.GetPrimaryIndex().IndexDesc(), txn, desc) if err != nil { return err } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 9771eecc7a3b..e4b4cebbeaa8 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -3078,7 +3078,7 @@ CREATE TABLE t.test (k INT NOT NULL, v INT); if len(tableDesc.Mutations) != 0 { return errors.Errorf("expected 0 mutations after cancellation, found %d", len(tableDesc.Mutations)) } - if tableDesc.PrimaryIndexInterface().NumColumns() != 1 || tableDesc.PrimaryIndexInterface().GetColumnName(0) != "rowid" { + if tableDesc.GetPrimaryIndex().NumColumns() != 1 || tableDesc.GetPrimaryIndex().GetColumnName(0) != "rowid" { return errors.Errorf("expected primary key change to not succeed after cancellation") } return nil diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 1b7fbb64fddb..e6e5f7416da9 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -286,8 +286,8 @@ func (n *scrubNode) startScrubTable( func getPrimaryColIdxs( tableDesc *tabledesc.Immutable, columns []*descpb.ColumnDescriptor, ) (primaryColIdxs []int, err error) { - for i := 0; i < tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - colID := tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := tableDesc.GetPrimaryIndex().GetColumnID(i) rowIdx := -1 for idx, col := range columns { if col.ID == colID { @@ -299,7 +299,7 @@ func getPrimaryColIdxs( return nil, errors.Errorf( "could not find primary index column in projection: columnID=%d columnName=%s", colID, - tableDesc.PrimaryIndexInterface().GetColumnName(i)) + tableDesc.GetPrimaryIndex().GetColumnName(i)) } primaryColIdxs = append(primaryColIdxs, rowIdx) } diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index b81fbd9c69af..ceba8cbff41e 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -114,8 +114,8 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { // Get primary key columns not included in the FK. var colIDs []descpb.ColumnID colIDs = append(colIDs, o.constraint.FK.OriginColumnIDs...) - for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - pkColID := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + pkColID := o.tableDesc.GetPrimaryIndex().GetColumnID(i) found := false for _, id := range o.constraint.FK.OriginColumnIDs { if pkColID == id { @@ -148,9 +148,9 @@ func (o *sqlForeignKeyCheckOperation) Next(params runParams) (tree.Datums, error // Collect the primary index values for generating the primary key // pretty string. - primaryKeyDatums := make(tree.Datums, 0, o.tableDesc.PrimaryIndexInterface().NumColumns()) - for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - id := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) + primaryKeyDatums := make(tree.Datums, 0, o.tableDesc.GetPrimaryIndex().NumColumns()) + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + id := o.tableDesc.GetPrimaryIndex().GetColumnID(i) idx := o.colIDToRowIdx.GetDefault(id) primaryKeyDatums = append(primaryKeyDatums, row[idx]) } @@ -165,8 +165,8 @@ func (o *sqlForeignKeyCheckOperation) Next(params runParams) (tree.Datums, error } rowDetails[col.Name] = row[idx].String() } - for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - id := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + id := o.tableDesc.GetPrimaryIndex().GetColumnID(i) found := false for _, fkID := range o.constraint.FK.OriginColumnIDs { if id == fkID { diff --git a/pkg/sql/scrub_index.go b/pkg/sql/scrub_index.go index ece75437b5ed..6a1d566e7c1b 100644 --- a/pkg/sql/scrub_index.go +++ b/pkg/sql/scrub_index.go @@ -83,8 +83,8 @@ func (o *indexCheckOperation) Start(params runParams) error { var pkColumns, otherColumns []*descpb.ColumnDescriptor - for i := 0; i < o.tableDesc.PrimaryIndexInterface().NumColumns(); i++ { - colID := o.tableDesc.PrimaryIndexInterface().GetColumnID(i) + for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { + colID := o.tableDesc.GetPrimaryIndex().GetColumnID(i) col := &o.tableDesc.Columns[colToIdx.GetDefault(colID)] pkColumns = append(pkColumns, col) colToIdx.Set(colID, -1) diff --git a/pkg/sql/scrub_test.go b/pkg/sql/scrub_test.go index 4ff368838957..5cb5cb2da662 100644 --- a/pkg/sql/scrub_test.go +++ b/pkg/sql/scrub_test.go @@ -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.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) 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.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), 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.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), 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.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), 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.PrimaryIndexInterface().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) + tableDesc, tableDesc.GetPrimaryIndex().IndexDesc(), colIDtoRowIndex, values, primaryIndexKeyPrefix) if err != nil { t.Fatalf("unexpected error: %s", err) } diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index c5193f14e5d9..bf0bf3a09403 100644 --- a/pkg/sql/show_create.go +++ b/pkg/sql/show_create.go @@ -96,16 +96,16 @@ func ShowCreateTable( return "", err } f.WriteString(colstr) - if desc.IsPhysicalTable() && desc.PrimaryIndexInterface().GetColumnID(0) == col.ID { + if desc.IsPhysicalTable() && desc.GetPrimaryIndex().GetColumnID(0) == col.ID { // Only set primaryKeyIsOnVisibleColumn to true if the primary key // is on a visible column (not rowid). primaryKeyIsOnVisibleColumn = true } } if primaryKeyIsOnVisibleColumn || - (desc.IsPhysicalTable() && desc.PrimaryIndexInterface().IsSharded()) { + (desc.IsPhysicalTable() && desc.GetPrimaryIndex().IsSharded()) { f.WriteString(",\n\tCONSTRAINT ") - formatQuoteNames(&f.Buffer, desc.PrimaryIndexInterface().GetName()) + formatQuoteNames(&f.Buffer, desc.GetPrimaryIndex().GetName()) f.WriteString(" ") f.WriteString(desc.PrimaryKeyString()) } @@ -142,7 +142,7 @@ func ShowCreateTable( } allIdx := append( append([]catalog.Index{}, desc.PublicNonPrimaryIndexes()...), - desc.PrimaryIndexInterface()) + desc.GetPrimaryIndex()) for _, idx := range allIdx { // Only add indexes to the create_statement column, and not to the // create_nofks column if they are not associated with an INTERLEAVE @@ -189,11 +189,11 @@ func ShowCreateTable( return "", err } - if err := showCreateInterleave(desc.PrimaryIndexInterface().IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { + if err := showCreateInterleave(desc.GetPrimaryIndex().IndexDesc(), &f.Buffer, dbPrefix, lCtx); err != nil { return "", err } if err := ShowCreatePartitioning( - a, p.ExecCfg().Codec, desc, desc.PrimaryIndexInterface().IndexDesc(), &desc.PrimaryIndexInterface().IndexDesc().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ + a, p.ExecCfg().Codec, desc, desc.GetPrimaryIndex().IndexDesc(), &desc.GetPrimaryIndex().IndexDesc().Partitioning, &f.Buffer, 0 /* indent */, 0, /* colOffset */ ); err != nil { return "", err } diff --git a/pkg/sql/tablewriter_delete.go b/pkg/sql/tablewriter_delete.go index ac0a84c0d86b..fed9c125ca18 100644 --- a/pkg/sql/tablewriter_delete.go +++ b/pkg/sql/tablewriter_delete.go @@ -123,7 +123,7 @@ func (td *tableDeleter) deleteAllRowsScan( var rf row.Fetcher tableArgs := row.FetcherTableArgs{ Desc: td.tableDesc(), - Index: td.tableDesc().PrimaryIndexInterface().IndexDesc(), + Index: td.tableDesc().GetPrimaryIndex().IndexDesc(), ColIdxMap: td.rd.FetchColIDtoRowIndex, Cols: td.rd.FetchCols, ValNeededForCol: valNeededForCol, @@ -254,7 +254,7 @@ func (td *tableDeleter) deleteIndexScan( var rf row.Fetcher tableArgs := row.FetcherTableArgs{ Desc: td.tableDesc(), - Index: td.tableDesc().PrimaryIndexInterface().IndexDesc(), + Index: td.tableDesc().GetPrimaryIndex().IndexDesc(), ColIdxMap: td.rd.FetchColIDtoRowIndex, Cols: td.rd.FetchCols, ValNeededForCol: valNeededForCol, diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index dd6fe8345f1c..db8aa866cba5 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -317,7 +317,7 @@ func resolveSubzone( indexName := string(zs.TableOrIndex.Index) var index catalog.Index if indexName == "" { - index = table.PrimaryIndexInterface() + index = table.GetPrimaryIndex() indexName = index.GetName() } else { var err error From 02a8b3d0a1d558438dacc137435fd0b4f6d62573 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Thu, 7 Jan 2021 11:00:18 -0500 Subject: [PATCH 11/13] sql: improve index caching in tabledesc.Immutable This patch changes the indexCache struct in a bid to reduce the number of memory allocations that take place when accessing its contents. The previous behaviour of GetPrimaryIndex() was particularly bad in that it resulted in an allocation in every call, which is easily avoided. Release note: None --- pkg/sql/catalog/tabledesc/index.go | 187 +++++++++++++++--------- pkg/sql/catalog/tabledesc/table_desc.go | 2 +- pkg/sql/rowenc/index_encoding.go | 2 +- 3 files changed, 123 insertions(+), 68 deletions(-) diff --git a/pkg/sql/catalog/tabledesc/index.go b/pkg/sql/catalog/tabledesc/index.go index 928d389e4be5..9a4f76792ab5 100644 --- a/pkg/sql/catalog/tabledesc/index.go +++ b/pkg/sql/catalog/tabledesc/index.go @@ -309,9 +309,29 @@ func (w index) GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID return w.desc.CompositeColumnIDs[compositeColumnOrdinal] } +type backingSlicesState int + +const ( + uninitialized backingSlicesState = iota + primary + active + all +) + // indexCache contains lazily precomputed slices of catalog.Index interfaces. -// A field value of nil indicates that the slice hasn't been precomputed yet. +// The valid initial state is indexCache{}. type indexCache struct { + // These first two slices grow as needed to derive all other slices below. + // Each interface in backingInterfaces points to the corresponding entry + // in backingStructs, this is done to minimize allocations. + backingStructs []index + backingInterfaces []catalog.Index + // backingSlicesState specifies how far the above slices have grown. + backingSlicesState backingSlicesState + + // The remaining fields store the precomputed slices of interest. + // A value of nil signifies it hasn't been precomputed yet, + // therefore empty precomputations need to be stored as []catalog.Index{}. all []catalog.Index active []catalog.Index nonDrop []catalog.Index @@ -322,19 +342,79 @@ type indexCache struct { partial []catalog.Index } +// primaryIndex returns the catalog.Index interface for the primary index. +func (c *indexCache) primaryIndex(desc *wrapper) catalog.Index { + c.ensureBackingSlices(desc, primary) + return c.backingInterfaces[0] +} + +// ensureBackingSlices ensures that backingStructs and backingInterfaces are +// populated with enough entries, as specified by requiredState. +func (c *indexCache) ensureBackingSlices(desc *wrapper, requiredState backingSlicesState) { + switch c.backingSlicesState { + case uninitialized: + // We need to transition the state to `primary` and perhaps beyond. + c.backingStructs = make([]index, 0, 1+len(desc.Indexes)+len(desc.Mutations)) + c.backingInterfaces = make([]catalog.Index, 0, cap(c.backingStructs)) + c.addToBackingSlices(index{desc: &desc.PrimaryIndex}) + c.backingSlicesState = primary + if requiredState <= primary { + return + } + fallthrough + case primary: + // We need to transition the state to `active` and perhaps beyond. + for i := range desc.Indexes { + c.addToBackingSlices(index{desc: &desc.Indexes[i]}) + } + c.backingSlicesState = active + if requiredState <= active { + return + } + fallthrough + case active: + // We need to transition the state to `all`. + for _, m := range desc.Mutations { + if idxDesc := m.GetIndex(); idxDesc != nil { + c.addToBackingSlices(index{ + desc: idxDesc, + mutationID: m.MutationID, + mutationState: m.State, + mutationDirection: m.Direction, + }) + } + } + c.backingSlicesState = all + fallthrough + case all: + // We're good, the `all` state is terminal. + return + } +} + +// addToBackingSlices adds a new index to the backing slices and sets its +// ordinal number in order of addition. +// This is a convenience method for ensureBackingSlices. +func (c *indexCache) addToBackingSlices(newIndex index) { + newIndex.ordinal = len(c.backingStructs) + c.backingStructs = append(c.backingStructs, newIndex) + c.backingInterfaces = append(c.backingInterfaces, &c.backingStructs[newIndex.ordinal]) +} + // cachedIndexes returns an already-build slice of catalog.Index interfaces if -// it exists, if not it builds it using the provided factory function and args. -// Notice that, as a result, empty slices need to be handled carefully. +// it exists, if not it builds it using the provided factory function and args, +// after populating the backing slices as much as required. +// Notice that, as a whole, empty slices need to be handled carefully. func (c *indexCache) cachedIndexes( cached *[]catalog.Index, factory func(c *indexCache, desc *wrapper) []catalog.Index, + requiredState backingSlicesState, desc *wrapper, ) []catalog.Index { if *cached == nil { - *cached = factory(c, desc) - if *cached == nil { - *cached = []catalog.Index{} - } + c.ensureBackingSlices(desc, requiredState) + s := factory(c, desc) + *cached = append(make([]catalog.Index, 0, len(s)), s...) } if len(*cached) == 0 { return nil @@ -344,69 +424,47 @@ func (c *indexCache) cachedIndexes( // buildPublicNonPrimary builds a fresh return value for // desc.PublicNonPrimaryIndexes(). -func buildPublicNonPrimary(_ *indexCache, desc *wrapper) []catalog.Index { - s := make([]catalog.Index, len(desc.Indexes)) - for i := range s { - s[i] = index{desc: &desc.Indexes[i], ordinal: i + 1} - } - return s +func buildPublicNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { + return c.backingInterfaces[1 : 1+len(desc.Indexes)] } func (c *indexCache) publicNonPrimaryIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.publicNonPrimary, buildPublicNonPrimary, desc) + return c.cachedIndexes(&c.publicNonPrimary, buildPublicNonPrimary, active, desc) } -// buildActive builds fresh return value for desc.ActiveIndexes(). +// buildActive builds a fresh return value for desc.ActiveIndexes(). func buildActive(c *indexCache, desc *wrapper) []catalog.Index { - publicNonPrimary := c.publicNonPrimaryIndexes(desc) - s := make([]catalog.Index, 1, 1+len(publicNonPrimary)) - s[0] = index{desc: &desc.PrimaryIndex} - return append(s, publicNonPrimary...) + return c.backingInterfaces[:1+len(desc.Indexes)] } func (c *indexCache) activeIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.active, buildActive, desc) -} - -// buildAll builds fresh return value for desc.AllIndexes(). -func buildAll(c *indexCache, desc *wrapper) []catalog.Index { - s := make([]catalog.Index, 0, 1+len(desc.Indexes)+len(desc.Mutations)) - s = append(s, c.activeIndexes(desc)...) - for _, m := range desc.Mutations { - if idxDesc := m.GetIndex(); idxDesc != nil { - idx := index{ - desc: idxDesc, - ordinal: len(s), - mutationID: m.MutationID, - mutationState: m.State, - mutationDirection: m.Direction, - } - s = append(s, idx) - } - } - return s + return c.cachedIndexes(&c.active, buildActive, active, desc) +} + +// buildAll builds a fresh return value for desc.AllIndexes(). +func buildAll(c *indexCache, _ *wrapper) []catalog.Index { + return c.backingInterfaces } func (c *indexCache) allIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.all, buildAll, desc) + return c.cachedIndexes(&c.all, buildAll, all, desc) } -// buildDeletableNonPrimary builds fresh return value for +// buildDeletableNonPrimary builds a fresh return value for // desc.DeletableNonPrimaryIndexes(). -func buildDeletableNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { - return c.allIndexes(desc)[1:] +func buildDeletableNonPrimary(c *indexCache, _ *wrapper) []catalog.Index { + return c.backingInterfaces[1:] } func (c *indexCache) deletableNonPrimaryIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.deletableNonPrimary, buildDeletableNonPrimary, desc) + return c.cachedIndexes(&c.deletableNonPrimary, buildDeletableNonPrimary, all, desc) } -// buildWritableNonPrimary builds fresh return value for +// buildWritableNonPrimary builds a fresh return value for // desc.WritableNonPrimaryIndexes(). -func buildWritableNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { - deletableNonPrimary := c.deletableNonPrimaryIndexes(desc) - s := make([]catalog.Index, 0, len(deletableNonPrimary)) - for _, idx := range deletableNonPrimary { +func buildWritableNonPrimary(c *indexCache, _ *wrapper) []catalog.Index { + s := make([]catalog.Index, 0, len(c.backingInterfaces[1:])) + for _, idx := range c.backingInterfaces[1:] { if idx.Public() || idx.WriteAndDeleteOnly() { s = append(s, idx) } @@ -415,15 +473,14 @@ func buildWritableNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { } func (c *indexCache) writableNonPrimaryIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.writableNonPrimary, buildWritableNonPrimary, desc) + return c.cachedIndexes(&c.writableNonPrimary, buildWritableNonPrimary, all, desc) } -// buildDeleteOnlyNonPrimary builds fresh return value for +// buildDeleteOnlyNonPrimary builds a fresh return value for // desc.DeleteOnlyNonPrimaryIndexes(). func buildDeleteOnlyNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { - deletableNonPublic := c.deletableNonPrimaryIndexes(desc)[len(desc.Indexes):] - s := make([]catalog.Index, 0, len(deletableNonPublic)) - for _, idx := range deletableNonPublic { + s := make([]catalog.Index, 0, len(desc.Mutations)) + for _, idx := range c.backingInterfaces[1+len(desc.Indexes):] { if idx.DeleteOnly() { s = append(s, idx) } @@ -432,14 +489,13 @@ func buildDeleteOnlyNonPrimary(c *indexCache, desc *wrapper) []catalog.Index { } func (c *indexCache) deleteOnlyNonPrimaryIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.deleteOnlyNonPrimary, buildDeleteOnlyNonPrimary, desc) + return c.cachedIndexes(&c.deleteOnlyNonPrimary, buildDeleteOnlyNonPrimary, all, desc) } -// buildNonDrop builds fresh return value for desc.NonDropIndexes(). +// buildNonDrop builds a fresh return value for desc.NonDropIndexes(). func buildNonDrop(c *indexCache, desc *wrapper) []catalog.Index { - all := c.allIndexes(desc) - s := make([]catalog.Index, 0, len(all)) - for _, idx := range all { + s := make([]catalog.Index, 0, len(c.backingInterfaces)) + for _, idx := range c.backingInterfaces { if !idx.Dropped() && (!idx.Primary() || desc.IsPhysicalTable()) { s = append(s, idx) } @@ -448,14 +504,13 @@ func buildNonDrop(c *indexCache, desc *wrapper) []catalog.Index { } func (c *indexCache) nonDropIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.nonDrop, buildNonDrop, desc) + return c.cachedIndexes(&c.nonDrop, buildNonDrop, all, desc) } -// buildPartial builds fresh return value for desc.PartialIndexes(). -func buildPartial(c *indexCache, desc *wrapper) []catalog.Index { - deletableNonPrimary := c.deletableNonPrimaryIndexes(desc) - s := make([]catalog.Index, 0, len(deletableNonPrimary)) - for _, idx := range deletableNonPrimary { +// buildPartial builds a fresh return value for desc.PartialIndexes(). +func buildPartial(c *indexCache, _ *wrapper) []catalog.Index { + s := make([]catalog.Index, 0, len(c.backingInterfaces[1:])) + for _, idx := range c.backingInterfaces[1:] { if idx.IsPartial() { s = append(s, idx) } @@ -464,5 +519,5 @@ func buildPartial(c *indexCache, desc *wrapper) []catalog.Index { } func (c *indexCache) partialIndexes(desc *wrapper) []catalog.Index { - return c.cachedIndexes(&c.partial, buildPartial, desc) + return c.cachedIndexes(&c.partial, buildPartial, all, desc) } diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index bd4fe92153f9..1290f2147a75 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -255,7 +255,7 @@ func (desc *Mutable) SetPublicNonPrimaryIndex(indexOrdinal int, index descpb.Ind // GetPrimaryIndex returns the primary index in the form of a catalog.Index // interface. func (desc *wrapper) GetPrimaryIndex() catalog.Index { - return index{desc: &desc.PrimaryIndex} + return desc.getExistingOrNewIndexCache().primaryIndex(desc) } // getExistingOrNewIndexCache should be the only place where the indexCache diff --git a/pkg/sql/rowenc/index_encoding.go b/pkg/sql/rowenc/index_encoding.go index 83a4857bbfa8..a04bda3da543 100644 --- a/pkg/sql/rowenc/index_encoding.go +++ b/pkg/sql/rowenc/index_encoding.go @@ -546,7 +546,7 @@ func DecodeIndexKeyPrefix( // TODO(dan): This whole operation is n^2 because of the interleaves // bookkeeping. We could improve it to n with a prefix tree of components. - interleaves := desc.ActiveIndexes() + interleaves := append(make([]catalog.Index, 0, len(desc.ActiveIndexes())), desc.ActiveIndexes()...) for component := 0; ; component++ { var tableID descpb.ID From e6448f435b0f073fc5e91f18081e82fd920b739d Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Thu, 7 Jan 2021 11:00:22 -0500 Subject: [PATCH 12/13] sql: turn some catalog.TableDescriptor methods into functions The catalog.TableDescriptor has ForEach*Index and Find*Index methods which can just as well be expressed as functions, thereby lightening that interface definition somewhat. This patch also adds iterutils.StopIteration() support for the above ForEach*Index functions, and otherwise also adds comments to many catalog.Index methods. Release note: None --- pkg/ccl/backupccl/backup_planning.go | 2 +- pkg/ccl/backupccl/restore_planning.go | 2 +- pkg/ccl/backupccl/targets.go | 2 +- pkg/ccl/importccl/import_processor.go | 2 +- pkg/ccl/partitionccl/partition.go | 2 +- pkg/sql/alter_table.go | 2 +- pkg/sql/catalog/BUILD.bazel | 1 + pkg/sql/catalog/descriptor.go | 283 +++++++++++++++++++++-- pkg/sql/catalog/tabledesc/index.go | 8 +- pkg/sql/catalog/tabledesc/safe_format.go | 2 +- pkg/sql/catalog/tabledesc/structured.go | 16 +- pkg/sql/catalog/tabledesc/table_desc.go | 209 ++++------------- pkg/sql/crdb_internal.go | 14 +- pkg/sql/create_table.go | 2 +- pkg/sql/drop_index.go | 4 +- pkg/sql/information_schema.go | 2 +- pkg/sql/partition_utils.go | 2 +- pkg/sql/pg_catalog.go | 8 +- pkg/sql/rowexec/joinreader_test.go | 2 +- pkg/sql/truncate.go | 2 +- 20 files changed, 346 insertions(+), 221 deletions(-) diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 0e3fab9a86cf..4276f65345d5 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -182,7 +182,7 @@ func getLogicallyMergedTableSpans( checkForKVInBounds func(start, end roachpb.Key, endTime hlc.Timestamp) (bool, error), ) ([]roachpb.Span, error) { var nonDropIndexIDs []descpb.IndexID - if err := table.ForEachNonDropIndex(func(idx catalog.Index) error { + if err := catalog.ForEachNonDropIndex(table, func(idx catalog.Index) error { key := tableAndIndex{tableID: table.GetID(), indexID: idx.GetID()} if added[key] { return nil diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 3e13127e297c..faf3da356b85 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -1021,7 +1021,7 @@ func RewriteTableDescs( return err } - if err := table.ForEachNonDropIndex(func(indexI catalog.Index) error { + if err := catalog.ForEachNonDropIndex(table, func(indexI catalog.Index) error { index := indexI.IndexDesc() // Verify that for any interleaved index being restored, the interleave // parent is also being restored. Otherwise, the interleave entries in the diff --git a/pkg/ccl/backupccl/targets.go b/pkg/ccl/backupccl/targets.go index 93b8ec72761b..15cd4b683fec 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -668,7 +668,7 @@ func ensureInterleavesIncluded(tables []catalog.TableDescriptor) error { } for _, table := range tables { - if err := table.ForEachIndex(catalog.IndexOpts{ + if err := catalog.ForEachIndex(table, catalog.IndexOpts{ AddMutations: true, }, func(index catalog.Index) error { for i := 0; i < index.NumInterleaveAncestors(); i++ { diff --git a/pkg/ccl/importccl/import_processor.go b/pkg/ccl/importccl/import_processor.go index 5bb40aa3fe75..9da93ace868f 100644 --- a/pkg/ccl/importccl/import_processor.go +++ b/pkg/ccl/importccl/import_processor.go @@ -191,7 +191,7 @@ func makeInputConverter( } if singleTable != nil { - if idx := singleTable.FindDeletableNonPrimaryIndex(func(idx catalog.Index) bool { + if idx := catalog.FindDeletableNonPrimaryIndex(singleTable, func(idx catalog.Index) bool { return idx.IsPartial() }); idx != nil { return nil, unimplemented.NewWithIssue(50225, "cannot import into table with partial indexes") diff --git a/pkg/ccl/partitionccl/partition.go b/pkg/ccl/partitionccl/partition.go index e61cd183e1d5..5b3941281c89 100644 --- a/pkg/ccl/partitionccl/partition.go +++ b/pkg/ccl/partitionccl/partition.go @@ -275,7 +275,7 @@ func selectPartitionExprs( a := &rowenc.DatumAlloc{} var prefixDatums []tree.Datum - if err := tableDesc.ForEachIndex(catalog.IndexOpts{ + if err := catalog.ForEachIndex(tableDesc, catalog.IndexOpts{ AddMutations: true, }, func(idx catalog.Index) error { return selectPartitionExprsByName( diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 46fed26abea7..6d4e170bdb40 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -816,7 +816,7 @@ func (n *alterTableNode) startExec(params runParams) error { // new name exists. This is what postgres does. switch details.Kind { case descpb.ConstraintTypeUnique, descpb.ConstraintTypePK: - if n.tableDesc.FindNonDropIndex(func(idx catalog.Index) bool { + if catalog.FindNonDropIndex(n.tableDesc, func(idx catalog.Index) bool { return idx.GetName() == string(t.NewName) }) != nil { return pgerror.Newf(pgcode.DuplicateRelation, diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index 2412f8841bab..36575a9cefe6 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -26,6 +26,7 @@ go_library( "//pkg/sql/types", "//pkg/util", "//pkg/util/hlc", + "//pkg/util/iterutil", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", ], diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index e83724143271..cc5001dfe33c 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -22,11 +22,13 @@ import ( "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" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) -// IndexOpts configures the behavior of TableDescriptor.ForEachIndex. +// IndexOpts configures the behavior of catalog.ForEachIndex and +// catalog.FindIndex. type IndexOpts struct { // NonPhysicalPrimaryIndex should be included. NonPhysicalPrimaryIndex bool @@ -112,40 +114,78 @@ type TableDescriptor interface { GetFormatVersion() descpb.FormatVersion GetPrimaryIndexID() descpb.IndexID + GetPrimaryIndex() Index PrimaryIndexSpan(codec keys.SQLCodec) roachpb.Span IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) KeysPerRow(id descpb.IndexID) (int, error) - GetPrimaryIndex() Index + // AllIndexes returns a slice with all indexes, public and non-public, + // in the underlying proto, in their canonical order: + // - the primary index, + // - the public non-primary indexes in the Indexes array, in order, + // - the non-public indexes present in the Mutations array, in order. + // + // See also Index.Ordinal(). AllIndexes() []Index + + // ActiveIndexes returns a slice with all public indexes in the underlying + // proto, in their canonical order: + // - the primary index, + // - the public non-primary indexes in the Indexes array, in order. + // + // See also Index.Ordinal(). ActiveIndexes() []Index + + // NonDropIndexes returns a slice of all non-drop indexes in the underlying + // proto, in their canonical order. This means: + // - the primary index, if the table is a physical table, + // - the public non-primary indexes in the Indexes array, in order, + // - the non-public indexes present in the Mutations array, in order, + // if the mutation is not a drop. + // + // See also Index.Ordinal(). NonDropIndexes() []Index + + // NonDropIndexes returns a slice of all partial indexes in the underlying + // proto, in their canonical order. This is equivalent to taking the slice + // produced by AllIndexes and removing indexes with empty expressions. PartialIndexes() []Index + + // PublicNonPrimaryIndexes returns a slice of all active secondary indexes, + // in their canonical order. This is equivalent to the Indexes array in the + // proto. PublicNonPrimaryIndexes() []Index + + // WritableNonPrimaryIndexes returns a slice of all non-primary indexes which + // allow being written to: public + delete-and-write-only, in their canonical + // order. This is equivalent to taking the slice produced by + // DeletableNonPrimaryIndexes and removing the indexes which are in mutations + // in the delete-only state. WritableNonPrimaryIndexes() []Index + + // DeletableNonPrimaryIndexes returns a slice of all non-primary indexes + // which allow being deleted from: public + delete-and-write-only + + // delete-only, in their canonical order. This is equivalent to taking + // the slice produced by AllIndexes and removing the primary index. DeletableNonPrimaryIndexes() []Index - DeleteOnlyNonPrimaryIndexes() []Index - ForEachIndex(opts IndexOpts, f func(idx Index) error) error - ForEachActiveIndex(f func(idx Index) error) error - ForEachNonDropIndex(f func(idx Index) error) error - ForEachPartialIndex(f func(idx Index) error) error - ForEachPublicNonPrimaryIndex(f func(idx Index) error) error - ForEachWritableNonPrimaryIndex(f func(idx Index) error) error - ForEachDeletableNonPrimaryIndex(f func(idx Index) error) error - ForEachDeleteOnlyNonPrimaryIndex(f func(idx Index) error) error - - FindIndex(opts IndexOpts, test func(idx Index) bool) Index - FindActiveIndex(test func(idx Index) bool) Index - FindNonDropIndex(test func(idx Index) bool) Index - FindPartialIndex(test func(idx Index) bool) Index - FindPublicNonPrimaryIndex(test func(idx Index) bool) Index - FindWritableNonPrimaryIndex(test func(idx Index) bool) Index - FindDeletableNonPrimaryIndex(test func(idx Index) bool) Index - FindDeleteOnlyNonPrimaryIndex(test func(idx Index) bool) Index + // DeleteOnlyNonPrimaryIndexes returns a slice of all non-primary indexes + // which allow only being deleted from, in their canonical order. This is + // equivalent to taking the slice produced by DeletableNonPrimaryIndexes and + // removing the indexes which are not in mutations or not in the delete-only + // state. + DeleteOnlyNonPrimaryIndexes() []Index + // FindIndexWithID returns the first catalog.Index that matches the id + // in the set of all indexes, or an error if none was found. The order of + // traversal is the canonical order, see Index.Ordinal(). FindIndexWithID(id descpb.IndexID) (Index, error) + + // FindIndexWithName returns the first catalog.Index that matches the name in + // the set of all indexes, excluding the primary index of non-physical + // tables, or an error if none was found. The order of traversal is the + // canonical order, see Index.Ordinal(). FindIndexWithName(name string) (Index, error) HasPrimaryKey() bool @@ -208,17 +248,52 @@ type TableDescriptor interface { // Index is an interface around the index descriptor types. type Index interface { + + // IndexDesc returns the underlying protobuf descriptor. + // Ideally, this method should be called as rarely as possible. IndexDesc() *descpb.IndexDescriptor + + // IndexDescDeepCopy returns a deep copy of the underlying proto. IndexDescDeepCopy() descpb.IndexDescriptor + // Ordinal returns the ordinal of the index in its parent table descriptor. + // + // The ordinal of an index in a `tableDesc descpb.TableDescriptor` is + // defined as follows: + // - 0 is the ordinal of the primary index, + // - [1:1+len(tableDesc.Indexes)] is the range of public non-primary indexes, + // - [1+len(tableDesc.Indexes):] is the range of non-public indexes. + // + // In terms of a `table catalog.TableDescriptor` interface, it is defined + // as the catalog.Index object's position in the table.AllIndexes() slice. Ordinal() int + + // Primary returns true iff the index is the primary index for the table + // descriptor. Primary() bool + + // Public returns true iff the index is active, i.e. readable, in the table + // descriptor. Public() bool + + // WriteAndDeleteOnly returns true iff the index is a mutation in the + // delete-and-write-only state in the table descriptor. WriteAndDeleteOnly() bool + + // DeleteOnly returns true iff the index is a mutation in the delete-only + // state in the table descriptor. DeleteOnly() bool + + // Adding returns true iff the index is an add mutation in the table + // descriptor. Adding() bool + + // Dropped returns true iff the index is a drop mutation in the table + // descriptor. Dropped() bool + // The remaining methods operate on the underlying descpb.IndexDescriptor object. + GetID() descpb.IndexID GetName() string IsInterleaved() bool @@ -229,34 +304,44 @@ type Index interface { IsCreatedExplicitly() bool GetPredicate() string GetType() descpb.IndexDescriptor_Type - IsValidOriginIndex(originColIDs descpb.ColumnIDs) bool - IsValidReferencedIndex(referencedColIDs descpb.ColumnIDs) bool GetGeoConfig() geoindex.Config - GetSharded() descpb.ShardedDescriptor - GetShardColumnName() string GetVersion() descpb.IndexDescriptorVersion GetEncodingType() descpb.IndexDescriptorEncodingType + + GetSharded() descpb.ShardedDescriptor + GetShardColumnName() string + + IsValidOriginIndex(originColIDs descpb.ColumnIDs) bool + IsValidReferencedIndex(referencedColIDs descpb.ColumnIDs) bool + GetPartitioning() descpb.PartitioningDescriptor FindPartitionByName(name string) descpb.PartitioningDescriptor PartitionNames() []string + NumInterleaveAncestors() int GetInterleaveAncestor(ancestorOrdinal int) descpb.InterleaveDescriptor_Ancestor + NumInterleavedBy() int GetInterleavedBy(interleavedByOrdinal int) descpb.ForeignKeyReference + NumColumns() int GetColumnID(columnOrdinal int) descpb.ColumnID GetColumnName(columnOrdinal int) string GetColumnDirection(columnOrdinal int) descpb.IndexDescriptor_Direction + + ForEachColumnID(func(id descpb.ColumnID) error) error ContainsColumnID(colID descpb.ColumnID) bool InvertedColumnID() descpb.ColumnID InvertedColumnName() string - ForEachColumnID(func(id descpb.ColumnID) error) error + NumStoredColumns() int GetStoredColumnID(storedColumnOrdinal int) descpb.ColumnID GetStoredColumnName(storedColumnOrdinal int) string HasOldStoredColumns() bool + NumExtraColumns() int GetExtraColumnID(extraColumnOrdinal int) descpb.ColumnID + NumCompositeColumns() int GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID } @@ -356,3 +441,151 @@ func FormatSafeDescriptorProperties(w *redact.StringBuilder, desc Descriptor) { w.Printf(", NumDrainingNames: %d", len(drainingNames)) } } + +func isIndexInSearchSet(desc TableDescriptor, opts IndexOpts, idx Index) bool { + if !opts.NonPhysicalPrimaryIndex && idx.Primary() && !desc.IsPhysicalTable() { + return false + } + if !opts.AddMutations && idx.Adding() { + return false + } + if !opts.DropMutations && idx.Dropped() { + return false + } + return true +} + +// ForEachIndex runs f over each index in the table descriptor according to +// filter parameters in opts. Indexes are visited in their canonical order, +// see Index.Ordinal(). ForEachIndex supports iterutil.StopIteration(). +func ForEachIndex(desc TableDescriptor, opts IndexOpts, f func(idx Index) error) error { + for _, idx := range desc.AllIndexes() { + if !isIndexInSearchSet(desc, opts, idx) { + continue + } + if err := f(idx); err != nil { + if iterutil.Done(err) { + return nil + } + return err + } + } + return nil +} + +func forEachIndex(slice []Index, f func(idx Index) error) error { + for _, idx := range slice { + if err := f(idx); err != nil { + if iterutil.Done(err) { + return nil + } + return err + } + } + return nil +} + +// ForEachActiveIndex is like ForEachIndex over ActiveIndexes(). +func ForEachActiveIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.ActiveIndexes(), f) +} + +// ForEachNonDropIndex is like ForEachIndex over NonDropIndexes(). +func ForEachNonDropIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.NonDropIndexes(), f) +} + +// ForEachPartialIndex is like ForEachIndex over PartialIndexes(). +func ForEachPartialIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.PartialIndexes(), f) +} + +// ForEachPublicNonPrimaryIndex is like ForEachIndex over +// PublicNonPrimaryIndexes(). +func ForEachPublicNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.PublicNonPrimaryIndexes(), f) +} + +// ForEachWritableNonPrimaryIndex is like ForEachIndex over +// WritableNonPrimaryIndexes(). +func ForEachWritableNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.WritableNonPrimaryIndexes(), f) +} + +// ForEachDeletableNonPrimaryIndex is like ForEachIndex over +// DeletableNonPrimaryIndexes(). +func ForEachDeletableNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.DeletableNonPrimaryIndexes(), f) +} + +// ForEachDeleteOnlyNonPrimaryIndex is like ForEachIndex over +// DeleteOnlyNonPrimaryIndexes(). +func ForEachDeleteOnlyNonPrimaryIndex(desc TableDescriptor, f func(idx Index) error) error { + return forEachIndex(desc.DeleteOnlyNonPrimaryIndexes(), f) +} + +// FindIndex returns the first index for which test returns true, nil otherwise, +// according to the parameters in opts just like ForEachIndex. +// Indexes are visited in their canonical order, see Index.Ordinal(). +func FindIndex(desc TableDescriptor, opts IndexOpts, test func(idx Index) bool) Index { + for _, idx := range desc.AllIndexes() { + if !isIndexInSearchSet(desc, opts, idx) { + continue + } + if test(idx) { + return idx + } + } + return nil +} + +func findIndex(slice []Index, test func(idx Index) bool) Index { + for _, idx := range slice { + if test(idx) { + return idx + } + } + return nil +} + +// FindActiveIndex returns the first index in ActiveIndex() for which test +// returns true. +func FindActiveIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.ActiveIndexes(), test) +} + +// FindNonDropIndex returns the first index in NonDropIndex() for which test +// returns true. +func FindNonDropIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.NonDropIndexes(), test) +} + +// FindPartialIndex returns the first index in PartialIndex() for which test +// returns true. +func FindPartialIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.PartialIndexes(), test) +} + +// FindPublicNonPrimaryIndex returns the first index in PublicNonPrimaryIndex() +// for which test returns true. +func FindPublicNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.PublicNonPrimaryIndexes(), test) +} + +// FindWritableNonPrimaryIndex returns the first index in +// WritableNonPrimaryIndex() for which test returns true. +func FindWritableNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.WritableNonPrimaryIndexes(), test) +} + +// FindDeletableNonPrimaryIndex returns the first index in +// DeletableNonPrimaryIndex() for which test returns true. +func FindDeletableNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.DeletableNonPrimaryIndexes(), test) +} + +// FindDeleteOnlyNonPrimaryIndex returns the first index in +// DeleteOnlyNonPrimaryIndex() for which test returns true. +func FindDeleteOnlyNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bool) Index { + return findIndex(desc.DeleteOnlyNonPrimaryIndexes(), test) +} diff --git a/pkg/sql/catalog/tabledesc/index.go b/pkg/sql/catalog/tabledesc/index.go index 9a4f76792ab5..e13cc7fcfe5a 100644 --- a/pkg/sql/catalog/tabledesc/index.go +++ b/pkg/sql/catalog/tabledesc/index.go @@ -40,7 +40,11 @@ func (w index) IndexDescDeepCopy() descpb.IndexDescriptor { return *protoutil.Clone(w.desc).(*descpb.IndexDescriptor) } -// Ordinal returns the ordinal of the index within the table descriptor. +// Ordinal returns the ordinal of the index in its parent TableDescriptor. +// The ordinal is defined as follows: +// - 0 is the ordinal of the primary index, +// - [1:1+len(desc.Indexes)] is the range of public non-primary indexes, +// - [1+len(desc.Indexes):] is the range of non-public indexes. func (w index) Ordinal() int { return w.ordinal } @@ -61,7 +65,7 @@ func (w index) Adding() bool { return w.mutationDirection == descpb.DescriptorMutation_ADD } -// Adding returns true iff the index is a drop mutation in the table descriptor. +// Dropped returns true iff the index is a drop mutation in the table descriptor. func (w index) Dropped() bool { return w.mutationDirection == descpb.DescriptorMutation_DROP } diff --git a/pkg/sql/catalog/tabledesc/safe_format.go b/pkg/sql/catalog/tabledesc/safe_format.go index 499205430a54..db975755e374 100644 --- a/pkg/sql/catalog/tabledesc/safe_format.go +++ b/pkg/sql/catalog/tabledesc/safe_format.go @@ -110,7 +110,7 @@ func formatSafeTableIndexes(w *redact.StringBuilder, desc catalog.TableDescripto w.Printf(", PrimaryIndex: %d", desc.GetPrimaryIndexID()) w.Printf(", NextIndexID: %d", desc.TableDesc().NextIndexID) w.Printf(", Indexes: [") - _ = desc.ForEachActiveIndex(func(idx catalog.Index) error { + _ = catalog.ForEachActiveIndex(desc, func(idx catalog.Index) error { if !idx.Primary() { w.Printf(", ") } diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 2b9019ddbea6..1ad6d78a04be 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -847,7 +847,7 @@ func ForEachExprStringInTableDesc(descI catalog.TableDescriptor, f func(expr *st } // Process all indexes. - if err := descI.ForEachIndex(catalog.IndexOpts{ + if err := catalog.ForEachIndex(descI, catalog.IndexOpts{ NonPhysicalPrimaryIndex: true, DropMutations: true, AddMutations: true, @@ -1048,7 +1048,7 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er } // Assign names to unnamed indexes. - _ = desc.ForEachDeletableNonPrimaryIndex(func(idx catalog.Index) error { + _ = catalog.ForEachDeletableNonPrimaryIndex(desc, func(idx catalog.Index) error { if len(idx.GetName()) == 0 { idx.IndexDesc().Name = buildIndexName(desc, idx) } @@ -1437,7 +1437,7 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des // un-upgraded foreign key references on the other table. This logic // somewhat parallels the logic in maybeUpgradeForeignKeyRepOnIndex. unupgradedFKsPresent := false - if err := referencedTable.ForEachIndex(catalog.IndexOpts{}, func(referencedIdx catalog.Index) error { + if err := catalog.ForEachIndex(referencedTable, catalog.IndexOpts{}, func(referencedIdx catalog.Index) error { if found { // TODO (lucy): If we ever revisit the tabledesc.Immutable methods, add // a way to break out of the index loop. @@ -1516,7 +1516,7 @@ func (desc *wrapper) validateCrossReferences(ctx context.Context, dg catalog.Des // un-upgraded foreign key references on the other table. This logic // somewhat parallels the logic in maybeUpgradeForeignKeyRepOnIndex. unupgradedFKsPresent := false - if err := originTable.ForEachIndex(catalog.IndexOpts{}, func(originIdx catalog.Index) error { + if err := catalog.ForEachIndex(originTable, catalog.IndexOpts{}, func(originIdx catalog.Index) error { if found { // TODO (lucy): If we ever revisit the tabledesc.Immutable methods, add // a way to break out of the index loop. @@ -1740,7 +1740,7 @@ func ValidateTableLocalityConfig( // ValidateIndexNameIsUnique validates that the index name does not exist. func (desc *wrapper) ValidateIndexNameIsUnique(indexName string) error { - if desc.FindNonDropIndex(func(idx catalog.Index) bool { + if catalog.FindNonDropIndex(desc, func(idx catalog.Index) bool { return idx.GetName() == indexName }) != nil { return sqlerrors.NewRelationAlreadyExistsError(indexName) @@ -2494,7 +2494,7 @@ func (desc *wrapper) validatePartitioning() error { partitionNames := make(map[string]string) a := &rowenc.DatumAlloc{} - return desc.ForEachNonDropIndex(func(idx catalog.Index) error { + return catalog.ForEachNonDropIndex(desc, func(idx catalog.Index) error { idxDesc := idx.IndexDesc() return desc.validatePartitioningDescriptor( a, idxDesc, &idxDesc.Partitioning, 0 /* colOffset */, partitionNames, @@ -3228,7 +3228,7 @@ func (desc *wrapper) FindFKByName(name string) (*descpb.ForeignKeyConstraint, er // IsInterleaved returns true if any part of this this table is interleaved with // another table's data. func (desc *wrapper) IsInterleaved() bool { - return nil != desc.FindNonDropIndex(func(idx catalog.Index) bool { + return nil != catalog.FindNonDropIndex(desc, func(idx catalog.Index) bool { return idx.IsInterleaved() }) } @@ -4071,7 +4071,7 @@ func (desc *Immutable) MutationColumns() []descpb.ColumnDescriptor { // 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 { - return nil != desc.FindNonDropIndex(func(idx catalog.Index) bool { + return nil != catalog.FindNonDropIndex(desc, func(idx catalog.Index) bool { return idx.IsSharded() && idx.GetShardColumnName() == col.Name }) } diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index 1290f2147a75..fa83f8907817 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -267,199 +267,84 @@ func (desc *wrapper) getExistingOrNewIndexCache() *indexCache { return desc.indexCache } -// AllIndexes returns a slice containing all indexes represented in the table -// descriptor, including mutations. +// AllIndexes returns a slice with all indexes, public and non-public, +// in the underlying proto, in their canonical order: +// - the primary index, +// - the public non-primary indexes in the Indexes array, in order, +// - the non-public indexes present in the Mutations array, in order. +// +// See also catalog.Index.Ordinal(). func (desc *wrapper) AllIndexes() []catalog.Index { return desc.getExistingOrNewIndexCache().allIndexes(desc) } -// ActiveIndexes returns a slice of all active (aka public) indexes. +// ActiveIndexes returns a slice with all public indexes in the underlying +// proto, in their canonical order: +// - the primary index, +// - the public non-primary indexes in the Indexes array, in order. +// +// See also catalog.Index.Ordinal(). func (desc *wrapper) ActiveIndexes() []catalog.Index { return desc.getExistingOrNewIndexCache().activeIndexes(desc) } -// NonDropIndexes returns a slice of all indexes (including mutations) which are -// not being dropped. +// NonDropIndexes returns a slice of all non-drop indexes in the underlying +// proto, in their canonical order. This means: +// - the primary index, if the table is a physical table, +// - the public non-primary indexes in the Indexes array, in order, +// - the non-public indexes present in the Mutations array, in order, +// if the mutation is not a drop. +// +// See also catalog.Index.Ordinal(). func (desc *wrapper) NonDropIndexes() []catalog.Index { return desc.getExistingOrNewIndexCache().nonDropIndexes(desc) } -// PartialIndexes returns a slice of all partial indexes in the table -// descriptor, including mutations. +// NonDropIndexes returns a slice of all partial indexes in the underlying +// proto, in their canonical order. This is equivalent to taking the slice +// produced by AllIndexes and filtering indexes with non-empty expressions. func (desc *wrapper) PartialIndexes() []catalog.Index { return desc.getExistingOrNewIndexCache().partialIndexes(desc) } -// PublicNonPrimaryIndexes returns a slice of all active secondary indexes. +// PublicNonPrimaryIndexes returns a slice of all active secondary indexes, +// in their canonical order. This is equivalent to the Indexes array in the +// proto. func (desc *wrapper) PublicNonPrimaryIndexes() []catalog.Index { return desc.getExistingOrNewIndexCache().publicNonPrimaryIndexes(desc) } -// WritableNonPrimaryIndexes returns a slice of all secondary indexes which -// allow being written to: active + delete-and-write-only. +// WritableNonPrimaryIndexes returns a slice of all non-primary indexes which +// allow being written to: public + delete-and-write-only, in their canonical +// order. This is equivalent to taking the slice produced by +// DeletableNonPrimaryIndexes and removing the indexes which are in mutations +// in the delete-only state. func (desc *wrapper) WritableNonPrimaryIndexes() []catalog.Index { return desc.getExistingOrNewIndexCache().writableNonPrimaryIndexes(desc) } -// DeletableNonPrimaryIndexes returns a slice of all secondary indexes which -// allow being deleted from: active + delete-and-write-only + delete-only. +// DeletableNonPrimaryIndexes returns a slice of all non-primary indexes +// which allow being deleted from: public + delete-and-write-only + +// delete-only, in their canonical order. This is equivalent to taking +// the slice produced by AllIndexes and removing the primary index. func (desc *wrapper) DeletableNonPrimaryIndexes() []catalog.Index { return desc.getExistingOrNewIndexCache().deletableNonPrimaryIndexes(desc) } -// DeletableNonPrimaryIndexes returns a slice of all secondary indexes which -// only allow being deleted from. +// DeleteOnlyNonPrimaryIndexes returns a slice of all non-primary indexes +// which allow only being deleted from, in their canonical order. This is +// equivalent to taking the slice produced by DeletableNonPrimaryIndexes and +// removing the indexes which are not in mutations or not in the delete-only +// state. func (desc *wrapper) DeleteOnlyNonPrimaryIndexes() []catalog.Index { return desc.getExistingOrNewIndexCache().deleteOnlyNonPrimaryIndexes(desc) } -// ForEachIndex runs f over each index in the table descriptor according to -// filter parameters in opts. -func (desc *wrapper) ForEachIndex(opts catalog.IndexOpts, f func(idx catalog.Index) error) error { - for _, idx := range desc.AllIndexes() { - if !opts.NonPhysicalPrimaryIndex && idx.Primary() && !desc.IsPhysicalTable() { - continue - } - if !opts.AddMutations && idx.Adding() { - continue - } - if !opts.DropMutations && idx.Dropped() { - continue - } - if err := f(idx); err != nil { - return err - } - } - return nil -} - -func forEachIndex(slice []catalog.Index, f func(idx catalog.Index) error) error { - for _, idx := range slice { - if err := f(idx); err != nil { - return err - } - } - return nil -} - -// ForEachActiveIndex is like ForEachIndex over ActiveIndexes(). -func (desc *wrapper) ForEachActiveIndex(f func(idx catalog.Index) error) error { - return forEachIndex(desc.ActiveIndexes(), f) -} - -// ForEachNonDropIndex is like ForEachIndex over NonDropIndexes(). -func (desc *wrapper) ForEachNonDropIndex(f func(idx catalog.Index) error) error { - return forEachIndex(desc.NonDropIndexes(), f) -} - -// ForEachPartialIndex is like ForEachIndex over PartialIndexes(). -func (desc *wrapper) ForEachPartialIndex(f func(idx catalog.Index) error) error { - return forEachIndex(desc.PartialIndexes(), f) -} - -// ForEachPublicNonPrimaryIndex is like ForEachIndex over -// PublicNonPrimaryIndexes(). -func (desc *wrapper) ForEachPublicNonPrimaryIndex(f func(idx catalog.Index) error) error { - return forEachIndex(desc.PublicNonPrimaryIndexes(), f) -} - -// ForEachWritableNonPrimaryIndex is like ForEachIndex over -// WritableNonPrimaryIndexes(). -func (desc *wrapper) ForEachWritableNonPrimaryIndex(f func(idx catalog.Index) error) error { - return forEachIndex(desc.WritableNonPrimaryIndexes(), f) -} - -// ForEachDeletableNonPrimaryIndex is like ForEachIndex over -// DeletableNonPrimaryIndexes(). -func (desc *wrapper) ForEachDeletableNonPrimaryIndex(f func(idx catalog.Index) error) error { - return forEachIndex(desc.DeletableNonPrimaryIndexes(), f) -} - -// ForEachDeleteOnlyNonPrimaryIndex is like ForEachIndex over -// DeleteOnlyNonPrimaryIndexes(). -func (desc *wrapper) ForEachDeleteOnlyNonPrimaryIndex(f func(idx catalog.Index) error) error { - return forEachIndex(desc.DeleteOnlyNonPrimaryIndexes(), f) -} - -// FindIndex returns the first index for which test returns true, nil otherwise, -// according to the parameters in opts just like ForEachIndex. -func (desc *wrapper) FindIndex( - opts catalog.IndexOpts, test func(idx catalog.Index) bool, -) catalog.Index { - for _, idx := range desc.AllIndexes() { - if !opts.NonPhysicalPrimaryIndex && idx.Primary() && !desc.IsPhysicalTable() { - continue - } - if !opts.AddMutations && idx.Adding() { - continue - } - if !opts.DropMutations && idx.Dropped() { - continue - } - if test(idx) { - return idx - } - } - return nil -} - -func findIndex(slice []catalog.Index, test func(idx catalog.Index) bool) catalog.Index { - for _, idx := range slice { - if test(idx) { - return idx - } - } - return nil -} - -// FindActiveIndex returns the first index in ActiveIndex() for which test -// returns true. -func (desc *wrapper) FindActiveIndex(test func(idx catalog.Index) bool) catalog.Index { - return findIndex(desc.ActiveIndexes(), test) -} - -// FindNonDropIndex returns the first index in NonDropIndex() for which test -// returns true. -func (desc *wrapper) FindNonDropIndex(test func(idx catalog.Index) bool) catalog.Index { - return findIndex(desc.NonDropIndexes(), test) -} - -// FindPartialIndex returns the first index in PartialIndex() for which test -// returns true. -func (desc *wrapper) FindPartialIndex(test func(idx catalog.Index) bool) catalog.Index { - return findIndex(desc.PartialIndexes(), test) -} - -// FindPublicNonPrimaryIndex returns the first index in PublicNonPrimaryIndex() -// for which test returns true. -func (desc *wrapper) FindPublicNonPrimaryIndex(test func(idx catalog.Index) bool) catalog.Index { - return findIndex(desc.PublicNonPrimaryIndexes(), test) -} - -// FindWritableNonPrimaryIndex returns the first index in -// WritableNonPrimaryIndex() for which test returns true. -func (desc *wrapper) FindWritableNonPrimaryIndex(test func(idx catalog.Index) bool) catalog.Index { - return findIndex(desc.WritableNonPrimaryIndexes(), test) -} - -// FindDeletableNonPrimaryIndex returns the first index in -// DeletableNonPrimaryIndex() for which test returns true. -func (desc *wrapper) FindDeletableNonPrimaryIndex(test func(idx catalog.Index) bool) catalog.Index { - return findIndex(desc.DeletableNonPrimaryIndexes(), test) -} - -// FindDeleteOnlyNonPrimaryIndex returns the first index in -// DeleteOnlyNonPrimaryIndex() for which test returns true. -func (desc *wrapper) FindDeleteOnlyNonPrimaryIndex( - test func(idx catalog.Index) bool, -) catalog.Index { - return findIndex(desc.DeleteOnlyNonPrimaryIndexes(), test) -} - // FindIndexWithID returns the first catalog.Index that matches the id -// in the set of all indexes. +// in the set of all indexes, or an error if none was found. The order of +// traversal is the canonical order, see catalog.Index.Ordinal(). func (desc *wrapper) FindIndexWithID(id descpb.IndexID) (catalog.Index, error) { - if idx := desc.FindIndex(catalog.IndexOpts{ + if idx := catalog.FindIndex(desc, catalog.IndexOpts{ NonPhysicalPrimaryIndex: true, DropMutations: true, AddMutations: true, @@ -477,9 +362,11 @@ func (desc *wrapper) FindIndexWithID(id descpb.IndexID) (catalog.Index, error) { } // FindIndexWithName returns the first catalog.Index that matches the name in -// the set of all indexes, excluding the primary index of non-physical tables. +// the set of all indexes, excluding the primary index of non-physical +// tables, or an error if none was found. The order of traversal is the +// canonical order, see catalog.Index.Ordinal(). func (desc *wrapper) FindIndexWithName(name string) (catalog.Index, error) { - if idx := desc.FindIndex(catalog.IndexOpts{ + if idx := catalog.FindIndex(desc, catalog.IndexOpts{ NonPhysicalPrimaryIndex: false, DropMutations: true, AddMutations: true, diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index ebb6313a1961..b5f1e76064e7 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1823,7 +1823,7 @@ CREATE TABLE crdb_internal.create_statements ( if createNofk == "" { createNofk = stmt } - hasPartitions := nil != table.FindIndex(catalog.IndexOpts{}, func(idx catalog.Index) bool { + hasPartitions := nil != catalog.FindIndex(table, catalog.IndexOpts{}, func(idx catalog.Index) bool { return idx.GetPartitioning().NumColumns != 0 }) return addRow( @@ -2050,7 +2050,7 @@ CREATE TABLE crdb_internal.table_indexes ( tableName := tree.NewDString(table.GetName()) // We report the primary index of non-physical tables here. These // indexes are not reported as a part of ForeachIndex. - return table.ForEachIndex(catalog.IndexOpts{ + return catalog.ForEachIndex(table, catalog.IndexOpts{ NonPhysicalPrimaryIndex: true, }, func(idx catalog.Index) error { row = row[:0] @@ -2179,7 +2179,7 @@ CREATE TABLE crdb_internal.index_columns ( return nil } - return table.ForEachIndex(catalog.IndexOpts{NonPhysicalPrimaryIndex: true}, reportIndex) + return catalog.ForEachIndex(table, catalog.IndexOpts{NonPhysicalPrimaryIndex: true}, reportIndex) }) }, } @@ -2261,7 +2261,7 @@ CREATE TABLE crdb_internal.backward_dependencies ( } // Record the backward references of the primary index. - if err := table.ForEachIndex(catalog.IndexOpts{}, reportIdxDeps); err != nil { + if err := catalog.ForEachIndex(table, catalog.IndexOpts{}, reportIdxDeps); err != nil { return err } @@ -2390,7 +2390,7 @@ CREATE TABLE crdb_internal.forward_dependencies ( } // Record the backward references of the primary index. - if err := table.ForEachIndex(catalog.IndexOpts{}, reportIdxDeps); err != nil { + if err := catalog.ForEachIndex(table, catalog.IndexOpts{}, reportIdxDeps); err != nil { return err } reportDependedOnBy := func( @@ -2820,7 +2820,7 @@ CREATE TABLE crdb_internal.zones ( } for i, s := range subzones { - index := table.FindActiveIndex(func(idx catalog.Index) bool { + index := catalog.FindActiveIndex(table, func(idx catalog.Index) bool { return idx.GetID() == descpb.IndexID(s.IndexID) }) if index == nil { @@ -3404,7 +3404,7 @@ CREATE TABLE crdb_internal.partitions ( worker := func(pusher rowPusher) error { return forEachTableDescAll(ctx, p, dbContext, hideVirtual, /* virtual tables have no partitions*/ func(db *dbdesc.Immutable, _ string, table catalog.TableDescriptor) error { - return table.ForEachIndex(catalog.IndexOpts{ + return catalog.ForEachIndex(table, catalog.IndexOpts{ AddMutations: true, }, func(index catalog.Index) error { return addPartitioningRows(ctx, p, dbName, table, index.IndexDesc(), &index.IndexDesc().Partitioning, diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 75e158efce9a..57df9d1981d7 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1886,7 +1886,7 @@ func NewTableDesc( } // Record the types of indexes that the table has. - if err := desc.ForEachNonDropIndex(func(idx catalog.Index) error { + if err := catalog.ForEachNonDropIndex(&desc, func(idx catalog.Index) error { if idx.IsSharded() { telemetry.Inc(sqltelemetry.HashShardedIndexCounter) } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 5627069b7dbf..2beb76a77bb6 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -202,7 +202,7 @@ func (n *dropIndexNode) maybeDropShardColumn( if dropped { return nil } - if tableDesc.FindNonDropIndex(func(otherIdx catalog.Index) bool { + if catalog.FindNonDropIndex(tableDesc, func(otherIdx catalog.Index) bool { return otherIdx.ContainsColumnID(shardColDesc.ID) }) != nil { return nil @@ -456,7 +456,7 @@ func (p *planner) dropIndexByName( ) } - foundIndex := tableDesc.FindPublicNonPrimaryIndex(func(idxEntry catalog.Index) bool { + foundIndex := catalog.FindPublicNonPrimaryIndex(tableDesc, func(idxEntry catalog.Index) bool { return idxEntry.GetID() == idx.ID }) diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index ccaaa4b6f489..d4dfae34c806 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -1297,7 +1297,7 @@ CREATE TABLE information_schema.statistics ( ) } - return table.ForEachIndex(catalog.IndexOpts{}, func(index catalog.Index) error { + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { // Columns in the primary key that aren't in index.ColumnNames or // index.StoreColumnNames are implicit columns in the index. var implicitCols map[string]struct{} diff --git a/pkg/sql/partition_utils.go b/pkg/sql/partition_utils.go index 3e74449a6ac1..f354a28c7e2b 100644 --- a/pkg/sql/partition_utils.go +++ b/pkg/sql/partition_utils.go @@ -100,7 +100,7 @@ func GenerateSubzoneSpans( var indexCovering covering.Covering var partitionCoverings []covering.Covering - if err := tableDesc.ForEachIndex(catalog.IndexOpts{ + if err := catalog.ForEachIndex(tableDesc, catalog.IndexOpts{ AddMutations: true, }, func(idx catalog.Index) error { _, indexSubzoneExists := subzoneIndexByIndexID[idx.GetID()] diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 741106caeb0a..0e0e18dce763 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -445,7 +445,7 @@ https://www.postgresql.org/docs/12/catalog-pg-attribute.html`, // Columns for each index. columnIdxMap := table.ColumnIdxMap() - return table.ForEachIndex(catalog.IndexOpts{}, func(index catalog.Index) error { + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { for i := 0; i < index.NumColumns(); i++ { colID := index.GetColumnID(i) idxID := h.IndexOid(table.GetID(), index.GetID()) @@ -625,7 +625,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-class.html`, } // Indexes. - return table.ForEachIndex(catalog.IndexOpts{}, func(index catalog.Index) error { + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { indexType := forwardIndexOid if index.GetType() == descpb.IndexDescriptor_INVERTED { indexType = invertedIndexOid @@ -1442,7 +1442,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, return forEachTableDesc(ctx, p, dbContext, hideVirtual, /* virtual tables do not have indexes */ func(db *dbdesc.Immutable, scName string, table catalog.TableDescriptor) error { tableOid := tableOid(table.GetID()) - return table.ForEachIndex(catalog.IndexOpts{}, func(index catalog.Index) error { + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { isMutation, isWriteOnly := table.GetIndexMutationCapabilities(index.GetID()) isReady := isMutation && isWriteOnly @@ -1524,7 +1524,7 @@ https://www.postgresql.org/docs/9.5/view-pg-indexes.html`, func(db *dbdesc.Immutable, scName string, table catalog.TableDescriptor, tableLookup tableLookupFn) error { scNameName := tree.NewDName(scName) tblName := tree.NewDName(table.GetName()) - return table.ForEachIndex(catalog.IndexOpts{}, func(index catalog.Index) error { + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(index catalog.Index) error { def, err := indexDefFromDescriptor(ctx, p, db, table, index.IndexDesc(), tableLookup) if err != nil { return err diff --git a/pkg/sql/rowexec/joinreader_test.go b/pkg/sql/rowexec/joinreader_test.go index 8ddbfde95fe8..83f738d3730d 100644 --- a/pkg/sql/rowexec/joinreader_test.go +++ b/pkg/sql/rowexec/joinreader_test.go @@ -1274,7 +1274,7 @@ func BenchmarkJoinReader(b *testing.B) { // Get the table descriptor and find the index that will provide us with // the expected match ratio. tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", tableName) - foundIndex := tableDesc.FindPublicNonPrimaryIndex(func(idx catalog.Index) bool { + foundIndex := catalog.FindPublicNonPrimaryIndex(tableDesc, func(idx catalog.Index) bool { require.Equal(b, 1, idx.NumColumns(), "all indexes created in this benchmark should only contain one column") return idx.GetColumnName(0) == columnDef.name }) diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 4de6cf76eab8..d1b54b0b40f6 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -376,7 +376,7 @@ func (p *planner) reassignInterleaveIndexReferences( ) error { for _, table := range tables { changed := false - if err := table.ForEachNonDropIndex(func(indexI catalog.Index) error { + if err := catalog.ForEachNonDropIndex(table, func(indexI catalog.Index) error { index := indexI.IndexDesc() for j, a := range index.Interleave.Ancestors { if a.TableID == truncatedID { From 5a0989417134d7ed73035b0077cd005007a9e301 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Thu, 7 Jan 2021 11:00:26 -0500 Subject: [PATCH 13/13] sql: add unit testing to new catalog.Index interface This commit introduces unit tests for the new catalog.Index interface, which were missing in my earlier commits. Release note: None --- pkg/sql/catalog/tabledesc/BUILD.bazel | 2 + pkg/sql/catalog/tabledesc/index_test.go | 319 ++++++++++++++++++++++++ 2 files changed, 321 insertions(+) create mode 100644 pkg/sql/catalog/tabledesc/index_test.go diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index 8d8b80d6ea2e..69ed02683e20 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -45,6 +45,7 @@ go_test( name = "tabledesc_test", srcs = [ "helpers_test.go", + "index_test.go", "main_test.go", "safe_format_test.go", "structured_test.go", @@ -54,6 +55,7 @@ go_test( embed = [":tabledesc"], deps = [ "//pkg/base", + "//pkg/geo/geoindex", "//pkg/keys", "//pkg/security", "//pkg/security/securitytest", diff --git a/pkg/sql/catalog/tabledesc/index_test.go b/pkg/sql/catalog/tabledesc/index_test.go new file mode 100644 index 000000000000..810d5902a49a --- /dev/null +++ b/pkg/sql/catalog/tabledesc/index_test.go @@ -0,0 +1,319 @@ +// 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 tabledesc_test + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/geo/geoindex" + "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/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestIndexInterface(t *testing.T) { + defer leaktest.AfterTest(t)() + + // This server is only used to turn a CREATE TABLE statement into a + // catalog.TableDescriptor. + s, conn, db := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(context.Background()) + if _, err := conn.Exec(`CREATE DATABASE d`); err != nil { + t.Fatalf("%+v", err) + } + sqlutils.MakeSQLRunner(conn).Exec(t, ` + SET experimental_enable_hash_sharded_indexes = on; + CREATE TABLE d.t ( + c1 INT, + c2 INT, + c3 INT, + c4 VARCHAR, + c5 VARCHAR, + c6 JSONB, + c7 GEOGRAPHY(GEOMETRY,4326) NULL, + CONSTRAINT pk PRIMARY KEY (c1 ASC, c2 ASC, c3 ASC), + INDEX s1 (c4 DESC, c5 DESC), + INVERTED INDEX s2 (c6), + INDEX s3 (c2, c3) STORING (c5, c6), + INDEX s4 (c5) USING HASH WITH BUCKET_COUNT=8, + UNIQUE INDEX s5 (c1, c4) WHERE c4 = 'x', + INVERTED INDEX s6 (c7) WITH (s2_level_mod=2) + ); + `) + + indexNames := []string{"pk", "s1", "s2", "s3", "s4", "s5", "s6"} + indexColumns := [][]string{ + {"c1", "c2", "c3"}, + {"c4", "c5"}, + {"c6"}, + {"c2", "c3"}, + {"crdb_internal_c5_shard_8", "c5"}, + {"c1", "c4"}, + {"c7"}, + } + extraColumnsAsPkColOrdinals := [][]int{ + {}, + {0, 1, 2}, + {0, 1, 2}, + {0}, + {0, 1, 2}, + {1, 2}, + {0, 1, 2}, + } + + immutable := catalogkv.TestingGetImmutableTableDescriptor(db, keys.SystemSQLCodec, "d", "t") + require.NotNil(t, immutable) + var tableI catalog.TableDescriptor = immutable + require.NotNil(t, tableI) + + // Find indexes by name, check that names are correct and that indexes + // are in the correct order. + indexes := make([]catalog.Index, len(indexNames)) + for i, name := range indexNames { + idx, err := tableI.FindIndexWithName(name) + require.NoError(t, err) + require.Equal(t, name, idx.GetName()) + require.Equal(t, i, idx.Ordinal()) + indexes[i] = idx + } + + pk := indexes[0] + s1 := indexes[1] + s2 := indexes[2] + s3 := indexes[3] + s4 := indexes[4] + s5 := indexes[5] + s6 := indexes[6] + + // Check that GetPrimaryIndex returns the primary index. + require.Equal(t, pk, tableI.GetPrimaryIndex()) + require.Equal(t, pk.GetID(), tableI.GetPrimaryIndexID()) + require.True(t, pk.Primary()) + require.True(t, pk.Public()) + require.Equal(t, descpb.PrimaryIndexEncoding, pk.GetEncodingType()) + + // Check that ActiveIndexes returns the same indexes in the same order. + require.Equal(t, indexes, tableI.ActiveIndexes()) + + // Check that PublicNonPrimaryIndexes returns the same indexes sans primary. + require.Equal(t, indexes[1:], tableI.PublicNonPrimaryIndexes()) + for _, idx := range tableI.PublicNonPrimaryIndexes() { + require.False(t, idx.Primary()) + require.True(t, idx.Public()) + require.Equal(t, descpb.SecondaryIndexEncoding, idx.GetEncodingType()) + } + + // Check that ForEachActiveIndex visits indexes in the same order as well. + { + expectedOrdinal := 0 + err := catalog.ForEachActiveIndex(tableI, func(idx catalog.Index) error { + if idx.Ordinal() != expectedOrdinal { + return fmt.Errorf("expected ordinal %d for index %s, instead got %d", + expectedOrdinal, idx.GetName(), idx.Ordinal()) + } + expectedOrdinal++ + return nil + }) + require.NoError(t, err) + } + + // Check various catalog.ForEach* and catalog.Find* functions. + for _, s := range []struct { + name string + expectedIndexNames []string + sliceFunc func(catalog.TableDescriptor) []catalog.Index + forEachFunc func(catalog.TableDescriptor, func(catalog.Index) error) error + findFunc func(catalog.TableDescriptor, func(catalog.Index) bool) catalog.Index + }{ + { + "ActiveIndex", + indexNames, + catalog.TableDescriptor.ActiveIndexes, + catalog.ForEachActiveIndex, + catalog.FindActiveIndex, + }, + { + "NonDropIndex", + indexNames, + catalog.TableDescriptor.NonDropIndexes, + catalog.ForEachNonDropIndex, + catalog.FindNonDropIndex, + }, + { + "PartialIndex", + []string{"s5"}, + catalog.TableDescriptor.PartialIndexes, + catalog.ForEachPartialIndex, + catalog.FindPartialIndex, + }, + { + "PublicNonPrimaryIndex", + indexNames[1:], + catalog.TableDescriptor.PublicNonPrimaryIndexes, + catalog.ForEachPublicNonPrimaryIndex, + catalog.FindPublicNonPrimaryIndex, + }, + { + "WritableNonPrimaryIndex", + indexNames[1:], + catalog.TableDescriptor.WritableNonPrimaryIndexes, + catalog.ForEachWritableNonPrimaryIndex, + catalog.FindWritableNonPrimaryIndex, + }, + { + "DeletableNonPrimaryIndex", + indexNames[1:], + catalog.TableDescriptor.DeletableNonPrimaryIndexes, + catalog.ForEachDeletableNonPrimaryIndex, + catalog.FindDeletableNonPrimaryIndex, + }, + { + "DeleteOnlyNonPrimaryIndex", + []string{}, + catalog.TableDescriptor.DeleteOnlyNonPrimaryIndexes, + catalog.ForEachDeleteOnlyNonPrimaryIndex, + catalog.FindDeleteOnlyNonPrimaryIndex, + }, + } { + expected := s.sliceFunc(tableI) + var actual []catalog.Index + err := s.forEachFunc(tableI, func(index catalog.Index) error { + actual = append(actual, index) + return nil + }) + require.NoErrorf(t, err, "Unexpected error from ForEach%s.", s.name) + require.Equalf(t, expected, actual, "Unexpected results from ForEach%s.", s.name) + actualNames := make([]string, len(actual)) + for i, idx := range actual { + actualNames[i] = idx.GetName() + } + require.Equalf(t, s.expectedIndexNames, actualNames, "Unexpected results from %ses.", s.name) + for _, expectedIndex := range expected { + foundIndex := s.findFunc(tableI, func(index catalog.Index) bool { + return index == expectedIndex + }) + require.Equalf(t, foundIndex, expectedIndex, "Unexpected results from Find%s.", s.name) + } + } + + // Check that finding indexes by ID is correct. + for _, idx := range indexes { + found, err := tableI.FindIndexWithID(idx.GetID()) + require.NoError(t, err) + require.Equalf(t, idx.GetID(), found.GetID(), + "mismatched IDs for index '%s'", idx.GetName()) + } + + // Check index metadata. + for _, idx := range indexes { + require.False(t, idx.WriteAndDeleteOnly()) + require.False(t, idx.DeleteOnly()) + require.False(t, idx.Adding()) + require.False(t, idx.Dropped()) + } + + errMsgFmt := "Unexpected %s result for index '%s'." + + // Check index methods on features not tested here. + for _, idx := range indexes { + require.False(t, idx.IsInterleaved(), + errMsgFmt, "IsInterleaved", idx.GetName()) + require.False(t, idx.IsDisabled(), + errMsgFmt, "IsDisabled", idx.GetName()) + require.False(t, idx.IsCreatedExplicitly(), + errMsgFmt, "IsCreatedExplicitly", idx.GetName()) + require.Equal(t, descpb.IndexDescriptorVersion(0x2), idx.GetVersion(), + errMsgFmt, "GetVersion", idx.GetName()) + require.Equal(t, descpb.PartitioningDescriptor{}, idx.GetPartitioning(), + errMsgFmt, "GetPartitioning", idx.GetName()) + require.Equal(t, []string(nil), idx.PartitionNames(), + errMsgFmt, "PartitionNames", idx.GetName()) + require.Equal(t, 0, idx.NumInterleaveAncestors(), + errMsgFmt, "NumInterleaveAncestors", idx.GetName()) + require.Equal(t, 0, idx.NumInterleavedBy(), + errMsgFmt, "NumInterleavedBy", idx.GetName()) + require.False(t, idx.HasOldStoredColumns(), + errMsgFmt, "HasOldStoredColumns", idx.GetName()) + require.Equalf(t, 0, idx.NumCompositeColumns(), + errMsgFmt, "NumCompositeColumns", idx.GetName()) + } + + // Check particular index features. + require.Equal(t, "c4 = 'x':::STRING", s5.GetPredicate()) + require.Equal(t, "crdb_internal_c5_shard_8", s4.GetShardColumnName()) + require.Equal(t, int32(2), s6.GetGeoConfig().S2Geography.S2Config.LevelMod) + for _, idx := range indexes { + require.Equalf(t, idx == s5, idx.IsPartial(), + errMsgFmt, "IsPartial", idx.GetName()) + require.Equal(t, idx == s5, idx.GetPredicate() != "", + errMsgFmt, "GetPredicate", idx.GetName()) + require.Equal(t, idx == s5 || idx == pk, idx.IsUnique(), + errMsgFmt, "IsUnique", idx.GetName()) + require.Equal(t, idx == s2 || idx == s6, idx.GetType() == descpb.IndexDescriptor_INVERTED, + errMsgFmt, "GetType", idx.GetName()) + require.Equal(t, idx == s4, idx.IsSharded(), + errMsgFmt, "IsSharded", idx.GetName()) + require.Equal(t, idx == s6, !(&geoindex.Config{}).Equal(idx.GetGeoConfig()), + errMsgFmt, "GetGeoConfig", idx.GetName()) + require.Equal(t, idx == s4, idx.GetShardColumnName() != "", + errMsgFmt, "GetShardColumnName", idx.GetName()) + require.Equal(t, idx == s4, !(&descpb.ShardedDescriptor{}).Equal(idx.GetSharded()), + errMsgFmt, "GetSharded", idx.GetName()) + require.Equalf(t, idx != s3, idx.NumStoredColumns() == 0, + errMsgFmt, "NumStoredColumns", idx.GetName()) + } + + // Check index columns. + for i, idx := range indexes { + expectedColNames := indexColumns[i] + actualColNames := make([]string, idx.NumColumns()) + for j := range actualColNames { + actualColNames[j] = idx.GetColumnName(j) + require.Equalf(t, idx == s1, idx.GetColumnDirection(j) == descpb.IndexDescriptor_DESC, + "mismatched column directions for index '%s'", idx.GetName()) + require.True(t, idx.ContainsColumnID(idx.GetColumnID(j)), + "column ID resolution failure for column '%s' in index '%s'", idx.GetColumnName(j), idx.GetName()) + } + require.Equalf(t, expectedColNames, actualColNames, + "mismatched columns for index '%s'", idx.GetName()) + } + + // Check index extra columns. + for i, idx := range indexes { + expectedExtraColIDs := make([]descpb.ColumnID, len(extraColumnsAsPkColOrdinals[i])) + for j, pkColOrdinal := range extraColumnsAsPkColOrdinals[i] { + expectedExtraColIDs[j] = pk.GetColumnID(pkColOrdinal) + } + actualExtraColIDs := make([]descpb.ColumnID, idx.NumExtraColumns()) + for j := range actualExtraColIDs { + actualExtraColIDs[j] = idx.GetExtraColumnID(j) + } + require.Equalf(t, expectedExtraColIDs, actualExtraColIDs, + "mismatched extra columns for index '%s'", idx.GetName()) + } + + // Check particular index column features. + require.Equal(t, "c6", s2.InvertedColumnName()) + require.Equal(t, s2.GetColumnID(0), s2.InvertedColumnID()) + require.Equal(t, "c7", s6.InvertedColumnName()) + require.Equal(t, s6.GetColumnID(0), s6.InvertedColumnID()) + require.Equal(t, 2, s3.NumStoredColumns()) + require.Equal(t, "c5", s3.GetStoredColumnName(0)) + require.Equal(t, "c6", s3.GetStoredColumnName(1)) +}