From 61c8f050746b4ad827ddae2867e3d5cea7f7cf42 Mon Sep 17 00:00:00 2001 From: Rohan Yadav Date: Mon, 16 Dec 2019 15:55:28 -0500 Subject: [PATCH] sql: remove explicit index IDs in foreign key descriptors Fixes #43195. This PR removes explicit index IDs in foreign key descriptors, and changing code that needed them to find a suitable index for the lookup they need. To be more specific, this PR removes all _reads_ of the explicit index IDs in foreign key descriptors, but continues to write them to maintain compatiblity in mutli-version cluster settings. In 20.2, these IDs can be finally fully removed. This PR also attempts to work around a situation where old descriptors on disk using the 19.1 format version could potentially lose information about the foreign key constraints when the index the old version descriptor points to has been deleted. This PR unblocks work on allowing primary key changes to proceed on tables that either reference or are referenced by foreign keys. This PR allows indexes that reference or are referenced by a foreign key constraint to be dropped if there is another suitable index for the constraint. Lastly, this PR adds in a mixed 19.2/20.1 cluster setting for logic tests. Release note (sql change): This PR allows indexes that reference or are referenced by a foreign key constraint to be dropped if there is another suitable index for the constraint. --- docs/generated/settings/settings.html | 2 +- pkg/settings/cluster/cockroach_versions.go | 10 + pkg/settings/cluster/versionkey_string.go | 5 +- pkg/sql/crdb_internal.go | 10 +- pkg/sql/create_table.go | 37 ++ pkg/sql/drop_index.go | 96 ++- pkg/sql/information_schema.go | 2 +- pkg/sql/logictest/logic.go | 9 + .../logictest/testdata/logic_test/drop_index | 49 ++ .../logic_test/drop_index_mixed_19.2_20.1 | 276 +++++++++ pkg/sql/old_foreign_key_desc_test.go | 134 +++++ pkg/sql/pg_catalog.go | 4 +- pkg/sql/row/cascader.go | 4 +- pkg/sql/row/fk_existence_delete.go | 8 +- pkg/sql/row/fk_existence_insert.go | 8 +- pkg/sql/sqlbase/structured.go | 43 +- pkg/sql/sqlbase/structured.pb.go | 546 +++++++++--------- pkg/sql/sqlbase/structured.proto | 14 +- pkg/sql/sqlbase/table.go | 30 +- 19 files changed, 955 insertions(+), 332 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/drop_index_mixed_19.2_20.1 create mode 100644 pkg/sql/old_foreign_key_desc_test.go diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index a5ee557891c0..8c9baea5daf8 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -61,6 +61,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen in the /debug page trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -versioncustom validation19.2-10set the active cluster version in the format '.' +versioncustom validation19.2-11set the active cluster version in the format '.' diff --git a/pkg/settings/cluster/cockroach_versions.go b/pkg/settings/cluster/cockroach_versions.go index 1196d69ea8ef..a8809e1eb893 100644 --- a/pkg/settings/cluster/cockroach_versions.go +++ b/pkg/settings/cluster/cockroach_versions.go @@ -55,6 +55,7 @@ const ( VersionAuthLocalAndTrustRejectMethods VersionPrimaryKeyColumnsOutOfFamilyZero VersionRootPassword + VersionNoExplicitForeignKeyIndexIDs // Add new versions here (step one of two). ) @@ -388,6 +389,15 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: VersionRootPassword, Version: roachpb.Version{Major: 19, Minor: 2, Unstable: 10}, }, + { + // VersionNoExplicitForeignKeyIndexIDs is https://github.com/cockroachdb/cockroach/pull/43332. + // + // It represents the migration away from using explicit index IDs in foreign + // key constraints, and instead allows all places that need these IDs to select + // an appropriate index to uphold the foreign key relationship. + Key: VersionNoExplicitForeignKeyIndexIDs, + Version: roachpb.Version{Major: 19, Minor: 2, Unstable: 11}, + }, // Add new versions here (step two of two). diff --git a/pkg/settings/cluster/versionkey_string.go b/pkg/settings/cluster/versionkey_string.go index c91a547ed494..67f179a3f984 100644 --- a/pkg/settings/cluster/versionkey_string.go +++ b/pkg/settings/cluster/versionkey_string.go @@ -31,11 +31,12 @@ func _() { _ = x[VersionAuthLocalAndTrustRejectMethods-20] _ = x[VersionPrimaryKeyColumnsOutOfFamilyZero-21] _ = x[VersionRootPassword-22] + _ = x[VersionNoExplicitForeignKeyIndexIDs-23] } -const _VersionKey_name = "Version19_1VersionStart19_2VersionQueryTxnTimestampVersionStickyBitVersionParallelCommitsVersionGenerationComparableVersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionTableDescModificationTimeFromMVCCVersionPartitionedBackupVersion19_2VersionStart20_1VersionContainsEstimatesCounterVersionChangeReplicasDemotionVersionSecondaryIndexColumnFamiliesVersionNamespaceTableWithSchemasVersionProtectedTimestampsVersionPrimaryKeyChangesVersionAuthLocalAndTrustRejectMethodsVersionPrimaryKeyColumnsOutOfFamilyZeroVersionRootPassword" +const _VersionKey_name = "Version19_1VersionStart19_2VersionQueryTxnTimestampVersionStickyBitVersionParallelCommitsVersionGenerationComparableVersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionTableDescModificationTimeFromMVCCVersionPartitionedBackupVersion19_2VersionStart20_1VersionContainsEstimatesCounterVersionChangeReplicasDemotionVersionSecondaryIndexColumnFamiliesVersionNamespaceTableWithSchemasVersionProtectedTimestampsVersionPrimaryKeyChangesVersionAuthLocalAndTrustRejectMethodsVersionPrimaryKeyColumnsOutOfFamilyZeroVersionRootPasswordVersionNoExplicitForeignKeyIndexIDs" -var _VersionKey_index = [...]uint16{0, 11, 27, 51, 67, 89, 116, 138, 164, 198, 225, 265, 289, 300, 316, 347, 376, 411, 443, 469, 493, 530, 569, 588} +var _VersionKey_index = [...]uint16{0, 11, 27, 51, 67, 89, 116, 138, 164, 198, 225, 265, 289, 300, 316, 347, 376, 411, 443, 469, 493, 530, 569, 588, 623} func (i VersionKey) String() string { if i < 0 || i >= VersionKey(len(_VersionKey_index)-1) { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index b39f1c570acf..1abc79d0163a 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1686,13 +1686,21 @@ CREATE TABLE crdb_internal.backward_dependencies ( for i := range table.OutboundFKs { fk := &table.OutboundFKs[i] + refTbl, err := tableLookup.getTableByID(fk.ReferencedTableID) + if err != nil { + return err + } + refIdx, err := sqlbase.FindFKReferencedIndex(refTbl, fk.ReferencedColumnIDs) + if err != nil { + return err + } if err := addRow( tableID, tableName, tree.DNull, tree.DNull, tree.NewDInt(tree.DInt(fk.ReferencedTableID)), fkDep, - tree.NewDInt(tree.DInt(fk.LegacyReferencedIndex)), + tree.NewDInt(tree.DInt(refIdx.ID)), tree.NewDString(fk.Name), tree.DNull, ); err != nil { diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 1bf6ecde233c..68f8ffdca0c5 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -442,6 +442,43 @@ const ( NonEmptyTable ) +// MaybeUpgradeDependentOldForeignKeyVersionTables upgrades the on-disk foreign key descriptor +// version of all table descriptors that have foreign key relationships with desc. This is intended +// to catch upgrade 19.1 version table descriptors that haven't been upgraded yet before an operation +// like drop index which could cause them to lose FK information in the old representation. +func (p *planner) MaybeUpgradeDependentOldForeignKeyVersionTables( + ctx context.Context, desc *sqlbase.MutableTableDescriptor, +) error { + // In order to avoid having old version foreign key descriptors that depend on this + // index lose information when this index is dropped, ensure that they get updated. + maybeUpgradeFKRepresentation := func(id sqlbase.ID) error { + // Read the referenced table and see if the foreign key representation has changed. If it has, write + // the upgraded descriptor back to disk. + tbl, didUpgrade, err := sqlbase.GetTableDescFromIDWithFKsChanged(ctx, p.txn, id) + if err != nil { + return err + } + if didUpgrade { + err := p.writeSchemaChange(ctx, sqlbase.NewMutableExistingTableDescriptor(*tbl), sqlbase.InvalidMutationID) + if err != nil { + return err + } + } + return nil + } + for i := range desc.OutboundFKs { + if err := maybeUpgradeFKRepresentation(desc.OutboundFKs[i].ReferencedTableID); err != nil { + return err + } + } + for i := range desc.InboundFKs { + if err := maybeUpgradeFKRepresentation(desc.InboundFKs[i].OriginTableID); err != nil { + return err + } + } + return nil +} + // ResolveFK looks up the tables and columns mentioned in a `REFERENCES` // constraint and adds metadata representing that constraint to the descriptor. // It may, in doing so, add to or alter descriptors in the passed in `backrefs` diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index a69e22fdb854..044018e911e4 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -16,6 +16,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" @@ -165,29 +166,84 @@ func (p *planner) dropIndexByName( // state consistent with the removal of the reference on the other table // involved in the FK, in case of rollbacks (#38733). + // TODO (rohany): switching all the checks from checking the legacy ID's to + // checking if the index has a prefix of the columns needed for the foreign + // key might result in some false positives for this index while it is in + // a mixed version cluster, but we have to remove all reads of the legacy + // explicit index fields. + + // 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([]*sqlbase.IndexDescriptor, 0, len(tableDesc.Indexes)+1) + remainingIndexes = append(remainingIndexes, &tableDesc.PrimaryIndex) + for i := range tableDesc.Indexes { + index := &tableDesc.Indexes[i] + if index.ID != idx.ID { + remainingIndexes = append(remainingIndexes, index) + } + } + + // indexHasReplacementCandidate runs isValidIndex on each index in remainingIndexes and returns + // true if at least one index satisfies isValidIndex. + indexHasReplacementCandidate := func(isValidIndex func(*sqlbase.IndexDescriptor) bool) bool { + foundReplacement := false + for _, index := range remainingIndexes { + if isValidIndex(index) { + foundReplacement = true + break + } + } + return foundReplacement + } + // If we aren't at the cluster version where we have removed explicit foreign key IDs + // from the foreign key descriptors, fall back to the existing drop index logic. + // That means we pretend that we can never find replacements for any indexes. + if !cluster.Version.IsActive(ctx, p.ExecCfg().Settings, cluster.VersionNoExplicitForeignKeyIndexIDs) { + indexHasReplacementCandidate = func(func(*sqlbase.IndexDescriptor) bool) bool { + return false + } + } + // Check for foreign key mutations referencing this index. for _, m := range tableDesc.Mutations { if c := m.GetConstraint(); c != nil && c.ConstraintType == sqlbase.ConstraintToUpdate_FOREIGN_KEY && - c.ForeignKey.LegacyOriginIndex == idx.ID { + // If the index being deleted could be used as a index for this outbound + // foreign key mutation, then make sure that we have another index that + // could be used for this mutation. + idx.IsValidOriginIndex(c.ForeignKey.OriginColumnIDs) && + !indexHasReplacementCandidate(func(idx *sqlbase.IndexDescriptor) bool { + return idx.IsValidOriginIndex(c.ForeignKey.OriginColumnIDs) + }) { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "referencing constraint %q in the middle of being added, try again later", c.ForeignKey.Name) } } + if err := p.MaybeUpgradeDependentOldForeignKeyVersionTables(ctx, tableDesc); err != nil { + return err + } + // Index for updating the FK slices in place when removing FKs. sliceIdx := 0 for i := range tableDesc.OutboundFKs { tableDesc.OutboundFKs[sliceIdx] = tableDesc.OutboundFKs[i] sliceIdx++ fk := &tableDesc.OutboundFKs[i] - if fk.LegacyOriginIndex == idx.ID { - if behavior != tree.DropCascade && constraintBehavior != ignoreIdxConstraint { - return errors.Errorf("index %q is in use as a foreign key constraint", idx.Name) - } - sliceIdx-- - if err := p.removeFKBackReference(ctx, tableDesc, fk); err != nil { - return err + canReplace := func(idx *sqlbase.IndexDescriptor) bool { + return idx.IsValidOriginIndex(fk.OriginColumnIDs) + } + // The index being deleted could be used as the origin index for this foreign key. + if idx.IsValidOriginIndex(fk.OriginColumnIDs) { + if !indexHasReplacementCandidate(canReplace) { + if behavior != tree.DropCascade && constraintBehavior != ignoreIdxConstraint { + return errors.Errorf("index %q is in use as a foreign key constraint", idx.Name) + } + sliceIdx-- + if err := p.removeFKBackReference(ctx, tableDesc, fk); err != nil { + return err + } } } } @@ -198,14 +254,22 @@ func (p *planner) dropIndexByName( tableDesc.InboundFKs[sliceIdx] = tableDesc.InboundFKs[i] sliceIdx++ fk := &tableDesc.InboundFKs[i] - if fk.LegacyReferencedIndex == idx.ID { - err := p.canRemoveFKBackreference(ctx, idx.Name, fk, behavior) - if err != nil { - return err - } - sliceIdx-- - if err := p.removeFKForBackReference(ctx, tableDesc, fk); err != nil { - return err + canReplace := func(idx *sqlbase.IndexDescriptor) bool { + return idx.IsValidReferencedIndex(fk.ReferencedColumnIDs) + } + // The index being deleted could potentially be the referenced index for this fk. + if idx.IsValidReferencedIndex(fk.ReferencedColumnIDs) { + // If we haven't found a replacement candidate for this foreign key, then + // we need a cascade to delete this index. + if !indexHasReplacementCandidate(canReplace) { + // If we found haven't found a replacement, then we check that the drop behavior is cascade. + if err := p.canRemoveFKBackreference(ctx, idx.Name, fk, behavior); err != nil { + return err + } + sliceIdx-- + if err := p.removeFKForBackReference(ctx, tableDesc, fk); err != nil { + return err + } } } } diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 93ef4eea6326..18797871d508 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -815,7 +815,7 @@ CREATE TABLE information_schema.referential_constraints ( if r, ok := matchOptionMap[fk.Match]; ok { matchType = r } - referencedIdx, err := refTable.FindIndexByID(fk.LegacyReferencedIndex) + referencedIdx, err := sqlbase.FindFKReferencedIndex(refTable, fk.ReferencedColumnIDs) if err != nil { return err } diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index bba1d2d97293..cecc6f49c6a9 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -448,6 +448,15 @@ var logicTestConfigs = []testClusterConfig{ overrideDistSQLMode: "on", overrideAutoStats: "false", }, + { + name: "local-mixed-19.2-20.1", + numNodes: 1, + overrideDistSQLMode: "off", + overrideAutoStats: "false", + bootstrapVersion: roachpb.Version{Major: 19, Minor: 2}, + serverVersion: roachpb.Version{Major: 20, Minor: 1}, + disableUpgrade: true, + }, { name: "fakedist-vec-off", numNodes: 3, diff --git a/pkg/sql/logictest/testdata/logic_test/drop_index b/pkg/sql/logictest/testdata/logic_test/drop_index index bc42b4ceff35..7deb55ba7101 100644 --- a/pkg/sql/logictest/testdata/logic_test/drop_index +++ b/pkg/sql/logictest/testdata/logic_test/drop_index @@ -265,3 +265,52 @@ DROP INDEX tu_a_key statement ok DROP INDEX tu_a + +# Test that we have more relaxed restrictions on dropping indexes referenced by fks. +subtest fk_drop + +# Ensure that we can drop an index used by a foreign key if there is another +# index that can take its place. +statement ok +CREATE TABLE fk1 (x INT); +CREATE TABLE fk2 (x INT PRIMARY KEY); +ALTER TABLE fk1 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES fk2 (x); +CREATE INDEX i ON fk1 (x); +DROP INDEX fk1_auto_index_fk1 + +statement error pq: index "i" is in use as a foreign key constraint +DROP INDEX fk1@i + +# Ensure that DROP INDEX CASCADE does not delete the foreign key when +# there is another index that can satisfy the foreign key constraint. +statement ok +DROP TABLE fk1; +DROP TABLE fk2; +CREATE TABLE fk1 (x int); +CREATE TABLE fk2 (x int PRIMARY KEY); +CREATE INDEX i ON fk1 (x); +CREATE INDEX i2 ON fk1 (x); +ALTER TABLE fk1 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES fk2 (x); +DROP INDEX fk1@i CASCADE + +query TT +SHOW CREATE fk1 +---- +fk1 CREATE TABLE fk1 ( + x INT8 NULL, + CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES fk2(x), + INDEX i2 (x ASC), + FAMILY "primary" (x, rowid) +) + +# Ensure that now the cascade deletes the foreign key constraint. +statement ok +DROP INDEX fk1@i2 CASCADE + +query TT +SHOW CREATE fk1 +---- +fk1 CREATE TABLE fk1 ( + x INT8 NULL, + FAMILY "primary" (x, rowid) +) diff --git a/pkg/sql/logictest/testdata/logic_test/drop_index_mixed_19.2_20.1 b/pkg/sql/logictest/testdata/logic_test/drop_index_mixed_19.2_20.1 new file mode 100644 index 000000000000..0d0aa8f13da8 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/drop_index_mixed_19.2_20.1 @@ -0,0 +1,276 @@ +# LogicTest: local-mixed-19.2-20.1 + +statement ok +CREATE TABLE users ( + id INT PRIMARY KEY, + name VARCHAR NOT NULL, + title VARCHAR, + INDEX foo (name), + UNIQUE INDEX bar (id, name), + INDEX baw (name, title), + FAMILY (id, name, title) +) + +statement ok +CREATE TABLE othertable ( + x INT, + y INT, + INDEX baw (x), + INDEX yak (y, x), + FAMILY (x, y) +) + +statement error index name "baw" is ambiguous +DROP INDEX baw + +statement error index name "baw" is ambiguous +DROP INDEX IF EXISTS baw + +statement error index "ark" does not exist +DROP INDEX ark + +statement ok +DROP INDEX IF EXISTS ark + +statement error index "ark" does not exist +DROP INDEX users@ark + +statement ok +DROP INDEX IF EXISTS users@ark + +statement ok +DROP INDEX yak + +statement ok +CREATE INDEX yak ON othertable (y, x) + +statement ok +DROP INDEX IF EXISTS yak + +statement ok +DROP TABLE othertable + +statement ok +DROP INDEX baw + +statement ok +INSERT INTO users VALUES (1, 'tom', 'cat'),(2, 'jerry', 'rat') + +query TTBITTBB colnames +SHOW INDEXES FROM users +---- +table_name index_name non_unique seq_in_index column_name direction storing implicit +users primary false 1 id ASC false false +users foo true 1 name ASC false false +users foo true 2 id ASC false true +users bar false 1 id ASC false false +users bar false 2 name ASC false false + +statement error index "zap" does not exist +DROP INDEX users@zap + +statement ok +DROP INDEX IF EXISTS users@zap + +query TTBITTBB colnames +SHOW INDEXES FROM users +---- +table_name index_name non_unique seq_in_index column_name direction storing implicit +users primary false 1 id ASC false false +users foo true 1 name ASC false false +users foo true 2 id ASC false true +users bar false 1 id ASC false false +users bar false 2 name ASC false false + +# Also test that dropping with a non-existing index still drops 'foo'. + +statement ok +DROP INDEX IF EXISTS users@foo, users@zap + +query TTBITTBB colnames +SHOW INDEXES FROM users +---- +table_name index_name non_unique seq_in_index column_name direction storing implicit +users primary false 1 id ASC false false +users bar false 1 id ASC false false +users bar false 2 name ASC false false + +user testuser + +statement error user testuser does not have CREATE privilege on relation users +DROP INDEX users@bar + +user root + +statement ok +GRANT CREATE ON TABLE users TO testuser + +user testuser + +statement error in use as unique constraint +DROP INDEX users@bar + +statement error in use as unique constraint +DROP INDEX users@bar RESTRICT + +statement ok +DROP INDEX users@bar CASCADE + +query TTBITTBB colnames +SHOW INDEXES FROM users +---- +table_name index_name non_unique seq_in_index column_name direction storing implicit +users primary false 1 id ASC false false + +user root + +query ITT rowsort +SELECT * FROM users +---- +1 tom cat +2 jerry rat + +statement ok +CREATE INDEX foo ON users (name) + +statement ok +CREATE INDEX bar ON users (title) + +statement ok +CREATE INDEX baz ON users (name, title) + +statement ok +DROP INDEX IF EXISTS users@invalid, users@baz + +query TTBITTBB colnames +SHOW INDEXES FROM users +---- +table_name index_name non_unique seq_in_index column_name direction storing implicit +users primary false 1 id ASC false false +users foo true 1 name ASC false false +users foo true 2 id ASC false true +users bar true 1 title ASC false false +users bar true 2 id ASC false true + +statement ok +CREATE VIEW v AS SELECT name FROM users@{FORCE_INDEX=foo} + +statement error cannot drop index "foo" because view "v" depends on it +DROP INDEX users@foo + +statement ok +DROP INDEX users@bar + +query TTBITTBB colnames +SHOW INDEXES FROM users +---- +table_name index_name non_unique seq_in_index column_name direction storing implicit +users primary false 1 id ASC false false +users foo true 1 name ASC false false +users foo true 2 id ASC false true + +statement ok +CREATE VIEW v2 AS SELECT name FROM v + +query T +SHOW TABLES +---- +users +v +v2 + +statement ok +GRANT ALL ON users to testuser + +statement ok +GRANT ALL ON v to testuser + +user testuser + +statement error user testuser does not have DROP privilege on relation v2 +DROP INDEX users@foo CASCADE + +user root + +statement ok +DROP INDEX users@foo CASCADE + +query TTBITTBB colnames +SHOW INDEXES FROM users +---- +table_name index_name non_unique seq_in_index column_name direction storing implicit +users primary false 1 id ASC false false + +query T +SHOW TABLES +---- +users + +# Test the syntax without a '@' + +statement ok +CREATE INDEX baz ON users (name) + +# Also test that dropping with a non-existing index still drops 'baz'. + +statement ok +DROP INDEX IF EXISTS baz, zap + +query TTBITTBB colnames +SHOW INDEXES FROM users +---- +table_name index_name non_unique seq_in_index column_name direction storing implicit +users primary false 1 id ASC false false + +# Test that it still succeeds when an index does not exist. + +statement ok +DROP INDEX IF EXISTS baz + +# Test that presence of a view or sequence doesn't break DROP INDEX (#21834) + +statement ok +CREATE DATABASE view_test + +statement ok +SET DATABASE = view_test + +statement ok +CREATE TABLE t (id INT) + +statement ok +CREATE VIEW v AS SELECT id FROM t + +statement error pgcode 42704 pq: index "nonexistent_index" does not exist +DROP INDEX nonexistent_index + +statement ok +CREATE DATABASE sequence_test + +statement ok +SET DATABASE = sequence_test + +statement ok +CREATE SEQUENCE s + +statement error pgcode 42704 pq: index "nonexistent_index" does not exist +DROP INDEX nonexistent_index + +statement ok +CREATE TABLE tu (a INT UNIQUE) + +statement ok +CREATE UNIQUE INDEX tu_a ON tu(a) + +statement error in use as unique constraint +DROP INDEX tu_a_key + +statement ok +DROP INDEX tu_a + +statement error pq: index "fk1_auto_index_fk1" is in use as a foreign key constraint +CREATE TABLE fk1 (x INT); +CREATE TABLE fk2 (x INT PRIMARY KEY); +ALTER TABLE fk1 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES fk2 (x); +CREATE INDEX i ON fk1 (x); +DROP INDEX fk1_auto_index_fk1 diff --git a/pkg/sql/old_foreign_key_desc_test.go b/pkg/sql/old_foreign_key_desc_test.go new file mode 100644 index 000000000000..3a9a1e63f28e --- /dev/null +++ b/pkg/sql/old_foreign_key_desc_test.go @@ -0,0 +1,134 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + "reflect" + "testing" + + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" +) + +// The goal of this test is to validate a case that could happen in a +// multi-version cluster setting. +// The foreign key representation has been updated in many ways, +// and it is possible that a pre 19.2 table descriptor foreign key +// representation could lead to descriptor information loss when +// performing index drops in some cases. This test constructs an +// old version descriptor and ensures that everything is OK. +func TestOldForeignKeyRepresentationGetsUpgraded(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + params, _ := tests.CreateTestServerParams() + s, sqlDB, kvDB := serverutils.StartServer(t, params) + defer s.Stopper().Stop(ctx) + if _, err := sqlDB.Exec(` +CREATE DATABASE t; +CREATE TABLE t.t1 (x INT); +CREATE TABLE t.t2 (x INT, UNIQUE INDEX (x)); +ALTER TABLE t.t1 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES t.t2 (x); +CREATE INDEX ON t.t1 (x); +`); err != nil { + t.Fatal(err) + } + desc := sqlbase.GetTableDescriptor(kvDB, "t", "t1") + desc = sqlbase.GetTableDescriptor(kvDB, "t", "t2") + // Remember the old foreign keys. + oldInboundFKs := append([]sqlbase.ForeignKeyConstraint{}, desc.InboundFKs...) + // downgradeForeignKey downgrades a table descriptor's foreign key representation + // to the pre-19.2 table descriptor format where foreign key information + // is stored on the index. + downgradeForeignKey := func(tbl *sqlbase.TableDescriptor) *sqlbase.TableDescriptor { + // Downgrade the outbound foreign keys. + for i := range tbl.OutboundFKs { + fk := &tbl.OutboundFKs[i] + idx, err := sqlbase.FindFKOriginIndex(tbl, fk.OriginColumnIDs) + if err != nil { + t.Fatal(err) + } + referencedTbl, err := sqlbase.GetTableDescFromID(ctx, kvDB, fk.ReferencedTableID) + if err != nil { + t.Fatal(err) + } + refIdx, err := sqlbase.FindFKReferencedIndex(referencedTbl, fk.ReferencedColumnIDs) + if err != nil { + t.Fatal(err) + } + idx.ForeignKey = sqlbase.ForeignKeyReference{ + Name: fk.Name, + Table: fk.ReferencedTableID, + Index: refIdx.ID, + Validity: fk.Validity, + SharedPrefixLen: int32(len(fk.OriginColumnIDs)), + OnDelete: fk.OnDelete, + OnUpdate: fk.OnUpdate, + Match: fk.Match, + } + } + tbl.OutboundFKs = nil + // Downgrade the inbound foreign keys. + for i := range tbl.InboundFKs { + fk := &tbl.InboundFKs[i] + idx, err := sqlbase.FindFKReferencedIndex(desc, fk.ReferencedColumnIDs) + if err != nil { + t.Fatal(err) + } + originTbl, err := sqlbase.GetTableDescFromID(ctx, kvDB, fk.OriginTableID) + if err != nil { + t.Fatal(err) + } + originIdx, err := sqlbase.FindFKOriginIndex(originTbl, fk.OriginColumnIDs) + if err != nil { + t.Fatal(err) + } + // Back references only contain the table and index IDs in old format versions. + fkRef := sqlbase.ForeignKeyReference{ + Table: fk.OriginTableID, + Index: originIdx.ID, + } + idx.ReferencedBy = append(idx.ReferencedBy, fkRef) + } + tbl.InboundFKs = nil + return tbl + } + err := kvDB.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + b := txn.NewBatch() + newDesc := downgradeForeignKey(desc) + if err := writeDescToBatch(ctx, false, s.ClusterSettings(), b, desc.ID, newDesc); err != nil { + return err + } + return txn.Run(ctx, b) + }) + if err != nil { + t.Fatal(err) + } + // Run a DROP INDEX statement and ensure that the downgraded descriptor gets upgraded successfully. + if _, err := sqlDB.Exec(`DROP INDEX t.t1@t1_auto_index_fk1`); err != nil { + t.Fatal(err) + } + desc = sqlbase.GetTableDescriptor(kvDB, "t", "t2") + // Remove the validity field on all the descriptors for comparison, since + // foreign keys on the referenced side's validity is not always updated correctly. + for i := range desc.InboundFKs { + desc.InboundFKs[i].Validity = sqlbase.ConstraintValidity_Validated + } + for i := range oldInboundFKs { + oldInboundFKs[i].Validity = sqlbase.ConstraintValidity_Validated + } + if !reflect.DeepEqual(desc.InboundFKs, oldInboundFKs) { + t.Error("expected fks", oldInboundFKs, "but found", desc.InboundFKs) + } +} diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 5b061bae479f..b219d8b491a8 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -871,7 +871,7 @@ CREATE TABLE pg_catalog.pg_constraint ( if err != nil { return err } - if idx, err := referencedTable.FindIndexByID(con.FK.LegacyReferencedIndex); err != nil { + if idx, err := sqlbase.FindFKReferencedIndex(referencedTable, con.FK.ReferencedColumnIDs); err != nil { // We couldn't find an index that matched. This shouldn't happen. log.Warningf(ctx, "broken fk reference: %v", err) } else { @@ -1155,7 +1155,7 @@ CREATE TABLE pg_catalog.pg_depend ( return err } refObjID := oidZero - if idx, err := referencedTable.FindIndexByID(con.FK.LegacyReferencedIndex); err != nil { + if idx, err := sqlbase.FindFKReferencedIndex(referencedTable, con.FK.ReferencedColumnIDs); err != nil { // We couldn't find an index that matched. This shouldn't happen. log.Warningf(ctx, "broken fk reference: %v", err) } else { diff --git a/pkg/sql/row/cascader.go b/pkg/sql/row/cascader.go index 3bc954ec5364..0de1b9a2a578 100644 --- a/pkg/sql/row/cascader.go +++ b/pkg/sql/row/cascader.go @@ -1119,11 +1119,11 @@ func (c *cascader) cascadeAll( if err != nil { return err } - referencedIndex, err := elem.table.TableDesc().FindIndexByID(ref.LegacyReferencedIndex) + referencedIndex, err := sqlbase.FindFKReferencedIndex(elem.table.TableDesc(), ref.ReferencedColumnIDs) if err != nil { return err } - referencingIndex, err := referencingTable.Desc.TableDesc().FindIndexByID(ref.LegacyOriginIndex) + referencingIndex, err := sqlbase.FindFKOriginIndex(referencingTable.Desc.TableDesc(), ref.OriginColumnIDs) if err != nil { return err } diff --git a/pkg/sql/row/fk_existence_delete.go b/pkg/sql/row/fk_existence_delete.go index 64569f79971b..f196bdc2777c 100644 --- a/pkg/sql/row/fk_existence_delete.go +++ b/pkg/sql/row/fk_existence_delete.go @@ -73,15 +73,15 @@ func makeFkExistenceCheckHelperForDelete( OnDelete: ref.OnDelete, OnUpdate: ref.OnUpdate, } - searchIdx, err := originTable.Desc.TableDesc().FindIndexByID(ref.LegacyOriginIndex) + searchIdx, err := sqlbase.FindFKOriginIndex(originTable.Desc.TableDesc(), ref.OriginColumnIDs) if err != nil { return fkExistenceCheckForDelete{}, errors.NewAssertionErrorWithWrappedErrf( - err, "failed to find index %d (table %d) for deletion", ref.LegacyOriginIndex, ref.OriginTableID) + err, "failed to find a suitable index on table %d for deletion", ref.OriginTableID) } - mutatedIdx, err := table.TableDesc().FindIndexByID(ref.LegacyReferencedIndex) + mutatedIdx, err := sqlbase.FindFKReferencedIndex(table.TableDesc(), ref.ReferencedColumnIDs) if err != nil { return fkExistenceCheckForDelete{}, errors.NewAssertionErrorWithWrappedErrf( - err, "failed to find available index %d (table %d) for deletion", ref.LegacyReferencedIndex, ref.ReferencedTableID) + err, "failed to find a suitable index on table %d for deletion", ref.ReferencedTableID) } fk, err := makeFkExistenceCheckBaseHelper(txn, otherTables, fakeRef, searchIdx, mutatedIdx, colMap, alloc, CheckDeletes) diff --git a/pkg/sql/row/fk_existence_insert.go b/pkg/sql/row/fk_existence_insert.go index 4b0f6a4a1180..98f1c6266df0 100644 --- a/pkg/sql/row/fk_existence_insert.go +++ b/pkg/sql/row/fk_existence_insert.go @@ -65,15 +65,15 @@ func makeFkExistenceCheckHelperForInsert( return h, errors.AssertionFailedf("referenced table %d not in provided table map %+v", ref.ReferencedTableID, otherTables) } - searchIdx, err := searchTable.TableDesc().FindIndexByID(ref.LegacyReferencedIndex) + searchIdx, err := sqlbase.FindFKReferencedIndex(searchTable.TableDesc(), ref.ReferencedColumnIDs) if err != nil { return h, errors.NewAssertionErrorWithWrappedErrf(err, - "failed to find search index %d for fk %q", ref.LegacyReferencedIndex, ref.Name) + "failed to find suitable search index for fk %q", ref.Name) } - mutatedIdx, err := table.TableDesc().FindIndexByID(ref.LegacyOriginIndex) + mutatedIdx, err := sqlbase.FindFKOriginIndex(table.TableDesc(), ref.OriginColumnIDs) if err != nil { return h, errors.NewAssertionErrorWithWrappedErrf(err, - "failed to find search index %d for fk %q", ref.LegacyOriginIndex, ref.Name) + "failed to find suitable search index for fk %q", ref.Name) } fk, err := makeFkExistenceCheckBaseHelper(txn, otherTables, ref, searchIdx, mutatedIdx, colMap, alloc, CheckInserts) if err == errSkipUnusedFK { diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index f6c2ee9dba18..0bd5563f3aa9 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -352,6 +352,8 @@ func GetDatabaseDescFromID( // GetTableDescFromID retrieves the table descriptor for the table // ID passed in using an existing proto getter. Returns an error if the // descriptor doesn't exist or if it exists and is not a table. +// NB: If this function changes, make sure to update GetTableDescFromIDWithFKsChanged +// in a similar way. func GetTableDescFromID( ctx context.Context, protoGetter protoGetter, id ID, ) (*TableDescriptor, error) { @@ -367,6 +369,26 @@ func GetTableDescFromID( return table, nil } +// GetTableDescFromIDWithFKsChanged retrieves the table descriptor for the table +// ID passed in using an existing proto getter. It returns the same things as +// GetTableDescFromID but additionally returns whether or not the table descriptor +// was changed during the foreign key upgrade process. +func GetTableDescFromIDWithFKsChanged( + ctx context.Context, protoGetter protoGetter, id ID, +) (*TableDescriptor, bool, error) { + table, err := getTableDescFromIDRaw(ctx, protoGetter, id) + if err != nil { + return nil, false, err + } + table.maybeUpgradeFormatVersion() + table.Privileges.MaybeFixPrivileges(table.ID) + changed, err := table.MaybeUpgradeForeignKeyRepresentation(ctx, protoGetter, false /* skipFKsWithNoMatchingTable */) + if err != nil { + return nil, false, err + } + return table, changed, err +} + // getTableDescFromIDRaw retrieves the table descriptor for the table // ID passed in using an existing proto getter. Returns an error if the // descriptor doesn't exist or if it exists and is not a table. Note that it @@ -817,6 +839,8 @@ func generatedFamilyName(familyID FamilyID, columnNames []string) string { // This includes format upgrades and optional changes that can be handled by all version // (for example: additional default privileges). // Returns true if any changes were made. +// NB: If this function changes, make sure to update GetTableDescFromIDWithFKsChanged +// in a similar way. func (desc *TableDescriptor) MaybeFillInDescriptor( ctx context.Context, protoGetter protoGetter, ) error { @@ -986,14 +1010,12 @@ func maybeUpgradeForeignKeyRepOnIndex( var forwardFK *ForeignKeyConstraint for i := range otherTable.OutboundFKs { otherFK := &otherTable.OutboundFKs[i] - // To find a match, we need to compare the reference's table id and - // index id, which are the only two available fields on backreferences - // in the old representation. Note that we have to compare the index id - // to the matching new forward reference's LegacyOriginIndex field, - // which although marked as Legacy, is populated every time we create - // a new-style fk during the duration of 19.2. + // To find a match, we find a foreign key reference that has the same + // referenced table ID, and that the index we point to is a valid + // index to satisfy the columns in the foreign key. + // TODO (rohany): I'm unsure about this... Could there be multiple FK's? if otherFK.ReferencedTableID == desc.ID && - otherFK.LegacyOriginIndex == ref.Index { + ColumnIDs(originIndex.ColumnIDs).HasPrefix(otherFK.OriginColumnIDs) { // Found a match. forwardFK = otherFK break @@ -2769,13 +2791,6 @@ func (desc *TableDescriptor) FindFKForBackRef( for i := range desc.OutboundFKs { fk := &desc.OutboundFKs[i] if fk.ReferencedTableID == referencedTableID && fk.Name == backref.Name { - if fk.LegacyReferencedIndex != backref.LegacyReferencedIndex || - fk.LegacyOriginIndex != backref.LegacyOriginIndex { - return nil, errors.AssertionFailedf("fk found for backref %s but pinned indexes were different: "+ - "%d != %d || %d != %d", fk.Name, fk.LegacyOriginIndex, backref.LegacyOriginIndex, - fk.LegacyReferencedIndex, backref.LegacyReferencedIndex) - } - return fk, nil } } diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index 2f29a4055e11..cbd6b4e953af 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -73,7 +73,7 @@ func (x *ConstraintValidity) UnmarshalJSON(data []byte) error { return nil } func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{0} } type ForeignKeyReference_Action int32 @@ -118,7 +118,7 @@ func (x *ForeignKeyReference_Action) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Action) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{0, 0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{0, 0} } // Match is the algorithm used to compare composite keys. @@ -158,7 +158,7 @@ func (x *ForeignKeyReference_Match) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Match) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{0, 1} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{0, 1} } // The direction of a column in the index. @@ -195,7 +195,7 @@ func (x *IndexDescriptor_Direction) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{6, 0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{6, 0} } // The type of the index. @@ -232,7 +232,7 @@ func (x *IndexDescriptor_Type) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{6, 1} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{6, 1} } type ConstraintToUpdate_ConstraintType int32 @@ -275,7 +275,7 @@ func (x *ConstraintToUpdate_ConstraintType) UnmarshalJSON(data []byte) error { return nil } func (ConstraintToUpdate_ConstraintType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{7, 0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{7, 0} } // A descriptor within a mutation is unavailable for reads, writes @@ -340,7 +340,7 @@ func (x *DescriptorMutation_State) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{9, 0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{9, 0} } // Direction of mutation. @@ -383,7 +383,7 @@ func (x *DescriptorMutation_Direction) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{9, 1} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{9, 1} } // State is set if this TableDescriptor is in the process of being added or deleted. @@ -434,7 +434,7 @@ func (x *TableDescriptor_State) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 0} } // AuditMode indicates which auditing actions to take when this table is used. @@ -471,7 +471,7 @@ func (x *TableDescriptor_AuditMode) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_AuditMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 1} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 1} } type ForeignKeyReference struct { @@ -493,7 +493,7 @@ func (m *ForeignKeyReference) Reset() { *m = ForeignKeyReference{} } func (m *ForeignKeyReference) String() string { return proto.CompactTextString(m) } func (*ForeignKeyReference) ProtoMessage() {} func (*ForeignKeyReference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{0} } func (m *ForeignKeyReference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -540,20 +540,22 @@ type ForeignKeyConstraint struct { // fields on the index that they use. In versions 19.2 and later, we preserve // the semantics of the older FKs which were tied to indexes by specifying // the index as a field on this proto, since the migration process to have - // top-level FK fields on the table descriptor requires two releases. In 20.1, - // when all 19.2 nodes will be correctly handling the new FK representation, - // we will perform a migration to upgrade all table descriptors. - LegacyOriginIndex IndexID `protobuf:"varint,10,opt,name=legacy_origin_index,json=legacyOriginIndex,casttype=IndexID" json:"legacy_origin_index"` + // top-level FK fields on the table descriptor requires two releases. + // In 20.1, these fields are no longer read, but must continue to be written + // to maintain compatibility in mixed 19.2/20.1 clusters. In 20.2 these fields + // can _finally_ be removed. + // * When using the foreign key constraint, do not read from these fields! * + LegacyOriginIndex IndexID `protobuf:"varint,10,opt,name=legacy_origin_index,json=legacyOriginIndex,casttype=IndexID" json:"legacy_origin_index"` // Deprecated: Do not use. // LegacyReferencedIndex is the ID of the index used for the FK on the // referenced side. See the comment for LegacyOriginIndex. - LegacyReferencedIndex IndexID `protobuf:"varint,11,opt,name=legacy_referenced_index,json=legacyReferencedIndex,casttype=IndexID" json:"legacy_referenced_index"` + LegacyReferencedIndex IndexID `protobuf:"varint,11,opt,name=legacy_referenced_index,json=legacyReferencedIndex,casttype=IndexID" json:"legacy_referenced_index"` // Deprecated: Do not use. } func (m *ForeignKeyConstraint) Reset() { *m = ForeignKeyConstraint{} } func (m *ForeignKeyConstraint) String() string { return proto.CompactTextString(m) } func (*ForeignKeyConstraint) ProtoMessage() {} func (*ForeignKeyConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{1} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{1} } func (m *ForeignKeyConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -600,7 +602,7 @@ func (m *ColumnDescriptor) Reset() { *m = ColumnDescriptor{} } func (m *ColumnDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnDescriptor) ProtoMessage() {} func (*ColumnDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{2} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{2} } func (m *ColumnDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -656,7 +658,7 @@ func (m *ColumnFamilyDescriptor) Reset() { *m = ColumnFamilyDescriptor{} func (m *ColumnFamilyDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnFamilyDescriptor) ProtoMessage() {} func (*ColumnFamilyDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{3} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{3} } func (m *ColumnFamilyDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -702,7 +704,7 @@ func (m *InterleaveDescriptor) Reset() { *m = InterleaveDescriptor{} } func (m *InterleaveDescriptor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor) ProtoMessage() {} func (*InterleaveDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{4} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{4} } func (m *InterleaveDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -746,7 +748,7 @@ func (m *InterleaveDescriptor_Ancestor) Reset() { *m = InterleaveDescrip func (m *InterleaveDescriptor_Ancestor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor_Ancestor) ProtoMessage() {} func (*InterleaveDescriptor_Ancestor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{4, 0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{4, 0} } func (m *InterleaveDescriptor_Ancestor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -791,7 +793,7 @@ func (m *PartitioningDescriptor) Reset() { *m = PartitioningDescriptor{} func (m *PartitioningDescriptor) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor) ProtoMessage() {} func (*PartitioningDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{5} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{5} } func (m *PartitioningDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -834,7 +836,7 @@ func (m *PartitioningDescriptor_List) Reset() { *m = PartitioningDescrip func (m *PartitioningDescriptor_List) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_List) ProtoMessage() {} func (*PartitioningDescriptor_List) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{5, 0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{5, 0} } func (m *PartitioningDescriptor_List) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -879,7 +881,7 @@ func (m *PartitioningDescriptor_Range) Reset() { *m = PartitioningDescri func (m *PartitioningDescriptor_Range) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_Range) ProtoMessage() {} func (*PartitioningDescriptor_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{5, 1} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{5, 1} } func (m *PartitioningDescriptor_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1020,7 +1022,7 @@ func (m *IndexDescriptor) Reset() { *m = IndexDescriptor{} } func (m *IndexDescriptor) String() string { return proto.CompactTextString(m) } func (*IndexDescriptor) ProtoMessage() {} func (*IndexDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{6} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{6} } func (m *IndexDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1071,7 +1073,7 @@ func (m *ConstraintToUpdate) Reset() { *m = ConstraintToUpdate{} } func (m *ConstraintToUpdate) String() string { return proto.CompactTextString(m) } func (*ConstraintToUpdate) ProtoMessage() {} func (*ConstraintToUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{7} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{7} } func (m *ConstraintToUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1112,7 +1114,7 @@ func (m *PrimaryKeySwap) Reset() { *m = PrimaryKeySwap{} } func (m *PrimaryKeySwap) String() string { return proto.CompactTextString(m) } func (*PrimaryKeySwap) ProtoMessage() {} func (*PrimaryKeySwap) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{8} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{8} } func (m *PrimaryKeySwap) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1167,7 +1169,7 @@ func (m *DescriptorMutation) Reset() { *m = DescriptorMutation{} } func (m *DescriptorMutation) String() string { return proto.CompactTextString(m) } func (*DescriptorMutation) ProtoMessage() {} func (*DescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{9} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{9} } func (m *DescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1515,7 +1517,7 @@ func (m *TableDescriptor) Reset() { *m = TableDescriptor{} } func (m *TableDescriptor) String() string { return proto.CompactTextString(m) } func (*TableDescriptor) ProtoMessage() {} func (*TableDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10} } func (m *TableDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1814,7 +1816,7 @@ func (m *TableDescriptor_SchemaChangeLease) Reset() { *m = TableDescript func (m *TableDescriptor_SchemaChangeLease) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SchemaChangeLease) ProtoMessage() {} func (*TableDescriptor_SchemaChangeLease) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 0} } func (m *TableDescriptor_SchemaChangeLease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1852,7 +1854,7 @@ func (m *TableDescriptor_CheckConstraint) Reset() { *m = TableDescriptor func (m *TableDescriptor_CheckConstraint) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_CheckConstraint) ProtoMessage() {} func (*TableDescriptor_CheckConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 1} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 1} } func (m *TableDescriptor_CheckConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1958,7 +1960,7 @@ func (m *TableDescriptor_NameInfo) Reset() { *m = TableDescriptor_NameIn func (m *TableDescriptor_NameInfo) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_NameInfo) ProtoMessage() {} func (*TableDescriptor_NameInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 2} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 2} } func (m *TableDescriptor_NameInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1998,7 +2000,7 @@ func (m *TableDescriptor_Reference) Reset() { *m = TableDescriptor_Refer func (m *TableDescriptor_Reference) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Reference) ProtoMessage() {} func (*TableDescriptor_Reference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 3} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 3} } func (m *TableDescriptor_Reference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2035,7 +2037,7 @@ func (m *TableDescriptor_MutationJob) Reset() { *m = TableDescriptor_Mut func (m *TableDescriptor_MutationJob) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_MutationJob) ProtoMessage() {} func (*TableDescriptor_MutationJob) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 4} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 4} } func (m *TableDescriptor_MutationJob) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2078,7 +2080,7 @@ func (m *TableDescriptor_SequenceOpts) Reset() { *m = TableDescriptor_Se func (m *TableDescriptor_SequenceOpts) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SequenceOpts) ProtoMessage() {} func (*TableDescriptor_SequenceOpts) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 5} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 5} } func (m *TableDescriptor_SequenceOpts) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2118,7 +2120,7 @@ func (m *TableDescriptor_SequenceOpts_SequenceOwner) String() string { } func (*TableDescriptor_SequenceOpts_SequenceOwner) ProtoMessage() {} func (*TableDescriptor_SequenceOpts_SequenceOwner) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 5, 0} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 5, 0} } func (m *TableDescriptor_SequenceOpts_SequenceOwner) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2158,7 +2160,7 @@ func (m *TableDescriptor_Replacement) Reset() { *m = TableDescriptor_Rep func (m *TableDescriptor_Replacement) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Replacement) ProtoMessage() {} func (*TableDescriptor_Replacement) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 6} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 6} } func (m *TableDescriptor_Replacement) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2195,7 +2197,7 @@ func (m *TableDescriptor_GCDescriptorMutation) Reset() { *m = TableDescr func (m *TableDescriptor_GCDescriptorMutation) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_GCDescriptorMutation) ProtoMessage() {} func (*TableDescriptor_GCDescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{10, 7} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{10, 7} } func (m *TableDescriptor_GCDescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2234,7 +2236,7 @@ func (m *DatabaseDescriptor) Reset() { *m = DatabaseDescriptor{} } func (m *DatabaseDescriptor) String() string { return proto.CompactTextString(m) } func (*DatabaseDescriptor) ProtoMessage() {} func (*DatabaseDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{11} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{11} } func (m *DatabaseDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2292,7 +2294,7 @@ func (m *Descriptor) Reset() { *m = Descriptor{} } func (m *Descriptor) String() string { return proto.CompactTextString(m) } func (*Descriptor) ProtoMessage() {} func (*Descriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_1b83adf6b8c06701, []int{12} + return fileDescriptor_structured_9e2c09aa50a30b8b, []int{12} } func (m *Descriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -11664,236 +11666,236 @@ var ( ) func init() { - proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_1b83adf6b8c06701) -} - -var fileDescriptor_structured_1b83adf6b8c06701 = []byte{ - // 3617 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x5a, 0x4b, 0x6f, 0x23, 0x57, - 0x76, 0x56, 0xf1, 0xcd, 0x53, 0x7c, 0x94, 0xae, 0xfa, 0x41, 0xcb, 0x3d, 0xa2, 0x9a, 0x76, 0xdb, - 0x9a, 0xb1, 0x2d, 0xb5, 0xd5, 0x79, 0x74, 0x9c, 0x60, 0x62, 0x8a, 0xa4, 0x5a, 0x94, 0xd4, 0xa4, - 0x5c, 0x52, 0x5b, 0x93, 0x60, 0x92, 0x4a, 0xa9, 0xea, 0x52, 0x2a, 0x77, 0xb1, 0x8a, 0x5d, 0x55, - 0x54, 0x4b, 0x40, 0x56, 0x59, 0x65, 0x15, 0x04, 0xc8, 0x36, 0x01, 0x06, 0x81, 0x91, 0xf8, 0x07, - 0x04, 0xc8, 0x2e, 0x48, 0x36, 0x81, 0x77, 0x33, 0xcb, 0x41, 0x80, 0x08, 0x89, 0xbc, 0xc9, 0x3f, - 0x08, 0xe0, 0x55, 0x70, 0x5f, 0xf5, 0xa0, 0x28, 0x0d, 0xd5, 0xed, 0xec, 0x78, 0xcf, 0x3d, 0xe7, - 0xbb, 0xaf, 0x73, 0xbe, 0x7b, 0xce, 0x2d, 0xc2, 0x03, 0xff, 0x95, 0xbd, 0xe6, 0xbf, 0xb2, 0x8f, - 0x74, 0x1f, 0xaf, 0xf9, 0x81, 0x37, 0x36, 0x82, 0xb1, 0x87, 0xcd, 0xd5, 0x91, 0xe7, 0x06, 0x2e, - 0xba, 0x6b, 0xb8, 0xc6, 0x4b, 0xcf, 0xd5, 0x8d, 0x93, 0x55, 0xff, 0x95, 0xbd, 0xca, 0xf5, 0x16, - 0x6b, 0xe3, 0xc0, 0xb2, 0xd7, 0x4e, 0x6c, 0x63, 0x2d, 0xb0, 0x86, 0xd8, 0x0f, 0xf4, 0xe1, 0x88, - 0x19, 0x2c, 0xbe, 0x1b, 0x87, 0x1b, 0x79, 0xd6, 0xa9, 0x65, 0xe3, 0x63, 0xcc, 0x3b, 0xef, 0x1c, - 0xbb, 0xc7, 0x2e, 0xfd, 0xb9, 0x46, 0x7e, 0x31, 0x69, 0xe3, 0x2f, 0xb2, 0xb0, 0xb0, 0xe9, 0x7a, - 0xd8, 0x3a, 0x76, 0x76, 0xf0, 0xb9, 0x8a, 0x07, 0xd8, 0xc3, 0x8e, 0x81, 0xd1, 0x32, 0x64, 0x03, - 0xfd, 0xc8, 0xc6, 0x35, 0x69, 0x59, 0x5a, 0x29, 0x6f, 0xc0, 0xb7, 0x17, 0xf5, 0xb9, 0xef, 0x2f, - 0xea, 0xa9, 0x6e, 0x5b, 0x65, 0x1d, 0xe8, 0x11, 0x64, 0x2d, 0xc7, 0xc4, 0x67, 0xb5, 0x14, 0xd5, - 0xa8, 0x72, 0x8d, 0x7c, 0x97, 0x08, 0x89, 0x1a, 0xed, 0x45, 0x35, 0xc8, 0x38, 0xfa, 0x10, 0xd7, - 0xd2, 0xcb, 0xd2, 0x4a, 0x71, 0x23, 0x43, 0xb4, 0x54, 0x2a, 0x41, 0x3b, 0x50, 0x38, 0xd5, 0x6d, - 0xcb, 0xb4, 0x82, 0xf3, 0x5a, 0x66, 0x59, 0x5a, 0xa9, 0xac, 0xff, 0x78, 0x75, 0xea, 0x8a, 0x57, - 0x5b, 0xae, 0xe3, 0x07, 0x9e, 0x6e, 0x39, 0xc1, 0x97, 0xdc, 0x80, 0x03, 0x85, 0x00, 0xe8, 0x31, - 0xcc, 0xfb, 0x27, 0xba, 0x87, 0x4d, 0x6d, 0xe4, 0xe1, 0x81, 0x75, 0xa6, 0xd9, 0xd8, 0xa9, 0x65, - 0x97, 0xa5, 0x95, 0x2c, 0x57, 0xad, 0xb2, 0xee, 0x3d, 0xda, 0xbb, 0x8b, 0x1d, 0x74, 0x00, 0x45, - 0xd7, 0xd1, 0x4c, 0x6c, 0xe3, 0x00, 0xd7, 0x72, 0x74, 0xfc, 0x4f, 0xaf, 0x19, 0x7f, 0xca, 0x06, - 0xad, 0x36, 0x8d, 0xc0, 0x72, 0x1d, 0x31, 0x0f, 0xd7, 0x69, 0x53, 0x20, 0x8e, 0x3a, 0x1e, 0x99, - 0x7a, 0x80, 0x6b, 0xf9, 0xb7, 0x46, 0x7d, 0x41, 0x81, 0xd0, 0x2e, 0x64, 0x87, 0x7a, 0x60, 0x9c, - 0xd4, 0x0a, 0x14, 0xf1, 0xf1, 0x2d, 0x10, 0x9f, 0x13, 0x3b, 0x0e, 0xc8, 0x40, 0x1a, 0x87, 0x90, - 0x63, 0xe3, 0xa0, 0x32, 0x14, 0x7b, 0x7d, 0xad, 0xd9, 0x3a, 0xe8, 0xf6, 0x7b, 0xca, 0x1c, 0x2a, - 0x41, 0x41, 0xed, 0xec, 0x1f, 0xa8, 0xdd, 0xd6, 0x81, 0x22, 0x91, 0xd6, 0x7e, 0xe7, 0x40, 0xeb, - 0xbd, 0xd8, 0xdd, 0x55, 0x52, 0xa8, 0x0a, 0x32, 0x69, 0xb5, 0x3b, 0x9b, 0xcd, 0x17, 0xbb, 0x07, - 0x4a, 0x1a, 0xc9, 0x90, 0x6f, 0x35, 0xf7, 0x5b, 0xcd, 0x76, 0x47, 0xc9, 0x2c, 0x66, 0xbe, 0xf9, - 0x7a, 0x69, 0xae, 0xf1, 0x18, 0xb2, 0x74, 0x38, 0x04, 0x90, 0xdb, 0xef, 0x3e, 0xdf, 0xdb, 0xed, - 0x28, 0x73, 0xa8, 0x00, 0x99, 0x4d, 0x02, 0x21, 0x11, 0x8b, 0xbd, 0xa6, 0x7a, 0xd0, 0x6d, 0xee, - 0x2a, 0x29, 0x66, 0xf1, 0x59, 0xe6, 0x7f, 0x7e, 0x51, 0x97, 0x1a, 0xbf, 0xcc, 0xc1, 0x9d, 0x68, - 0xee, 0xd1, 0x69, 0xa3, 0x16, 0x54, 0x5d, 0xcf, 0x3a, 0xb6, 0x1c, 0x8d, 0xfa, 0x9c, 0x66, 0x99, - 0xdc, 0x1f, 0xdf, 0x25, 0xeb, 0xb9, 0xbc, 0xa8, 0x97, 0xfb, 0xb4, 0xfb, 0x80, 0xf4, 0x76, 0xdb, - 0xdc, 0x41, 0xcb, 0x6e, 0x4c, 0x68, 0xa2, 0x1d, 0x98, 0xe7, 0x20, 0x86, 0x6b, 0x8f, 0x87, 0x8e, - 0x66, 0x99, 0x7e, 0x2d, 0xb5, 0x9c, 0x5e, 0x29, 0x6f, 0xd4, 0x2f, 0x2f, 0xea, 0x55, 0x06, 0xd1, - 0xa2, 0x7d, 0xdd, 0xb6, 0xff, 0xfd, 0x45, 0xbd, 0x20, 0x1a, 0x2a, 0x1f, 0x9e, 0xb7, 0x4d, 0x1f, - 0x1d, 0xc2, 0x5d, 0x4f, 0xec, 0xad, 0x19, 0x07, 0x4c, 0x53, 0xc0, 0xf7, 0x2e, 0x2f, 0xea, 0x0b, - 0xe1, 0xe6, 0x9b, 0xd3, 0x41, 0x17, 0xbc, 0x49, 0x05, 0xd3, 0x47, 0x7d, 0x88, 0x89, 0xa3, 0xe5, - 0x66, 0xe8, 0x72, 0xeb, 0x7c, 0xb9, 0xf3, 0x11, 0x74, 0x72, 0xc9, 0xf3, 0xde, 0x44, 0x87, 0x19, - 0x06, 0x5e, 0xf6, 0xc6, 0xc0, 0xcb, 0xbd, 0x6d, 0xe0, 0x25, 0xc2, 0x28, 0xff, 0xff, 0x12, 0x46, - 0x85, 0x1f, 0x3c, 0x8c, 0x8a, 0x3f, 0x40, 0x18, 0xa1, 0x3f, 0x84, 0x05, 0x1b, 0x1f, 0xeb, 0xc6, - 0xb9, 0xc6, 0xdd, 0x8b, 0xd1, 0x21, 0x4c, 0xa7, 0xc3, 0x79, 0xa6, 0xcb, 0x9c, 0x8d, 0x0a, 0xd1, - 0x33, 0xb8, 0xcf, 0x01, 0x62, 0x27, 0xcf, 0x40, 0xe4, 0xe9, 0x20, 0x77, 0x99, 0x7e, 0xe4, 0x05, - 0xb4, 0x83, 0x45, 0xd1, 0x76, 0xa6, 0x50, 0x52, 0xca, 0xdb, 0x99, 0x42, 0x59, 0xa9, 0x34, 0xbe, - 0x49, 0x83, 0xc2, 0x7c, 0xab, 0x8d, 0x7d, 0xc3, 0xb3, 0x46, 0x81, 0xeb, 0x85, 0x1e, 0x21, 0x5d, - 0xf1, 0x88, 0x0f, 0x20, 0x65, 0x99, 0x9c, 0xc8, 0xef, 0x71, 0x5f, 0x4b, 0x51, 0xe7, 0x8a, 0xbc, - 0x36, 0x65, 0x99, 0x68, 0x17, 0x32, 0xc1, 0xf9, 0x88, 0x91, 0x79, 0x69, 0xe3, 0x29, 0xd1, 0xfc, - 0x8f, 0x8b, 0xfa, 0xe3, 0x63, 0x2b, 0x38, 0x19, 0x1f, 0xad, 0x1a, 0xee, 0x70, 0x2d, 0xdc, 0x51, - 0xf3, 0x28, 0xfa, 0xbd, 0x36, 0x7a, 0x79, 0x4c, 0xee, 0xa5, 0x35, 0x62, 0xec, 0xaf, 0x1e, 0xa8, - 0x14, 0x05, 0x2d, 0x43, 0xc1, 0x19, 0xdb, 0x36, 0xbd, 0x66, 0x88, 0x9f, 0x17, 0xc4, 0x81, 0x09, - 0x29, 0x7a, 0x08, 0x25, 0x13, 0x0f, 0xf4, 0xb1, 0x1d, 0x68, 0xf8, 0x6c, 0xe4, 0x31, 0x5f, 0x56, - 0x65, 0x2e, 0xeb, 0x9c, 0x8d, 0x3c, 0xf4, 0x00, 0x72, 0x27, 0x96, 0x69, 0x62, 0x87, 0xba, 0xb2, - 0x80, 0xe0, 0x32, 0xb4, 0x0e, 0xf3, 0x63, 0x1f, 0xfb, 0x9a, 0x8f, 0x5f, 0x8d, 0xc9, 0x86, 0xd1, - 0x50, 0x05, 0x1a, 0xaa, 0x39, 0x1e, 0x3a, 0x55, 0xa2, 0xb0, 0xcf, 0xfb, 0x49, 0x24, 0xae, 0xc3, - 0xbc, 0xfb, 0xda, 0x99, 0xb0, 0x91, 0x93, 0x36, 0x44, 0x21, 0x6e, 0xf3, 0x10, 0x4a, 0x86, 0x3b, - 0x1c, 0x8d, 0x03, 0xcc, 0x26, 0x5a, 0x62, 0x13, 0xe5, 0x32, 0x32, 0xd1, 0xf0, 0x90, 0x0a, 0x4a, - 0x71, 0x3b, 0x53, 0x28, 0x2a, 0xb0, 0x9d, 0x29, 0xe4, 0x95, 0x42, 0xe3, 0xaf, 0x52, 0x70, 0x8f, - 0x6d, 0xf2, 0xa6, 0x3e, 0xb4, 0xec, 0xf3, 0xb7, 0x3d, 0x30, 0x86, 0xc2, 0x0f, 0x8c, 0xce, 0x8b, - 0x72, 0x14, 0x31, 0x63, 0x2c, 0x45, 0xe7, 0x45, 0x64, 0x3d, 0x22, 0x42, 0x4f, 0x01, 0x62, 0x34, - 0x96, 0xa1, 0xeb, 0x7c, 0xe7, 0xf2, 0xa2, 0x5e, 0x9c, 0x4e, 0x5e, 0x45, 0x23, 0x46, 0x59, 0xf3, - 0xe2, 0x74, 0x42, 0x04, 0x7a, 0x44, 0xe5, 0x8d, 0xf7, 0xf8, 0x9c, 0xaa, 0x6d, 0xa6, 0x20, 0xcc, - 0x93, 0xe4, 0x6a, 0x26, 0x3a, 0x4d, 0x7e, 0x1b, 0xfc, 0x73, 0x0a, 0xee, 0x74, 0x9d, 0x00, 0x7b, - 0x36, 0xd6, 0x4f, 0x71, 0x6c, 0x3b, 0x7e, 0x06, 0x45, 0xdd, 0x31, 0xb0, 0x1f, 0xb8, 0x9e, 0x5f, - 0x93, 0x96, 0xd3, 0x2b, 0xf2, 0xfa, 0x6f, 0x5d, 0x13, 0xc2, 0xd3, 0xec, 0x57, 0x9b, 0xdc, 0x98, - 0xef, 0x64, 0x04, 0xb6, 0xf8, 0x2f, 0x12, 0x14, 0x44, 0x2f, 0x7a, 0x0c, 0x85, 0x89, 0xdb, 0xe6, - 0x2e, 0x5f, 0x4d, 0x3e, 0x49, 0xba, 0xf9, 0x80, 0x53, 0xed, 0x6f, 0x43, 0x81, 0x86, 0xad, 0x16, - 0x9e, 0xc9, 0xa2, 0xb0, 0xe0, 0x91, 0x1b, 0x0f, 0xe2, 0x3c, 0xd5, 0xed, 0x9a, 0xa8, 0x35, 0x2d, - 0x67, 0x49, 0x53, 0xfb, 0xfb, 0x62, 0xff, 0xf6, 0x93, 0x59, 0xcb, 0x95, 0x34, 0x86, 0xed, 0x19, - 0xdf, 0xb9, 0xff, 0x4d, 0xc3, 0xbd, 0x3d, 0xdd, 0x0b, 0x2c, 0xc2, 0x7e, 0x96, 0x73, 0x1c, 0xdb, - 0xbb, 0x47, 0x20, 0x3b, 0xe3, 0x21, 0x3f, 0x27, 0x9f, 0xaf, 0x8b, 0xed, 0x03, 0x38, 0xe3, 0x21, - 0x3b, 0x02, 0x9f, 0x04, 0xb8, 0x6d, 0xf9, 0x01, 0xbd, 0x1e, 0xe5, 0xf5, 0xf5, 0x6b, 0x76, 0x77, - 0xfa, 0x18, 0xab, 0xbb, 0x96, 0x1f, 0x08, 0x2f, 0x25, 0x28, 0xa8, 0x0f, 0x59, 0x4f, 0x77, 0x8e, - 0x31, 0x75, 0x3b, 0x79, 0xfd, 0xc9, 0xed, 0xe0, 0x54, 0x62, 0x2a, 0x28, 0x97, 0xe2, 0x2c, 0xfe, - 0x9d, 0x04, 0x19, 0x32, 0xca, 0x0d, 0x91, 0x71, 0x0f, 0x72, 0xa7, 0xba, 0x3d, 0xc6, 0xec, 0x8a, - 0x2f, 0xa9, 0xbc, 0x85, 0xfe, 0x04, 0xaa, 0xfe, 0xf8, 0x68, 0x14, 0x1b, 0x8a, 0x6e, 0xb5, 0xbc, - 0xfe, 0xc9, 0xad, 0x66, 0x15, 0x66, 0x93, 0x49, 0x2c, 0x76, 0x00, 0x8b, 0xaf, 0x20, 0x4b, 0x67, - 0x7d, 0xc3, 0xfc, 0x1e, 0x42, 0x29, 0x70, 0x35, 0x7c, 0x66, 0xd8, 0x63, 0xdf, 0x3a, 0xc5, 0xd4, - 0x5f, 0x4a, 0xaa, 0x1c, 0xb8, 0x1d, 0x21, 0x42, 0x8f, 0xa0, 0x32, 0xf0, 0xdc, 0xa1, 0x66, 0x39, - 0x42, 0x89, 0xf2, 0xad, 0x5a, 0x26, 0xd2, 0xae, 0x10, 0x26, 0x4e, 0xfe, 0x5f, 0x01, 0xaa, 0xd4, - 0xbf, 0x66, 0x62, 0x8f, 0x47, 0x31, 0xf6, 0xb8, 0x9b, 0x60, 0x8f, 0xd0, 0x49, 0x09, 0x79, 0x3c, - 0x80, 0xdc, 0xd8, 0xb1, 0x5e, 0x8d, 0xd9, 0xf8, 0x21, 0xb5, 0x32, 0xd9, 0x15, 0x6a, 0xc9, 0x5c, - 0xa5, 0x96, 0x8f, 0x01, 0x91, 0x88, 0xc2, 0x5a, 0x42, 0x31, 0x4b, 0x15, 0x15, 0xda, 0xd3, 0xba, - 0x96, 0x88, 0x72, 0xb7, 0x20, 0xa2, 0x2d, 0x50, 0xf0, 0x59, 0xe0, 0xe9, 0xf1, 0x7c, 0x2c, 0x4f, - 0xed, 0x97, 0x2e, 0x2f, 0xea, 0x95, 0x0e, 0xe9, 0x9b, 0x0e, 0x52, 0xc1, 0xb1, 0x3e, 0x93, 0x78, - 0xc9, 0x3c, 0xc7, 0x30, 0x2d, 0x0f, 0xd3, 0x2c, 0xc2, 0xaf, 0x15, 0x96, 0xd3, 0x37, 0x64, 0x0b, - 0x13, 0xdb, 0xbe, 0xda, 0x16, 0x86, 0xaa, 0xc2, 0xa0, 0x42, 0x81, 0x8f, 0xf6, 0x41, 0x1e, 0xb0, - 0xe4, 0x42, 0x7b, 0x89, 0xcf, 0x69, 0x1a, 0x22, 0xaf, 0xff, 0x64, 0xf6, 0x34, 0x64, 0x23, 0x47, - 0x8e, 0xa0, 0x26, 0xa9, 0x30, 0x08, 0x3b, 0xd1, 0x21, 0x94, 0x63, 0xf9, 0xc3, 0xd1, 0x39, 0xbd, - 0xdf, 0xde, 0x0c, 0xb6, 0x14, 0x01, 0x6d, 0x9c, 0xa3, 0x2f, 0x00, 0xac, 0x90, 0x47, 0x69, 0x4a, - 0x22, 0xaf, 0x7f, 0x74, 0x0b, 0xc2, 0x15, 0xfc, 0x12, 0x81, 0xa0, 0x43, 0xa8, 0x44, 0x2d, 0x3a, - 0xd9, 0xd2, 0xad, 0x27, 0xcb, 0x50, 0xcb, 0x31, 0x9c, 0x0d, 0x92, 0x86, 0xde, 0x21, 0x97, 0xad, - 0xeb, 0x5b, 0x01, 0x8e, 0xbb, 0x41, 0x99, 0xba, 0x41, 0xe3, 0xf2, 0xa2, 0x8e, 0x5a, 0xa2, 0x7f, - 0xba, 0x2b, 0x20, 0x63, 0xa2, 0x9f, 0x39, 0x56, 0xc2, 0x81, 0x09, 0x62, 0x25, 0x72, 0xac, 0xfd, - 0xc8, 0x85, 0xaf, 0x38, 0x56, 0xcc, 0xbd, 0x59, 0xdd, 0x50, 0x4a, 0x70, 0x4f, 0xf5, 0xcd, 0xb9, - 0x27, 0x01, 0x84, 0x3a, 0x3c, 0x25, 0x53, 0x68, 0x4a, 0xfb, 0xd1, 0x8c, 0x4e, 0x7a, 0x70, 0x3e, - 0x12, 0x1b, 0xc9, 0x72, 0xb1, 0x27, 0x80, 0x0c, 0x0f, 0xeb, 0x01, 0x36, 0x49, 0x02, 0x63, 0x5b, - 0x86, 0x15, 0xd8, 0xe7, 0xb5, 0xf9, 0x58, 0xdc, 0xcf, 0xf3, 0xfe, 0x4e, 0xd8, 0x8d, 0x9e, 0x42, - 0xfe, 0x14, 0x7b, 0xbe, 0xe5, 0x3a, 0x35, 0x44, 0xc9, 0x64, 0x89, 0x27, 0xac, 0xf7, 0x26, 0xc6, - 0xfb, 0x92, 0x69, 0xa9, 0x42, 0x1d, 0x6d, 0x41, 0x19, 0x3b, 0x86, 0x6b, 0x5a, 0xce, 0xb1, 0x46, - 0xa7, 0xbf, 0x10, 0xa5, 0x0d, 0xdf, 0x5f, 0xd4, 0xdf, 0x9d, 0xb0, 0xef, 0x70, 0x5d, 0x32, 0x6d, - 0xb5, 0x84, 0x63, 0xad, 0xc6, 0x12, 0x14, 0xc3, 0x00, 0x43, 0x79, 0x48, 0x37, 0xf7, 0x5b, 0xac, - 0xe8, 0x6c, 0x77, 0xf6, 0x5b, 0x8a, 0xd4, 0x78, 0x08, 0x19, 0xa2, 0x47, 0x8a, 0xcf, 0xcd, 0xbe, - 0x7a, 0xd8, 0x54, 0xdb, 0xac, 0xd0, 0xed, 0xf6, 0xbe, 0xec, 0xa8, 0x07, 0x9d, 0xb6, 0x22, 0x28, - 0xf4, 0xdf, 0xd2, 0x80, 0xa2, 0x7a, 0xe7, 0xc0, 0xe5, 0x35, 0xc3, 0x31, 0x54, 0x8d, 0x50, 0xca, - 0xe6, 0x2a, 0x2d, 0xa7, 0x56, 0x2a, 0xeb, 0x4f, 0x7f, 0x63, 0xcd, 0x24, 0x30, 0xe2, 0xa2, 0x68, - 0xdf, 0x2b, 0x46, 0x42, 0x1a, 0xd2, 0x75, 0x6a, 0x39, 0x35, 0x41, 0xd7, 0x2a, 0x64, 0x8d, 0x13, - 0x6c, 0xbc, 0xe4, 0x17, 0xd6, 0xef, 0x5c, 0x33, 0x30, 0x4d, 0x4e, 0x62, 0x67, 0xdc, 0x22, 0x36, - 0xd1, 0xd0, 0xe2, 0x26, 0xa5, 0x50, 0x48, 0x4d, 0x32, 0x51, 0xe6, 0xc6, 0xe0, 0x9e, 0x56, 0x9b, - 0x8b, 0xe0, 0x8e, 0x11, 0xd1, 0x53, 0xa8, 0x3a, 0x6e, 0xa0, 0x91, 0xec, 0x9d, 0x07, 0x0c, 0xcd, - 0xc9, 0xcb, 0x1b, 0x0a, 0x3f, 0xd6, 0x28, 0x3c, 0xca, 0x8e, 0x1b, 0xf4, 0xc6, 0xb6, 0xcd, 0x04, - 0x8d, 0xcf, 0xa0, 0x92, 0xdc, 0x23, 0x54, 0x84, 0x6c, 0x6b, 0xab, 0xd3, 0xda, 0x51, 0xe6, 0x50, - 0x15, 0xe4, 0xcd, 0xbe, 0xda, 0xe9, 0x3e, 0xeb, 0x69, 0x3b, 0x9d, 0x3f, 0x62, 0x0f, 0x13, 0xbd, - 0xbe, 0x78, 0x98, 0x08, 0xf3, 0xea, 0xac, 0x92, 0x6b, 0xfc, 0x93, 0x04, 0x95, 0x3d, 0xcf, 0x1a, - 0xea, 0xde, 0xf9, 0x0e, 0x3e, 0xdf, 0x7f, 0xad, 0x8f, 0xd0, 0xe7, 0x70, 0xc7, 0xc1, 0xaf, 0xb5, - 0x11, 0x93, 0x6a, 0x61, 0x9e, 0x26, 0x5d, 0x53, 0xa6, 0x39, 0xf8, 0x35, 0x47, 0xe8, 0xf2, 0x34, - 0xed, 0x63, 0x90, 0x5d, 0x9b, 0x17, 0x66, 0x58, 0xbc, 0x1c, 0xc8, 0x71, 0x23, 0x70, 0x6d, 0x56, - 0x89, 0xd1, 0x3b, 0x4f, 0x26, 0xe3, 0x09, 0xed, 0xf4, 0x14, 0x6d, 0x07, 0xbf, 0xe6, 0xda, 0xdc, - 0xf5, 0xfe, 0x3d, 0x0b, 0x28, 0x3a, 0xb8, 0xe7, 0xe3, 0x40, 0xa7, 0xde, 0xdc, 0x84, 0x1c, 0xdf, - 0x46, 0x89, 0x1e, 0xcf, 0x87, 0xd7, 0x7a, 0x5c, 0xb2, 0xd0, 0xdb, 0x9a, 0x53, 0xb9, 0x21, 0xfa, - 0x69, 0xfc, 0x91, 0x4e, 0x5e, 0xff, 0x60, 0x36, 0x7a, 0xd8, 0x9a, 0x13, 0xaf, 0x77, 0x3b, 0x90, - 0xf5, 0x03, 0x52, 0x83, 0xa7, 0x29, 0xbd, 0xac, 0x5d, 0x63, 0x7f, 0x75, 0xf2, 0xab, 0xfb, 0xc4, - 0x4c, 0xf8, 0x1c, 0xc5, 0x40, 0x87, 0x50, 0x0c, 0x6f, 0x55, 0xfe, 0xe2, 0xf7, 0x64, 0x76, 0xc0, - 0x30, 0xca, 0x45, 0xfa, 0x1e, 0x62, 0xa1, 0x26, 0xc8, 0x43, 0xae, 0x16, 0x95, 0x20, 0xcb, 0x3c, - 0xb1, 0x01, 0x81, 0x40, 0x13, 0x9c, 0x58, 0x4b, 0x05, 0x61, 0xd4, 0x35, 0x49, 0x2d, 0xea, 0xb9, - 0xb6, 0x7d, 0xa4, 0x1b, 0x2f, 0xe9, 0x2b, 0x46, 0x58, 0x8b, 0x0a, 0x29, 0xda, 0x21, 0xe9, 0x89, - 0xf0, 0x51, 0xfa, 0x26, 0x21, 0xcf, 0xf0, 0x6e, 0x22, 0x38, 0x60, 0x6b, 0x4e, 0x8d, 0x99, 0xa3, - 0x3e, 0x54, 0x46, 0x09, 0x3f, 0xe5, 0xb9, 0xc0, 0xa3, 0xeb, 0x2e, 0x84, 0x84, 0xf2, 0xd6, 0x9c, - 0x3a, 0x61, 0xde, 0xf8, 0x1c, 0xb2, 0x74, 0xc7, 0x09, 0xcf, 0xbd, 0xe8, 0xed, 0xf4, 0xfa, 0x87, - 0x3d, 0x16, 0x3a, 0xed, 0xce, 0x6e, 0xe7, 0xa0, 0xa3, 0xf5, 0x7b, 0xbb, 0x24, 0x74, 0xde, 0x81, - 0xbb, 0x5c, 0xd0, 0xec, 0xb5, 0xb5, 0x43, 0xb5, 0x2b, 0xba, 0x52, 0x8d, 0x95, 0x38, 0x91, 0x16, - 0x20, 0xd3, 0xeb, 0xf7, 0x3a, 0xca, 0x1c, 0xa5, 0xd4, 0x76, 0x5b, 0x91, 0x28, 0xa5, 0xaa, 0xfd, - 0x3d, 0x11, 0x71, 0x1b, 0x25, 0x00, 0x33, 0x3c, 0xa5, 0xed, 0x4c, 0x21, 0xa7, 0xe4, 0x1b, 0xff, - 0xd9, 0x80, 0xea, 0x04, 0x0d, 0xdd, 0x90, 0x86, 0x2e, 0xd3, 0x34, 0x34, 0x1d, 0x51, 0x44, 0x98, - 0x86, 0xa6, 0x78, 0x06, 0xfa, 0x04, 0x8a, 0x23, 0xdd, 0xc3, 0x4e, 0x10, 0x3d, 0x85, 0x89, 0x6a, - 0xb7, 0xb0, 0x47, 0x3b, 0x42, 0xf5, 0x02, 0x53, 0xec, 0x12, 0xa3, 0xf0, 0x56, 0x62, 0x9e, 0xf0, - 0x0e, 0x0f, 0xf2, 0xf9, 0x1b, 0x2e, 0xa4, 0x3d, 0x98, 0x1f, 0xba, 0xa6, 0x35, 0xb0, 0x0c, 0xe6, - 0x46, 0x81, 0x35, 0x64, 0xcf, 0x59, 0xf2, 0xfa, 0x8f, 0x62, 0x67, 0x32, 0x0e, 0x2c, 0x7b, 0xf5, - 0xc4, 0x36, 0x56, 0x0f, 0xc4, 0xd3, 0x3b, 0x5f, 0x91, 0x12, 0xb7, 0x26, 0x9d, 0xe8, 0x19, 0xe4, - 0x45, 0xb5, 0x55, 0xa0, 0x39, 0xce, 0xac, 0xe1, 0xcb, 0x11, 0x85, 0x35, 0xda, 0x84, 0x8a, 0x83, - 0xcf, 0xe2, 0x35, 0x76, 0x31, 0xe1, 0xe0, 0xa5, 0x1e, 0x3e, 0x9b, 0x5e, 0x60, 0x97, 0x9c, 0xa8, - 0xc7, 0x44, 0x5f, 0x40, 0x39, 0xc1, 0x82, 0xf4, 0xa5, 0x6a, 0x66, 0x4e, 0x08, 0x93, 0x8f, 0x18, - 0x39, 0xa2, 0x4d, 0xc8, 0x0b, 0xa2, 0x93, 0xe9, 0x1a, 0x6f, 0x07, 0x26, 0x8c, 0xd1, 0x06, 0x94, - 0xe9, 0x12, 0x43, 0x76, 0x2e, 0x45, 0xe9, 0xc4, 0xe5, 0x45, 0x5d, 0x26, 0x2b, 0x9c, 0x52, 0x49, - 0xcb, 0x4e, 0x28, 0x37, 0xd1, 0x36, 0x40, 0xf8, 0xc9, 0x83, 0xe4, 0x7d, 0x37, 0xa5, 0xd6, 0x7b, - 0x42, 0x31, 0x9a, 0x92, 0x1a, 0xb3, 0x46, 0xcf, 0xa1, 0x28, 0xb8, 0x81, 0x25, 0x7c, 0xd7, 0x87, - 0xfa, 0x55, 0xa6, 0x12, 0xfc, 0x14, 0x22, 0xa0, 0x1e, 0x64, 0x6d, 0xac, 0xfb, 0x98, 0x67, 0x7d, - 0x4f, 0x67, 0xbc, 0xc0, 0xf7, 0x8d, 0x13, 0x3c, 0xd4, 0x5b, 0x27, 0xa4, 0xa2, 0xdc, 0x25, 0xf6, - 0x2a, 0x83, 0x41, 0x3d, 0x50, 0xe8, 0x76, 0xc5, 0x49, 0x4f, 0xa1, 0x3b, 0xf6, 0x3e, 0xdf, 0xb1, - 0x0a, 0xd9, 0xb1, 0x6b, 0x89, 0x8f, 0xfa, 0xd3, 0xf3, 0x88, 0xfc, 0xfe, 0x00, 0x2a, 0x03, 0xd7, - 0x1b, 0xea, 0x81, 0x26, 0x02, 0x67, 0x3e, 0xaa, 0x0d, 0xbf, 0xbf, 0xa8, 0x97, 0x37, 0x69, 0xaf, - 0x08, 0x9a, 0xf2, 0x20, 0xde, 0x44, 0x5b, 0xe2, 0x8e, 0x58, 0xa0, 0x94, 0xfe, 0xf1, 0xac, 0xab, - 0xbb, 0x7a, 0x41, 0xf4, 0x20, 0x47, 0xb3, 0x13, 0xbf, 0x76, 0x87, 0xee, 0xf9, 0x1b, 0x66, 0x3a, - 0x2a, 0x47, 0x41, 0x3f, 0x87, 0x8a, 0x49, 0x24, 0x24, 0xcb, 0x64, 0xb5, 0xe7, 0x5d, 0x8a, 0xbb, - 0x36, 0x23, 0x2e, 0xa9, 0x4b, 0xbb, 0xce, 0xc0, 0x15, 0x25, 0x87, 0x00, 0x63, 0xf5, 0x6a, 0x1f, - 0x0a, 0x03, 0x7d, 0x68, 0xd9, 0x16, 0xf6, 0x6b, 0xf7, 0x28, 0xee, 0x27, 0x37, 0x46, 0xf8, 0xe4, - 0xf3, 0x9e, 0xb8, 0x61, 0x04, 0x48, 0x18, 0xe8, 0x54, 0x70, 0x4e, 0x0e, 0xf5, 0xfe, 0xd5, 0x40, - 0x17, 0xcf, 0x7b, 0x89, 0xa7, 0x3e, 0x1a, 0xe8, 0xbc, 0x65, 0xa2, 0xf7, 0x00, 0x4e, 0x2d, 0xfc, - 0x5a, 0x7b, 0x35, 0xc6, 0xde, 0x79, 0xad, 0x16, 0xe3, 0xdd, 0x22, 0x91, 0x7f, 0x41, 0xc4, 0xe8, - 0x53, 0x28, 0x9a, 0x78, 0x84, 0x1d, 0xd3, 0xef, 0x3b, 0xb5, 0x77, 0x68, 0x96, 0xb2, 0x40, 0x8a, - 0xed, 0xb6, 0x10, 0x72, 0x5e, 0x8d, 0xb4, 0xd0, 0x57, 0x50, 0x62, 0x0d, 0x6c, 0xf6, 0x9d, 0x8d, - 0xf3, 0xda, 0x22, 0x5d, 0xf4, 0xe3, 0x19, 0x37, 0x33, 0x2a, 0xe0, 0xee, 0x88, 0xf5, 0xb4, 0x63, - 0x68, 0x6a, 0x02, 0x1b, 0xfd, 0x1c, 0x4a, 0xc2, 0xbb, 0xb7, 0xdd, 0x23, 0xbf, 0xf6, 0xee, 0x8d, - 0x0f, 0x52, 0x93, 0x63, 0x3d, 0x8f, 0x4c, 0x05, 0x6f, 0xc5, 0xd1, 0xd0, 0xcf, 0xa0, 0x1c, 0xbe, - 0xee, 0xba, 0xa3, 0xc0, 0xaf, 0x3d, 0xa0, 0x81, 0xf9, 0x64, 0x56, 0xd7, 0xe5, 0xb6, 0xfd, 0x51, - 0xe0, 0xab, 0x25, 0x3f, 0xd6, 0x42, 0x0f, 0xa1, 0x68, 0x7a, 0xee, 0x88, 0xdd, 0x1f, 0x3f, 0x5a, - 0x96, 0x56, 0xd2, 0xe2, 0x98, 0x89, 0x98, 0x5e, 0x0c, 0x1a, 0x54, 0x3c, 0x3c, 0xb2, 0x75, 0x03, - 0x0f, 0xc9, 0xcd, 0xe6, 0x0e, 0x6a, 0x4b, 0x74, 0xf4, 0xf5, 0x99, 0x37, 0x32, 0x34, 0x16, 0x8e, - 0x19, 0xc3, 0xeb, 0x0f, 0xd0, 0x0b, 0x00, 0x7d, 0x6c, 0x5a, 0x81, 0x36, 0x74, 0x4d, 0x5c, 0xab, - 0xdf, 0xf8, 0xad, 0x63, 0x12, 0xbc, 0x49, 0x0c, 0x9f, 0xbb, 0x26, 0x0e, 0x1f, 0x49, 0x85, 0x00, - 0x7d, 0x0a, 0x32, 0x5d, 0xda, 0x57, 0xee, 0x11, 0xf1, 0xcd, 0x65, 0xba, 0xb8, 0x79, 0x7e, 0x96, - 0xc5, 0xb6, 0xe7, 0x8e, 0xb6, 0xdd, 0x23, 0xea, 0x31, 0xfc, 0xa7, 0x89, 0x7c, 0x28, 0x1d, 0x1b, - 0x5a, 0x44, 0xa5, 0x0f, 0xe9, 0x29, 0xfe, 0xfe, 0x8c, 0x73, 0x79, 0xd6, 0x9a, 0x42, 0xae, 0x0b, - 0xe2, 0x4e, 0x78, 0xd6, 0x12, 0x32, 0x5f, 0x95, 0x8f, 0x8d, 0xb0, 0x81, 0x3e, 0x84, 0x12, 0x2b, - 0x55, 0x79, 0x00, 0x34, 0x62, 0x01, 0x20, 0xb3, 0x1e, 0x16, 0x02, 0x3d, 0xe0, 0x35, 0xad, 0xa6, - 0xfb, 0x9a, 0x3b, 0x60, 0x67, 0xf6, 0xde, 0xec, 0x77, 0x7e, 0x85, 0x59, 0x37, 0xfd, 0xfe, 0x80, - 0x1e, 0xac, 0x01, 0x25, 0x77, 0x1c, 0x1c, 0xb9, 0x63, 0xc7, 0xd4, 0x06, 0x2f, 0xfd, 0xda, 0xfb, - 0x74, 0xb5, 0xb7, 0x2a, 0xaa, 0xc2, 0xd5, 0xf5, 0x39, 0xd0, 0xe6, 0x8e, 0xaf, 0xca, 0x02, 0x75, - 0xf3, 0xa5, 0x8f, 0xfe, 0x0c, 0x64, 0xcb, 0x89, 0xc6, 0x78, 0x74, 0xfb, 0x31, 0x90, 0x48, 0x8c, - 0xbb, 0x4e, 0x38, 0x04, 0x70, 0x4c, 0x32, 0xc2, 0x47, 0x50, 0x71, 0x07, 0x03, 0xdb, 0x72, 0xb0, - 0xe6, 0x61, 0xdd, 0x77, 0x9d, 0xda, 0x07, 0xb1, 0x1d, 0x2c, 0xf3, 0x3e, 0x95, 0x76, 0xa1, 0x06, - 0x14, 0x03, 0x3c, 0x1c, 0xb9, 0x9e, 0xee, 0x9d, 0xd7, 0x3e, 0x8c, 0x25, 0xce, 0x91, 0x18, 0x1d, - 0xc1, 0xe2, 0xd8, 0xc1, 0x67, 0x23, 0xd7, 0xc7, 0xa6, 0xc6, 0xf3, 0x39, 0x9f, 0xde, 0x6d, 0xc4, - 0x8f, 0x56, 0x28, 0xc7, 0x3d, 0xe2, 0x93, 0xba, 0xff, 0x42, 0x68, 0xb2, 0xfc, 0x8e, 0xdd, 0x81, - 0x61, 0x96, 0x77, 0x7f, 0x3c, 0xb5, 0xdb, 0x5c, 0xfc, 0x46, 0x82, 0xf9, 0x2b, 0xf7, 0x25, 0xfa, - 0x53, 0xc8, 0x3b, 0xae, 0x19, 0x7b, 0xc9, 0xef, 0xf0, 0x61, 0x72, 0x3d, 0xd7, 0x64, 0x0f, 0xf9, - 0x4f, 0x66, 0xfa, 0x78, 0x45, 0x7f, 0x8d, 0x8e, 0x56, 0x99, 0x99, 0x9a, 0x23, 0xa8, 0x5d, 0x13, - 0x7d, 0x02, 0x55, 0x7c, 0x36, 0xb2, 0xbc, 0x58, 0xce, 0x98, 0x8a, 0xc5, 0x7c, 0x25, 0xea, 0x24, - 0x0e, 0xc2, 0x1f, 0x89, 0xff, 0x26, 0x05, 0xd5, 0x89, 0x1b, 0x8b, 0x24, 0xc9, 0xf4, 0xbb, 0x51, - 0x22, 0x49, 0x26, 0x92, 0xd8, 0xb3, 0xc0, 0x4d, 0x9f, 0x71, 0xd3, 0x6f, 0xfb, 0x19, 0x37, 0xf9, - 0xf8, 0x9a, 0xbd, 0xc5, 0xe3, 0xeb, 0xef, 0xc1, 0x3d, 0xcb, 0xd7, 0x1c, 0xd7, 0x11, 0x85, 0x7f, - 0x58, 0x23, 0xc5, 0x3f, 0xc8, 0x2d, 0x58, 0x7e, 0xcf, 0x75, 0x58, 0xc9, 0x2f, 0x14, 0xc2, 0xd2, - 0x3d, 0xa3, 0x64, 0x17, 0xff, 0x51, 0x82, 0x82, 0xb8, 0x6f, 0x93, 0x79, 0xbf, 0x34, 0x63, 0xde, - 0x7f, 0xfd, 0x4e, 0x6d, 0x82, 0x72, 0xc5, 0xed, 0x58, 0xd9, 0xf1, 0x40, 0xe4, 0x4b, 0x53, 0xbd, - 0xad, 0x32, 0x4a, 0x38, 0x19, 0x3f, 0xbf, 0xaf, 0x25, 0x28, 0xc6, 0xff, 0x28, 0x93, 0x0a, 0xe7, - 0x38, 0xbd, 0x88, 0x79, 0xc3, 0xaf, 0x43, 0xc9, 0x13, 0x49, 0xcf, 0x7e, 0x22, 0x7c, 0x9a, 0x7f, - 0x0e, 0x72, 0xec, 0x1a, 0x9c, 0xac, 0x91, 0xa5, 0x37, 0xa8, 0x91, 0xdf, 0x87, 0x1c, 0xe7, 0x7e, - 0xe6, 0xe4, 0x65, 0x6e, 0x9d, 0x65, 0xbc, 0x9f, 0xfd, 0x8a, 0x70, 0x3e, 0x1f, 0xfd, 0x97, 0x69, - 0x28, 0xc5, 0xaf, 0x49, 0x42, 0x14, 0x96, 0x63, 0x78, 0xf4, 0x8e, 0xa2, 0xa3, 0x8b, 0x20, 0x89, - 0xc4, 0xe4, 0xf2, 0x1c, 0x5a, 0x8e, 0x46, 0xbf, 0xd8, 0x24, 0x02, 0xa9, 0x30, 0xb4, 0x9c, 0x2f, - 0x89, 0x94, 0xaa, 0xe8, 0x67, 0x5c, 0x25, 0x9d, 0x50, 0xd1, 0xcf, 0x98, 0xca, 0x22, 0xcd, 0x47, - 0xbd, 0x80, 0x16, 0x8c, 0xe9, 0x58, 0x86, 0xe9, 0x05, 0x68, 0x09, 0xf2, 0xa7, 0x96, 0x17, 0x8c, - 0x75, 0x9b, 0xd6, 0x86, 0xc2, 0x3b, 0x85, 0x10, 0x39, 0x50, 0x89, 0x12, 0x83, 0xd7, 0x0e, 0xf6, - 0xa8, 0x13, 0xcb, 0xeb, 0xcd, 0x37, 0xc8, 0x0c, 0xa2, 0x06, 0x01, 0x12, 0xf4, 0xe9, 0xc7, 0x85, - 0x8b, 0x7f, 0x2f, 0x41, 0x39, 0xa1, 0x86, 0xba, 0x50, 0xa5, 0x03, 0xc7, 0xca, 0x3d, 0x76, 0x56, - 0x0f, 0xc3, 0xbf, 0xbc, 0x90, 0xee, 0xa9, 0xf5, 0x5e, 0xd9, 0x8d, 0x75, 0x99, 0xe8, 0x73, 0xa8, - 0x30, 0xa8, 0xf0, 0x73, 0x66, 0xd2, 0xfd, 0x4a, 0x14, 0x29, 0xf9, 0x4d, 0xb3, 0xe4, 0x46, 0x32, - 0x33, 0xfe, 0x89, 0x69, 0xd1, 0x01, 0x39, 0x96, 0x79, 0xcc, 0xe0, 0xf7, 0xbf, 0x0b, 0x99, 0x90, - 0x11, 0x67, 0xbc, 0x51, 0xa9, 0x01, 0x1f, 0xef, 0x6f, 0x25, 0xb8, 0x33, 0x2d, 0x03, 0x48, 0xc4, - 0x13, 0x73, 0xa4, 0x99, 0xe2, 0x29, 0x91, 0x99, 0xa5, 0xa6, 0x66, 0x66, 0x91, 0x83, 0xa7, 0x7f, - 0x93, 0x83, 0x37, 0x9e, 0x88, 0x07, 0x17, 0x80, 0xdc, 0xde, 0x8b, 0x8d, 0xdd, 0x6e, 0x6b, 0xea, - 0x63, 0x09, 0x92, 0x21, 0xdf, 0xdf, 0xdc, 0xdc, 0xed, 0xf6, 0x3a, 0x4a, 0xba, 0xb1, 0x02, 0xc5, - 0x30, 0xc1, 0x42, 0x25, 0x28, 0xb4, 0xbb, 0xfb, 0xcd, 0x8d, 0xdd, 0x4e, 0x5b, 0x99, 0x43, 0x65, - 0x28, 0xaa, 0x9d, 0x66, 0x9b, 0x3e, 0xc9, 0x28, 0xd2, 0x67, 0x85, 0xbf, 0xfc, 0x45, 0x5d, 0xe2, - 0xf4, 0x98, 0x53, 0xf2, 0xdb, 0x99, 0x02, 0x52, 0x16, 0x1a, 0xff, 0x20, 0x01, 0x6a, 0xeb, 0x81, - 0x4e, 0x7c, 0xef, 0x16, 0x4f, 0x2c, 0xa9, 0x1b, 0x4e, 0x29, 0x59, 0x36, 0xa7, 0xdf, 0xa6, 0x6c, - 0x8e, 0x26, 0xdd, 0xf8, 0x5a, 0x02, 0x88, 0x4d, 0xf0, 0xa7, 0xf1, 0x7f, 0x13, 0x5e, 0xff, 0x4a, - 0x30, 0x11, 0x4d, 0x5b, 0x73, 0xe2, 0xbf, 0x86, 0xcf, 0xa0, 0x60, 0xf2, 0x65, 0x73, 0x77, 0xba, - 0xb6, 0x1c, 0xbf, 0xb2, 0x3b, 0x5b, 0xe4, 0x84, 0xb9, 0x94, 0x3f, 0x5d, 0xe5, 0x21, 0x3b, 0x76, - 0x2c, 0xd7, 0xf9, 0x89, 0x1a, 0x7f, 0xf2, 0x17, 0x77, 0x23, 0x39, 0x0a, 0xfa, 0x5b, 0x0f, 0xb0, - 0xc9, 0x1e, 0xd1, 0x5e, 0x38, 0xa7, 0xa1, 0x40, 0x42, 0x15, 0x00, 0xde, 0x6f, 0x39, 0xc7, 0x4a, - 0x8a, 0x1e, 0xa4, 0xe7, 0x8e, 0x46, 0xa4, 0x95, 0xde, 0xf8, 0xf1, 0xb7, 0xff, 0xbd, 0x34, 0xf7, - 0xed, 0xe5, 0x92, 0xf4, 0xab, 0xcb, 0x25, 0xe9, 0xd7, 0x97, 0x4b, 0xd2, 0x7f, 0x5d, 0x2e, 0x49, - 0x7f, 0xfd, 0xdd, 0xd2, 0xdc, 0xaf, 0xbe, 0x5b, 0x9a, 0xfb, 0xf5, 0x77, 0x4b, 0x73, 0x7f, 0x9c, - 0xe7, 0x93, 0xfd, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x95, 0x8b, 0x20, 0x00, 0xff, 0x29, 0x00, - 0x00, + proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_9e2c09aa50a30b8b) +} + +var fileDescriptor_structured_9e2c09aa50a30b8b = []byte{ + // 3624 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x5a, 0xcb, 0x6f, 0x23, 0x67, + 0x72, 0x57, 0xf3, 0xcd, 0x6a, 0x3e, 0x9a, 0x9f, 0xe6, 0x41, 0xcb, 0xb3, 0xa2, 0x86, 0xf6, 0xd8, + 0xda, 0xb5, 0x2d, 0x8d, 0x35, 0x79, 0x4c, 0x9c, 0x60, 0x61, 0x8a, 0xa4, 0x46, 0x1c, 0x69, 0x48, + 0xb9, 0xa5, 0xf1, 0x6c, 0x82, 0x4d, 0x3a, 0xad, 0xee, 0x8f, 0x52, 0x7b, 0x9a, 0xdd, 0x9c, 0xee, + 0xa6, 0x46, 0x02, 0x72, 0xca, 0x29, 0xa7, 0x20, 0x40, 0xae, 0x09, 0xb0, 0x08, 0x8c, 0xc4, 0x7f, + 0x40, 0x80, 0xdc, 0x82, 0xe4, 0x12, 0xf8, 0x96, 0x05, 0x72, 0x59, 0x04, 0x88, 0x90, 0xc8, 0x97, + 0xfc, 0x07, 0x01, 0x7c, 0x0a, 0xbe, 0x57, 0x3f, 0x28, 0x4a, 0x4b, 0xcd, 0x78, 0x6f, 0xfc, 0xea, + 0xab, 0xaa, 0xef, 0x51, 0x55, 0xbf, 0xaa, 0xfa, 0x9a, 0x70, 0xcf, 0x7f, 0x65, 0xaf, 0xfb, 0xaf, + 0xec, 0x43, 0xdd, 0xc7, 0xeb, 0x7e, 0xe0, 0x4d, 0x8c, 0x60, 0xe2, 0x61, 0x73, 0x6d, 0xec, 0xb9, + 0x81, 0x8b, 0x6e, 0x1b, 0xae, 0xf1, 0xd2, 0x73, 0x75, 0xe3, 0x78, 0xcd, 0x7f, 0x65, 0xaf, 0x71, + 0xbe, 0xa5, 0xfa, 0x24, 0xb0, 0xec, 0xf5, 0x63, 0xdb, 0x58, 0x0f, 0xac, 0x11, 0xf6, 0x03, 0x7d, + 0x34, 0x66, 0x02, 0x4b, 0xef, 0xc6, 0xd5, 0x8d, 0x3d, 0xeb, 0xc4, 0xb2, 0xf1, 0x11, 0xe6, 0x93, + 0xb7, 0x8e, 0xdc, 0x23, 0x97, 0xfe, 0x5c, 0x27, 0xbf, 0x18, 0xb5, 0xf9, 0xe7, 0x59, 0x58, 0xdc, + 0x72, 0x3d, 0x6c, 0x1d, 0x39, 0x3b, 0xf8, 0x4c, 0xc5, 0x43, 0xec, 0x61, 0xc7, 0xc0, 0x68, 0x05, + 0xb2, 0x81, 0x7e, 0x68, 0xe3, 0xba, 0xb4, 0x22, 0xad, 0x96, 0x37, 0xe1, 0xdb, 0xf3, 0xc6, 0xc2, + 0xf7, 0xe7, 0x8d, 0x54, 0xaf, 0xa3, 0xb2, 0x09, 0xf4, 0x00, 0xb2, 0x96, 0x63, 0xe2, 0xd3, 0x7a, + 0x8a, 0x72, 0x54, 0x39, 0x47, 0xbe, 0x47, 0x88, 0x84, 0x8d, 0xce, 0xa2, 0x3a, 0x64, 0x1c, 0x7d, + 0x84, 0xeb, 0xe9, 0x15, 0x69, 0xb5, 0xb8, 0x99, 0x21, 0x5c, 0x2a, 0xa5, 0xa0, 0x1d, 0x28, 0x9c, + 0xe8, 0xb6, 0x65, 0x5a, 0xc1, 0x59, 0x3d, 0xb3, 0x22, 0xad, 0x56, 0x36, 0x7e, 0xbc, 0x36, 0xf3, + 0xc4, 0x6b, 0x6d, 0xd7, 0xf1, 0x03, 0x4f, 0xb7, 0x9c, 0xe0, 0x4b, 0x2e, 0xc0, 0x15, 0x85, 0x0a, + 0xd0, 0x43, 0xa8, 0xf9, 0xc7, 0xba, 0x87, 0x4d, 0x6d, 0xec, 0xe1, 0xa1, 0x75, 0xaa, 0xd9, 0xd8, + 0xa9, 0x67, 0x57, 0xa4, 0xd5, 0x2c, 0x67, 0xad, 0xb2, 0xe9, 0x3d, 0x3a, 0xbb, 0x8b, 0x1d, 0x74, + 0x00, 0x45, 0xd7, 0xd1, 0x4c, 0x6c, 0xe3, 0x00, 0xd7, 0x73, 0x74, 0xfd, 0x4f, 0xaf, 0x58, 0x7f, + 0xc6, 0x05, 0xad, 0xb5, 0x8c, 0xc0, 0x72, 0x1d, 0xb1, 0x0f, 0xd7, 0xe9, 0x50, 0x45, 0x5c, 0xeb, + 0x64, 0x6c, 0xea, 0x01, 0xae, 0xe7, 0xdf, 0x5a, 0xeb, 0x73, 0xaa, 0x08, 0xed, 0x42, 0x76, 0xa4, + 0x07, 0xc6, 0x71, 0xbd, 0x40, 0x35, 0x3e, 0xbc, 0x81, 0xc6, 0x67, 0x44, 0x8e, 0x2b, 0x64, 0x4a, + 0x9a, 0x2f, 0x20, 0xc7, 0xd6, 0x41, 0x65, 0x28, 0xf6, 0x07, 0x5a, 0xab, 0x7d, 0xd0, 0x1b, 0xf4, + 0x95, 0x05, 0x54, 0x82, 0x82, 0xda, 0xdd, 0x3f, 0x50, 0x7b, 0xed, 0x03, 0x45, 0x22, 0xa3, 0xfd, + 0xee, 0x81, 0xd6, 0x7f, 0xbe, 0xbb, 0xab, 0xa4, 0x50, 0x15, 0x64, 0x32, 0xea, 0x74, 0xb7, 0x5a, + 0xcf, 0x77, 0x0f, 0x94, 0x34, 0x92, 0x21, 0xdf, 0x6e, 0xed, 0xb7, 0x5b, 0x9d, 0xae, 0x92, 0x59, + 0xca, 0x7c, 0xf3, 0xf5, 0xf2, 0x42, 0xf3, 0x21, 0x64, 0xe9, 0x72, 0x08, 0x20, 0xb7, 0xdf, 0x7b, + 0xb6, 0xb7, 0xdb, 0x55, 0x16, 0x50, 0x01, 0x32, 0x5b, 0x44, 0x85, 0x44, 0x24, 0xf6, 0x5a, 0xea, + 0x41, 0xaf, 0xb5, 0xab, 0xa4, 0x98, 0xc4, 0x67, 0x99, 0xff, 0xfd, 0x45, 0x43, 0x6a, 0xfe, 0x47, + 0x0e, 0x6e, 0x45, 0x7b, 0x8f, 0xac, 0x8d, 0xda, 0x50, 0x75, 0x3d, 0xeb, 0xc8, 0x72, 0x34, 0xea, + 0x73, 0x9a, 0x65, 0x72, 0x7f, 0x7c, 0x97, 0x9c, 0xe7, 0xe2, 0xbc, 0x51, 0x1e, 0xd0, 0xe9, 0x03, + 0x32, 0xdb, 0xeb, 0x70, 0x07, 0x2d, 0xbb, 0x31, 0xa2, 0x89, 0x76, 0xa0, 0xc6, 0x95, 0x18, 0xae, + 0x3d, 0x19, 0x39, 0x9a, 0x65, 0xfa, 0xf5, 0xd4, 0x4a, 0x7a, 0xb5, 0xbc, 0xd9, 0xb8, 0x38, 0x6f, + 0x54, 0x99, 0x8a, 0x36, 0x9d, 0xeb, 0x75, 0xfc, 0xef, 0xcf, 0x1b, 0x05, 0x31, 0x50, 0xf9, 0xf2, + 0x7c, 0x6c, 0xfa, 0xe8, 0x05, 0xdc, 0xf6, 0xc4, 0xdd, 0x9a, 0x71, 0x85, 0x69, 0xaa, 0xf0, 0xbd, + 0x8b, 0xf3, 0xc6, 0x62, 0x78, 0xf9, 0xe6, 0x6c, 0xa5, 0x8b, 0xde, 0x34, 0x83, 0xe9, 0xa3, 0x01, + 0xc4, 0xc8, 0xd1, 0x71, 0x33, 0xf4, 0xb8, 0x0d, 0x7e, 0xdc, 0x5a, 0xa4, 0x3a, 0x79, 0xe4, 0x9a, + 0x37, 0x35, 0x61, 0x86, 0x81, 0x97, 0xbd, 0x36, 0xf0, 0x72, 0x6f, 0x1b, 0x78, 0x89, 0x30, 0xca, + 0xff, 0x46, 0xc2, 0xa8, 0xf0, 0x83, 0x87, 0x51, 0xf1, 0x07, 0x08, 0x23, 0xd4, 0x82, 0x45, 0x1b, + 0x1f, 0xe9, 0xc6, 0x99, 0xc6, 0xdd, 0x8b, 0xc1, 0x21, 0x50, 0x8b, 0xd5, 0xa6, 0xe0, 0xb0, 0x2e, + 0xa9, 0x35, 0xc6, 0xcd, 0xdc, 0x8d, 0x92, 0x51, 0x0f, 0xee, 0x72, 0x15, 0x31, 0xdb, 0x33, 0x35, + 0xf2, 0x55, 0x6a, 0x6e, 0x33, 0x89, 0xc8, 0x13, 0xe8, 0x14, 0x8b, 0xa4, 0xa7, 0x99, 0x42, 0x49, + 0x29, 0x3f, 0xcd, 0x14, 0xca, 0x4a, 0xa5, 0xf9, 0x4d, 0x1a, 0x14, 0xe6, 0x5f, 0x1d, 0xec, 0x1b, + 0x9e, 0x35, 0x0e, 0x5c, 0x2f, 0xf4, 0x0a, 0xe9, 0x92, 0x57, 0x7c, 0x00, 0x29, 0xcb, 0xe4, 0x60, + 0x7e, 0x87, 0xfb, 0x5b, 0x8a, 0x3a, 0x58, 0xe4, 0xb9, 0x29, 0xcb, 0x44, 0xbb, 0x90, 0x09, 0xce, + 0xc6, 0x0c, 0xd0, 0x4b, 0x9b, 0x8f, 0x09, 0xe7, 0x7f, 0x9e, 0x37, 0x1e, 0x1e, 0x59, 0xc1, 0xf1, + 0xe4, 0x70, 0xcd, 0x70, 0x47, 0xeb, 0xe1, 0xad, 0x9a, 0x87, 0xd1, 0xef, 0xf5, 0xf1, 0xcb, 0x23, + 0x92, 0x9b, 0xd6, 0x89, 0xb0, 0xbf, 0x76, 0xa0, 0x52, 0x2d, 0x68, 0x05, 0x0a, 0xce, 0xc4, 0xb6, + 0x69, 0xaa, 0x21, 0xbe, 0x5e, 0x10, 0x46, 0x13, 0x54, 0x74, 0x1f, 0x4a, 0x26, 0x1e, 0xea, 0x13, + 0x3b, 0xd0, 0xf0, 0xe9, 0xd8, 0x63, 0xfe, 0xac, 0xca, 0x9c, 0xd6, 0x3d, 0x1d, 0x7b, 0xe8, 0x1e, + 0xe4, 0x8e, 0x2d, 0xd3, 0xc4, 0x0e, 0x75, 0x67, 0xa1, 0x82, 0xd3, 0xd0, 0x06, 0xd4, 0x26, 0x3e, + 0xf6, 0x35, 0x1f, 0xbf, 0x9a, 0x90, 0x0b, 0xa3, 0xe1, 0x0a, 0x34, 0x5c, 0x73, 0x3c, 0x7c, 0xaa, + 0x84, 0x61, 0x9f, 0xcf, 0x93, 0x68, 0xdc, 0x80, 0x9a, 0xfb, 0xda, 0x99, 0x92, 0x91, 0x93, 0x32, + 0x84, 0x21, 0x2e, 0x73, 0x1f, 0x4a, 0x86, 0x3b, 0x1a, 0x4f, 0x02, 0xcc, 0x36, 0x5a, 0x62, 0x1b, + 0xe5, 0x34, 0xb2, 0xd1, 0xd0, 0x48, 0x05, 0xa5, 0xf8, 0x34, 0x53, 0x28, 0x2a, 0xf0, 0x34, 0x53, + 0xc8, 0x2b, 0x85, 0xe6, 0x5f, 0xa6, 0xe0, 0x0e, 0xbb, 0xe4, 0x2d, 0x7d, 0x64, 0xd9, 0x67, 0x6f, + 0x6b, 0x30, 0xa6, 0x85, 0x1b, 0x8c, 0xee, 0x8b, 0xe2, 0x14, 0x11, 0x63, 0x48, 0x45, 0xf7, 0x45, + 0x68, 0x7d, 0x42, 0x42, 0x8f, 0x01, 0x62, 0x50, 0x96, 0xa1, 0xe7, 0x7c, 0xe7, 0xe2, 0xbc, 0x51, + 0x9c, 0x0d, 0x60, 0x45, 0x23, 0x06, 0x5b, 0x35, 0x61, 0x9d, 0x50, 0x03, 0x35, 0x51, 0x79, 0xf3, + 0x3d, 0xbe, 0xa7, 0x6a, 0x87, 0x31, 0x08, 0xf1, 0x24, 0xc0, 0x9a, 0x89, 0x49, 0x93, 0x67, 0x84, + 0x7f, 0x4a, 0xc1, 0xad, 0x9e, 0x13, 0x60, 0xcf, 0xc6, 0xfa, 0x09, 0x8e, 0x5d, 0xc7, 0xcf, 0xa0, + 0xa8, 0x3b, 0x06, 0xf6, 0x03, 0xd7, 0xf3, 0xeb, 0xd2, 0x4a, 0x7a, 0x55, 0xde, 0xf8, 0xad, 0x2b, + 0xc2, 0x78, 0x96, 0xfc, 0x5a, 0x8b, 0x0b, 0xf3, 0x9b, 0x8c, 0x94, 0x2d, 0xfd, 0xb3, 0x04, 0x05, + 0x31, 0x8b, 0x1e, 0x42, 0x61, 0x2a, 0xe3, 0xdc, 0xe6, 0xa7, 0xc9, 0x27, 0x81, 0x37, 0x1f, 0x70, + 0xb8, 0xfd, 0x6d, 0x28, 0xd0, 0xc0, 0xd5, 0x42, 0x9b, 0x2c, 0x09, 0x09, 0x1e, 0xbb, 0xf1, 0xe2, + 0x28, 0x4f, 0x79, 0x7b, 0x26, 0x6a, 0xcf, 0xaa, 0x5b, 0xd2, 0x54, 0xfe, 0xae, 0xb8, 0xbf, 0xfd, + 0x64, 0xe5, 0x72, 0xa9, 0x94, 0x61, 0x77, 0xc6, 0x6f, 0xee, 0xff, 0xd2, 0x70, 0x67, 0x4f, 0xf7, + 0x02, 0x8b, 0x20, 0xa0, 0xe5, 0x1c, 0xc5, 0xee, 0xee, 0x01, 0xc8, 0xce, 0x64, 0xc4, 0xed, 0xe4, + 0xf3, 0x73, 0xb1, 0x7b, 0x00, 0x67, 0x32, 0x62, 0x26, 0xf0, 0x49, 0x80, 0xdb, 0x96, 0x1f, 0xd0, + 0x14, 0x29, 0x6f, 0x6c, 0x5c, 0x71, 0xbb, 0xb3, 0xd7, 0x58, 0xdb, 0xb5, 0xfc, 0x40, 0x78, 0x29, + 0xd1, 0x82, 0x06, 0x90, 0xf5, 0x74, 0xe7, 0x08, 0x53, 0xb7, 0x93, 0x37, 0x1e, 0xdd, 0x4c, 0x9d, + 0x4a, 0x44, 0x05, 0xec, 0x52, 0x3d, 0x4b, 0x7f, 0x2b, 0x41, 0x86, 0xac, 0x72, 0x4d, 0x64, 0xdc, + 0x81, 0xdc, 0x89, 0x6e, 0x4f, 0x30, 0x4b, 0xf3, 0x25, 0x95, 0x8f, 0xd0, 0x1f, 0x43, 0xd5, 0x9f, + 0x1c, 0x8e, 0x63, 0x4b, 0xd1, 0xab, 0x96, 0x37, 0x3e, 0xb9, 0xd1, 0xae, 0xc2, 0x8a, 0x32, 0xa9, + 0x8b, 0x19, 0x60, 0xe9, 0x15, 0x64, 0xe9, 0xae, 0xaf, 0xd9, 0xdf, 0x7d, 0x28, 0x05, 0xae, 0x86, + 0x4f, 0x0d, 0x7b, 0xe2, 0x5b, 0x27, 0x98, 0xfa, 0x4b, 0x49, 0x95, 0x03, 0xb7, 0x2b, 0x48, 0xe8, + 0x01, 0x54, 0x86, 0x9e, 0x3b, 0xd2, 0x2c, 0x47, 0x30, 0x51, 0xbc, 0x55, 0xcb, 0x84, 0xda, 0x13, + 0xc4, 0x84, 0xe5, 0xff, 0x05, 0xa0, 0x4a, 0xfd, 0x6b, 0x2e, 0xf4, 0x78, 0x10, 0x43, 0x8f, 0xdb, + 0x09, 0xf4, 0x08, 0x9d, 0x94, 0x80, 0xc7, 0x3d, 0xc8, 0x4d, 0x1c, 0xeb, 0xd5, 0x84, 0xad, 0x1f, + 0x42, 0x2b, 0xa3, 0x5d, 0x82, 0x96, 0xcc, 0x65, 0x68, 0xf9, 0x18, 0x10, 0x89, 0x28, 0xac, 0x25, + 0x18, 0xb3, 0x94, 0x51, 0xa1, 0x33, 0xed, 0x2b, 0x81, 0x28, 0x77, 0x03, 0x20, 0xda, 0x06, 0x05, + 0x9f, 0x06, 0x9e, 0x1e, 0xaf, 0xc9, 0xf2, 0x54, 0x7e, 0xf9, 0xe2, 0xbc, 0x51, 0xe9, 0x92, 0xb9, + 0xd9, 0x4a, 0x2a, 0x38, 0x36, 0x67, 0x12, 0x2f, 0xa9, 0x71, 0x1d, 0xa6, 0xe5, 0x61, 0x5a, 0x49, + 0xf8, 0xf5, 0xc2, 0x4a, 0xfa, 0x9a, 0x8a, 0x61, 0xea, 0xda, 0xd7, 0x3a, 0x42, 0x50, 0x55, 0x98, + 0xaa, 0x90, 0xe0, 0xa3, 0x7d, 0x90, 0x87, 0xac, 0xc0, 0xd0, 0x5e, 0xe2, 0x33, 0x5a, 0x8a, 0xc8, + 0x1b, 0x3f, 0x99, 0xbf, 0x14, 0xd9, 0xcc, 0x11, 0x13, 0xd4, 0x25, 0x15, 0x86, 0xe1, 0x24, 0x7a, + 0x01, 0xe5, 0x58, 0x05, 0x71, 0x78, 0x46, 0xf3, 0xdb, 0x9b, 0xa9, 0x2d, 0x45, 0x8a, 0x36, 0xcf, + 0xd0, 0x17, 0x00, 0x56, 0x88, 0xa3, 0xb4, 0x28, 0x91, 0x37, 0x3e, 0xba, 0x01, 0xe0, 0x0a, 0x7c, + 0x89, 0x94, 0xa0, 0x17, 0x50, 0x89, 0x46, 0x74, 0xb3, 0xa5, 0x1b, 0x6f, 0x96, 0x69, 0x2d, 0xc7, + 0xf4, 0x6c, 0x92, 0x52, 0xf4, 0x16, 0x49, 0xb6, 0xae, 0x6f, 0x05, 0x38, 0xee, 0x06, 0x65, 0xea, + 0x06, 0xcd, 0x8b, 0xf3, 0x06, 0x6a, 0x8b, 0xf9, 0xd9, 0xae, 0x80, 0x8c, 0xa9, 0x79, 0xe6, 0x58, + 0x09, 0x07, 0x26, 0x1a, 0x2b, 0x91, 0x63, 0xed, 0x47, 0x2e, 0x7c, 0xc9, 0xb1, 0x62, 0xee, 0xcd, + 0x7a, 0x87, 0x52, 0x02, 0x7b, 0xaa, 0x6f, 0x8e, 0x3d, 0x09, 0x45, 0xa8, 0xcb, 0x4b, 0x32, 0x85, + 0x96, 0xb5, 0x1f, 0xcd, 0xe9, 0xa4, 0x07, 0x67, 0x63, 0x71, 0x91, 0xac, 0x16, 0x7b, 0x04, 0xc8, + 0xf0, 0xb0, 0x1e, 0x60, 0x93, 0x14, 0x30, 0xb6, 0x65, 0x58, 0x81, 0x7d, 0x56, 0xaf, 0xc5, 0xe2, + 0xbe, 0xc6, 0xe7, 0xbb, 0xe1, 0x34, 0x7a, 0x0c, 0xf9, 0x13, 0xec, 0xf9, 0x96, 0xeb, 0xd4, 0x11, + 0x05, 0x93, 0x65, 0x5e, 0xb2, 0xde, 0x99, 0x5a, 0xef, 0x4b, 0xc6, 0xa5, 0x0a, 0x76, 0xb4, 0x0d, + 0x65, 0xec, 0x18, 0xae, 0x69, 0x39, 0x47, 0x1a, 0xdd, 0xfe, 0x62, 0x54, 0x36, 0x7c, 0x7f, 0xde, + 0x78, 0x77, 0x4a, 0xbe, 0xcb, 0x79, 0xc9, 0xb6, 0xd5, 0x12, 0x8e, 0x8d, 0x9a, 0xcb, 0x50, 0x0c, + 0x03, 0x0c, 0xe5, 0x21, 0xdd, 0xda, 0x6f, 0xb3, 0xc6, 0xb3, 0xd3, 0xdd, 0x6f, 0x2b, 0x52, 0xf3, + 0x3e, 0x64, 0x08, 0x1f, 0x69, 0x40, 0xb7, 0x06, 0xea, 0x8b, 0x96, 0xda, 0x61, 0xcd, 0x6e, 0xaf, + 0xff, 0x65, 0x57, 0x3d, 0xe8, 0x76, 0x14, 0x01, 0xa1, 0xff, 0x9a, 0x06, 0x14, 0xf5, 0x3c, 0x07, + 0x2e, 0xef, 0x1b, 0x8e, 0xa0, 0x6a, 0x84, 0x54, 0xb6, 0x57, 0x69, 0x25, 0xb5, 0x5a, 0xd9, 0x78, + 0xfc, 0x6b, 0xfb, 0x26, 0xa1, 0x23, 0x4e, 0x8a, 0xee, 0xbd, 0x62, 0x24, 0xa8, 0x21, 0x5c, 0xa7, + 0x56, 0x52, 0x53, 0x70, 0xad, 0x42, 0xd6, 0x38, 0xc6, 0xc6, 0x4b, 0x9e, 0xb0, 0x7e, 0xe7, 0x8a, + 0x85, 0x69, 0x71, 0x12, 0xb3, 0x71, 0x9b, 0xc8, 0x44, 0x4b, 0x8b, 0x4c, 0x4a, 0x55, 0x21, 0x35, + 0x89, 0x44, 0x99, 0x6b, 0x83, 0x7b, 0x56, 0x7f, 0x2e, 0x82, 0x3b, 0x06, 0x44, 0x8f, 0xa1, 0xea, + 0xb8, 0x81, 0x46, 0xaa, 0x77, 0x1e, 0x30, 0xb4, 0x26, 0x2f, 0x6f, 0x2a, 0xdc, 0xac, 0x51, 0x78, + 0x94, 0x1d, 0x37, 0xe8, 0x4f, 0x6c, 0x9b, 0x11, 0x9a, 0x9f, 0x41, 0x25, 0x79, 0x47, 0xa8, 0x08, + 0xd9, 0xf6, 0x76, 0xb7, 0xbd, 0xa3, 0x2c, 0xa0, 0x2a, 0xc8, 0x5b, 0x03, 0xb5, 0xdb, 0x7b, 0xd2, + 0xd7, 0x76, 0xba, 0x7f, 0xc8, 0x1e, 0x27, 0xfa, 0x03, 0xf1, 0x38, 0x11, 0xd6, 0xd5, 0x59, 0x25, + 0xd7, 0xfc, 0x47, 0x09, 0x2a, 0x7b, 0x9e, 0x35, 0xd2, 0xbd, 0xb3, 0x1d, 0x7c, 0xb6, 0xff, 0x5a, + 0x1f, 0xa3, 0xcf, 0xe1, 0x96, 0x83, 0x5f, 0x6b, 0x63, 0x46, 0xd5, 0xc2, 0x3a, 0x4d, 0x9a, 0xfd, + 0x72, 0x55, 0x73, 0xf0, 0x6b, 0xae, 0xa1, 0xc7, 0xcb, 0xb4, 0x8f, 0x41, 0x76, 0x6d, 0xde, 0x9a, + 0x61, 0xf1, 0x7a, 0x20, 0xc7, 0x85, 0xc0, 0xb5, 0x59, 0x27, 0x46, 0x73, 0x9e, 0x4c, 0xd6, 0x13, + 0xdc, 0xe9, 0x19, 0xdc, 0x0e, 0x7e, 0xcd, 0xb9, 0xb9, 0xeb, 0xfd, 0x5b, 0x16, 0x50, 0x64, 0xb8, + 0x67, 0x93, 0x40, 0xa7, 0xde, 0xdc, 0x82, 0x1c, 0xbf, 0x46, 0x89, 0x9a, 0xe7, 0xc3, 0x2b, 0x3d, + 0x2e, 0xd9, 0xe8, 0x6d, 0x2f, 0xa8, 0x5c, 0x10, 0xfd, 0x34, 0xfe, 0x50, 0x27, 0x6f, 0x7c, 0x30, + 0x1f, 0x3c, 0x6c, 0x2f, 0x88, 0x17, 0xbc, 0x1d, 0xc8, 0xfa, 0x01, 0xe9, 0xc3, 0xd3, 0x14, 0x5e, + 0xd6, 0xaf, 0x90, 0xbf, 0xbc, 0xf9, 0xb5, 0x7d, 0x22, 0x26, 0x7c, 0x8e, 0xea, 0x40, 0x2f, 0xa0, + 0x18, 0x66, 0x55, 0xfe, 0xea, 0xf7, 0x68, 0x7e, 0x85, 0x61, 0x94, 0x8b, 0xf2, 0x3d, 0xd4, 0x85, + 0x5a, 0x20, 0x8f, 0x38, 0x5b, 0xd4, 0x82, 0xac, 0xf0, 0xc2, 0x06, 0x84, 0x06, 0x5a, 0xe0, 0xc4, + 0x46, 0x2a, 0x08, 0xa1, 0x9e, 0x49, 0x7a, 0x51, 0xcf, 0xb5, 0xed, 0x43, 0xdd, 0x78, 0x49, 0x5f, + 0x32, 0xc2, 0x5e, 0x54, 0x50, 0xd1, 0x0e, 0x29, 0x4f, 0x84, 0x8f, 0xd2, 0x77, 0x09, 0x79, 0x8e, + 0xb7, 0x13, 0x81, 0x01, 0xdb, 0x0b, 0x6a, 0x4c, 0x1c, 0x0d, 0xa0, 0x32, 0x4e, 0xf8, 0x29, 0xaf, + 0x05, 0x1e, 0x5c, 0x95, 0x10, 0x12, 0xcc, 0xdb, 0x0b, 0xea, 0x94, 0x78, 0xf3, 0x73, 0xc8, 0xd2, + 0x1b, 0x27, 0x38, 0xf7, 0xbc, 0xbf, 0xd3, 0x1f, 0xbc, 0xe8, 0xb3, 0xd0, 0xe9, 0x74, 0x77, 0xbb, + 0x07, 0x5d, 0x6d, 0xd0, 0xdf, 0x25, 0xa1, 0xf3, 0x0e, 0xdc, 0xe6, 0x84, 0x56, 0xbf, 0xa3, 0xbd, + 0x50, 0x7b, 0x62, 0x2a, 0xd5, 0x5c, 0x8d, 0x03, 0x69, 0x01, 0x32, 0xfd, 0x41, 0xbf, 0xab, 0x2c, + 0x50, 0x48, 0xed, 0x74, 0x14, 0x89, 0x42, 0xaa, 0x3a, 0xd8, 0x13, 0x11, 0xb7, 0x59, 0x02, 0x30, + 0x43, 0x2b, 0x3d, 0xcd, 0x14, 0x72, 0x4a, 0xbe, 0xf9, 0x5f, 0x4d, 0xa8, 0x4e, 0xc1, 0xd0, 0x35, + 0x65, 0xe8, 0x0a, 0x2d, 0x43, 0xd3, 0x11, 0x44, 0x84, 0x65, 0x68, 0x8a, 0x57, 0xa0, 0x8f, 0xa0, + 0x38, 0xd6, 0x3d, 0xec, 0x04, 0xd1, 0x73, 0x98, 0xe8, 0x76, 0x0b, 0x7b, 0x74, 0x22, 0x64, 0x2f, + 0x30, 0xc6, 0x1e, 0x11, 0x0a, 0xb3, 0x12, 0xf3, 0x84, 0x77, 0x78, 0x90, 0xd7, 0xae, 0x49, 0x48, + 0x7b, 0x50, 0x1b, 0xb9, 0xa6, 0x35, 0xb4, 0x0c, 0xe6, 0x46, 0x81, 0x35, 0x62, 0x4f, 0x5a, 0xf2, + 0xc6, 0x8f, 0x62, 0x36, 0x99, 0x04, 0x96, 0xbd, 0x76, 0x6c, 0x1b, 0x6b, 0x07, 0xe2, 0xf9, 0x9d, + 0x9f, 0x48, 0x89, 0x4b, 0x93, 0x49, 0xf4, 0x04, 0xf2, 0xa2, 0xdb, 0x2a, 0xd0, 0x1a, 0x67, 0xde, + 0xf0, 0xe5, 0x1a, 0x85, 0x34, 0xda, 0x82, 0x8a, 0x83, 0x4f, 0xe3, 0x3d, 0x76, 0x31, 0xe1, 0xe0, + 0xa5, 0x3e, 0x3e, 0x9d, 0xdd, 0x60, 0x97, 0x9c, 0x68, 0xc6, 0x44, 0x5f, 0x40, 0x39, 0x81, 0x82, + 0xf4, 0xb5, 0x6a, 0x6e, 0x4c, 0x08, 0x8b, 0x8f, 0x18, 0x38, 0xa2, 0x2d, 0xc8, 0x0b, 0xa0, 0x93, + 0xe9, 0x19, 0x6f, 0xa6, 0x4c, 0x08, 0xa3, 0x4d, 0x28, 0xd3, 0x23, 0x86, 0xe8, 0x5c, 0x8a, 0xca, + 0x89, 0x8b, 0xf3, 0x86, 0x4c, 0x4e, 0x38, 0xa3, 0x93, 0x96, 0x9d, 0x90, 0x6e, 0xa2, 0xa7, 0x00, + 0xe1, 0x67, 0x0f, 0x52, 0xf7, 0x5d, 0x57, 0x5a, 0xef, 0x09, 0xc6, 0x68, 0x4b, 0x6a, 0x4c, 0x1a, + 0x3d, 0x83, 0xa2, 0xc0, 0x06, 0x56, 0xf0, 0x5d, 0x1d, 0xea, 0x97, 0x91, 0x4a, 0xe0, 0x53, 0xa8, + 0x01, 0xf5, 0x21, 0x6b, 0x63, 0xdd, 0xc7, 0xbc, 0xea, 0x7b, 0x3c, 0x67, 0x02, 0xdf, 0x37, 0x8e, + 0xf1, 0x48, 0x6f, 0x1f, 0x93, 0x8e, 0x72, 0x97, 0xc8, 0xab, 0x4c, 0x0d, 0xea, 0x83, 0x42, 0xaf, + 0x2b, 0x0e, 0x7a, 0x0a, 0xbd, 0xb1, 0xf7, 0xf9, 0x8d, 0x55, 0xc8, 0x8d, 0x5d, 0x09, 0x7c, 0xd4, + 0x9f, 0x9e, 0x45, 0xe0, 0xf7, 0x07, 0x50, 0x19, 0xba, 0xde, 0x48, 0x0f, 0x34, 0x11, 0x38, 0xb5, + 0xa8, 0x37, 0xfc, 0xfe, 0xbc, 0x51, 0xde, 0xa2, 0xb3, 0x22, 0x68, 0xca, 0xc3, 0xf8, 0x10, 0x6d, + 0x8b, 0x1c, 0xb1, 0x48, 0x21, 0xfd, 0xe3, 0x79, 0x4f, 0x77, 0x39, 0x41, 0xf4, 0x21, 0x47, 0xab, + 0x13, 0xbf, 0x7e, 0x8b, 0xde, 0xf9, 0x1b, 0x56, 0x3a, 0x2a, 0xd7, 0x82, 0x7e, 0x0e, 0x15, 0x93, + 0x50, 0x48, 0x95, 0xc9, 0x7a, 0xcf, 0xdb, 0x54, 0xef, 0xfa, 0x9c, 0x7a, 0x49, 0x5f, 0xda, 0x73, + 0x86, 0xae, 0x68, 0x39, 0x84, 0x32, 0xd6, 0xaf, 0x0e, 0xa0, 0x30, 0xd4, 0x47, 0x96, 0x6d, 0x61, + 0xbf, 0x7e, 0x87, 0xea, 0xfd, 0xe4, 0xda, 0x08, 0x9f, 0x7e, 0xde, 0x13, 0x19, 0x46, 0x28, 0x09, + 0x03, 0x9d, 0x12, 0xce, 0x88, 0x51, 0xef, 0x5e, 0x0e, 0x74, 0xf1, 0xbc, 0x97, 0x78, 0xea, 0xa3, + 0x81, 0xce, 0x47, 0x26, 0x7a, 0x0f, 0xe0, 0xc4, 0xc2, 0xaf, 0xb5, 0x57, 0x13, 0xec, 0x9d, 0xd5, + 0xeb, 0x31, 0xdc, 0x2d, 0x12, 0xfa, 0x17, 0x84, 0x8c, 0x3e, 0x85, 0xa2, 0x89, 0xc7, 0xd8, 0x31, + 0xfd, 0x81, 0x53, 0x7f, 0x87, 0x56, 0x29, 0x8b, 0xa4, 0xd9, 0xee, 0x08, 0x22, 0xc7, 0xd5, 0x88, + 0x0b, 0x7d, 0x05, 0x25, 0x36, 0xc0, 0xe6, 0xc0, 0xd9, 0x3c, 0xab, 0x2f, 0xd1, 0x43, 0x3f, 0x9c, + 0xf3, 0x32, 0xa3, 0x06, 0xee, 0x96, 0x38, 0x4f, 0x27, 0xa6, 0x4d, 0x4d, 0xe8, 0x46, 0x3f, 0x87, + 0x92, 0xf0, 0xee, 0xa7, 0xee, 0xa1, 0x5f, 0x7f, 0xf7, 0xda, 0x07, 0xa9, 0xe9, 0xb5, 0x9e, 0x45, + 0xa2, 0x02, 0xb7, 0xe2, 0xda, 0xd0, 0xcf, 0xa0, 0x1c, 0xbe, 0xee, 0xba, 0xe3, 0xc0, 0xaf, 0xdf, + 0xa3, 0x81, 0xf9, 0x68, 0x5e, 0xd7, 0xe5, 0xb2, 0x83, 0x71, 0xe0, 0xab, 0x25, 0x3f, 0x36, 0x42, + 0xf7, 0xa1, 0x68, 0x7a, 0xee, 0x98, 0xe5, 0x8f, 0x1f, 0xad, 0x48, 0xab, 0x69, 0x61, 0x66, 0x42, + 0xa6, 0x89, 0x41, 0x83, 0x8a, 0x87, 0xc7, 0xb6, 0x6e, 0xe0, 0x11, 0xc9, 0x6c, 0xee, 0xb0, 0xbe, + 0x4c, 0x57, 0xdf, 0x98, 0xfb, 0x22, 0x43, 0x61, 0xe1, 0x98, 0x31, 0x7d, 0x83, 0x21, 0x7a, 0x0e, + 0xa0, 0x4f, 0x4c, 0x2b, 0xd0, 0x46, 0xae, 0x89, 0xeb, 0x8d, 0x6b, 0xbf, 0x77, 0x4c, 0x2b, 0x6f, + 0x11, 0xc1, 0x67, 0xae, 0x89, 0xc3, 0x47, 0x52, 0x41, 0x40, 0x9f, 0x82, 0x4c, 0x8f, 0xf6, 0x95, + 0x7b, 0x48, 0x7c, 0x73, 0x85, 0x1e, 0xae, 0xc6, 0x6d, 0x59, 0xec, 0x78, 0xee, 0xf8, 0xa9, 0x7b, + 0x48, 0x3d, 0x86, 0xff, 0x34, 0x91, 0x0f, 0xa5, 0x23, 0x43, 0x8b, 0xa0, 0xf4, 0x3e, 0xb5, 0xe2, + 0xef, 0xcf, 0xb9, 0x97, 0x27, 0xed, 0x19, 0xe0, 0xba, 0x28, 0x72, 0xc2, 0x93, 0xb6, 0xa0, 0xf9, + 0xaa, 0x7c, 0x64, 0x84, 0x03, 0xf4, 0x21, 0x94, 0x58, 0xab, 0xca, 0x03, 0xa0, 0x19, 0x0b, 0x00, + 0x99, 0xcd, 0xb0, 0x10, 0xe8, 0x03, 0xef, 0x69, 0x35, 0xdd, 0xd7, 0xdc, 0x21, 0xb3, 0xd9, 0x7b, + 0xf3, 0xe7, 0xfc, 0x0a, 0x93, 0x6e, 0xf9, 0x83, 0x21, 0x35, 0xac, 0x01, 0x25, 0x77, 0x12, 0x1c, + 0xba, 0x13, 0xc7, 0xd4, 0x86, 0x2f, 0xfd, 0xfa, 0xfb, 0xf4, 0xb4, 0x37, 0x6a, 0xaa, 0xc2, 0xd3, + 0x0d, 0xb8, 0xa2, 0xad, 0x1d, 0x5f, 0x95, 0x85, 0xd6, 0xad, 0x97, 0x3e, 0xfa, 0x53, 0x90, 0x2d, + 0x27, 0x5a, 0xe3, 0xc1, 0xcd, 0xd7, 0x40, 0xa2, 0x30, 0xee, 0x39, 0xe1, 0x12, 0xc0, 0x75, 0x92, + 0x15, 0x3e, 0x82, 0x8a, 0x3b, 0x1c, 0xda, 0x96, 0x83, 0x35, 0x0f, 0xeb, 0xbe, 0xeb, 0xd4, 0x3f, + 0x88, 0xdd, 0x60, 0x99, 0xcf, 0xa9, 0x74, 0x0a, 0x35, 0xa1, 0x18, 0xe0, 0xd1, 0xd8, 0xf5, 0x74, + 0xef, 0xac, 0xfe, 0x61, 0xac, 0x70, 0x8e, 0xc8, 0xe8, 0x10, 0x96, 0x26, 0x0e, 0x3e, 0x1d, 0xbb, + 0x3e, 0x36, 0x35, 0x5e, 0xcf, 0xf9, 0x34, 0xb7, 0x11, 0x3f, 0x5a, 0xa5, 0x18, 0xf7, 0x80, 0x6f, + 0xea, 0xee, 0x73, 0xc1, 0xc9, 0xea, 0x3b, 0x96, 0x03, 0xc3, 0x2a, 0xef, 0xee, 0x64, 0xe6, 0xb4, + 0xb9, 0xf4, 0x8d, 0x04, 0xb5, 0x4b, 0xf9, 0x12, 0xfd, 0x09, 0xe4, 0x1d, 0xd7, 0x8c, 0xbd, 0xe4, + 0x77, 0xf9, 0x32, 0xb9, 0xbe, 0x6b, 0xb2, 0x87, 0xfc, 0x47, 0x73, 0x7d, 0xbc, 0xa2, 0xbf, 0xc6, + 0x87, 0x6b, 0x4c, 0x4c, 0xcd, 0x11, 0xad, 0x3d, 0x13, 0x7d, 0x02, 0x55, 0x7c, 0x3a, 0xb6, 0xbc, + 0x58, 0xcd, 0x98, 0x8a, 0xc5, 0x7c, 0x25, 0x9a, 0x24, 0x0e, 0xc2, 0x1f, 0x89, 0xff, 0x3a, 0x05, + 0xd5, 0xa9, 0x8c, 0x45, 0x8a, 0x64, 0xfa, 0xdd, 0x28, 0x51, 0x24, 0x13, 0x4a, 0xec, 0x59, 0xe0, + 0xba, 0x4f, 0xb9, 0xe9, 0xb7, 0xfd, 0x94, 0x9b, 0x7c, 0x7c, 0xcd, 0xde, 0xe0, 0xf1, 0xf5, 0xf7, + 0xe0, 0x8e, 0xe5, 0x6b, 0x8e, 0xeb, 0x88, 0xc6, 0x3f, 0xec, 0x91, 0xe2, 0x1f, 0xe4, 0x16, 0x2d, + 0xbf, 0xef, 0x3a, 0xac, 0xe5, 0x17, 0x0c, 0x61, 0xeb, 0x9e, 0x51, 0xb2, 0x4b, 0xff, 0x20, 0x41, + 0x41, 0xe4, 0xdb, 0x64, 0xdd, 0x2f, 0xcd, 0x59, 0xf7, 0x5f, 0x7d, 0x53, 0x5b, 0xa0, 0x5c, 0x72, + 0x3b, 0xd6, 0x76, 0xdc, 0x13, 0xf5, 0xd2, 0x4c, 0x6f, 0xab, 0x8c, 0x13, 0x4e, 0xc6, 0xed, 0xf7, + 0xb5, 0x04, 0xc5, 0xf8, 0x9f, 0x65, 0x52, 0xe1, 0x1e, 0x67, 0x37, 0x31, 0x6f, 0xf8, 0x75, 0x28, + 0x69, 0x91, 0xf4, 0xfc, 0x16, 0xe1, 0xdb, 0xfc, 0x33, 0x90, 0x63, 0x69, 0x70, 0xba, 0x47, 0x96, + 0xde, 0xa0, 0x47, 0x7e, 0x1f, 0x72, 0x1c, 0xfb, 0x99, 0x93, 0x97, 0xb9, 0x74, 0x96, 0xe1, 0x7e, + 0xf6, 0x2b, 0x82, 0xf9, 0x7c, 0xf5, 0x7f, 0x4f, 0x43, 0x29, 0x9e, 0x26, 0x09, 0x50, 0x58, 0x8e, + 0xe1, 0xd1, 0x1c, 0x45, 0x57, 0x17, 0x41, 0x12, 0x91, 0x49, 0xf2, 0x1c, 0x59, 0x8e, 0x46, 0xbf, + 0xd8, 0x24, 0x02, 0xa9, 0x30, 0xb2, 0x9c, 0x2f, 0x09, 0x95, 0xb2, 0xe8, 0xa7, 0x9c, 0x25, 0x9d, + 0x60, 0xd1, 0x4f, 0x19, 0xcb, 0x12, 0xad, 0x47, 0xbd, 0x80, 0x36, 0x8c, 0xe9, 0x58, 0x85, 0xe9, + 0x05, 0x68, 0x19, 0xf2, 0x27, 0x96, 0x17, 0x4c, 0x74, 0x9b, 0xf6, 0x86, 0xc2, 0x3b, 0x05, 0x11, + 0x39, 0x50, 0x89, 0x0a, 0x83, 0xd7, 0x0e, 0xf6, 0xa8, 0x13, 0xcb, 0x1b, 0xad, 0x37, 0xa8, 0x0c, + 0xa2, 0x01, 0x51, 0x24, 0xe0, 0xd3, 0x8f, 0x13, 0x97, 0xfe, 0x4e, 0x82, 0x72, 0x82, 0x0d, 0xf5, + 0xa0, 0x4a, 0x17, 0x8e, 0xb5, 0x7b, 0xcc, 0x56, 0xf7, 0xc3, 0xbf, 0xbd, 0x90, 0xe9, 0x99, 0xfd, + 0x5e, 0xd9, 0x8d, 0x4d, 0x99, 0xe8, 0x73, 0xa8, 0x30, 0x55, 0xe1, 0xe7, 0xcc, 0xa4, 0xfb, 0x95, + 0xa8, 0xa6, 0xe4, 0x37, 0xcd, 0x92, 0x1b, 0xd1, 0xcc, 0xf8, 0x27, 0xa6, 0x25, 0x07, 0xe4, 0x58, + 0xe5, 0x31, 0x87, 0xdf, 0xff, 0x2e, 0x64, 0x42, 0x44, 0x9c, 0x33, 0xa3, 0x52, 0x01, 0xbe, 0xde, + 0xdf, 0x48, 0x70, 0x6b, 0x56, 0x05, 0x90, 0x88, 0x27, 0xe6, 0x48, 0x73, 0xc5, 0x53, 0xa2, 0x32, + 0x4b, 0xcd, 0xac, 0xcc, 0x22, 0x07, 0x4f, 0xff, 0x3a, 0x07, 0x6f, 0x3e, 0x12, 0x0f, 0x2e, 0x00, + 0xb9, 0xbd, 0xe7, 0x9b, 0xbb, 0xbd, 0xf6, 0xcc, 0xc7, 0x12, 0x24, 0x43, 0x7e, 0xb0, 0xb5, 0xb5, + 0xdb, 0xeb, 0x77, 0x95, 0x74, 0x73, 0x15, 0x8a, 0x61, 0x81, 0x85, 0x4a, 0x50, 0xe8, 0xf4, 0xf6, + 0x5b, 0x9b, 0xbb, 0xdd, 0x8e, 0xb2, 0x80, 0xca, 0x50, 0x54, 0xbb, 0xad, 0x0e, 0x7d, 0x92, 0x51, + 0xa4, 0xcf, 0x0a, 0x7f, 0xf1, 0x8b, 0x86, 0xc4, 0xe1, 0x31, 0xa7, 0xe4, 0x9f, 0x66, 0x0a, 0x48, + 0x59, 0x6c, 0xfe, 0xbd, 0x04, 0xa8, 0xa3, 0x07, 0x3a, 0xf1, 0xbd, 0x1b, 0x3c, 0xb1, 0xa4, 0xae, + 0xb1, 0x52, 0xb2, 0x6d, 0x4e, 0xbf, 0x4d, 0xdb, 0x1c, 0x6d, 0xba, 0xf9, 0xb5, 0x04, 0x10, 0xdb, + 0xe0, 0x4f, 0xe3, 0xff, 0x28, 0xbc, 0xfa, 0x95, 0x60, 0x2a, 0x9a, 0xb6, 0x17, 0xc4, 0xff, 0x0d, + 0x9f, 0x40, 0xc1, 0xe4, 0xc7, 0xe6, 0xee, 0x74, 0x65, 0x3b, 0x7e, 0xe9, 0x76, 0xb6, 0x89, 0x85, + 0x39, 0x95, 0x3f, 0x5d, 0xe5, 0x21, 0x3b, 0x71, 0x2c, 0xd7, 0xf9, 0x89, 0x1a, 0x7f, 0xf2, 0x17, + 0xb9, 0x91, 0x98, 0x82, 0xfe, 0xd6, 0x03, 0x6c, 0xb2, 0x47, 0xb4, 0xe7, 0xce, 0x49, 0x48, 0x90, + 0x50, 0x05, 0x80, 0xcf, 0x5b, 0xce, 0x91, 0x92, 0xa2, 0x86, 0xf4, 0xdc, 0xf1, 0x98, 0x8c, 0xd2, + 0x9b, 0x3f, 0xfe, 0xf6, 0x7f, 0x96, 0x17, 0xbe, 0xbd, 0x58, 0x96, 0x7e, 0x79, 0xb1, 0x2c, 0xfd, + 0xea, 0x62, 0x59, 0xfa, 0xef, 0x8b, 0x65, 0xe9, 0xaf, 0xbe, 0x5b, 0x5e, 0xf8, 0xe5, 0x77, 0xcb, + 0x0b, 0xbf, 0xfa, 0x6e, 0x79, 0xe1, 0x8f, 0xf2, 0x7c, 0xb3, 0xff, 0x1f, 0x00, 0x00, 0xff, 0xff, + 0x2f, 0xe5, 0x73, 0xb1, 0x03, 0x2a, 0x00, 0x00, } diff --git a/pkg/sql/sqlbase/structured.proto b/pkg/sql/sqlbase/structured.proto index d88a798434d8..c67a997d0750 100644 --- a/pkg/sql/sqlbase/structured.proto +++ b/pkg/sql/sqlbase/structured.proto @@ -93,13 +93,17 @@ message ForeignKeyConstraint { // fields on the index that they use. In versions 19.2 and later, we preserve // the semantics of the older FKs which were tied to indexes by specifying // the index as a field on this proto, since the migration process to have - // top-level FK fields on the table descriptor requires two releases. In 20.1, - // when all 19.2 nodes will be correctly handling the new FK representation, - // we will perform a migration to upgrade all table descriptors. - optional uint32 legacy_origin_index = 10 [(gogoproto.nullable) = false, (gogoproto.casttype) = "IndexID"]; + // top-level FK fields on the table descriptor requires two releases. + // In 20.1, these fields are no longer read, but must continue to be written + // to maintain compatibility in mixed 19.2/20.1 clusters. In 20.2 these fields + // can _finally_ be removed. + // * When using the foreign key constraint, do not read from these fields! * + optional uint32 legacy_origin_index = 10 + [(gogoproto.nullable) = false, (gogoproto.casttype) = "IndexID", deprecated = true]; // LegacyReferencedIndex is the ID of the index used for the FK on the // referenced side. See the comment for LegacyOriginIndex. - optional uint32 legacy_referenced_index = 11 [(gogoproto.nullable) = false, (gogoproto.casttype) = "IndexID"]; + optional uint32 legacy_referenced_index = 11 + [(gogoproto.nullable) = false, (gogoproto.casttype) = "IndexID", deprecated = true]; // These fields were used for the 19.1 -> 19.2 foreign key migration. reserved 12, 13; } diff --git a/pkg/sql/sqlbase/table.go b/pkg/sql/sqlbase/table.go index 9175bc708e17..37de7043501d 100644 --- a/pkg/sql/sqlbase/table.go +++ b/pkg/sql/sqlbase/table.go @@ -416,6 +416,18 @@ func (desc *TableDescriptor) collectConstraintInfo( return info, nil } +// IsValidOriginIndex returns whether the index can serve as an origin index for a foreign +// key constraint with the provided set of originColIDs. +func (idx *IndexDescriptor) IsValidOriginIndex(originColIDs ColumnIDs) bool { + return ColumnIDs(idx.ColumnIDs).HasPrefix(originColIDs) +} + +// IsValidReferencedIndex returns whether the index can serve as a referenced index for a foreign +// key constraint with the provided set of referencedColumnIDs. +func (idx *IndexDescriptor) IsValidReferencedIndex(referencedColIDs ColumnIDs) bool { + return idx.Unique && ColumnIDs(idx.ColumnIDs).Equals(referencedColIDs) +} + // FindFKReferencedIndex finds the first index in the supplied referencedTable // that can satisfy a foreign key of the supplied column ids. func FindFKReferencedIndex( @@ -423,13 +435,14 @@ func FindFKReferencedIndex( ) (*IndexDescriptor, error) { // Search for a unique index on the referenced table that matches our foreign // key columns. - if ColumnIDs(referencedTable.PrimaryIndex.ColumnIDs).Equals(referencedColIDs) { + if referencedTable.PrimaryIndex.IsValidReferencedIndex(referencedColIDs) { return &referencedTable.PrimaryIndex, nil } // If the PK doesn't match, find the index corresponding to the referenced column. - for _, idx := range referencedTable.Indexes { - if idx.Unique && ColumnIDs(idx.ColumnIDs).Equals(referencedColIDs) { - return &idx, nil + for i := range referencedTable.Indexes { + idx := &referencedTable.Indexes[i] + if idx.IsValidReferencedIndex(referencedColIDs) { + return idx, nil } } return nil, pgerror.Newf( @@ -446,13 +459,14 @@ func FindFKOriginIndex( ) (*IndexDescriptor, error) { // Search for an index on the origin table that matches our foreign // key columns. - if ColumnIDs(originTable.PrimaryIndex.ColumnIDs).HasPrefix(originColIDs) { + if originTable.PrimaryIndex.IsValidOriginIndex(originColIDs) { return &originTable.PrimaryIndex, nil } // If the PK doesn't match, find the index corresponding to the origin column. - for _, idx := range originTable.Indexes { - if ColumnIDs(idx.ColumnIDs).HasPrefix(originColIDs) { - return &idx, nil + for i := range originTable.Indexes { + idx := &originTable.Indexes[i] + if idx.IsValidOriginIndex(originColIDs) { + return idx, nil } } return nil, pgerror.Newf(