From d8ac1f1674089c7212c802598ae6f6a895fb705c Mon Sep 17 00:00:00 2001 From: David Taylor Date: Wed, 12 Aug 2020 20:56:49 +0000 Subject: [PATCH 1/9] importccl: speed up revert of IMPORT INTO empty table When IMPORT INTO fails, it reverts the tables to their pre-IMPORT state. Typically this requires running a somewhat expensive RevertRange operation that finds the keys written by the IMPORT in amongst all the table data and deletes just those keys. This is somewhat expensive -- we need to iterate the keys in the target table and check them to see if they need to be reverted. Non-INTO style IMPORTs create the table into which they will IMPORT and thus can just drop it wholesale on failure, instead of doing this expensive revert. However INTO-style IMPORTs could use a similarly fast/cheap wholesale delete *if they knew the table was empty* when the IMPORT was started. This change tracks which tables were empty when the IMPORT started and then deletes, rather than reverts, the table span on failure. Release note (performance improvement): Cleaning up after a failure during IMPORT INTO a table which was empty is now faster. --- pkg/ccl/importccl/import_stmt.go | 36 +- pkg/jobs/jobspb/jobs.pb.go | 613 ++++++++++++---------- pkg/jobs/jobspb/jobs.proto | 1 + pkg/sql/gcjob/table_garbage_collection.go | 6 +- 4 files changed, 362 insertions(+), 294 deletions(-) diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index abd6d9483d0b..4e899391b83c 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/gcjob" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" @@ -1196,12 +1197,30 @@ func (r *importResumer) Resume( if details.Walltime == 0 { // TODO(dt): update job status to mention waiting for tables to go offline. for _, i := range details.Tables { - if _, err := p.ExecCfg().LeaseManager.WaitForOneVersion(ctx, i.Desc.ID, retry.Options{}); err != nil { - return err + if !i.IsNew { + if _, err := p.ExecCfg().LeaseManager.WaitForOneVersion(ctx, i.Desc.ID, retry.Options{}); err != nil { + return err + } } } + // Now that we know all the tables are offline, pick a walltime at which we + // will write. details.Walltime = p.ExecCfg().Clock.Now().WallTime + + // Check if the tables being imported into are starting empty, in which + // case we can cheaply clear-range instead of revert-range to cleanup. + for i := range details.Tables { + if !details.Tables[i].IsNew { + tblSpan := sqlbase.NewImmutableTableDescriptor(*details.Tables[i].Desc).TableSpan(keys.TODOSQLCodec) + res, err := p.ExecCfg().DB.Scan(ctx, tblSpan.Key, tblSpan.EndKey, 1 /* maxRows */) + if err != nil { + return errors.Wrap(err, "checking if existing table is empty") + } + details.Tables[i].WasEmpty = len(res) == 0 + } + } + if err := r.job.WithTxn(nil).SetDetails(ctx, details); err != nil { return err } @@ -1422,9 +1441,14 @@ func (r *importResumer) dropTables( } var revert []*sqlbase.ImmutableTableDescriptor + var empty []*sqlbase.ImmutableTableDescriptor for _, tbl := range details.Tables { if !tbl.IsNew { - revert = append(revert, sqlbase.NewImmutableTableDescriptor(*tbl.Desc)) + if tbl.WasEmpty { + empty = append(empty, sqlbase.NewImmutableTableDescriptor(*tbl.Desc)) + } else { + revert = append(revert, sqlbase.NewImmutableTableDescriptor(*tbl.Desc)) + } } } @@ -1447,6 +1471,12 @@ func (r *importResumer) dropTables( } } + for i := range empty { + if err := gcjob.ClearTableData(ctx, execCfg.DB, execCfg.DistSender, execCfg.Codec, empty[i]); err != nil { + return errors.Wrapf(err, "clearing data for table %d", empty[i].ID) + } + } + b := txn.NewBatch() dropTime := int64(1) tablesToGC := make([]descpb.ID, 0, len(details.Tables)) diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 4d71d6e0a7ad..17bb21f07dc5 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -54,7 +54,7 @@ func (x EncryptionMode) String() string { return proto.EnumName(EncryptionMode_name, int32(x)) } func (EncryptionMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{0} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{0} } type Status int32 @@ -83,7 +83,7 @@ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } func (Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{1} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{1} } type Type int32 @@ -129,7 +129,7 @@ var Type_value = map[string]int32{ } func (Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{2} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{2} } type SchemaChangeGCProgress_Status int32 @@ -159,7 +159,7 @@ func (x SchemaChangeGCProgress_Status) String() string { return proto.EnumName(SchemaChangeGCProgress_Status_name, int32(x)) } func (SchemaChangeGCProgress_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{15, 0} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{15, 0} } type Lease struct { @@ -173,7 +173,7 @@ func (m *Lease) Reset() { *m = Lease{} } func (m *Lease) String() string { return proto.CompactTextString(m) } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{0} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{0} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -214,7 +214,7 @@ func (m *BackupEncryptionOptions) Reset() { *m = BackupEncryptionOptions func (m *BackupEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*BackupEncryptionOptions) ProtoMessage() {} func (*BackupEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{1} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{1} } func (m *BackupEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -248,7 +248,7 @@ func (m *BackupEncryptionOptions_KMSInfo) Reset() { *m = BackupEncryptio func (m *BackupEncryptionOptions_KMSInfo) String() string { return proto.CompactTextString(m) } func (*BackupEncryptionOptions_KMSInfo) ProtoMessage() {} func (*BackupEncryptionOptions_KMSInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{1, 0} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{1, 0} } func (m *BackupEncryptionOptions_KMSInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -302,7 +302,7 @@ func (m *BackupDetails) Reset() { *m = BackupDetails{} } func (m *BackupDetails) String() string { return proto.CompactTextString(m) } func (*BackupDetails) ProtoMessage() {} func (*BackupDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{2} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{2} } func (m *BackupDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -334,7 +334,7 @@ func (m *BackupProgress) Reset() { *m = BackupProgress{} } func (m *BackupProgress) String() string { return proto.CompactTextString(m) } func (*BackupProgress) ProtoMessage() {} func (*BackupProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{3} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{3} } func (m *BackupProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -389,7 +389,7 @@ func (m *RestoreDetails) Reset() { *m = RestoreDetails{} } func (m *RestoreDetails) String() string { return proto.CompactTextString(m) } func (*RestoreDetails) ProtoMessage() {} func (*RestoreDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{4} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{4} } func (m *RestoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -426,7 +426,7 @@ func (m *RestoreDetails_DescriptorRewrite) Reset() { *m = RestoreDetails func (m *RestoreDetails_DescriptorRewrite) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_DescriptorRewrite) ProtoMessage() {} func (*RestoreDetails_DescriptorRewrite) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{4, 0} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{4, 0} } func (m *RestoreDetails_DescriptorRewrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -459,7 +459,7 @@ func (m *RestoreDetails_BackupLocalityInfo) Reset() { *m = RestoreDetail func (m *RestoreDetails_BackupLocalityInfo) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_BackupLocalityInfo) ProtoMessage() {} func (*RestoreDetails_BackupLocalityInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{4, 1} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{4, 1} } func (m *RestoreDetails_BackupLocalityInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -493,7 +493,7 @@ func (m *RestoreDetails_Tenant) Reset() { *m = RestoreDetails_Tenant{} } func (m *RestoreDetails_Tenant) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_Tenant) ProtoMessage() {} func (*RestoreDetails_Tenant) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{4, 3} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{4, 3} } func (m *RestoreDetails_Tenant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -526,7 +526,7 @@ func (m *RestoreProgress) Reset() { *m = RestoreProgress{} } func (m *RestoreProgress) String() string { return proto.CompactTextString(m) } func (*RestoreProgress) ProtoMessage() {} func (*RestoreProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{5} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{5} } func (m *RestoreProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -585,7 +585,7 @@ func (m *ImportDetails) Reset() { *m = ImportDetails{} } func (m *ImportDetails) String() string { return proto.CompactTextString(m) } func (*ImportDetails) ProtoMessage() {} func (*ImportDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{6} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{6} } func (m *ImportDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -615,6 +615,7 @@ type ImportDetails_Table struct { Name string `protobuf:"bytes,18,opt,name=name,proto3" json:"name,omitempty"` SeqVal int64 `protobuf:"varint,19,opt,name=seq_val,json=seqVal,proto3" json:"seq_val,omitempty"` IsNew bool `protobuf:"varint,20,opt,name=is_new,json=isNew,proto3" json:"is_new,omitempty"` + WasEmpty bool `protobuf:"varint,22,opt,name=was_empty,json=wasEmpty,proto3" json:"was_empty,omitempty"` TargetCols []string `protobuf:"bytes,21,rep,name=target_cols,json=targetCols,proto3" json:"target_cols,omitempty"` } @@ -622,7 +623,7 @@ func (m *ImportDetails_Table) Reset() { *m = ImportDetails_Table{} } func (m *ImportDetails_Table) String() string { return proto.CompactTextString(m) } func (*ImportDetails_Table) ProtoMessage() {} func (*ImportDetails_Table) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{6, 0} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{6, 0} } func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -665,7 +666,7 @@ func (m *ImportProgress) Reset() { *m = ImportProgress{} } func (m *ImportProgress) String() string { return proto.CompactTextString(m) } func (*ImportProgress) ProtoMessage() {} func (*ImportProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{7} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{7} } func (m *ImportProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -699,7 +700,7 @@ func (m *TypeSchemaChangeDetails) Reset() { *m = TypeSchemaChangeDetails func (m *TypeSchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*TypeSchemaChangeDetails) ProtoMessage() {} func (*TypeSchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{8} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{8} } func (m *TypeSchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -732,7 +733,7 @@ func (m *TypeSchemaChangeProgress) Reset() { *m = TypeSchemaChangeProgre func (m *TypeSchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*TypeSchemaChangeProgress) ProtoMessage() {} func (*TypeSchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{9} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{9} } func (m *TypeSchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -765,7 +766,7 @@ func (m *ResumeSpanList) Reset() { *m = ResumeSpanList{} } func (m *ResumeSpanList) String() string { return proto.CompactTextString(m) } func (*ResumeSpanList) ProtoMessage() {} func (*ResumeSpanList) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{10} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{10} } func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -800,7 +801,7 @@ func (m *DroppedTableDetails) Reset() { *m = DroppedTableDetails{} } func (m *DroppedTableDetails) String() string { return proto.CompactTextString(m) } func (*DroppedTableDetails) ProtoMessage() {} func (*DroppedTableDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{11} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{11} } func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -858,7 +859,7 @@ func (m *SchemaChangeGCDetails) Reset() { *m = SchemaChangeGCDetails{} } func (m *SchemaChangeGCDetails) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails) ProtoMessage() {} func (*SchemaChangeGCDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{12} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{12} } func (m *SchemaChangeGCDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -892,7 +893,7 @@ func (m *SchemaChangeGCDetails_DroppedIndex) Reset() { *m = SchemaChange func (m *SchemaChangeGCDetails_DroppedIndex) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedIndex) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedIndex) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{12, 0} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{12, 0} } func (m *SchemaChangeGCDetails_DroppedIndex) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -926,7 +927,7 @@ func (m *SchemaChangeGCDetails_DroppedID) Reset() { *m = SchemaChangeGCD func (m *SchemaChangeGCDetails_DroppedID) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedID) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedID) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{12, 1} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{12, 1} } func (m *SchemaChangeGCDetails_DroppedID) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -990,7 +991,7 @@ func (m *SchemaChangeDetails) Reset() { *m = SchemaChangeDetails{} } func (m *SchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*SchemaChangeDetails) ProtoMessage() {} func (*SchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{13} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{13} } func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1022,7 +1023,7 @@ func (m *SchemaChangeProgress) Reset() { *m = SchemaChangeProgress{} } func (m *SchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeProgress) ProtoMessage() {} func (*SchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{14} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{14} } func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1058,7 +1059,7 @@ func (m *SchemaChangeGCProgress) Reset() { *m = SchemaChangeGCProgress{} func (m *SchemaChangeGCProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress) ProtoMessage() {} func (*SchemaChangeGCProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{15} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{15} } func (m *SchemaChangeGCProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1092,7 +1093,7 @@ func (m *SchemaChangeGCProgress_IndexProgress) Reset() { *m = SchemaChan func (m *SchemaChangeGCProgress_IndexProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_IndexProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_IndexProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{15, 0} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{15, 0} } func (m *SchemaChangeGCProgress_IndexProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1126,7 +1127,7 @@ func (m *SchemaChangeGCProgress_TableProgress) Reset() { *m = SchemaChan func (m *SchemaChangeGCProgress_TableProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_TableProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_TableProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{15, 1} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{15, 1} } func (m *SchemaChangeGCProgress_TableProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1159,7 +1160,7 @@ func (m *ChangefeedTarget) Reset() { *m = ChangefeedTarget{} } func (m *ChangefeedTarget) String() string { return proto.CompactTextString(m) } func (*ChangefeedTarget) ProtoMessage() {} func (*ChangefeedTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{16} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{16} } func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1212,7 +1213,7 @@ func (m *ChangefeedDetails) Reset() { *m = ChangefeedDetails{} } func (m *ChangefeedDetails) String() string { return proto.CompactTextString(m) } func (*ChangefeedDetails) ProtoMessage() {} func (*ChangefeedDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{17} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{17} } func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1247,7 +1248,7 @@ func (m *ResolvedSpan) Reset() { *m = ResolvedSpan{} } func (m *ResolvedSpan) String() string { return proto.CompactTextString(m) } func (*ResolvedSpan) ProtoMessage() {} func (*ResolvedSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{18} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{18} } func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1290,7 +1291,7 @@ func (m *ChangefeedProgress) Reset() { *m = ChangefeedProgress{} } func (m *ChangefeedProgress) String() string { return proto.CompactTextString(m) } func (*ChangefeedProgress) ProtoMessage() {} func (*ChangefeedProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{19} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{19} } func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1334,7 +1335,7 @@ func (m *CreateStatsDetails) Reset() { *m = CreateStatsDetails{} } func (m *CreateStatsDetails) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails) ProtoMessage() {} func (*CreateStatsDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{20} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{20} } func (m *CreateStatsDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1375,7 +1376,7 @@ func (m *CreateStatsDetails_ColStat) Reset() { *m = CreateStatsDetails_C func (m *CreateStatsDetails_ColStat) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails_ColStat) ProtoMessage() {} func (*CreateStatsDetails_ColStat) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{20, 0} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{20, 0} } func (m *CreateStatsDetails_ColStat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1407,7 +1408,7 @@ func (m *CreateStatsProgress) Reset() { *m = CreateStatsProgress{} } func (m *CreateStatsProgress) String() string { return proto.CompactTextString(m) } func (*CreateStatsProgress) ProtoMessage() {} func (*CreateStatsProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{21} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{21} } func (m *CreateStatsProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1474,7 +1475,7 @@ func (m *Payload) Reset() { *m = Payload{} } func (m *Payload) String() string { return proto.CompactTextString(m) } func (*Payload) ProtoMessage() {} func (*Payload) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{22} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{22} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1813,7 +1814,7 @@ func (m *Progress) Reset() { *m = Progress{} } func (m *Progress) String() string { return proto.CompactTextString(m) } func (*Progress) ProtoMessage() {} func (*Progress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{23} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{23} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2219,7 +2220,7 @@ func (m *Job) Reset() { *m = Job{} } func (m *Job) String() string { return proto.CompactTextString(m) } func (*Job) ProtoMessage() {} func (*Job) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_afeabab6bdaa84f6, []int{24} + return fileDescriptor_jobs_d49aea886d9cc8b8, []int{24} } func (m *Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3106,6 +3107,18 @@ func (m *ImportDetails_Table) MarshalTo(dAtA []byte) (int, error) { i += copy(dAtA[i:], s) } } + if m.WasEmpty { + dAtA[i] = 0xb0 + i++ + dAtA[i] = 0x1 + i++ + if m.WasEmpty { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -4791,6 +4804,9 @@ func (m *ImportDetails_Table) Size() (n int) { n += 2 + l + sovJobs(uint64(l)) } } + if m.WasEmpty { + n += 3 + } return n } @@ -7813,6 +7829,26 @@ func (m *ImportDetails_Table) Unmarshal(dAtA []byte) error { } m.TargetCols = append(m.TargetCols, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 22: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WasEmpty", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.WasEmpty = bool(v != 0) default: iNdEx = preIndex skippy, err := skipJobs(dAtA[iNdEx:]) @@ -11952,256 +11988,257 @@ var ( ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_afeabab6bdaa84f6) } +func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_d49aea886d9cc8b8) } -var fileDescriptor_jobs_afeabab6bdaa84f6 = []byte{ - // 3957 bytes of a gzipped FileDescriptorProto +var fileDescriptor_jobs_d49aea886d9cc8b8 = []byte{ + // 3975 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xcd, 0x6f, 0x23, 0x47, - 0x76, 0x57, 0x93, 0x4d, 0xb2, 0xf9, 0x28, 0x52, 0xad, 0x92, 0x66, 0x86, 0xcb, 0xcc, 0x8a, 0x0a, - 0x6d, 0xef, 0x7c, 0xd8, 0xa6, 0xbc, 0xf2, 0x66, 0xd7, 0x3b, 0x59, 0x8f, 0x57, 0xfc, 0xd0, 0x88, - 0x9c, 0xd1, 0x87, 0x9b, 0xd2, 0x78, 0xed, 0x85, 0xd3, 0x69, 0xb2, 0x4b, 0x54, 0x47, 0x64, 0x37, - 0xa7, 0xab, 0x39, 0x33, 0x5a, 0x04, 0x09, 0xb0, 0x41, 0x02, 0x63, 0x4e, 0x09, 0xb0, 0xc9, 0x21, - 0xc9, 0x04, 0x01, 0x92, 0x00, 0x39, 0x04, 0x08, 0x10, 0x04, 0x49, 0xfe, 0x80, 0x1c, 0x7c, 0xc8, + 0x76, 0x57, 0x93, 0x2d, 0xb2, 0xf9, 0x28, 0x52, 0xad, 0x92, 0x66, 0x86, 0xcb, 0xd8, 0xa2, 0x42, + 0xdb, 0x3b, 0x1f, 0xb6, 0x29, 0xaf, 0xbc, 0xd9, 0xf5, 0x4e, 0xd6, 0xe3, 0x15, 0x3f, 0x34, 0x22, + 0x67, 0xf4, 0xe1, 0xa6, 0x34, 0x5e, 0x7b, 0xe1, 0x74, 0x9a, 0xec, 0x12, 0xd5, 0x11, 0xd9, 0xdd, + 0xd3, 0xd5, 0x9c, 0x19, 0x2d, 0x82, 0x04, 0xd8, 0x20, 0xc1, 0x62, 0x4e, 0x09, 0xb0, 0xc9, 0x21, + 0xc9, 0x04, 0x01, 0x12, 0x03, 0x39, 0x04, 0x08, 0x10, 0x04, 0x49, 0xfe, 0x80, 0x1c, 0x7c, 0xc8, 0x61, 0x8f, 0x8b, 0x1c, 0xb8, 0x09, 0x7d, 0xc9, 0x21, 0x87, 0x45, 0x72, 0x9b, 0x53, 0x50, 0x1f, - 0xdd, 0x6c, 0x52, 0x1a, 0x8a, 0x1a, 0xd9, 0x9b, 0x8b, 0xd4, 0xf5, 0xaa, 0xea, 0x57, 0x5f, 0xef, - 0xfd, 0xde, 0xab, 0x57, 0x84, 0xab, 0xbf, 0xe5, 0x34, 0xc9, 0x1a, 0xfd, 0xd3, 0x6b, 0xb2, 0x7f, - 0xc5, 0x9e, 0xeb, 0x78, 0x0e, 0xfa, 0x5a, 0xcb, 0x69, 0x1d, 0xbb, 0x8e, 0xd1, 0x3a, 0x2a, 0x92, - 0x47, 0x9d, 0x22, 0xab, 0xe1, 0xad, 0x72, 0x57, 0xb0, 0xeb, 0x3a, 0x2e, 0x6d, 0xcf, 0x3f, 0x78, - 0x8f, 0xdc, 0x72, 0xdb, 0x69, 0x3b, 0xec, 0x73, 0x8d, 0x7e, 0x09, 0x29, 0x62, 0x18, 0xbd, 0xe6, - 0x9a, 0x69, 0x78, 0x86, 0x90, 0x65, 0x7d, 0x99, 0xe5, 0xbc, 0x7d, 0xe8, 0xb8, 0x5d, 0xc3, 0xf3, - 0x31, 0x5e, 0x23, 0x8f, 0x3a, 0x6b, 0x2d, 0xc3, 0x33, 0x3a, 0x4e, 0x7b, 0xcd, 0xc4, 0xa4, 0xd5, - 0x6b, 0xae, 0x11, 0xcf, 0xed, 0xb7, 0xbc, 0xbe, 0x8b, 0x4d, 0xbf, 0x7b, 0xdf, 0xb3, 0x3a, 0x6b, - 0x47, 0x9d, 0xd6, 0x9a, 0x67, 0x75, 0x31, 0xf1, 0x8c, 0x6e, 0x8f, 0xd7, 0x14, 0x7e, 0x17, 0x62, - 0x0f, 0xb0, 0x41, 0x30, 0xfa, 0x04, 0x12, 0xb6, 0x63, 0x62, 0xdd, 0x32, 0xb3, 0xd2, 0xaa, 0x74, - 0x33, 0x5d, 0xda, 0x18, 0x0e, 0xf2, 0xf1, 0x1d, 0xc7, 0xc4, 0xb5, 0xca, 0x8b, 0x41, 0xfe, 0xdd, - 0xb6, 0xe5, 0x1d, 0xf5, 0x9b, 0xc5, 0x96, 0xd3, 0x5d, 0x0b, 0xd6, 0x69, 0x36, 0x47, 0xdf, 0x6b, - 0xbd, 0xe3, 0xf6, 0x9a, 0x98, 0x65, 0x91, 0x77, 0xd3, 0xe2, 0x14, 0xb1, 0x66, 0xa2, 0x65, 0x88, - 0xe1, 0x9e, 0xd3, 0x3a, 0xca, 0x46, 0x56, 0xa5, 0x9b, 0x51, 0x8d, 0x17, 0xee, 0xc8, 0xff, 0xf5, - 0x97, 0x79, 0xa9, 0xf0, 0x37, 0x11, 0xb8, 0x56, 0x32, 0x5a, 0xc7, 0xfd, 0x5e, 0xd5, 0x6e, 0xb9, - 0x27, 0x3d, 0xcf, 0x72, 0xec, 0x5d, 0xf6, 0x97, 0x20, 0x15, 0xa2, 0xc7, 0xf8, 0x84, 0xcd, 0x67, - 0x5e, 0xa3, 0x9f, 0xe8, 0x7d, 0x90, 0xbb, 0x8e, 0x89, 0x19, 0x50, 0x66, 0xfd, 0x56, 0xf1, 0xa5, - 0x5b, 0x5e, 0x1c, 0xa1, 0x6d, 0x3b, 0x26, 0xd6, 0x58, 0x37, 0xd4, 0x04, 0xe5, 0xb8, 0x4b, 0x74, - 0xcb, 0x3e, 0x74, 0xb2, 0xd1, 0x55, 0xe9, 0x66, 0x6a, 0xfd, 0xce, 0x14, 0x88, 0x97, 0x4c, 0xab, - 0x78, 0x7f, 0xbb, 0x51, 0xb3, 0x0f, 0x9d, 0x52, 0x6a, 0x38, 0xc8, 0x27, 0x44, 0x41, 0x4b, 0x1c, - 0x77, 0x09, 0xfd, 0xc8, 0xed, 0x82, 0x2f, 0xa3, 0xf3, 0xef, 0xbb, 0x16, 0x9b, 0x7f, 0x52, 0xa3, - 0x9f, 0xe8, 0x2d, 0x40, 0x98, 0xe3, 0x61, 0x53, 0xa7, 0xe7, 0xab, 0xd3, 0x05, 0x46, 0xd8, 0x02, - 0xd5, 0xa0, 0xa6, 0x62, 0x78, 0xc6, 0x7d, 0x7c, 0xc2, 0x77, 0x48, 0xec, 0xd3, 0xbf, 0xc6, 0x20, - 0xcd, 0x27, 0x54, 0xc1, 0x9e, 0x61, 0x75, 0x08, 0x2a, 0x01, 0x10, 0xcf, 0x70, 0x3d, 0x9d, 0x9e, - 0x29, 0x1b, 0x24, 0xb5, 0xfe, 0xf5, 0xd0, 0x72, 0xe8, 0x99, 0x17, 0x8f, 0x3a, 0xad, 0xe2, 0xbe, - 0x7f, 0xe6, 0x25, 0xf9, 0xf3, 0x41, 0x7e, 0x4e, 0x4b, 0xb2, 0x6e, 0x54, 0x8a, 0xee, 0x82, 0x82, - 0x6d, 0x93, 0x23, 0x44, 0x66, 0x47, 0x48, 0x60, 0xdb, 0x64, 0xfd, 0xbf, 0xc6, 0x57, 0x48, 0xf7, - 0x32, 0x59, 0x4a, 0x0c, 0x07, 0xf9, 0xe8, 0x81, 0x56, 0xe3, 0x4b, 0xbd, 0x01, 0x0b, 0x4d, 0x36, - 0x5f, 0xbd, 0x6b, 0xd8, 0xd6, 0x21, 0x26, 0x5e, 0x56, 0x66, 0xeb, 0xcc, 0x70, 0xf1, 0xb6, 0x90, - 0xa2, 0xdf, 0x93, 0x60, 0xa9, 0xef, 0x5a, 0x44, 0x6f, 0x9e, 0xe8, 0x1d, 0xa7, 0x65, 0x74, 0x2c, - 0xef, 0x44, 0x3f, 0x7e, 0x9c, 0x8d, 0xad, 0x46, 0x6f, 0xa6, 0xd6, 0xef, 0x9e, 0x7b, 0x40, 0x62, - 0x3f, 0x8a, 0x07, 0xae, 0x45, 0x4a, 0x27, 0x0f, 0x04, 0xc2, 0xfd, 0xc7, 0x55, 0xdb, 0x73, 0x4f, - 0x4a, 0xcb, 0xc3, 0x41, 0x5e, 0x3d, 0xd0, 0x6a, 0xe1, 0xaa, 0x87, 0x9a, 0xda, 0x9f, 0x68, 0x8c, - 0x34, 0x00, 0x1c, 0x9c, 0x74, 0x36, 0xce, 0xf6, 0x62, 0xfd, 0xe2, 0xca, 0xa1, 0x85, 0x50, 0xd0, - 0x1f, 0x49, 0x90, 0xa3, 0x66, 0x86, 0x5b, 0xf4, 0xb8, 0x03, 0xd3, 0xd3, 0x5d, 0xdc, 0x72, 0x5c, - 0x33, 0x9b, 0xa0, 0xdb, 0x51, 0x6a, 0xfc, 0xfb, 0xac, 0xd6, 0xc5, 0x8c, 0xb8, 0xdf, 0xb7, 0xcc, - 0xe2, 0xc1, 0x41, 0xad, 0x32, 0x1c, 0xe4, 0xb3, 0x7b, 0x3e, 0x78, 0x70, 0x40, 0x1a, 0x83, 0xd6, - 0xb2, 0xbd, 0x97, 0xd4, 0xa0, 0xf7, 0x20, 0xd3, 0x72, 0x3a, 0x1d, 0xdc, 0xa2, 0x33, 0xd4, 0x0f, - 0xb4, 0x5a, 0x56, 0x61, 0x87, 0xb7, 0x38, 0x1c, 0xe4, 0xd3, 0xe5, 0xa0, 0x86, 0x1e, 0x63, 0xba, - 0x15, 0x2e, 0xe6, 0xca, 0x70, 0xe5, 0xcc, 0x2d, 0x0e, 0x9b, 0x69, 0x92, 0x9b, 0xe9, 0x32, 0xc4, - 0x1e, 0x1b, 0x9d, 0x3e, 0xd7, 0xa9, 0xa4, 0xc6, 0x0b, 0x77, 0x22, 0xef, 0x49, 0x05, 0x15, 0x32, - 0x7c, 0xe7, 0xf6, 0x5c, 0xa7, 0xed, 0x62, 0x42, 0x0a, 0x7f, 0x96, 0x81, 0x8c, 0x86, 0x89, 0xe7, - 0xb8, 0xd8, 0xd7, 0xec, 0x7f, 0x90, 0x60, 0x89, 0x52, 0x99, 0x6b, 0xf5, 0x3c, 0xc7, 0xd5, 0x5d, - 0xfc, 0xc4, 0xb5, 0x3c, 0x4c, 0xb2, 0x11, 0xa6, 0x11, 0x1b, 0x53, 0x4e, 0x65, 0x1c, 0xa8, 0x58, - 0x09, 0x40, 0x34, 0x81, 0xc1, 0x95, 0xe2, 0xee, 0x8f, 0x7f, 0x9e, 0xbf, 0x33, 0xd3, 0x9e, 0x9f, - 0x66, 0xd7, 0x62, 0xad, 0xa2, 0x21, 0xf3, 0x14, 0x30, 0xba, 0x0e, 0x32, 0x55, 0xaa, 0x6c, 0x74, - 0x35, 0x7a, 0x33, 0x59, 0x52, 0x86, 0x83, 0xbc, 0x4c, 0xd5, 0x4e, 0x63, 0xd2, 0x31, 0x43, 0x93, - 0x5f, 0xc1, 0xd0, 0xee, 0x41, 0xca, 0x33, 0x9a, 0x1d, 0xac, 0xd3, 0x91, 0x89, 0xb0, 0x8d, 0x6f, - 0x4c, 0xec, 0x04, 0x79, 0xd4, 0x69, 0x1a, 0x04, 0x17, 0xf7, 0x69, 0xcb, 0xd0, 0xda, 0xc1, 0xf3, - 0x05, 0x04, 0xad, 0x41, 0xca, 0x79, 0x8c, 0x5d, 0xd7, 0x32, 0xb1, 0x6e, 0x36, 0x99, 0xa2, 0x27, - 0x4b, 0x99, 0xe1, 0x20, 0x0f, 0xbb, 0x42, 0x5c, 0x29, 0x69, 0xe0, 0x37, 0xa9, 0x34, 0x91, 0x07, - 0xcb, 0xc2, 0x8e, 0x03, 0xe3, 0x64, 0xfc, 0x99, 0x60, 0x53, 0xf8, 0xde, 0xec, 0x87, 0xc1, 0xcf, - 0xdd, 0x57, 0x1e, 0xc6, 0xa0, 0x7c, 0x91, 0xa8, 0x79, 0xaa, 0x06, 0xbd, 0x09, 0x8b, 0x3d, 0x17, - 0xf7, 0x0c, 0x17, 0xeb, 0x2d, 0xa7, 0xdb, 0xeb, 0x60, 0x0f, 0x9b, 0x4c, 0x53, 0x15, 0x4d, 0x15, - 0x15, 0x65, 0x5f, 0x8e, 0xde, 0x80, 0x0c, 0xf1, 0x0c, 0x8f, 0x12, 0x3b, 0xc1, 0x2e, 0x6d, 0x99, - 0x64, 0x2d, 0xd3, 0x4c, 0x5a, 0x13, 0x42, 0x74, 0x0b, 0x54, 0xb6, 0x11, 0x44, 0xef, 0xf5, 0x9b, - 0x1d, 0x8b, 0x1c, 0x61, 0x33, 0x0b, 0xac, 0xe1, 0x02, 0x97, 0xef, 0xf9, 0x62, 0x74, 0x32, 0xa6, - 0x80, 0x2d, 0xba, 0x1d, 0x46, 0x1b, 0x67, 0x53, 0xab, 0xd2, 0xcd, 0x58, 0x69, 0xeb, 0xc5, 0x20, - 0x5f, 0x99, 0x59, 0x7b, 0x08, 0xee, 0xae, 0x79, 0x2e, 0xc6, 0x21, 0x65, 0x2c, 0x0b, 0xbc, 0xb0, - 0x1e, 0xf9, 0xb2, 0x09, 0x22, 0x9a, 0xff, 0x52, 0x88, 0x68, 0x0f, 0x12, 0x1e, 0xb6, 0x0d, 0xdb, - 0x23, 0xd9, 0x34, 0x3b, 0xb6, 0x77, 0x66, 0x3f, 0xb6, 0x7d, 0xd6, 0xd1, 0xd7, 0x47, 0x01, 0x83, - 0x2a, 0x00, 0xde, 0x49, 0xcf, 0x57, 0xc7, 0x0c, 0x03, 0x7d, 0xe3, 0x65, 0xea, 0x78, 0xd2, 0x0b, - 0x6b, 0x63, 0xd2, 0x13, 0x65, 0x82, 0xea, 0x30, 0x4f, 0x5a, 0x47, 0xb8, 0x6b, 0x08, 0x9c, 0x05, - 0x86, 0x73, 0xe3, 0x25, 0x38, 0x0d, 0xd6, 0x34, 0x84, 0x94, 0x22, 0x81, 0x84, 0xe4, 0xfe, 0x57, - 0x82, 0xc5, 0x53, 0xf6, 0x8e, 0xf6, 0x21, 0x12, 0x44, 0x34, 0x94, 0x32, 0x23, 0x2c, 0x9a, 0xb9, - 0x8c, 0xed, 0x47, 0x2c, 0x13, 0xb5, 0x21, 0x49, 0x35, 0xd0, 0xf6, 0x68, 0xb8, 0x14, 0x61, 0xe0, - 0xf5, 0xe1, 0x20, 0xaf, 0xec, 0x31, 0xe1, 0xa5, 0x87, 0x50, 0x38, 0x78, 0xcd, 0x44, 0x79, 0x48, - 0x79, 0x8e, 0x8e, 0x9f, 0x5a, 0xc4, 0xb3, 0xec, 0x36, 0xf3, 0xb3, 0x8a, 0x06, 0x9e, 0x53, 0x15, - 0x92, 0xdc, 0x9f, 0x47, 0x00, 0x9d, 0x36, 0x2c, 0xf4, 0x2f, 0x12, 0x5c, 0xf7, 0x7d, 0xaa, 0xe3, - 0x5a, 0x6d, 0xcb, 0x36, 0x3a, 0x63, 0xce, 0x55, 0x62, 0x3b, 0xfd, 0xc9, 0x65, 0xac, 0x57, 0x38, - 0xdc, 0x5d, 0x01, 0x3f, 0xe9, 0x78, 0xaf, 0x53, 0x07, 0xc5, 0x1d, 0xef, 0xa9, 0x26, 0x0f, 0xb5, - 0x6c, 0xff, 0x25, 0x9d, 0x73, 0xf7, 0xe1, 0xeb, 0x53, 0x81, 0x2f, 0xe2, 0x6e, 0x72, 0x3f, 0x96, - 0xe0, 0xda, 0x4b, 0x9c, 0x40, 0x18, 0x27, 0xcd, 0x71, 0x3e, 0x0c, 0xe3, 0xa4, 0xd6, 0x7f, 0xfd, - 0x12, 0x8e, 0x26, 0x3c, 0x89, 0x6f, 0x41, 0x9c, 0x1b, 0x11, 0xba, 0x1a, 0x68, 0xa3, 0x5c, 0x8a, - 0x73, 0x6d, 0x64, 0xfa, 0x84, 0x40, 0x66, 0x9c, 0xca, 0x03, 0x41, 0xf6, 0x5d, 0x97, 0x15, 0x49, - 0x8d, 0x14, 0xde, 0x81, 0x05, 0x31, 0x94, 0xef, 0x30, 0xd1, 0xd7, 0x01, 0x8e, 0xac, 0xf6, 0x91, - 0xfe, 0xc4, 0xf0, 0xb0, 0x2b, 0x82, 0xe3, 0x24, 0x95, 0x7c, 0x44, 0x05, 0x85, 0xcf, 0x14, 0x48, - 0xd7, 0xba, 0x3d, 0xc7, 0xf5, 0x7c, 0x77, 0xfa, 0x00, 0xe2, 0x9c, 0xdf, 0xc4, 0xa9, 0x17, 0xa7, - 0xac, 0x6b, 0xac, 0x27, 0x77, 0x24, 0xc2, 0xf4, 0x05, 0x46, 0xe0, 0xe7, 0x22, 0x67, 0xfa, 0xb9, - 0xf7, 0x21, 0xce, 0xef, 0x27, 0x22, 0xbe, 0xce, 0x87, 0xc6, 0xf2, 0x6f, 0x07, 0xb5, 0xdd, 0x4d, - 0xab, 0x83, 0x37, 0x59, 0x33, 0x1f, 0x9c, 0x77, 0x42, 0xdf, 0x00, 0x85, 0x10, 0x4f, 0x27, 0xd6, - 0x8f, 0xb8, 0x9b, 0x8c, 0xf2, 0x20, 0xbb, 0xd1, 0xd8, 0x6f, 0x58, 0x3f, 0xc2, 0x5a, 0x82, 0x10, - 0x8f, 0x7e, 0xa0, 0x1c, 0x28, 0x4f, 0x8c, 0x4e, 0x87, 0xb9, 0xd3, 0x18, 0xbb, 0x54, 0x04, 0xe5, - 0x71, 0xe3, 0x8c, 0x7f, 0xb5, 0xc6, 0x29, 0x3c, 0x63, 0xcf, 0xf0, 0x8e, 0x58, 0x38, 0x97, 0xd4, - 0x80, 0x8b, 0xf6, 0x0c, 0xef, 0x08, 0x65, 0x21, 0x41, 0x0c, 0xea, 0xa4, 0x48, 0x56, 0x59, 0x8d, - 0xde, 0x9c, 0xd7, 0xfc, 0x22, 0x5a, 0x01, 0xe6, 0x62, 0x79, 0x91, 0x79, 0xab, 0xa8, 0x16, 0x92, - 0xb0, 0x7d, 0x38, 0xb6, 0x7a, 0xfa, 0xe1, 0x31, 0xe1, 0x2e, 0x4a, 0xec, 0xc3, 0xb1, 0xd5, 0xdb, - 0xbc, 0x4f, 0xb4, 0x04, 0xad, 0xdc, 0x3c, 0x26, 0x34, 0xc8, 0xb6, 0xec, 0x36, 0x26, 0x9e, 0x6e, - 0x5a, 0x2e, 0x6e, 0x79, 0x9d, 0x13, 0xe6, 0xa3, 0x14, 0x2d, 0xc3, 0xc5, 0x15, 0x21, 0xa5, 0xbe, - 0x6f, 0xd2, 0x9f, 0x32, 0xdf, 0xa2, 0x68, 0x0b, 0x13, 0xee, 0xf4, 0x4c, 0x37, 0x99, 0x3e, 0xdb, - 0x4d, 0x9e, 0x13, 0xe0, 0x5e, 0xfd, 0x7f, 0x08, 0x70, 0x73, 0x9f, 0x45, 0x20, 0xc6, 0xf4, 0x16, - 0xdd, 0x01, 0x99, 0x1e, 0x9b, 0xb8, 0x1a, 0xcd, 0x1a, 0x2c, 0xb1, 0x3e, 0xd4, 0x22, 0x6d, 0xa3, - 0x8b, 0xb3, 0x88, 0x1d, 0x2a, 0xfb, 0x46, 0xd7, 0x20, 0x41, 0xf0, 0x23, 0xfd, 0xb1, 0xd1, 0xc9, - 0x2e, 0xb1, 0x13, 0x8b, 0x13, 0xfc, 0xe8, 0xa1, 0xd1, 0x41, 0x57, 0x20, 0x6e, 0x11, 0xdd, 0xc6, - 0x4f, 0xb2, 0xcb, 0x6c, 0x9f, 0x62, 0x16, 0xd9, 0xc1, 0x4f, 0x18, 0x79, 0x1b, 0x6e, 0x1b, 0x7b, - 0x7a, 0xcb, 0xe9, 0x90, 0xec, 0x15, 0x6a, 0x30, 0x34, 0x16, 0xa3, 0xa2, 0xb2, 0xd3, 0x21, 0x75, - 0x59, 0x89, 0xa8, 0xd1, 0xba, 0xac, 0x44, 0x55, 0xb9, 0x2e, 0x2b, 0xb2, 0x1a, 0xab, 0xcb, 0x4a, - 0x4c, 0x8d, 0xd7, 0x65, 0x25, 0xae, 0x26, 0xea, 0xb2, 0x92, 0x50, 0x95, 0xba, 0xac, 0x28, 0x6a, - 0xb2, 0x2e, 0x2b, 0x49, 0x15, 0xea, 0xb2, 0x02, 0x6a, 0xaa, 0x2e, 0x2b, 0x29, 0x75, 0xbe, 0x2e, - 0x2b, 0xf3, 0x6a, 0xba, 0x2e, 0x2b, 0x69, 0x35, 0x53, 0x97, 0x95, 0x8c, 0xba, 0x50, 0x97, 0x95, - 0x05, 0x55, 0xad, 0xcb, 0x8a, 0xaa, 0x2e, 0xd6, 0x65, 0x65, 0x51, 0x45, 0x85, 0xff, 0x96, 0x20, - 0xc3, 0x0d, 0x3a, 0x20, 0x8f, 0x37, 0x61, 0x91, 0xa9, 0x98, 0x65, 0xb7, 0xf5, 0x9e, 0x10, 0x32, - 0x5a, 0x88, 0x68, 0xaa, 0x5f, 0x11, 0x34, 0x7e, 0x0d, 0xd2, 0x2e, 0x36, 0xcc, 0x51, 0xc3, 0x08, - 0x6b, 0x38, 0x4f, 0x85, 0x41, 0xa3, 0x37, 0x20, 0xc3, 0x18, 0x6f, 0xd4, 0x2a, 0xca, 0x5a, 0xa5, - 0x99, 0x34, 0x68, 0x56, 0x82, 0x34, 0xe9, 0x19, 0xf6, 0xa8, 0x95, 0xcc, 0xb8, 0xe8, 0xda, 0x19, - 0xfc, 0xd0, 0xe8, 0x19, 0xb6, 0xe0, 0x85, 0x79, 0xda, 0x27, 0xcc, 0x7c, 0x2e, 0x26, 0xfd, 0x2e, - 0xd6, 0x7b, 0x0e, 0x8f, 0x81, 0xa3, 0x5a, 0x92, 0x4b, 0xf6, 0x1c, 0x52, 0xf8, 0x6d, 0xb8, 0x46, - 0x43, 0x0d, 0x1e, 0x26, 0x94, 0x8f, 0x0c, 0xbb, 0x1d, 0xdc, 0x28, 0x0c, 0x48, 0xb0, 0x70, 0x25, - 0x88, 0x05, 0xb6, 0x86, 0x83, 0x7c, 0x9c, 0xb6, 0xbe, 0x34, 0x1f, 0xc4, 0x29, 0x70, 0xcd, 0x2c, - 0xe4, 0x20, 0x3b, 0x39, 0x7a, 0x70, 0xc7, 0xd1, 0xd8, 0x15, 0xa7, 0xdf, 0xc5, 0x74, 0x69, 0x0f, - 0x2c, 0xe2, 0xa1, 0xef, 0xc3, 0xbc, 0x58, 0x0a, 0x5d, 0xa1, 0xcf, 0xcc, 0xe7, 0xec, 0x46, 0xca, - 0x0d, 0x40, 0x48, 0xe1, 0x1f, 0x25, 0x58, 0xaa, 0xb8, 0x4e, 0xaf, 0x87, 0x4d, 0xa1, 0xc2, 0x7c, - 0xa9, 0xbe, 0xe6, 0x4a, 0x21, 0xcd, 0xdd, 0x81, 0x48, 0xad, 0x22, 0x02, 0x95, 0xbb, 0x97, 0x8d, - 0x7f, 0x6a, 0x15, 0xf4, 0x5d, 0x88, 0xd3, 0xd0, 0xba, 0x4f, 0x18, 0xcb, 0x67, 0xd6, 0x7f, 0x75, - 0x8a, 0x47, 0x69, 0xb0, 0x86, 0x9a, 0xe8, 0x50, 0xf8, 0x83, 0x38, 0x5c, 0x09, 0xef, 0xd1, 0xbd, - 0xb2, 0x3f, 0xf1, 0x4f, 0x21, 0x61, 0xd9, 0x26, 0x7e, 0x1a, 0xf8, 0xa9, 0xf7, 0xa7, 0xa1, 0x9e, - 0x05, 0x51, 0x14, 0xfb, 0x51, 0xa3, 0x30, 0x7e, 0xc4, 0x2a, 0x30, 0xd1, 0x0f, 0x02, 0x2f, 0xc8, - 0xaf, 0x91, 0x77, 0x5e, 0x19, 0xbd, 0x32, 0xe1, 0x11, 0xc7, 0x1c, 0x4e, 0x94, 0x79, 0xad, 0xaf, - 0xc6, 0xe1, 0x34, 0x60, 0xd1, 0xb2, 0x3d, 0xec, 0x76, 0xb0, 0xf1, 0x98, 0xf2, 0x2d, 0x1d, 0x5e, - 0xdc, 0x26, 0x67, 0x65, 0x37, 0x35, 0x04, 0xc0, 0x59, 0xf2, 0x53, 0x58, 0x0a, 0x83, 0xfa, 0x47, - 0x30, 0xfd, 0x86, 0xc9, 0x76, 0x78, 0x04, 0xeb, 0x5f, 0xe4, 0x42, 0x40, 0x35, 0x8e, 0x93, 0xfb, - 0x13, 0x09, 0xe6, 0xc3, 0xc7, 0x82, 0x2c, 0x50, 0xd8, 0x18, 0xbe, 0x2d, 0x46, 0x4b, 0x3b, 0xd4, - 0xb5, 0xb1, 0x4a, 0xb6, 0x57, 0x1f, 0xbc, 0xf2, 0x5e, 0x71, 0x08, 0x71, 0xe4, 0x35, 0x13, 0xfd, - 0x0a, 0x24, 0x4d, 0xd7, 0xe9, 0x8d, 0xd2, 0x5b, 0x51, 0x4d, 0xa1, 0x02, 0xea, 0x32, 0x72, 0xbf, - 0x03, 0xc9, 0xe0, 0x40, 0x43, 0xd7, 0x84, 0xe8, 0x97, 0x78, 0x4d, 0x98, 0x36, 0x7e, 0xe1, 0xe7, - 0x71, 0x58, 0x3a, 0x8b, 0xaa, 0x3e, 0x06, 0x35, 0xc4, 0x0c, 0x7a, 0xc7, 0x22, 0x9e, 0xd0, 0xd8, - 0x5b, 0xd3, 0xe3, 0xd1, 0x10, 0xbd, 0x88, 0xf3, 0xc8, 0xb8, 0xe3, 0xa4, 0xf3, 0x43, 0xc8, 0x98, - 0x7c, 0xc9, 0xba, 0x30, 0x85, 0xe8, 0xb9, 0x01, 0xe1, 0x19, 0x14, 0x23, 0xd0, 0xd3, 0x66, 0xa8, - 0x8a, 0xb0, 0x34, 0x9e, 0x8f, 0x6e, 0x1a, 0x9e, 0x41, 0xf5, 0x84, 0x9e, 0xb1, 0xcc, 0x58, 0xa7, - 0x31, 0x1c, 0xe4, 0x17, 0x05, 0x56, 0x45, 0xd4, 0x5e, 0x7a, 0x8f, 0x17, 0xcd, 0x09, 0x40, 0x93, - 0x12, 0x3d, 0xad, 0xa7, 0x03, 0xc7, 0x46, 0x44, 0x4f, 0x35, 0xf5, 0xf2, 0x44, 0x4f, 0x3f, 0x6b, - 0x26, 0xfa, 0x7d, 0x09, 0x16, 0x79, 0x2e, 0xa6, 0xdb, 0xf7, 0x0c, 0x96, 0x46, 0x0b, 0x02, 0xcd, - 0x8f, 0x87, 0x83, 0xfc, 0x02, 0xdb, 0x90, 0x6d, 0x51, 0xc7, 0x86, 0x2d, 0xbd, 0xea, 0xb0, 0x23, - 0x14, 0x11, 0x7c, 0x05, 0x02, 0x13, 0xdd, 0x87, 0x0c, 0x8f, 0x9a, 0x75, 0x1a, 0x37, 0x5a, 0x8e, - 0xcd, 0x22, 0xd0, 0x74, 0xe9, 0xf5, 0x17, 0x83, 0xfc, 0xea, 0x19, 0x9a, 0xc5, 0x03, 0xee, 0x87, - 0xbc, 0xad, 0x96, 0x3e, 0x0c, 0x17, 0x51, 0x0b, 0xd2, 0x81, 0x6a, 0x9c, 0xf4, 0x44, 0xc0, 0x7a, - 0x79, 0x67, 0x31, 0xef, 0xeb, 0x08, 0xc5, 0x44, 0x6d, 0x58, 0xf0, 0x07, 0xe1, 0x37, 0x77, 0x92, - 0x4d, 0x7e, 0x29, 0xc3, 0xf8, 0x6a, 0xcd, 0x57, 0x4d, 0xc4, 0xdd, 0xe9, 0x2a, 0x2c, 0x9f, 0xe9, - 0x8d, 0xff, 0x34, 0x06, 0x57, 0xc7, 0x19, 0x3e, 0x88, 0x30, 0xf4, 0x49, 0x1f, 0xf4, 0xc1, 0xcc, - 0x5e, 0xc2, 0xc7, 0xe0, 0x34, 0xe4, 0x97, 0x26, 0xbd, 0xd0, 0xa7, 0x13, 0x5e, 0xe8, 0x15, 0xf0, - 0x99, 0x7a, 0x4d, 0xe0, 0x0b, 0xd0, 0xdc, 0xbf, 0x49, 0x90, 0x1e, 0x1b, 0xff, 0x97, 0x49, 0xb7, - 0x7b, 0x41, 0x54, 0xc0, 0x9f, 0x67, 0xde, 0xbb, 0xf8, 0xda, 0xc6, 0x83, 0x85, 0xdc, 0x3f, 0x4b, - 0x90, 0x1e, 0x5b, 0xee, 0x57, 0x44, 0xd4, 0x5f, 0xfa, 0xcc, 0x0b, 0xdf, 0x81, 0x38, 0x97, 0x20, - 0x04, 0x99, 0x8f, 0x36, 0x6a, 0xfb, 0xb5, 0x9d, 0x7b, 0xfa, 0xe6, 0xae, 0xa6, 0xdf, 0x2b, 0xab, - 0x73, 0x68, 0x1e, 0x94, 0x4a, 0xf5, 0x41, 0x95, 0x0a, 0x55, 0x09, 0xa5, 0x20, 0xc1, 0x4a, 0xd5, - 0x8a, 0x1a, 0x29, 0x94, 0x40, 0xe5, 0xd8, 0x87, 0x98, 0x52, 0x2b, 0xbd, 0x2b, 0xa0, 0x22, 0x2c, - 0x51, 0x58, 0xdc, 0xa5, 0x31, 0x06, 0x75, 0x26, 0x7a, 0x28, 0xc2, 0x5b, 0x0c, 0xaa, 0xa8, 0x5b, - 0xd9, 0x31, 0xba, 0xb8, 0xf0, 0x4f, 0x32, 0x2c, 0x8e, 0x40, 0x7c, 0xc7, 0x42, 0xef, 0x94, 0x96, - 0x7d, 0xac, 0x8f, 0x1e, 0x6c, 0xf8, 0x9d, 0xd2, 0xb2, 0x8f, 0x0f, 0xb4, 0x9a, 0x96, 0xa0, 0x95, - 0x07, 0xae, 0x85, 0xea, 0x20, 0x3b, 0x3d, 0xcf, 0x0f, 0xd0, 0xbf, 0x3d, 0x65, 0x2b, 0x4e, 0x8d, - 0x51, 0xdc, 0xed, 0x79, 0x3c, 0xbb, 0xa2, 0x31, 0x0c, 0xf4, 0x77, 0x12, 0x24, 0xf8, 0x85, 0x87, - 0x64, 0xe3, 0x0c, 0xef, 0xbb, 0x17, 0xc2, 0xe3, 0x1b, 0x20, 0xb2, 0xf6, 0x1f, 0x51, 0x55, 0x7f, - 0x31, 0xc8, 0x2f, 0x4e, 0x6e, 0x10, 0xb9, 0x64, 0x3a, 0xdf, 0x9f, 0x22, 0xaa, 0xf3, 0x44, 0xf2, - 0x68, 0xa3, 0x19, 0xa5, 0xce, 0x98, 0xab, 0x4f, 0x8f, 0x1d, 0x44, 0xae, 0x0d, 0xf3, 0xe1, 0xd9, - 0x9f, 0x91, 0x6e, 0xda, 0x18, 0x4f, 0x37, 0xbd, 0x39, 0xd3, 0xce, 0x70, 0xcc, 0x70, 0x7a, 0xe9, - 0x3b, 0x90, 0x0c, 0xb6, 0xfd, 0x22, 0xc9, 0x31, 0xce, 0x92, 0xc1, 0x25, 0x34, 0xa6, 0xc6, 0x0b, - 0x7f, 0x2f, 0xc1, 0xbc, 0x86, 0x89, 0xd3, 0x79, 0x8c, 0x4d, 0x1a, 0x35, 0xa0, 0x6f, 0x82, 0x4c, - 0xa3, 0x10, 0x71, 0x85, 0x3e, 0xe7, 0x7a, 0xc2, 0x9a, 0xa2, 0x0d, 0x48, 0x06, 0x89, 0x80, 0x8b, - 0xbc, 0x29, 0x8e, 0x7a, 0xa1, 0x5b, 0xa0, 0x36, 0x9d, 0xbe, 0x6d, 0x1a, 0xee, 0x89, 0xee, 0x62, - 0xa3, 0x75, 0x84, 0x4d, 0x91, 0xfa, 0x5c, 0xf0, 0xe5, 0x1a, 0x17, 0x17, 0x3e, 0x8b, 0x00, 0x1a, - 0x6d, 0x4e, 0x88, 0x26, 0x68, 0xec, 0xc3, 0xd6, 0x21, 0x2e, 0x58, 0x91, 0x33, 0x53, 0xcb, 0x13, - 0x21, 0x54, 0xb0, 0x70, 0xff, 0x48, 0xdd, 0x90, 0x8c, 0xa0, 0x3f, 0x9e, 0x9e, 0xee, 0x88, 0xb2, - 0x74, 0x07, 0xd3, 0xd2, 0x5f, 0x6a, 0xca, 0x43, 0xb8, 0xbb, 0xff, 0x91, 0x01, 0x95, 0x5d, 0x6c, - 0x78, 0x98, 0x32, 0x0f, 0x99, 0x76, 0x1f, 0x2c, 0x41, 0x8c, 0x5f, 0x1e, 0x22, 0x17, 0xb9, 0x3c, - 0x88, 0x4d, 0xe1, 0x5d, 0xd1, 0x6f, 0xc0, 0x7c, 0xcb, 0xe9, 0xf4, 0xbb, 0xb6, 0xce, 0x5e, 0x59, - 0x44, 0x28, 0xf9, 0x6b, 0xd3, 0x94, 0xf8, 0xd4, 0xe4, 0x8a, 0x65, 0xa7, 0x43, 0xcb, 0xfe, 0xfd, - 0x96, 0x03, 0xb2, 0x16, 0xe8, 0x3a, 0x24, 0x03, 0x83, 0x62, 0x41, 0x64, 0x52, 0x1b, 0x09, 0xd0, - 0x3a, 0xc4, 0x0c, 0xa2, 0x3b, 0x87, 0x2c, 0xca, 0x3b, 0x4f, 0xc3, 0x34, 0xd9, 0x20, 0xbb, 0x87, - 0xe8, 0x5d, 0x48, 0x1f, 0x3e, 0xe2, 0x91, 0x2f, 0x27, 0x50, 0xfe, 0xf8, 0xb5, 0x30, 0x1c, 0xe4, - 0x53, 0x9b, 0x1f, 0xb2, 0xc5, 0x52, 0xfa, 0xd4, 0x52, 0x87, 0x8f, 0x82, 0x02, 0xba, 0x0d, 0x8b, - 0x5d, 0xe3, 0xa9, 0x7e, 0xe8, 0x1a, 0x2d, 0x11, 0xea, 0x75, 0x38, 0x2b, 0x48, 0xda, 0x42, 0xd7, - 0x78, 0xba, 0x29, 0xe4, 0x35, 0xb3, 0x83, 0x73, 0xbf, 0x90, 0x20, 0x21, 0x56, 0x84, 0x7a, 0x00, - 0x62, 0x7b, 0x2c, 0x93, 0x07, 0x13, 0xe9, 0xd2, 0x87, 0xc3, 0x41, 0x3e, 0x59, 0x66, 0xd2, 0x5a, - 0x85, 0xbc, 0x18, 0xe4, 0xbf, 0xff, 0xaa, 0xa4, 0xe5, 0x83, 0x68, 0x49, 0x3e, 0x48, 0xcd, 0x64, - 0xd9, 0x9a, 0x23, 0x83, 0xe8, 0x47, 0x16, 0xf1, 0x9c, 0xb6, 0x6b, 0x74, 0xd9, 0xe1, 0x2a, 0xda, - 0xfc, 0x91, 0x41, 0xb6, 0x7c, 0x19, 0xca, 0xd1, 0x70, 0xe0, 0x31, 0x7f, 0x24, 0xe3, 0x26, 0x15, - 0x94, 0xd1, 0x3a, 0x5c, 0x09, 0x3a, 0xeb, 0x74, 0xd1, 0xcd, 0x7e, 0xeb, 0x18, 0x33, 0x4f, 0x40, - 0x39, 0x6b, 0x29, 0xa8, 0xdc, 0x36, 0x9e, 0x96, 0x78, 0x55, 0xe1, 0x0a, 0x2c, 0x85, 0x8e, 0x35, - 0x08, 0xb1, 0xfe, 0x02, 0x20, 0xb1, 0x67, 0x9c, 0x74, 0x1c, 0xc3, 0x44, 0xab, 0x90, 0xf2, 0x9f, - 0xb9, 0x68, 0x90, 0xca, 0xf5, 0x30, 0x2c, 0xa2, 0x93, 0xea, 0x13, 0xec, 0xb2, 0x33, 0xe1, 0x2c, - 0x15, 0x94, 0xc5, 0xdb, 0x1e, 0x9d, 0x9f, 0xde, 0xb5, 0x5a, 0xae, 0xc3, 0x53, 0x0e, 0x51, 0xc6, - 0xb6, 0x54, 0xba, 0xcd, 0x84, 0xe8, 0x06, 0x2c, 0x1c, 0x5a, 0x36, 0xcb, 0x4a, 0xfa, 0xed, 0x58, - 0xfe, 0x58, 0xcb, 0xf8, 0x62, 0xd1, 0xf0, 0x31, 0x64, 0x42, 0x2f, 0x7b, 0xf4, 0x6c, 0xe2, 0xec, - 0x6c, 0x76, 0x87, 0x83, 0x7c, 0x7a, 0xa4, 0xeb, 0xfc, 0x7c, 0x2e, 0xe3, 0x54, 0xd2, 0xa3, 0x61, - 0xe8, 0xe9, 0x2c, 0x43, 0x8c, 0xfd, 0xf6, 0x87, 0x3f, 0xb7, 0x6b, 0xbc, 0x80, 0xbe, 0x0d, 0xb1, - 0x0e, 0x36, 0x08, 0x4f, 0x01, 0xa7, 0xd6, 0x57, 0xa7, 0x58, 0x0f, 0xfb, 0x99, 0x8e, 0xc6, 0x9b, - 0xa3, 0x12, 0xc4, 0x79, 0x9e, 0x99, 0x65, 0x87, 0x53, 0xeb, 0x37, 0x67, 0xfd, 0x95, 0xc4, 0xd6, - 0x9c, 0x26, 0x7a, 0xa2, 0x2a, 0x24, 0x5c, 0xfe, 0xb4, 0xc0, 0x72, 0xc6, 0xe7, 0xde, 0x2f, 0x43, - 0xef, 0x1d, 0x5b, 0x73, 0x9a, 0xdf, 0x17, 0xed, 0xfb, 0x6f, 0x78, 0x9c, 0x86, 0xc5, 0x8b, 0x65, - 0x71, 0xc6, 0x08, 0x6a, 0x04, 0x38, 0x86, 0x42, 0x17, 0x68, 0xb1, 0xcc, 0x25, 0x4b, 0x3d, 0x4f, - 0x5f, 0xe0, 0xd8, 0x9b, 0x05, 0x5d, 0x20, 0xef, 0x89, 0x76, 0x00, 0x5a, 0x81, 0x6b, 0xc8, 0x66, - 0x18, 0xce, 0x5b, 0x17, 0x09, 0x3f, 0xb6, 0xe6, 0xb4, 0x10, 0x02, 0xfa, 0x10, 0x52, 0xad, 0x91, - 0xae, 0x67, 0x17, 0x18, 0xe0, 0xdb, 0x17, 0x22, 0xbc, 0x2d, 0x4a, 0x72, 0x23, 0xe9, 0x38, 0xc9, - 0xa9, 0x93, 0x24, 0x57, 0x85, 0xb4, 0x48, 0x05, 0xf0, 0x9f, 0x8d, 0x65, 0x17, 0x19, 0xc7, 0x86, - 0xb5, 0xc4, 0xff, 0x61, 0x59, 0xb1, 0x6a, 0xb7, 0x1c, 0x13, 0x9b, 0x55, 0x5a, 0xd6, 0x44, 0x6e, - 0x91, 0x15, 0x08, 0xba, 0x07, 0x99, 0x56, 0x07, 0x1b, 0x76, 0xbf, 0xe7, 0xe3, 0xa0, 0x19, 0x71, - 0xd2, 0xa2, 0x9f, 0x00, 0xda, 0x01, 0x74, 0xc8, 0x5e, 0x12, 0xc3, 0xb3, 0x62, 0xb9, 0xf0, 0x59, - 0xc0, 0x54, 0xd6, 0x57, 0x1b, 0xcd, 0x0c, 0xbd, 0x0e, 0x69, 0xdb, 0xb1, 0x5b, 0x86, 0xdd, 0xc2, - 0x1d, 0xe6, 0x8e, 0x78, 0xfa, 0x7c, 0x5c, 0x88, 0x3e, 0x81, 0x0c, 0x19, 0x8b, 0xb9, 0xb3, 0x57, - 0xd8, 0x88, 0xef, 0x5c, 0x34, 0x81, 0xb7, 0x35, 0xa7, 0x4d, 0x20, 0xa1, 0xdf, 0x04, 0xd5, 0x9b, - 0x48, 0xda, 0xb2, 0x57, 0x8b, 0xe9, 0x4f, 0xee, 0x2f, 0xc9, 0x32, 0x6f, 0xcd, 0x69, 0xa7, 0xd0, - 0x4a, 0x49, 0x48, 0x98, 0xbc, 0x3a, 0x48, 0xee, 0x27, 0x54, 0xa5, 0xf0, 0x8b, 0x38, 0x28, 0x41, - 0xb4, 0xb2, 0x06, 0x28, 0xf0, 0x2f, 0xa3, 0x5f, 0x3b, 0x50, 0xa2, 0x8c, 0x6c, 0xcd, 0x69, 0x8b, - 0x7e, 0xdd, 0xe8, 0x07, 0x0f, 0x37, 0x60, 0xa1, 0xeb, 0x98, 0xd6, 0xa1, 0x35, 0x62, 0x3b, 0x9e, - 0x5e, 0xca, 0xf8, 0x62, 0xc1, 0x76, 0x77, 0xc7, 0xde, 0x0a, 0xa3, 0x33, 0xf8, 0xca, 0xad, 0xb9, - 0xd0, 0x63, 0x22, 0x65, 0x5f, 0xb7, 0x6f, 0xdb, 0x96, 0xdd, 0xd6, 0xc5, 0x05, 0x89, 0x7b, 0xe2, - 0xb4, 0x90, 0x8a, 0x3b, 0x4e, 0x79, 0x82, 0x8e, 0x6e, 0x9d, 0x4b, 0x47, 0xfe, 0xda, 0xb7, 0xa4, - 0x80, 0x8f, 0x36, 0x27, 0xf9, 0xe8, 0xf6, 0xf9, 0x7c, 0x14, 0x82, 0x09, 0x08, 0xe9, 0xe0, 0x4c, - 0x42, 0x5a, 0x9b, 0x51, 0x5b, 0x42, 0x88, 0xe3, 0x8c, 0x54, 0x9e, 0x60, 0xa4, 0x5b, 0xe7, 0x32, - 0x52, 0x78, 0x8d, 0x82, 0x92, 0x76, 0xcf, 0xa0, 0xa4, 0xb7, 0x67, 0xa2, 0xa4, 0x10, 0x58, 0x98, - 0x93, 0xb4, 0xb3, 0x38, 0xa9, 0x38, 0x1b, 0x27, 0x85, 0x20, 0xc7, 0x48, 0xe9, 0x87, 0xa7, 0x0c, - 0x4e, 0x65, 0xb0, 0xdf, 0xbc, 0xf0, 0xad, 0x78, 0x4b, 0x3a, 0x65, 0x71, 0xc6, 0x19, 0x16, 0xb7, - 0xc8, 0xe0, 0xdf, 0xbd, 0x80, 0xc5, 0x85, 0x06, 0x38, 0x6d, 0x72, 0x00, 0x8a, 0xff, 0xca, 0x14, - 0x32, 0xbf, 0xc2, 0x4f, 0x24, 0x88, 0xd6, 0x9d, 0x26, 0xca, 0x8c, 0x52, 0x08, 0xec, 0xf2, 0xff, - 0xc1, 0xa8, 0xb9, 0x88, 0x87, 0x5f, 0x9b, 0x32, 0x13, 0x7f, 0x64, 0x2d, 0xe8, 0x84, 0xbe, 0x07, - 0x89, 0x1e, 0x8f, 0x75, 0x84, 0x85, 0x15, 0xa6, 0xf5, 0xe7, 0x2d, 0x35, 0xbf, 0xcb, 0xed, 0x5b, - 0x90, 0x19, 0xff, 0xad, 0x2a, 0xca, 0x00, 0xec, 0x19, 0x84, 0xf4, 0x8e, 0x5c, 0x83, 0x60, 0x75, - 0x0e, 0x25, 0x20, 0x7a, 0x7f, 0xbb, 0xa1, 0x4a, 0xb7, 0x7f, 0x10, 0x4e, 0x2a, 0x54, 0xb4, 0x8d, - 0xda, 0x4e, 0x6d, 0xe7, 0x9e, 0xbe, 0xb3, 0xb1, 0x5d, 0x6d, 0xa8, 0x73, 0x28, 0x0b, 0xcb, 0x1f, - 0x6d, 0xd4, 0xf6, 0x45, 0x96, 0x41, 0xaf, 0xed, 0xec, 0x57, 0xb5, 0x87, 0x1b, 0x0f, 0x54, 0x09, - 0x5d, 0x05, 0xa4, 0xed, 0x96, 0xef, 0x37, 0x2a, 0x25, 0xbd, 0xbc, 0xbb, 0xbd, 0xb7, 0x51, 0xde, - 0xaf, 0xed, 0xee, 0xa8, 0x11, 0xa4, 0x80, 0x5c, 0xd9, 0xdd, 0xa9, 0xaa, 0x70, 0xfb, 0x27, 0x51, - 0x90, 0xe9, 0x1e, 0xa3, 0xd7, 0x21, 0x75, 0xb0, 0xd3, 0xd8, 0xab, 0x96, 0x6b, 0x9b, 0xb5, 0x6a, - 0x45, 0x9d, 0xcb, 0x2d, 0x3d, 0x7b, 0xbe, 0xba, 0x40, 0xab, 0x0e, 0x6c, 0xd2, 0xc3, 0x2d, 0x46, - 0x2e, 0x28, 0x07, 0xf1, 0xd2, 0x46, 0xf9, 0xfe, 0xc1, 0x9e, 0x2a, 0xe5, 0x32, 0xcf, 0x9e, 0xaf, - 0x02, 0x6d, 0xc0, 0x0d, 0x1b, 0x5d, 0x87, 0x84, 0x56, 0x6d, 0xec, 0xef, 0x6a, 0x55, 0x35, 0x92, - 0x5b, 0x78, 0xf6, 0x7c, 0x35, 0x45, 0x2b, 0x85, 0xbd, 0xa2, 0x1b, 0x90, 0x6e, 0x94, 0xb7, 0xaa, - 0xdb, 0x1b, 0x7a, 0x79, 0x6b, 0x63, 0xe7, 0x5e, 0x55, 0x8d, 0xe6, 0x96, 0x9f, 0x3d, 0x5f, 0x55, - 0x27, 0x0f, 0x98, 0x0e, 0x51, 0xdb, 0xde, 0xdb, 0xd5, 0xf6, 0x55, 0x79, 0x34, 0x04, 0xb7, 0x2b, - 0x54, 0x00, 0xe0, 0xbd, 0x37, 0xab, 0xd5, 0x8a, 0x1a, 0xcb, 0xa1, 0x67, 0xcf, 0x57, 0x33, 0xb4, - 0x7e, 0x64, 0x2e, 0xe8, 0x0d, 0x98, 0x2f, 0x6b, 0xd5, 0x8d, 0xfd, 0xaa, 0xde, 0xd8, 0xdf, 0xd8, - 0x6f, 0xa8, 0xf1, 0xd1, 0x4a, 0x42, 0x26, 0x80, 0x8a, 0xb0, 0xb8, 0x71, 0xb0, 0xbf, 0xab, 0x8f, - 0xb5, 0x4d, 0xe4, 0xae, 0x3d, 0x7b, 0xbe, 0xba, 0x44, 0xdb, 0x6e, 0xf4, 0x3d, 0x27, 0xdc, 0xfe, - 0x2d, 0x50, 0xc7, 0xe6, 0xaf, 0xdf, 0x2b, 0xab, 0x4a, 0xee, 0xea, 0xb3, 0xe7, 0xab, 0x68, 0x72, - 0x09, 0xf7, 0xca, 0xe8, 0x5b, 0x70, 0x75, 0xff, 0xe3, 0xbd, 0x6a, 0xa5, 0xda, 0x28, 0xeb, 0xe3, - 0xcb, 0x4e, 0xe6, 0xb2, 0xcf, 0x9e, 0xaf, 0x2e, 0xd3, 0x3e, 0x93, 0xfd, 0x72, 0xca, 0x67, 0x7f, - 0xb5, 0x32, 0xf7, 0xb7, 0x7f, 0xbd, 0x32, 0x57, 0xba, 0xf9, 0xf9, 0x7f, 0xae, 0xcc, 0x7d, 0x3e, - 0x5c, 0x91, 0x7e, 0x3a, 0x5c, 0x91, 0x7e, 0x36, 0x5c, 0x91, 0xfe, 0x63, 0xb8, 0x22, 0xfd, 0xe1, - 0x17, 0x2b, 0x73, 0x3f, 0xfd, 0x62, 0x65, 0xee, 0x67, 0x5f, 0xac, 0xcc, 0x7d, 0x12, 0xe7, 0x6a, - 0xd5, 0x8c, 0xb3, 0x9f, 0x73, 0xbf, 0xfb, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x7c, 0xcd, 0x9c, - 0xd5, 0x9d, 0x2e, 0x00, 0x00, + 0xdd, 0x6c, 0x52, 0x1a, 0x89, 0x1a, 0xd9, 0xbb, 0x17, 0xa9, 0xeb, 0x55, 0xd5, 0xaf, 0xbe, 0xde, + 0xfb, 0xbd, 0x57, 0xaf, 0x08, 0x57, 0x7f, 0xc7, 0x69, 0x91, 0x55, 0xfa, 0xc7, 0x6d, 0xb1, 0x7f, + 0x25, 0xd7, 0x73, 0x7c, 0x07, 0x7d, 0xad, 0xed, 0xb4, 0x8f, 0x3c, 0xc7, 0x68, 0x1f, 0x96, 0xc8, + 0xc3, 0x6e, 0x89, 0xd5, 0xf0, 0x56, 0xf9, 0x2b, 0xd8, 0xf3, 0x1c, 0x8f, 0xb6, 0xe7, 0x1f, 0xbc, + 0x47, 0x7e, 0xa9, 0xe3, 0x74, 0x1c, 0xf6, 0xb9, 0x4a, 0xbf, 0x84, 0x14, 0x31, 0x0c, 0xb7, 0xb5, + 0x6a, 0x1a, 0xbe, 0x21, 0x64, 0xb9, 0x40, 0x66, 0x39, 0x6f, 0x1f, 0x38, 0x5e, 0xcf, 0xf0, 0x03, + 0x8c, 0xd7, 0xc8, 0xc3, 0xee, 0x6a, 0xdb, 0xf0, 0x8d, 0xae, 0xd3, 0x59, 0x35, 0x31, 0x69, 0xbb, + 0xad, 0x55, 0xe2, 0x7b, 0xfd, 0xb6, 0xdf, 0xf7, 0xb0, 0x19, 0x74, 0xef, 0xfb, 0x56, 0x77, 0xf5, + 0xb0, 0xdb, 0x5e, 0xf5, 0xad, 0x1e, 0x26, 0xbe, 0xd1, 0x73, 0x79, 0x4d, 0xf1, 0xf7, 0x61, 0xf6, + 0x3e, 0x36, 0x08, 0x46, 0x9f, 0x40, 0xd2, 0x76, 0x4c, 0xac, 0x5b, 0x66, 0x4e, 0x5a, 0x91, 0x6e, + 0x64, 0xca, 0xeb, 0xc3, 0x41, 0x21, 0xb1, 0xed, 0x98, 0xb8, 0x5e, 0x7d, 0x3e, 0x28, 0xbc, 0xdb, + 0xb1, 0xfc, 0xc3, 0x7e, 0xab, 0xd4, 0x76, 0x7a, 0xab, 0xe1, 0x3a, 0xcd, 0xd6, 0xe8, 0x7b, 0xd5, + 0x3d, 0xea, 0xac, 0x8a, 0x59, 0x96, 0x78, 0x37, 0x2d, 0x41, 0x11, 0xeb, 0x26, 0x5a, 0x82, 0x59, + 0xec, 0x3a, 0xed, 0xc3, 0x5c, 0x6c, 0x45, 0xba, 0x11, 0xd7, 0x78, 0xe1, 0xb6, 0xfc, 0xdf, 0x7f, + 0x5d, 0x90, 0x8a, 0x9f, 0xc5, 0xe0, 0x5a, 0xd9, 0x68, 0x1f, 0xf5, 0xdd, 0x9a, 0xdd, 0xf6, 0x8e, + 0x5d, 0xdf, 0x72, 0xec, 0x1d, 0xf6, 0x97, 0x20, 0x15, 0xe2, 0x47, 0xf8, 0x98, 0xcd, 0x67, 0x4e, + 0xa3, 0x9f, 0xe8, 0x7d, 0x90, 0x7b, 0x8e, 0x89, 0x19, 0x50, 0x76, 0xed, 0x66, 0xe9, 0x85, 0x5b, + 0x5e, 0x1a, 0xa1, 0x6d, 0x39, 0x26, 0xd6, 0x58, 0x37, 0xd4, 0x02, 0xe5, 0xa8, 0x47, 0x74, 0xcb, + 0x3e, 0x70, 0x72, 0xf1, 0x15, 0xe9, 0x46, 0x7a, 0xed, 0xf6, 0x19, 0x10, 0x2f, 0x98, 0x56, 0xe9, + 0xde, 0x56, 0xb3, 0x6e, 0x1f, 0x38, 0xe5, 0xf4, 0x70, 0x50, 0x48, 0x8a, 0x82, 0x96, 0x3c, 0xea, + 0x11, 0xfa, 0x91, 0xdf, 0x81, 0x40, 0x46, 0xe7, 0xdf, 0xf7, 0x2c, 0x36, 0xff, 0x94, 0x46, 0x3f, + 0xd1, 0x5b, 0x80, 0x30, 0xc7, 0xc3, 0xa6, 0x4e, 0xcf, 0x57, 0xa7, 0x0b, 0x8c, 0xb1, 0x05, 0xaa, + 0x61, 0x4d, 0xd5, 0xf0, 0x8d, 0x7b, 0xf8, 0x98, 0xef, 0x90, 0xd8, 0xa7, 0x7f, 0x9b, 0x85, 0x0c, + 0x9f, 0x50, 0x15, 0xfb, 0x86, 0xd5, 0x25, 0xa8, 0x0c, 0x40, 0x7c, 0xc3, 0xf3, 0x75, 0x7a, 0xa6, + 0x6c, 0x90, 0xf4, 0xda, 0xab, 0x91, 0xe5, 0xd0, 0x33, 0x2f, 0x1d, 0x76, 0xdb, 0xa5, 0xbd, 0xe0, + 0xcc, 0xcb, 0xf2, 0xe7, 0x83, 0xc2, 0x8c, 0x96, 0x62, 0xdd, 0xa8, 0x14, 0xdd, 0x01, 0x05, 0xdb, + 0x26, 0x47, 0x88, 0x4d, 0x8f, 0x90, 0xc4, 0xb6, 0xc9, 0xfa, 0x7f, 0x8d, 0xaf, 0x90, 0xee, 0x65, + 0xaa, 0x9c, 0x1c, 0x0e, 0x0a, 0xf1, 0x7d, 0xad, 0xce, 0x97, 0x7a, 0x1d, 0xe6, 0x5b, 0x6c, 0xbe, + 0x7a, 0xcf, 0xb0, 0xad, 0x03, 0x4c, 0xfc, 0x9c, 0xcc, 0xd6, 0x99, 0xe5, 0xe2, 0x2d, 0x21, 0x45, + 0x7f, 0x20, 0xc1, 0x62, 0xdf, 0xb3, 0x88, 0xde, 0x3a, 0xd6, 0xbb, 0x4e, 0xdb, 0xe8, 0x5a, 0xfe, + 0xb1, 0x7e, 0xf4, 0x28, 0x37, 0xbb, 0x12, 0xbf, 0x91, 0x5e, 0xbb, 0x73, 0xee, 0x01, 0x89, 0xfd, + 0x28, 0xed, 0x7b, 0x16, 0x29, 0x1f, 0xdf, 0x17, 0x08, 0xf7, 0x1e, 0xd5, 0x6c, 0xdf, 0x3b, 0x2e, + 0x2f, 0x0d, 0x07, 0x05, 0x75, 0x5f, 0xab, 0x47, 0xab, 0x1e, 0x68, 0x6a, 0x7f, 0xa2, 0x31, 0xd2, + 0x00, 0x70, 0x78, 0xd2, 0xb9, 0x04, 0xdb, 0x8b, 0xb5, 0x8b, 0x2b, 0x87, 0x16, 0x41, 0x41, 0x7f, + 0x22, 0x41, 0x9e, 0x9a, 0x19, 0x6e, 0xd3, 0xe3, 0x0e, 0x4d, 0x4f, 0xf7, 0x70, 0xdb, 0xf1, 0xcc, + 0x5c, 0x92, 0x6e, 0x47, 0xb9, 0xf9, 0x1f, 0xd3, 0x5a, 0x17, 0x33, 0xe2, 0x7e, 0xdf, 0x32, 0x4b, + 0xfb, 0xfb, 0xf5, 0xea, 0x70, 0x50, 0xc8, 0xed, 0x06, 0xe0, 0xe1, 0x01, 0x69, 0x0c, 0x5a, 0xcb, + 0xb9, 0x2f, 0xa8, 0x41, 0xef, 0x41, 0xb6, 0xed, 0x74, 0xbb, 0xb8, 0x4d, 0x67, 0xa8, 0xef, 0x6b, + 0xf5, 0x9c, 0xc2, 0x0e, 0x6f, 0x61, 0x38, 0x28, 0x64, 0x2a, 0x61, 0x0d, 0x3d, 0xc6, 0x4c, 0x3b, + 0x5a, 0xcc, 0x57, 0xe0, 0xca, 0xa9, 0x5b, 0x1c, 0x35, 0xd3, 0x14, 0x37, 0xd3, 0x25, 0x98, 0x7d, + 0x64, 0x74, 0xfb, 0x5c, 0xa7, 0x52, 0x1a, 0x2f, 0xdc, 0x8e, 0xbd, 0x27, 0x15, 0x55, 0xc8, 0xf2, + 0x9d, 0xdb, 0xf5, 0x9c, 0x8e, 0x87, 0x09, 0x29, 0xfe, 0x45, 0x16, 0xb2, 0x1a, 0x26, 0xbe, 0xe3, + 0xe1, 0x40, 0xb3, 0xff, 0x51, 0x82, 0x45, 0x4a, 0x65, 0x9e, 0xe5, 0xfa, 0x8e, 0xa7, 0x7b, 0xf8, + 0xb1, 0x67, 0xf9, 0x98, 0xe4, 0x62, 0x4c, 0x23, 0xd6, 0xcf, 0x38, 0x95, 0x71, 0xa0, 0x52, 0x35, + 0x04, 0xd1, 0x04, 0x06, 0x57, 0x8a, 0x3b, 0x3f, 0xfa, 0x79, 0xe1, 0xf6, 0x54, 0x7b, 0x7e, 0x92, + 0x5d, 0x4b, 0xf5, 0xaa, 0x86, 0xcc, 0x13, 0xc0, 0xe8, 0x15, 0x90, 0xa9, 0x52, 0xe5, 0xe2, 0x2b, + 0xf1, 0x1b, 0xa9, 0xb2, 0x32, 0x1c, 0x14, 0x64, 0xaa, 0x76, 0x1a, 0x93, 0x8e, 0x19, 0x9a, 0xfc, + 0x12, 0x86, 0x76, 0x17, 0xd2, 0xbe, 0xd1, 0xea, 0x62, 0x9d, 0x8e, 0x4c, 0x84, 0x6d, 0x7c, 0x7d, + 0x62, 0x27, 0xc8, 0xc3, 0x6e, 0xcb, 0x20, 0xb8, 0xb4, 0x47, 0x5b, 0x46, 0xd6, 0x0e, 0x7e, 0x20, + 0x20, 0x68, 0x15, 0xd2, 0xce, 0x23, 0xec, 0x79, 0x96, 0x89, 0x75, 0xb3, 0xc5, 0x14, 0x3d, 0x55, + 0xce, 0x0e, 0x07, 0x05, 0xd8, 0x11, 0xe2, 0x6a, 0x59, 0x83, 0xa0, 0x49, 0xb5, 0x85, 0x7c, 0x58, + 0x12, 0x76, 0x1c, 0x1a, 0x27, 0xe3, 0xcf, 0x24, 0x9b, 0xc2, 0x77, 0xa7, 0x3f, 0x0c, 0x7e, 0xee, + 0x81, 0xf2, 0x30, 0x06, 0xe5, 0x8b, 0x44, 0xad, 0x13, 0x35, 0xe8, 0x4d, 0x58, 0x70, 0x3d, 0xec, + 0x1a, 0x1e, 0xd6, 0xdb, 0x4e, 0xcf, 0xed, 0x62, 0x1f, 0x9b, 0x4c, 0x53, 0x15, 0x4d, 0x15, 0x15, + 0x95, 0x40, 0x8e, 0xde, 0x80, 0x2c, 0xf1, 0x0d, 0x9f, 0x12, 0x3b, 0xc1, 0x1e, 0x6d, 0x99, 0x62, + 0x2d, 0x33, 0x4c, 0x5a, 0x17, 0x42, 0x74, 0x13, 0x54, 0xb6, 0x11, 0x44, 0x77, 0xfb, 0xad, 0xae, + 0x45, 0x0e, 0xb1, 0x99, 0x03, 0xd6, 0x70, 0x9e, 0xcb, 0x77, 0x03, 0x31, 0x3a, 0x1e, 0x53, 0xc0, + 0x36, 0xdd, 0x0e, 0xa3, 0x83, 0x73, 0xe9, 0x15, 0xe9, 0xc6, 0x6c, 0x79, 0xf3, 0xf9, 0xa0, 0x50, + 0x9d, 0x5a, 0x7b, 0x08, 0xee, 0xad, 0xfa, 0x1e, 0xc6, 0x11, 0x65, 0xac, 0x08, 0xbc, 0xa8, 0x1e, + 0x05, 0xb2, 0x09, 0x22, 0x9a, 0xfb, 0x52, 0x88, 0x68, 0x17, 0x92, 0x3e, 0xb6, 0x0d, 0xdb, 0x27, + 0xb9, 0x0c, 0x3b, 0xb6, 0x77, 0xa6, 0x3f, 0xb6, 0x3d, 0xd6, 0x31, 0xd0, 0x47, 0x01, 0x83, 0xaa, + 0x00, 0xfe, 0xb1, 0x1b, 0xa8, 0x63, 0x96, 0x81, 0xbe, 0xf1, 0x22, 0x75, 0x3c, 0x76, 0xa3, 0xda, + 0x98, 0xf2, 0x45, 0x99, 0xa0, 0x06, 0xcc, 0x91, 0xf6, 0x21, 0xee, 0x19, 0x02, 0x67, 0x9e, 0xe1, + 0x5c, 0x7f, 0x01, 0x4e, 0x93, 0x35, 0x8d, 0x20, 0xa5, 0x49, 0x28, 0x21, 0xf9, 0xff, 0x93, 0x60, + 0xe1, 0x84, 0xbd, 0xa3, 0x3d, 0x88, 0x85, 0x11, 0x0d, 0xa5, 0xcc, 0x18, 0x8b, 0x66, 0x2e, 0x63, + 0xfb, 0x31, 0xcb, 0x44, 0x1d, 0x48, 0x51, 0x0d, 0xb4, 0x7d, 0x1a, 0x2e, 0xc5, 0x18, 0x78, 0x63, + 0x38, 0x28, 0x28, 0xbb, 0x4c, 0x78, 0xe9, 0x21, 0x14, 0x0e, 0x5e, 0x37, 0x51, 0x01, 0xd2, 0xbe, + 0xa3, 0xe3, 0x27, 0x16, 0xf1, 0x2d, 0xbb, 0xc3, 0xfc, 0xac, 0xa2, 0x81, 0xef, 0xd4, 0x84, 0x24, + 0xff, 0x97, 0x31, 0x40, 0x27, 0x0d, 0x0b, 0xfd, 0xab, 0x04, 0xaf, 0x04, 0x3e, 0xd5, 0xf1, 0xac, + 0x8e, 0x65, 0x1b, 0xdd, 0x31, 0xe7, 0x2a, 0xb1, 0x9d, 0xfe, 0xe4, 0x32, 0xd6, 0x2b, 0x1c, 0xee, + 0x8e, 0x80, 0x9f, 0x74, 0xbc, 0xaf, 0x50, 0x07, 0xc5, 0x1d, 0xef, 0x89, 0x26, 0x0f, 0xb4, 0x5c, + 0xff, 0x05, 0x9d, 0xf3, 0xf7, 0xe0, 0xd5, 0x33, 0x81, 0x2f, 0xe2, 0x6e, 0xf2, 0x3f, 0x92, 0xe0, + 0xda, 0x0b, 0x9c, 0x40, 0x14, 0x27, 0xc3, 0x71, 0x3e, 0x8c, 0xe2, 0xa4, 0xd7, 0x7e, 0xf3, 0x12, + 0x8e, 0x26, 0x3a, 0x89, 0x6f, 0x42, 0x82, 0x1b, 0x11, 0xba, 0x1a, 0x6a, 0xa3, 0x5c, 0x4e, 0x70, + 0x6d, 0x64, 0xfa, 0x84, 0x40, 0x66, 0x9c, 0xca, 0x03, 0x41, 0xf6, 0xdd, 0x90, 0x15, 0x49, 0x8d, + 0x15, 0xdf, 0x81, 0x79, 0x31, 0x54, 0xe0, 0x30, 0xd1, 0xab, 0x00, 0x87, 0x56, 0xe7, 0x50, 0x7f, + 0x6c, 0xf8, 0xd8, 0x13, 0xc1, 0x71, 0x8a, 0x4a, 0x3e, 0xa2, 0x82, 0xe2, 0x67, 0x0a, 0x64, 0xea, + 0x3d, 0xd7, 0xf1, 0xfc, 0xc0, 0x9d, 0xde, 0x87, 0x04, 0xe7, 0x37, 0x71, 0xea, 0xa5, 0x33, 0xd6, + 0x35, 0xd6, 0x93, 0x3b, 0x12, 0x61, 0xfa, 0x02, 0x23, 0xf4, 0x73, 0xb1, 0x53, 0xfd, 0xdc, 0xfb, + 0x90, 0xe0, 0xf7, 0x13, 0x11, 0x5f, 0x17, 0x22, 0x63, 0x05, 0xb7, 0x83, 0xfa, 0xce, 0x86, 0xd5, + 0xc5, 0x1b, 0xac, 0x59, 0x00, 0xce, 0x3b, 0xa1, 0xaf, 0x83, 0x42, 0x88, 0xaf, 0x13, 0xeb, 0x87, + 0xdc, 0x4d, 0xc6, 0x79, 0x90, 0xdd, 0x6c, 0xee, 0x35, 0xad, 0x1f, 0x62, 0x2d, 0x49, 0x88, 0x4f, + 0x3f, 0x50, 0x1e, 0x94, 0xc7, 0x46, 0xb7, 0xcb, 0xdc, 0xe9, 0x2c, 0xbb, 0x54, 0x84, 0xe5, 0x71, + 0xe3, 0x4c, 0x7c, 0xb5, 0xc6, 0x29, 0x3c, 0xa3, 0x6b, 0xf8, 0x87, 0x2c, 0x9c, 0x4b, 0x69, 0xc0, + 0x45, 0xbb, 0x86, 0x7f, 0x88, 0x72, 0x90, 0x24, 0x06, 0x75, 0x52, 0x24, 0xa7, 0xac, 0xc4, 0x6f, + 0xcc, 0x69, 0x41, 0x11, 0x2d, 0x03, 0x73, 0xb1, 0xbc, 0xc8, 0xbc, 0x55, 0x5c, 0x8b, 0x48, 0xd8, + 0x3e, 0x1c, 0x59, 0xae, 0x7e, 0x70, 0x44, 0xb8, 0x8b, 0x12, 0xfb, 0x70, 0x64, 0xb9, 0x1b, 0xf7, + 0x88, 0x96, 0xa4, 0x95, 0x1b, 0x47, 0x84, 0x06, 0xd9, 0x96, 0xdd, 0xc1, 0xc4, 0xd7, 0x4d, 0xcb, + 0xc3, 0x6d, 0xbf, 0x7b, 0xcc, 0x7c, 0x94, 0xa2, 0x65, 0xb9, 0xb8, 0x2a, 0xa4, 0xd4, 0xf7, 0x4d, + 0xfa, 0x53, 0xe6, 0x5b, 0x14, 0x6d, 0x7e, 0xc2, 0x9d, 0x9e, 0xea, 0x26, 0x33, 0xa7, 0xbb, 0xc9, + 0x73, 0x02, 0xdc, 0xab, 0xbf, 0x82, 0x00, 0x37, 0xff, 0x59, 0x0c, 0x66, 0x99, 0xde, 0xa2, 0xdb, + 0x20, 0xd3, 0x63, 0x13, 0x57, 0xa3, 0x69, 0x83, 0x25, 0xd6, 0x87, 0x5a, 0xa4, 0x6d, 0xf4, 0x70, + 0x0e, 0xb1, 0x43, 0x65, 0xdf, 0xe8, 0x1a, 0x24, 0x09, 0x7e, 0xa8, 0x3f, 0x32, 0xba, 0xb9, 0x45, + 0x76, 0x62, 0x09, 0x82, 0x1f, 0x3e, 0x30, 0xba, 0xe8, 0x0a, 0x24, 0x2c, 0xa2, 0xdb, 0xf8, 0x71, + 0x6e, 0x89, 0xed, 0xd3, 0xac, 0x45, 0xb6, 0xf1, 0x63, 0x46, 0xde, 0x86, 0xd7, 0xc1, 0xbe, 0xde, + 0x76, 0xba, 0x24, 0x77, 0x85, 0x1a, 0x0c, 0x8d, 0xc5, 0xa8, 0xa8, 0xe2, 0x74, 0x09, 0xfa, 0x35, + 0x48, 0x3d, 0x36, 0x88, 0x8e, 0x7b, 0xae, 0x7f, 0xcc, 0x36, 0x4b, 0xa1, 0x6a, 0x4c, 0x6a, 0xb4, + 0xdc, 0x90, 0x95, 0x98, 0x1a, 0x6f, 0xc8, 0x4a, 0x5c, 0x95, 0x1b, 0xb2, 0x22, 0xab, 0xb3, 0x0d, + 0x59, 0x99, 0x55, 0x13, 0x0d, 0x59, 0x49, 0xa8, 0xc9, 0x86, 0xac, 0x24, 0x55, 0xa5, 0x21, 0x2b, + 0x8a, 0x9a, 0x6a, 0xc8, 0x4a, 0x4a, 0x85, 0x86, 0xac, 0x80, 0x9a, 0x6e, 0xc8, 0x4a, 0x5a, 0x9d, + 0x6b, 0xc8, 0xca, 0x9c, 0x9a, 0x69, 0xc8, 0x4a, 0x46, 0xcd, 0x36, 0x64, 0x25, 0xab, 0xce, 0x37, + 0x64, 0x65, 0x5e, 0x55, 0x1b, 0xb2, 0xa2, 0xaa, 0x0b, 0x0d, 0x59, 0x59, 0x50, 0x51, 0xf1, 0x7f, + 0x24, 0xc8, 0x72, 0x6b, 0x0f, 0x99, 0xe5, 0x4d, 0x58, 0x60, 0xfa, 0x67, 0xd9, 0x1d, 0xdd, 0x15, + 0x42, 0xc6, 0x19, 0x31, 0x4d, 0x0d, 0x2a, 0xc2, 0xc6, 0xaf, 0x41, 0xc6, 0xc3, 0x86, 0x39, 0x6a, + 0x18, 0x63, 0x0d, 0xe7, 0xa8, 0x30, 0x6c, 0xf4, 0x06, 0x64, 0x19, 0x1d, 0x8e, 0x5a, 0xc5, 0x59, + 0xab, 0x0c, 0x93, 0x86, 0xcd, 0xca, 0x90, 0x21, 0xae, 0x61, 0x8f, 0x5a, 0xc9, 0x8c, 0xa8, 0xae, + 0x9d, 0x42, 0x1e, 0x4d, 0xd7, 0xb0, 0x05, 0x69, 0xcc, 0xd1, 0x3e, 0x51, 0x5a, 0xf4, 0x30, 0xe9, + 0xf7, 0xb0, 0xee, 0x3a, 0x3c, 0x40, 0x8e, 0x6b, 0x29, 0x2e, 0xd9, 0x75, 0x48, 0xf1, 0x77, 0xe1, + 0x1a, 0x8d, 0x43, 0x78, 0x0c, 0x51, 0x39, 0x34, 0xec, 0x4e, 0x78, 0xdd, 0x30, 0x20, 0xc9, 0x62, + 0x99, 0x30, 0x50, 0xd8, 0x1c, 0x0e, 0x0a, 0x09, 0xda, 0xfa, 0xd2, 0x64, 0x91, 0xa0, 0xc0, 0x75, + 0xb3, 0x98, 0x87, 0xdc, 0xe4, 0xe8, 0xe1, 0x05, 0x48, 0x63, 0xf7, 0x9f, 0x7e, 0x0f, 0xd3, 0xa5, + 0xdd, 0xb7, 0x88, 0x8f, 0xbe, 0x07, 0x73, 0x62, 0x29, 0x74, 0x85, 0x01, 0x6d, 0x9f, 0xb3, 0x1b, + 0x69, 0x2f, 0x04, 0x21, 0xc5, 0x7f, 0x92, 0x60, 0xb1, 0xea, 0x39, 0xae, 0x8b, 0x4d, 0xa1, 0xdf, + 0x7c, 0xa9, 0x81, 0x5a, 0x4b, 0x11, 0xb5, 0xde, 0x86, 0x58, 0xbd, 0x2a, 0xa2, 0x98, 0x3b, 0x97, + 0x0d, 0x8e, 0xea, 0x55, 0xf4, 0x1d, 0x48, 0xd0, 0xb8, 0xbb, 0x4f, 0x98, 0x0b, 0xc8, 0xae, 0xfd, + 0xfa, 0x19, 0xee, 0xa6, 0xc9, 0x1a, 0x6a, 0xa2, 0x43, 0xf1, 0x8f, 0x12, 0x70, 0x25, 0xba, 0x47, + 0x77, 0x2b, 0xc1, 0xc4, 0x3f, 0x85, 0xa4, 0x65, 0x9b, 0xf8, 0x49, 0xe8, 0xc4, 0xde, 0x3f, 0x0b, + 0xf5, 0x34, 0x88, 0x92, 0xd8, 0x8f, 0x3a, 0x85, 0x09, 0xc2, 0x59, 0x81, 0x89, 0xbe, 0x1f, 0xba, + 0x48, 0x7e, 0xc7, 0xbc, 0xfd, 0xd2, 0xe8, 0xd5, 0x09, 0x77, 0x39, 0xe6, 0x8d, 0xe2, 0xcc, 0xa5, + 0x7d, 0x35, 0xde, 0xa8, 0x09, 0x0b, 0x96, 0xed, 0x63, 0xaf, 0x8b, 0x8d, 0x47, 0x94, 0x8c, 0xe9, + 0xf0, 0xe2, 0xaa, 0x39, 0x2d, 0xf5, 0xa9, 0x11, 0x00, 0x4e, 0xa1, 0x9f, 0xc2, 0x62, 0x14, 0x34, + 0x38, 0x82, 0xb3, 0xaf, 0x9f, 0x6c, 0x87, 0x47, 0xb0, 0xc1, 0x2d, 0x2f, 0x02, 0x54, 0xe7, 0x38, + 0xf9, 0x3f, 0x93, 0x60, 0x2e, 0x7a, 0x2c, 0xc8, 0x02, 0x85, 0x8d, 0x11, 0xd8, 0x62, 0xbc, 0xbc, + 0x4d, 0xfd, 0x1e, 0xab, 0x64, 0x7b, 0xf5, 0xc1, 0x4b, 0xef, 0x15, 0x87, 0x10, 0x47, 0x5e, 0x37, + 0x29, 0xf9, 0x9a, 0x9e, 0xe3, 0x8e, 0x72, 0x5f, 0x71, 0x4d, 0xa1, 0x02, 0xea, 0x4f, 0xf2, 0xbf, + 0x07, 0xa9, 0xf0, 0x40, 0x23, 0x77, 0x88, 0xf8, 0x97, 0x78, 0x87, 0x38, 0x6b, 0xfc, 0xe2, 0xcf, + 0x13, 0xb0, 0x78, 0x1a, 0x55, 0x7d, 0x0c, 0x6a, 0x84, 0x19, 0xf4, 0xae, 0x45, 0x7c, 0xa1, 0xb1, + 0x37, 0xcf, 0x0e, 0x56, 0x23, 0xf4, 0x22, 0xce, 0x23, 0xeb, 0x8d, 0x93, 0xce, 0x0f, 0x20, 0x6b, + 0xf2, 0x25, 0xeb, 0xc2, 0x14, 0xe2, 0xe7, 0x46, 0x8b, 0xa7, 0x50, 0x8c, 0x40, 0xcf, 0x98, 0x91, + 0x2a, 0xc2, 0x72, 0x7c, 0x01, 0xba, 0x69, 0xf8, 0x06, 0xd5, 0x13, 0x7a, 0xc6, 0x32, 0x63, 0x9d, + 0xe6, 0x70, 0x50, 0x58, 0x10, 0x58, 0x55, 0x51, 0x7b, 0xe9, 0x3d, 0x5e, 0x30, 0x27, 0x00, 0x4d, + 0x4a, 0xf4, 0xb4, 0x9e, 0x0e, 0x3c, 0x3b, 0x22, 0x7a, 0xaa, 0xa9, 0x97, 0x27, 0x7a, 0xfa, 0x59, + 0x37, 0xd1, 0x1f, 0x4a, 0xb0, 0xc0, 0x13, 0x35, 0xbd, 0xbe, 0x6f, 0xb0, 0x1c, 0x5b, 0x18, 0x85, + 0x7e, 0x3c, 0x1c, 0x14, 0xe6, 0xd9, 0x86, 0x6c, 0x89, 0x3a, 0x36, 0x6c, 0xf9, 0x65, 0x87, 0x1d, + 0xa1, 0x88, 0xc8, 0x2c, 0x14, 0x98, 0xe8, 0x1e, 0x64, 0x79, 0x48, 0xad, 0xd3, 0xa0, 0xd2, 0x72, + 0x6c, 0x16, 0x9e, 0x66, 0xca, 0xaf, 0x3f, 0x1f, 0x14, 0x56, 0x4e, 0xd1, 0x2c, 0x1e, 0x8d, 0x3f, + 0xe0, 0x6d, 0xb5, 0xcc, 0x41, 0xb4, 0x88, 0xda, 0x90, 0x09, 0x55, 0xe3, 0xd8, 0x15, 0xd1, 0xec, + 0xe5, 0x9d, 0xc5, 0x5c, 0xa0, 0x23, 0x14, 0x13, 0x75, 0x60, 0x3e, 0x18, 0x84, 0x5f, 0xeb, 0x49, + 0x2e, 0xf5, 0xa5, 0x0c, 0x13, 0xa8, 0x35, 0x5f, 0x35, 0x11, 0x17, 0xab, 0xab, 0xb0, 0x74, 0xaa, + 0x37, 0xfe, 0xf3, 0x59, 0xb8, 0x3a, 0xce, 0xf0, 0x61, 0x84, 0xa1, 0x4f, 0xfa, 0xa0, 0x0f, 0xa6, + 0xf6, 0x12, 0x01, 0x06, 0xa7, 0xa1, 0xa0, 0x34, 0xe9, 0x85, 0x3e, 0x9d, 0xf0, 0x42, 0x2f, 0x81, + 0xcf, 0xd4, 0x6b, 0x02, 0x5f, 0x80, 0xe6, 0xff, 0x5d, 0x82, 0xcc, 0xd8, 0xf8, 0xbf, 0x4c, 0xba, + 0xdd, 0x0d, 0xa3, 0x02, 0xfe, 0x76, 0xf3, 0xde, 0xc5, 0xd7, 0x36, 0x1e, 0x2c, 0xe4, 0xff, 0x45, + 0x82, 0xcc, 0xd8, 0x72, 0xbf, 0x22, 0xa2, 0xfe, 0xd2, 0x67, 0x5e, 0xfc, 0x36, 0x24, 0xb8, 0x04, + 0x21, 0xc8, 0x7e, 0xb4, 0x5e, 0xdf, 0xab, 0x6f, 0xdf, 0xd5, 0x37, 0x76, 0x34, 0xfd, 0x6e, 0x45, + 0x9d, 0x41, 0x73, 0xa0, 0x54, 0x6b, 0xf7, 0x6b, 0x54, 0xa8, 0x4a, 0x28, 0x0d, 0x49, 0x56, 0xaa, + 0x55, 0xd5, 0x58, 0xb1, 0x0c, 0x2a, 0xc7, 0x3e, 0xc0, 0x94, 0x5a, 0xe9, 0x45, 0x02, 0x95, 0x60, + 0x91, 0xc2, 0xe2, 0x1e, 0x8d, 0x31, 0xa8, 0x33, 0xd1, 0x23, 0x11, 0xde, 0x42, 0x58, 0x45, 0xdd, + 0xca, 0xb6, 0xd1, 0xc3, 0xc5, 0x7f, 0x96, 0x61, 0x61, 0x04, 0x12, 0x38, 0x16, 0x7a, 0xe1, 0xb4, + 0xec, 0x23, 0x7d, 0xf4, 0x9a, 0xc3, 0x2f, 0x9c, 0x96, 0x7d, 0xb4, 0xaf, 0xd5, 0xb5, 0x24, 0xad, + 0xdc, 0xf7, 0x2c, 0xd4, 0x00, 0xd9, 0x71, 0xfd, 0x20, 0x40, 0xff, 0xd6, 0x19, 0x5b, 0x71, 0x62, + 0x8c, 0xd2, 0x8e, 0xeb, 0xf3, 0xd4, 0x8b, 0xc6, 0x30, 0xd0, 0xdf, 0x4b, 0x90, 0xe4, 0xb7, 0x21, + 0x92, 0x4b, 0x30, 0xbc, 0xef, 0x5c, 0x08, 0x8f, 0x6f, 0x80, 0x48, 0xe9, 0x7f, 0x44, 0x55, 0xfd, + 0xf9, 0xa0, 0xb0, 0x30, 0xb9, 0x41, 0xe4, 0x92, 0xb9, 0xfe, 0x60, 0x8a, 0xa8, 0xc1, 0xb3, 0xcc, + 0xa3, 0x8d, 0x66, 0x94, 0x3a, 0x65, 0x22, 0x3f, 0x33, 0x76, 0x10, 0xf9, 0x0e, 0xcc, 0x45, 0x67, + 0x7f, 0x4a, 0x2e, 0x6a, 0x7d, 0x3c, 0x17, 0xf5, 0xe6, 0x54, 0x3b, 0xc3, 0x31, 0xa3, 0xb9, 0xa7, + 0x6f, 0x43, 0x2a, 0xdc, 0xf6, 0x8b, 0x64, 0xce, 0x38, 0x4b, 0x86, 0x97, 0xd0, 0x59, 0x35, 0x51, + 0xfc, 0x07, 0x09, 0xe6, 0x34, 0x4c, 0x9c, 0xee, 0x23, 0x6c, 0xd2, 0xa8, 0x01, 0x7d, 0x03, 0x64, + 0x1a, 0x85, 0x88, 0xfb, 0xf5, 0x39, 0xd7, 0x13, 0xd6, 0x14, 0xad, 0x43, 0x2a, 0xcc, 0x12, 0x5c, + 0xe4, 0xc1, 0x71, 0xd4, 0x0b, 0xdd, 0x04, 0xb5, 0xe5, 0xf4, 0x6d, 0xd3, 0xf0, 0x8e, 0x75, 0x0f, + 0x1b, 0xed, 0x43, 0x6c, 0x8a, 0xbc, 0xe8, 0x7c, 0x20, 0xd7, 0xb8, 0xb8, 0xf8, 0xe3, 0x18, 0xa0, + 0xd1, 0xe6, 0x44, 0x68, 0x82, 0xc6, 0x3e, 0x6c, 0x1d, 0xe2, 0x82, 0x15, 0x3b, 0x35, 0xef, 0x3c, + 0x11, 0x42, 0x85, 0x0b, 0x0f, 0x8e, 0xd4, 0x8b, 0xc8, 0x08, 0xfa, 0xd3, 0xb3, 0x73, 0x21, 0x71, + 0x96, 0x0b, 0x61, 0x5a, 0xfa, 0x4b, 0xcd, 0x87, 0x08, 0x77, 0xf7, 0xbf, 0x32, 0xa0, 0x8a, 0x87, + 0x0d, 0x1f, 0x53, 0xe6, 0x21, 0x67, 0xdd, 0x07, 0xcb, 0x30, 0xcb, 0x2f, 0x0f, 0xb1, 0x8b, 0x5c, + 0x1e, 0xc4, 0xa6, 0xf0, 0xae, 0xe8, 0xb7, 0x60, 0xae, 0xed, 0x74, 0xfb, 0x3d, 0x5b, 0x67, 0x4f, + 0x30, 0x22, 0x94, 0xfc, 0x8d, 0xb3, 0x94, 0xf8, 0xc4, 0xe4, 0x4a, 0x15, 0xa7, 0x4b, 0xcb, 0xc1, + 0xfd, 0x96, 0x03, 0xb2, 0x16, 0xe8, 0x15, 0x48, 0x85, 0x06, 0xc5, 0x82, 0xc8, 0x94, 0x36, 0x12, + 0xa0, 0x35, 0x98, 0x35, 0x88, 0xee, 0x1c, 0xb0, 0x28, 0xef, 0x3c, 0x0d, 0xd3, 0x64, 0x83, 0xec, + 0x1c, 0xa0, 0x77, 0x21, 0x73, 0xf0, 0x90, 0x47, 0xbe, 0x9c, 0x40, 0xf9, 0xcb, 0xd8, 0xfc, 0x70, + 0x50, 0x48, 0x6f, 0x7c, 0xc8, 0x16, 0x4b, 0xe9, 0x53, 0x4b, 0x1f, 0x3c, 0x0c, 0x0b, 0xe8, 0x16, + 0x2c, 0xf4, 0x8c, 0x27, 0xfa, 0x81, 0x67, 0xb4, 0x45, 0xa8, 0xd7, 0xe5, 0xac, 0x20, 0x69, 0xf3, + 0x3d, 0xe3, 0xc9, 0x86, 0x90, 0xd7, 0xcd, 0x2e, 0xce, 0xff, 0x42, 0x82, 0xa4, 0x58, 0x11, 0x72, + 0x01, 0xc4, 0xf6, 0x58, 0x26, 0x0f, 0x26, 0x32, 0xe5, 0x0f, 0x87, 0x83, 0x42, 0xaa, 0xc2, 0xa4, + 0xf5, 0x2a, 0x79, 0x3e, 0x28, 0x7c, 0xef, 0x65, 0x49, 0x2b, 0x00, 0xd1, 0x52, 0x7c, 0x90, 0xba, + 0xc9, 0xb2, 0x35, 0x87, 0x06, 0xd1, 0x0f, 0x2d, 0xe2, 0x3b, 0x1d, 0xcf, 0xe8, 0xb1, 0xc3, 0x55, + 0xb4, 0xb9, 0x43, 0x83, 0x6c, 0x06, 0x32, 0x94, 0xa7, 0xe1, 0xc0, 0x23, 0xfe, 0x82, 0xc6, 0x4d, + 0x2a, 0x2c, 0xa3, 0x35, 0xb8, 0x12, 0x76, 0xd6, 0xe9, 0xa2, 0x5b, 0xfd, 0xf6, 0x11, 0x66, 0x9e, + 0x80, 0x72, 0xd6, 0x62, 0x58, 0xb9, 0x65, 0x3c, 0x29, 0xf3, 0xaa, 0xe2, 0x15, 0x58, 0x8c, 0x1c, + 0x6b, 0x18, 0x62, 0xfd, 0x15, 0x40, 0x72, 0xd7, 0x38, 0xee, 0x3a, 0x86, 0x89, 0x56, 0x20, 0x1d, + 0xbc, 0x81, 0xd1, 0x20, 0x95, 0xeb, 0x61, 0x54, 0x44, 0x27, 0xd5, 0x27, 0xd8, 0x63, 0x67, 0xc2, + 0x59, 0x2a, 0x2c, 0x8b, 0x87, 0x3f, 0x3a, 0x3f, 0xbd, 0x67, 0xb5, 0x3d, 0x87, 0xa7, 0x1c, 0xe2, + 0x8c, 0x6d, 0xa9, 0x74, 0x8b, 0x09, 0xd1, 0x75, 0x98, 0x3f, 0xb0, 0x6c, 0x96, 0xb2, 0x0c, 0xda, + 0xb1, 0xe4, 0xb2, 0x96, 0x0d, 0xc4, 0xa2, 0xe1, 0x23, 0xc8, 0x46, 0x9e, 0xfd, 0xe8, 0xd9, 0x24, + 0xd8, 0xd9, 0xec, 0x0c, 0x07, 0x85, 0xcc, 0x48, 0xd7, 0xf9, 0xf9, 0x5c, 0xc6, 0xa9, 0x64, 0x46, + 0xc3, 0xd0, 0xd3, 0x59, 0x82, 0x59, 0xf6, 0xc3, 0x20, 0xfe, 0x16, 0xaf, 0xf1, 0x02, 0xfa, 0x16, + 0xcc, 0x76, 0xb1, 0x41, 0x78, 0x7e, 0x38, 0xbd, 0xb6, 0x72, 0x86, 0xf5, 0xb0, 0xdf, 0xf0, 0x68, + 0xbc, 0x39, 0x2a, 0x43, 0x82, 0x27, 0xa1, 0x59, 0xea, 0x38, 0xbd, 0x76, 0x63, 0xda, 0x9f, 0x50, + 0x6c, 0xce, 0x68, 0xa2, 0x27, 0xaa, 0x41, 0xd2, 0xe3, 0xef, 0x0e, 0x2c, 0xa1, 0x7c, 0xee, 0xfd, + 0x32, 0xf2, 0x18, 0xb2, 0x39, 0xa3, 0x05, 0x7d, 0xd1, 0x5e, 0xf0, 0xc0, 0xc7, 0x69, 0x58, 0x3c, + 0x67, 0x96, 0xa6, 0x8c, 0xa0, 0x46, 0x80, 0x63, 0x28, 0x74, 0x81, 0x16, 0xcb, 0x5c, 0xb2, 0xbc, + 0xf4, 0xd9, 0x0b, 0x1c, 0x7b, 0xd0, 0xa0, 0x0b, 0xe4, 0x3d, 0xd1, 0x36, 0x40, 0x3b, 0x74, 0x0d, + 0xb9, 0x2c, 0xc3, 0x79, 0xeb, 0x22, 0xe1, 0xc7, 0xe6, 0x8c, 0x16, 0x41, 0x40, 0x1f, 0x42, 0xba, + 0x3d, 0xd2, 0xf5, 0xdc, 0x3c, 0x03, 0x7c, 0xfb, 0x42, 0x84, 0xb7, 0x49, 0x49, 0x6e, 0x24, 0x1d, + 0x27, 0x39, 0x75, 0x92, 0xe4, 0x6a, 0x90, 0x11, 0xa9, 0x00, 0xfe, 0x9b, 0xb2, 0xdc, 0x02, 0xe3, + 0xd8, 0xa8, 0x96, 0x04, 0xbf, 0x3a, 0x2b, 0xd5, 0xec, 0xb6, 0x63, 0x62, 0xb3, 0x46, 0xcb, 0x9a, + 0xc8, 0x2d, 0xb2, 0x02, 0x41, 0x77, 0x21, 0xdb, 0xee, 0x62, 0xc3, 0xee, 0xbb, 0x01, 0x0e, 0x9a, + 0x12, 0x27, 0x23, 0xfa, 0x09, 0xa0, 0x6d, 0x40, 0x07, 0xec, 0x99, 0x31, 0x3a, 0x2b, 0x96, 0x28, + 0x9f, 0x06, 0x4c, 0x65, 0x7d, 0xb5, 0xd1, 0xcc, 0xd0, 0xeb, 0x90, 0xb1, 0x1d, 0xbb, 0x6d, 0xd8, + 0x6d, 0xdc, 0x65, 0xee, 0x88, 0xe7, 0xd6, 0xc7, 0x85, 0xe8, 0x13, 0xc8, 0x92, 0xb1, 0x98, 0x3b, + 0x77, 0x85, 0x8d, 0xf8, 0xce, 0x45, 0x13, 0x78, 0x9b, 0x33, 0xda, 0x04, 0x12, 0xfa, 0x6d, 0x50, + 0xfd, 0x89, 0xa4, 0x2d, 0xcb, 0xd2, 0x9f, 0xfd, 0x1e, 0xff, 0x82, 0x2c, 0xf3, 0xe6, 0x8c, 0x76, + 0x02, 0xad, 0x9c, 0x82, 0xa4, 0xc9, 0xab, 0xc3, 0xe4, 0x7e, 0x52, 0x55, 0x8a, 0xbf, 0x48, 0x80, + 0x12, 0x46, 0x2b, 0xab, 0x80, 0x42, 0xff, 0x32, 0xfa, 0x29, 0x04, 0x25, 0xca, 0xd8, 0xe6, 0x8c, + 0xb6, 0x10, 0xd4, 0x8d, 0x7e, 0x0d, 0x71, 0x1d, 0xe6, 0x7b, 0x8e, 0x69, 0x1d, 0x58, 0x23, 0xb6, + 0xe3, 0xe9, 0xa5, 0x6c, 0x20, 0x16, 0x6c, 0x77, 0x67, 0xec, 0x21, 0x31, 0x3e, 0x85, 0xaf, 0xdc, + 0x9c, 0x89, 0xbc, 0x34, 0x52, 0xf6, 0xf5, 0xfa, 0xb6, 0x6d, 0xd9, 0x1d, 0x5d, 0x5c, 0x90, 0xb8, + 0x27, 0xce, 0x08, 0xa9, 0xb8, 0xe3, 0x54, 0x26, 0xe8, 0xe8, 0xe6, 0xb9, 0x74, 0x14, 0xac, 0x7d, + 0x53, 0x0a, 0xf9, 0x68, 0x63, 0x92, 0x8f, 0x6e, 0x9d, 0xcf, 0x47, 0x11, 0x98, 0x90, 0x90, 0xf6, + 0x4f, 0x25, 0xa4, 0xd5, 0x29, 0xb5, 0x25, 0x82, 0x38, 0xce, 0x48, 0x95, 0x09, 0x46, 0xba, 0x79, + 0x2e, 0x23, 0x45, 0xd7, 0x28, 0x28, 0x69, 0xe7, 0x14, 0x4a, 0x7a, 0x7b, 0x2a, 0x4a, 0x8a, 0x80, + 0x45, 0x39, 0x49, 0x3b, 0x8d, 0x93, 0x4a, 0xd3, 0x71, 0x52, 0x04, 0x72, 0x8c, 0x94, 0x7e, 0x70, + 0xc2, 0xe0, 0x54, 0x06, 0xfb, 0x8d, 0x0b, 0xdf, 0x8a, 0x37, 0xa5, 0x13, 0x16, 0x67, 0x9c, 0x62, + 0x71, 0x0b, 0x0c, 0xfe, 0xdd, 0x0b, 0x58, 0x5c, 0x64, 0x80, 0x93, 0x26, 0x07, 0xa0, 0x04, 0xaf, + 0x4c, 0x11, 0xf3, 0x2b, 0xfe, 0x44, 0x82, 0x78, 0xc3, 0x69, 0xa1, 0xec, 0x28, 0x85, 0xc0, 0x2e, + 0xff, 0x1f, 0x8c, 0x9a, 0x8b, 0x78, 0xf8, 0xb5, 0x33, 0x66, 0x12, 0x8c, 0xac, 0x85, 0x9d, 0xd0, + 0x77, 0x21, 0xe9, 0xf2, 0x58, 0x47, 0x58, 0x58, 0xf1, 0xac, 0xfe, 0xbc, 0xa5, 0x16, 0x74, 0xb9, + 0x75, 0x13, 0xb2, 0xe3, 0x3f, 0x64, 0x45, 0x59, 0x80, 0x5d, 0x83, 0x10, 0xf7, 0xd0, 0x33, 0x08, + 0x56, 0x67, 0x50, 0x12, 0xe2, 0xf7, 0xb6, 0x9a, 0xaa, 0x74, 0xeb, 0xfb, 0xd1, 0xa4, 0x42, 0x55, + 0x5b, 0xaf, 0x6f, 0xd7, 0xb7, 0xef, 0xea, 0xdb, 0xeb, 0x5b, 0xb5, 0xa6, 0x3a, 0x83, 0x72, 0xb0, + 0xf4, 0xd1, 0x7a, 0x7d, 0x4f, 0x64, 0x19, 0xf4, 0xfa, 0xf6, 0x5e, 0x4d, 0x7b, 0xb0, 0x7e, 0x5f, + 0x95, 0xd0, 0x55, 0x40, 0xda, 0x4e, 0xe5, 0x5e, 0xb3, 0x5a, 0xd6, 0x2b, 0x3b, 0x5b, 0xbb, 0xeb, + 0x95, 0xbd, 0xfa, 0xce, 0xb6, 0x1a, 0x43, 0x0a, 0xc8, 0xd5, 0x9d, 0xed, 0x9a, 0x0a, 0xb7, 0x7e, + 0x12, 0x07, 0x99, 0xee, 0x31, 0x7a, 0x1d, 0xd2, 0xfb, 0xdb, 0xcd, 0xdd, 0x5a, 0xa5, 0xbe, 0x51, + 0xaf, 0x55, 0xd5, 0x99, 0xfc, 0xe2, 0xd3, 0x67, 0x2b, 0xf3, 0xb4, 0x6a, 0xdf, 0x26, 0x2e, 0x6e, + 0x33, 0x72, 0x41, 0x79, 0x48, 0x94, 0xd7, 0x2b, 0xf7, 0xf6, 0x77, 0x55, 0x29, 0x9f, 0x7d, 0xfa, + 0x6c, 0x05, 0x68, 0x03, 0x6e, 0xd8, 0xe8, 0x15, 0x48, 0x6a, 0xb5, 0xe6, 0xde, 0x8e, 0x56, 0x53, + 0x63, 0xf9, 0xf9, 0xa7, 0xcf, 0x56, 0xd2, 0xb4, 0x52, 0xd8, 0x2b, 0xba, 0x0e, 0x99, 0x66, 0x65, + 0xb3, 0xb6, 0xb5, 0xae, 0x57, 0x36, 0xd7, 0xb7, 0xef, 0xd6, 0xd4, 0x78, 0x7e, 0xe9, 0xe9, 0xb3, + 0x15, 0x75, 0xf2, 0x80, 0xe9, 0x10, 0xf5, 0xad, 0xdd, 0x1d, 0x6d, 0x4f, 0x95, 0x47, 0x43, 0x70, + 0xbb, 0x42, 0x45, 0x00, 0xde, 0x7b, 0xa3, 0x56, 0xab, 0xaa, 0xb3, 0x79, 0xf4, 0xf4, 0xd9, 0x4a, + 0x96, 0xd6, 0x8f, 0xcc, 0x05, 0xbd, 0x01, 0x73, 0x15, 0xad, 0xb6, 0xbe, 0x57, 0xd3, 0x9b, 0x7b, + 0xeb, 0x7b, 0x4d, 0x35, 0x31, 0x5a, 0x49, 0xc4, 0x04, 0x50, 0x09, 0x16, 0xd6, 0xf7, 0xf7, 0x76, + 0xf4, 0xb1, 0xb6, 0xc9, 0xfc, 0xb5, 0xa7, 0xcf, 0x56, 0x16, 0x69, 0xdb, 0xf5, 0xbe, 0xef, 0x44, + 0xdb, 0xbf, 0x05, 0xea, 0xd8, 0xfc, 0xf5, 0xbb, 0x15, 0x55, 0xc9, 0x5f, 0x7d, 0xfa, 0x6c, 0x05, + 0x4d, 0x2e, 0xe1, 0x6e, 0x05, 0x7d, 0x13, 0xae, 0xee, 0x7d, 0xbc, 0x5b, 0xab, 0xd6, 0x9a, 0x15, + 0x7d, 0x7c, 0xd9, 0xa9, 0x7c, 0xee, 0xe9, 0xb3, 0x95, 0x25, 0xda, 0x67, 0xb2, 0x5f, 0x5e, 0xf9, + 0xf1, 0xdf, 0x2c, 0xcf, 0xfc, 0xdd, 0xdf, 0x2e, 0xcf, 0x94, 0x6f, 0x7c, 0xfe, 0x5f, 0xcb, 0x33, + 0x9f, 0x0f, 0x97, 0xa5, 0x9f, 0x0e, 0x97, 0xa5, 0x9f, 0x0d, 0x97, 0xa5, 0xff, 0x1c, 0x2e, 0x4b, + 0x7f, 0xfc, 0xc5, 0xf2, 0xcc, 0x4f, 0xbf, 0x58, 0x9e, 0xf9, 0xd9, 0x17, 0xcb, 0x33, 0x9f, 0x24, + 0xb8, 0x5a, 0xb5, 0x12, 0xec, 0xb7, 0xde, 0xef, 0xfe, 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x7b, + 0xcd, 0x87, 0x15, 0xba, 0x2e, 0x00, 0x00, } diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 39bcb4e38c3c..8358a3d0005e 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -156,6 +156,7 @@ message ImportDetails { string name = 18; int64 seq_val = 19; bool is_new = 20; + bool was_empty = 22; repeated string target_cols = 21; reserved 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17; } diff --git a/pkg/sql/gcjob/table_garbage_collection.go b/pkg/sql/gcjob/table_garbage_collection.go index 29c96c1e292f..b7b70bf0c1e5 100644 --- a/pkg/sql/gcjob/table_garbage_collection.go +++ b/pkg/sql/gcjob/table_garbage_collection.go @@ -67,7 +67,7 @@ func gcTables( } // First, delete all the table data. - if err := clearTableData(ctx, execCfg.DB, execCfg.DistSender, execCfg.Codec, table); err != nil { + if err := ClearTableData(ctx, execCfg.DB, execCfg.DistSender, execCfg.Codec, table); err != nil { return false, errors.Wrapf(err, "clearing data for table %d", table.ID) } @@ -83,8 +83,8 @@ func gcTables( return didGC, nil } -// clearTableData deletes all of the data in the specified table. -func clearTableData( +// ClearTableData deletes all of the data in the specified table. +func ClearTableData( ctx context.Context, db *kv.DB, distSender *kvcoord.DistSender, From 19b14805f7ab0dd22c691f2c5a8f12198e677975 Mon Sep 17 00:00:00 2001 From: Rebecca Taft Date: Tue, 18 Aug 2020 20:48:14 -0500 Subject: [PATCH 2/9] opt: add index acceleration support for ~ and && bounding box operators This commit adds index acceleration support for the bounding box comparison operators, ~ and &&. It maps ~ to Covers and && to Intersects. Release note (performance improvement): The ~ and && geospatial bounding box operations can now benefit from index acceleration if one of the operands is an indexed geometry column. --- .../logic_test/inverted_filter_geospatial | 54 ++++ .../inverted_filter_geospatial_dist | 28 ++ .../inverted_filter_geospatial_explain_local | 72 +++++ .../logic_test/inverted_join_geospatial | 68 +++++ .../logic_test/inverted_join_geospatial_dist | 22 ++ .../inverted_join_geospatial_explain | 76 +++++ pkg/sql/opt/invertedidx/geo.go | 262 +++++++++++------- pkg/sql/opt/invertedidx/geo_test.go | 126 ++++++++- pkg/sql/opt/operator.go | 7 +- pkg/sql/opt/ops/scalar.opt | 16 ++ pkg/sql/opt/optbuilder/scalar.go | 22 +- pkg/sql/opt/optbuilder/testdata/scalar | 94 +++++++ pkg/sql/opt/xform/custom_funcs.go | 2 +- pkg/sql/opt/xform/testdata/rules/join | 101 +++++++ pkg/sql/opt/xform/testdata/rules/select | 114 ++++++++ 15 files changed, 957 insertions(+), 107 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial b/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial index 6bd2dcb43a8d..cfa6bd388616 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial +++ b/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial @@ -45,6 +45,60 @@ SELECT k FROM geo_table WHERE ST_DWithin('POINT(2.5 2.5)'::geometry, geom, 1) OR 3 6 +# Bounding box operations. +statement ok +SET CLUSTER SETTING sql.spatial.experimental_box2d_comparison_operators.enabled = on + +query I +SELECT k FROM geo_table WHERE 'POINT(3.0 3.0)'::geometry && geom ORDER BY k +---- +3 +6 + +query I +SELECT k FROM geo_table WHERE 'POINT(3.0 3.0)'::geometry::box2d && geom ORDER BY k +---- +3 +6 + +query I +SELECT k FROM geo_table WHERE ST_Covers('LINESTRING(1.0 1.0, 5.0 5.0)'::geometry, geom) ORDER BY k +---- +1 +2 +3 +4 + +# Note that the result of the `~` bounding box operation includes an extra +# result not present in the previous result of ST_Covers. +query I +SELECT k FROM geo_table WHERE 'LINESTRING(1.0 1.0, 5.0 5.0)'::geometry ~ geom ORDER BY k +---- +1 +2 +3 +4 +6 + +query I +SELECT k FROM geo_table WHERE 'LINESTRING(1.0 1.0, 5.0 5.0)'::geometry::box2d ~ geom ORDER BY k +---- +1 +2 +3 +4 +6 + +query I +SELECT k FROM geo_table WHERE geom ~ 'LINESTRING(1.0 1.0, 5.0 5.0)'::geometry ORDER BY k +---- +6 + +query I +SELECT k FROM geo_table WHERE geom ~ 'LINESTRING(1.0 1.0, 5.0 5.0)'::geometry::box2d ORDER BY k +---- +6 + statement ok CREATE TABLE geo_table2( k int, diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_dist b/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_dist index 83bf1424e826..a18e2bd4b696 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_dist +++ b/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_dist @@ -143,3 +143,31 @@ SELECT url FROM [EXPLAIN (DISTSQL) SELECT k FROM geo_table WHERE ST_CoveredBy('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k] ---- https://cockroachdb.github.io/distsqlplan/decode.html#eJyUU99P2zAQft9fcboXWs1rbScU6qeyEbZMhbK004ZIhEJzYhEhzmwXgVD_9ykNA1rWjvohyf347vvuLn5A-7tAheNgGHyawMwUcBSNjuE8-Hk6PAhPoHUYjifjb8M2PKZcNwlXpC9celkQ_PgSRAFYdzHVt2Qou7xv7Rx_H07C01F4Mmm1ZEeC7Mg2g5bX4eB1eLu9o9TnYHQcTKIzVpe6acMoOgwi-HgG1wkyLHVGJ-kNWVTnKDBhWBk9JWu1qV0Pi4Qwu0PFGeZlNXO1O2E41YZQPaDLXUGocFJrjCjNyHQ5MszIpXmxKPvUwqBWcJGXGd0hw3GVllZBV4hd2Zdil_s93tv3e_293od_OPcgLTPw90G7X2QsJnOGeuaeFVmXXhEqMWdvVx2Wt2QcZUd54ciQ6Ypl6X_jwV1lQJcwEApsrRusS41TMcaxt7cbx1zyOOb8fw8EKrMtUSJGWOmd4WjmFAzE2inIbabwVefl4-rkutVVJr9Jzf0zNRvItezeNuxjbRyZrrfMPBDvkWGzF7Xy13PBfd4c-fgWXDQf_YP-0-HCf2U_Zy6dff-VvaNeXp6BbL9h7P42jUdkK11aWmp8XWU-TxhSdkXNtbR6ZqZ0avR0QdOYowVu4cjIuiYqGiMsm1At8CVYbATLzWC5EextBnsbwf4KOJm_-xMAAP__NiSaTw== + +# Bounding box operations. +statement ok +SET CLUSTER SETTING sql.spatial.experimental_box2d_comparison_operators.enabled = on + +query T +SELECT url FROM [EXPLAIN (DISTSQL) +SELECT k FROM geo_table WHERE geom && 'POINT(3.0 3.0)'::geometry] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJyUk29v2j4Qx5__XsXpntBKbrET_rR-xE9bumWiwABpmxpUZeTEogU7s52JCfHepyTdWuhISx5Yuj-fy33P5y3aHxlKnAXD4M0cCpPBzXR8C3fB58nw_3AEZ2_D2Xz2cXgODynf64QV6XsXf80IPr0PpkFpryEqOPd69QmtyTgczc_8Sw7-JT9vSfkuGN8G8-mXBTJUOqFRvCaL8g4FMvRwwTA3eknWalO6t1VSmGxQcoapygtXuhcMl9oQyi261GWEEkf6QudtHxkm5OI0q9J2DHXhHiHr4hWh7OzYk8KiufC8VDilOCHT5nvl8e8ABqX0-1QltEGGszxWVkJbiK537Yku7_R476rTu-73Lv7h7EOsEvA5aPeNjMVjXYtTug7VTzKOkps0c2TItMV-63_iwSY3oBUMhARb9g3WxcbJCKPI73ejiHs8ijh_6UAglZxIiQjhQDvDceEkDMTRKXinTOGDTtXD1XnHri436To2v5BhPSsJA-9gi7nggj__rjrP7JZ8XPFXiPH3xLyw4VOyuVaWXrXifLdgSMmK6ldkdWGWNDF6Wf2mNscVVzkSsq6OdmojVHWobPApLBphrxn2GmG_GfYbYX4AL3b__Q4AAP__ZFp--A== + +query T +SELECT url FROM [EXPLAIN (DISTSQL) +SELECT k FROM geo_table WHERE 'POINT(3.0 3.0)'::geometry::box2d && geom] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJyUk99v2jAQx9_3V5zuhVZyhe1QaP3EuqYaE4MOkIbUoCojJxYt2JntTEyI_31K0m2lLWnJg6X78bl873zeovuZocJpOAw_zKCwGdxMxp_hLpzfDt8PRnByPZjOpl-Gp_CQ8qNOWJG59_G3jODrx3ASQosLLvjz76LzzG4pdTWey2uICs5ltz7LgusFMtQmoVG8JofqDgUylLhgmFuzJOeMLd3bKmmQbFBxhqnOC1-6FwyXxhKqLfrUZ4QKR-bM5O0AGSbk4zSr0nYMTeH_Q87HK0LV2bFHhUVz4VnZ-oTihGyb75XHf5Pply3dpzqhDTKc5rF2CtpCnMtLKc55p8u7F53uZa979oKzB7FOIOBg_HeyDg-pFseoHuhfZD0lN2nmyZJti33pf-PhJrdgNPSFAlfqBudj61WEURT0zqOISx5FnL92IJBOjqREhPCkd4bjwivoi4NTkMdM4ZNJ9cPVyUNXl9t0HdvfyLCelYLW1Xh-EkDAAghOW-rFHe7LN2gN9rS-ssATcrnRjt60wXy3YEjJiupH4kxhl3RrzbL6TW2OK65yJOR8He3UxkDXoVLgY1g0wrIZlo1w0AwHjTB_Ai927_4EAAD__0vdfDM= + +query T +SELECT url FROM [EXPLAIN (DISTSQL) +SELECT k FROM geo_table WHERE 'LINESTRING(1.0 1.0, 5.0 5.0)'::geometry ~ geom] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJyUk99v2jAQx9_3V5zuhVZyi50EWP3EtIUuE4UuIG1Tg6qMnLpowc5sMzEh9rdPIe0GaKTFUhLdj8_lvmd7jfZHgRIn4TB8O4WlKWAQj2_gLvx8O3wTjeDsXTSZTj4Oz-Ex5Xud8ED63qVfC4JP78M4hNYwGoWTaRyNrs_EJQdxyRl0Lnn1nLekvA7HN-E0_gK_K3QxQ4ZKZzRKF2RR3qFAhh7OGJZGz8labSr3epsUZSuUnGGuyqWr3DOGc20I5Rpd7gpCiSN9ocu2jwwzcmlebNM2DPXS_YOsSx8IZbBhO4VFc-FpJTKmNCPT5nvl8e8M-pWk-1xltEKGkzJVVkJbiI535YkOD7q8-zroXvW6F_9x9iBVGfg90O4bGYvHuhandB2pn2QcZYO8cGTItMV-60_xcFUa0Ar6QoKt-gbrUuNkgkni9zpJwj2eJJw_90IglZ1IiQThQDvD8dJJ6IujU_BOmcIHnavHrfOObV1p8kVqfiHDelYSWlxwj9fr6buzBtwfNNkiCPih3ZJ7N6DvvUCpv6f0meMfky21svSi8883M4aUPVB9xaxemjndGj3f_qY2x1tu68jIujoa1Eak6lDV4C4sGmGvGfYaYb8Z9hthfgDPNq_-BAAA___AfYR8 + +query T +SELECT url FROM [EXPLAIN (DISTSQL) +SELECT k FROM geo_table WHERE geom ~ 'LINESTRING(1.0 1.0, 5.0 5.0)'::geometry::box2d] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJyUkt9v2jAQx9_3V5zupa3kCdv8Gn5iXYPGxEoHSENqoirDJxYt2JltJibE_vYpCd1aNFjxA-h7d1_nc-fbov-eo8JpNIrezWDtchhMxh_hPprfjd4Ob-HyZjidTT-NrmBf8q0uWJJ9COmXnODz-2gSlXoFv-CCCy55fR7_n5wBbw5OadFq8UN9odT1eC5vEmRorKbbdEUe1T0KTBgWzi7Ie-vK0LYqGOoNKs4wM8U6lOGE4cI6QrXFkIWcUOGsRJ9Qqsk1ODLUFNIsr67901m_7OkhM5o2yHBapMYraAjRlj0p2rzV4Z03rU6v23n9j2AXUqOhJ8GGr-Q8JjuGdh3-EvmQLgmV2LGXUw_ND3KB9CDLAzlyDfEc_TEfbQoH1kBfKPAlN_iQuqBijONmtx3HXPI45vx_Pwhk9JkuESMc9M5wvA4K-uLoFOQ5U_hgM7N_Onns6QqXrVL3ExnWs1LQl-V6Xo_nlwIEa0P76kLtN-sFgM1zACfkC2s8PYM7djPfJQxJL6leYG_XbkF3zi6qz9RyXPmqgCYf6qyoxdDUqRLwqVmcNMvTZnnS3DwwJ7tXvwMAAP__dHFMpA== diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_explain_local b/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_explain_local index c25759aa79f1..47fe7c6a846e 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_explain_local +++ b/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_explain_local @@ -57,3 +57,75 @@ filter · · · missing stats · · table geo_table2@geom_index · spans 20 spans + +# Bounding box operations. +statement ok +SET CLUSTER SETTING sql.spatial.experimental_box2d_comparison_operators.enabled = on + +query TTT +EXPLAIN SELECT k FROM geo_table2 WHERE geom && 'POINT(3.0 3.0)'::geometry +---- +· distribution local +· vectorized false +filter · · + │ filter geom && '010100000000000000000008400000000000000840' + └── index join · · + │ table geo_table2@primary + └── inverted filter · · + │ inverted column geom_inverted_key + │ num spans 31 + └── scan · · +· missing stats · +· table geo_table2@geom_index +· spans 31 spans + +query TTT +EXPLAIN SELECT k FROM geo_table2 WHERE 'POINT(3.0 3.0)'::geometry::box2d && geom +---- +· distribution local +· vectorized false +filter · · + │ filter 'BOX(3 3,3 3)' && geom + └── index join · · + │ table geo_table2@primary + └── inverted filter · · + │ inverted column geom_inverted_key + │ num spans 31 + └── scan · · +· missing stats · +· table geo_table2@geom_index +· spans 31 spans + +query TTT +EXPLAIN SELECT k FROM geo_table2 WHERE 'LINESTRING(1.0 1.0, 5.0 5.0)'::geometry ~ geom +---- +· distribution local +· vectorized false +filter · · + │ filter '010200000002000000000000000000F03F000000000000F03F00000000000014400000000000001440' ~ geom + └── index join · · + │ table geo_table2@primary + └── inverted filter · · + │ inverted column geom_inverted_key + │ num spans 38 + └── scan · · +· missing stats · +· table geo_table2@geom_index +· spans 38 spans + +query TTT +EXPLAIN SELECT k FROM geo_table2 WHERE geom ~ 'LINESTRING(1.0 1.0, 5.0 5.0)'::geometry::box2d +---- +· distribution local +· vectorized false +filter · · + │ filter geom ~ 'BOX(1 1,5 5)' + └── index join · · + │ table geo_table2@primary + └── inverted filter · · + │ inverted column geom_inverted_key + │ num spans 93 + └── scan · · +· missing stats · +· table geo_table2@geom_index +· spans 93 spans diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial b/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial index fc7b99ef3943..d75878cd1dcd 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial +++ b/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial @@ -198,3 +198,71 @@ ORDER BY rk 12 14 15 + +# Bounding box operations. +statement ok +SET CLUSTER SETTING sql.spatial.experimental_box2d_comparison_operators.enabled = on + +query II +SELECT lk, rk FROM ltable JOIN rtable@geom_index ON ltable.geom1 ~ rtable.geom +ORDER BY lk, rk +---- +1 13 + +query II +SELECT lk, rk FROM ltable JOIN rtable@geom_index ON rtable.geom ~ ltable.geom1 +ORDER BY lk, rk +---- +1 13 +1 16 +2 14 +2 16 +3 12 +3 16 +5 12 +5 16 + +query II +SELECT lk, rk FROM ltable JOIN rtable@geom_index ON rtable.geom && ltable.geom1 +ORDER BY lk, rk +---- +1 13 +1 16 +2 14 +2 16 +3 12 +3 16 +5 12 +5 16 + +query II +SELECT lk, rk FROM ltable JOIN rtable@geom_index ON ltable.geom1::box2d ~ rtable.geom +ORDER BY lk, rk +---- +1 13 + +query II +SELECT lk, rk FROM ltable JOIN rtable@geom_index ON rtable.geom ~ ltable.geom1::box2d +ORDER BY lk, rk +---- +1 13 +1 16 +2 14 +2 16 +3 12 +3 16 +5 12 +5 16 + +query II +SELECT lk, rk FROM ltable JOIN rtable@geom_index ON ltable.geom1::box2d && rtable.geom +ORDER BY lk, rk +---- +1 13 +1 16 +2 14 +2 16 +3 12 +3 16 +5 12 +5 16 diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist b/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist index 5d737daf16d7..6e5813f846f7 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist +++ b/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist @@ -92,3 +92,25 @@ SELECT url FROM [EXPLAIN (DISTSQL) SELECT lk FROM ltable WHERE NOT EXISTS (SELECT * FROM rtable WHERE ST_Intersects(ltable.geom2, rtable.geom))] ---- https://cockroachdb.github.io/distsqlplan/decode.html#eJy0U9FumzAUfd9XWPcpmZwRTLIHP1FtVKXKoAtIq1ShicFVhkYws420Ksq_T-BoKWh0dFEfr-8595xjXx9A_SyBQ-RtvA8xaWRJrrfhJ_Lg3d9trvyAzD76URx93szJCVL-MIhSp99KJF9uvK1HgjAm3n2LJLMT7q2Byacwpb8WlUapMNNqZia826HYM3oCdtV8ngCFSuQYpHtUwB_ABgoMKDiQUKilyFApIdvWoQP6-S_gSwpFVTe6PU4oZEIi8APoQpcIHOJWYItpjtJaAoUcdVqU3Xhjxa1lsU_lI1CI6rRSnCysVjRsNCeuTV0HkiMF0eiziNLpDoHbRzrdSCAWorbWPQtjg1e9wfb0hPa0hBZbWM7_ZGTTrbCJVpzFKxtx-kbk2Kv_scFGPbBRD2fpphIyR4n58KX_DflLkJtUfb8VRYXSWvVzxI81crLxrmNyFcQ-uQ39oI0QDD6cyyhxnfn5kkfTOS9Z5y2qWlQKJ-3zso2G-Q7NVSnRyAzvpMg6GVOGHa87yFFp012bwq-6VrcCT8n2s-T3PfJySGaXKDuXkFcvILMhef0s2R5kTo5vfgcAAP__M3_rTA== + +# Bounding box operations. +statement ok +SET CLUSTER SETTING sql.spatial.experimental_box2d_comparison_operators.enabled = on + +query T +SELECT url FROM [EXPLAIN (DISTSQL) +SELECT lk, rk FROM ltable JOIN rtable@geom_index ON ltable.geom1 ~ rtable.geom] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJzElMGO2jwQgO__U1hz2pXMD0mAhZxStVTKiiZb4FBpFa1SPEIpwU5tZ0WF2GevkqBCKJjQSuUW2_N5Pnsm3oD6noIL09F49H5GcpmSj5PwE3kefXkav_MDcvfBn86mn8f3ZBeSLimRyyoq1fHXFMlj6AdElt_eAsXqJeEM1yQMdgH_F5MWedvFlMMIKHDBMIhXqMB9Bgso2EDBgYhCJsUclRKyWNqUgT5bg9uhkPAs18V0RGEuJIK7AZ3oFMGFWbH9BGOGst0BCgx1nKTl9pWJl8lkFcsfQGGaxVy5pNUukoa5dolnUc-GaEtB5HqfROl4geBaW9pcxOevKDWyR5FwlG2n7vLbVQH9RYzWmSRKv8zFK0p159mUeN37miL1nLOW9jWWhd3utvonDfe3NRZimWfkm0g4EdwlXnGkMCCeTd6I1zv0Oy_nnJXbO-VcSIYSWU0o2p7QD0RLZO3hUeDp1N1aaqt5G1nN2qhtt8oqX91IF1SOGql7o0a6YHnQSA__vpHs5tW0G1bTaf1JLS-IHNWyd6NaXrA8qOXgto_CCbkJqkxwhY3--U7xaCBbYPXCKJHLOT5JMS_TVMOw5MoJhkpXq1Y18Hm1VAgewpYRts2wbYSdGmwdw45Zu2NO3TXSPTPcM8J9M9z_m0M_GOGBOfPACA_N8PAq7Wj7388AAAD__5r_BqA= + +query T +SELECT url FROM [EXPLAIN (DISTSQL) +SELECT lk, rk FROM ltable JOIN rtable@geom_index ON rtable.geom ~ ltable.geom1] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJzElFFv2jAQx9_3Kax7aiUzSAIU8pRpY1IqRjrgYVIVVSk5oYxgZ7ZTUSH62ScnbBAGJmzSeIvP9_f_57uL1yB_pODCZDAcfJySXKTk8zj4Qh4H3x6GH_wRufnkT6aTr8Nbsk1JF5SIRZmVqug5RXIf-CMiim9vjnz5lLAYVyT4FXyvg-Rtm16srBAoMB7jKFqiBPcRLKBgAwUHQgqZ4DOUkgu9tS4S_XgFbotCwrJc6XBIYcYFgrsGlagUwYWpPn-MUYyi2QIKMaooSYvjS28vE8kyEq9AYZJFTLqk0dSmQa5c4lnUsyHcUOC52plIFc0RXGtD64P47AWFwvieJwxF06my_FEqoL8Vg1UmiFRPM_6CAuPn1xvPpsRr31YoqeecBLUvAdWA24J1j0LuCjbkfJFn5DtPGOHMJZ6-VTAiXoe8Ea9SxdNwzkm4HVPOuIj15StA4eYI_og3eNbsHyQet25XrK36k2TVm6Sm3SgaffEsnUE5mKX29WbpDOjeLN39_1my6zfUrtlQp_E37TwDctDOzvXaeQZ0r5296z4NR-DGKDPOJNb681v66cB4juU7I3kuZvgg-KywKZdBoSsCMUpV7lrlwmfllgbcF1tGsW0W20axUxFbh2LHjN0yW7eN6o5Z3DGKu2Zx918ufWcU98zOPaO4bxb3L8ION-9-BgAA__9vVwoz + +query T +SELECT url FROM [EXPLAIN (DISTSQL) +SELECT lk, rk FROM ltable JOIN rtable@geom_index ON rtable.geom && ltable.geom1] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJzMlM-O2jAQh-99CmtOu5Ip-QMs5JSqpVJWlGyBQ6VttErxCKUEO7Wdigrx7lUSWggFE9oDvSBi-_N89m_kDahvKXgwHY6Gb2cklyl5Pwk_kOfhp6fRm2BM7t4F09n04-ie7JakS0rkslqV6vhLiuQxDMZElv_9BYrVS8IZrkn4a_B1MUg-55bl9KrfHVlO2BFQ4ILhOF6hAu8ZbKDgAAUXIgqZFHNUSshialMuDNgaPItCwrNcF8MRhbmQCN4GdKJTBA9mxf4TjBnKtgUUGOo4Scvtq9p-JpNVLH8AhWkWc-WRVrsoGubaI75NfQeiLQWR630RpeMFgmdvaXORgH9HqZE9ioSjbLt1lz9uDehvYrjOJFH6JeEapcK5Vne-Q4nfua9pUt89a-pcY1oY7m6sd9Jyf2MjIZZ5Rr6KhBPBPeIXxwrHxO_WY_ZrN3re0z3rudfLuZAMJbKaW7Q9cZKxaImsPThaeLp0p1babt5VdrOuajutMvSr--qCylFfdW7YVxdMD_rq4aZ95TQP12kYrtv6m2gviBxF271htBdMD6Lt_zdPxgnPCapMcIWNXgSreFKQLbB6f5TI5RyfpJiXZarPsOTKAYZKV7N29RHwaqoQPIRtI-yYYccIuzXYPoZds7ZlLt0x0l0z3DXCPTPc-5dDPxjhvrly3wgPzPDgKu1o--pnAAAA___waBaG diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_explain b/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_explain index f9b1764ca876..9ede2ee238d1 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_explain +++ b/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_explain @@ -186,3 +186,79 @@ project · · (lk) · estimated row count 1000 (missing stats) · · · table rtable@primary · · · spans FULL SCAN · · + +# Bounding box operations. +statement ok +SET CLUSTER SETTING sql.spatial.experimental_box2d_comparison_operators.enabled = on + +query TTTTT +EXPLAIN (VERBOSE) +SELECT lk, rk1, rk2 FROM ltable JOIN rtable@geom_index ON ltable.geom1 ~ rtable.geom +---- +· distribution local · · +· vectorized true · · +project · · (lk, rk1, rk2) · + │ estimated row count 326700 (missing stats) · · + └── lookup join (inner) · · (lk, geom1, rk1, rk2, geom) · + │ estimated row count 326700 (missing stats) · · + │ table rtable@primary · · + │ equality (rk1, rk2) = (rk1,rk2) · · + │ equality cols are key · · · + │ pred geom1 ~ geom · · + └── project · · (lk, geom1, rk1, rk2) · + │ estimated row count 10000 (missing stats) · · + └── inverted join · · (lk, geom1, rk1, rk2, geom_inverted_key) · + │ table rtable@geom_index · · + │ inverted expr st_covers(geom1, geom_inverted_key) · · + └── scan · · (lk, geom1) · +· estimated row count 1000 (missing stats) · · +· table ltable@primary · · +· spans FULL SCAN · · + +query TTTTT +EXPLAIN (VERBOSE) +SELECT lk, rk1, rk2 FROM ltable JOIN rtable@geom_index ON rtable.geom ~ ltable.geom1 +---- +· distribution local · · +· vectorized true · · +project · · (lk, rk1, rk2) · + │ estimated row count 326700 (missing stats) · · + └── lookup join (inner) · · (lk, geom1, rk1, rk2, geom) · + │ estimated row count 326700 (missing stats) · · + │ table rtable@primary · · + │ equality (rk1, rk2) = (rk1,rk2) · · + │ equality cols are key · · · + │ pred geom ~ geom1 · · + └── project · · (lk, geom1, rk1, rk2) · + │ estimated row count 10000 (missing stats) · · + └── inverted join · · (lk, geom1, rk1, rk2, geom_inverted_key) · + │ table rtable@geom_index · · + │ inverted expr st_coveredby(geom1, geom_inverted_key) · · + └── scan · · (lk, geom1) · +· estimated row count 1000 (missing stats) · · +· table ltable@primary · · +· spans FULL SCAN · · + +query TTTTT +EXPLAIN (VERBOSE) +SELECT lk, rk1, rk2 FROM ltable JOIN rtable@geom_index ON rtable.geom && ltable.geom1 +---- +· distribution local · · +· vectorized true · · +project · · (lk, rk1, rk2) · + │ estimated row count 326700 (missing stats) · · + └── lookup join (inner) · · (lk, geom1, rk1, rk2, geom) · + │ estimated row count 326700 (missing stats) · · + │ table rtable@primary · · + │ equality (rk1, rk2) = (rk1,rk2) · · + │ equality cols are key · · · + │ pred geom && geom1 · · + └── project · · (lk, geom1, rk1, rk2) · + │ estimated row count 10000 (missing stats) · · + └── inverted join · · (lk, geom1, rk1, rk2, geom_inverted_key) · + │ table rtable@geom_index · · + │ inverted expr st_intersects(geom1, geom_inverted_key) · · + └── scan · · (lk, geom1) · +· estimated row count 1000 (missing stats) · · +· table ltable@primary · · +· spans FULL SCAN · · diff --git a/pkg/sql/opt/invertedidx/geo.go b/pkg/sql/opt/invertedidx/geo.go index 5991a5cccfd9..865f89bc4cf7 100644 --- a/pkg/sql/opt/invertedidx/geo.go +++ b/pkg/sql/opt/invertedidx/geo.go @@ -31,12 +31,22 @@ import ( // This file contains functions for building geospatial inverted index scans // and joins that are used throughout the xform package. -// IsGeoIndexFunction returns true if the given function is a geospatial -// function that can be index-accelerated. -func IsGeoIndexFunction(fn opt.ScalarExpr) bool { - function := fn.(*memo.FunctionExpr) - _, ok := geoindex.RelationshipMap[function.Name] - return ok +// GetGeoIndexRelationship returns the corresponding geospatial relationship +// and ok=true if the given expression is either a geospatial function or +// bounding box comparison operator that can be index-accelerated. Otherwise +// returns ok=false. +func GetGeoIndexRelationship(expr opt.ScalarExpr) (_ geoindex.RelationshipType, ok bool) { + if function, ok := expr.(*memo.FunctionExpr); ok { + rel, ok := geoindex.RelationshipMap[function.Name] + return rel, ok + } + if _, ok := expr.(*memo.BBoxCoversExpr); ok { + return geoindex.Covers, true + } + if _, ok := expr.(*memo.BBoxIntersectsExpr); ok { + return geoindex.Intersects, true + } + return 0, false } // getSpanExprForGeoIndexFn is a function that returns a SpanExpression that @@ -103,7 +113,11 @@ func TryJoinGeoIndex( // derived, it is returned with ok=true. If no constraint can be derived, // then TryConstrainGeoIndex returns ok=false. func TryConstrainGeoIndex( - ctx context.Context, filters memo.FiltersExpr, tabID opt.TableID, index cat.Index, + ctx context.Context, + factory *norm.Factory, + filters memo.FiltersExpr, + tabID opt.TableID, + index cat.Index, ) (invertedConstraint *invertedexpr.SpanExpression, ok bool) { config := index.GeoConfig() var getSpanExpr getSpanExprForGeoIndexFn @@ -118,7 +132,7 @@ func TryConstrainGeoIndex( var invertedExpr invertedexpr.InvertedExpression for i := range filters { invertedExprLocal := constrainGeoIndex( - ctx, filters[i].Condition, tabID, index, getSpanExpr, + ctx, factory, filters[i].Condition, tabID, index, getSpanExpr, ) if invertedExpr == nil { invertedExpr = invertedExprLocal @@ -299,6 +313,7 @@ func joinGeoIndex( inputCols opt.ColSet, getSpanExpr getSpanExprForGeoIndexFn, ) opt.ScalarExpr { + var args memo.ScalarListExpr switch t := filterCond.(type) { case *memo.AndExpr: leftExpr := joinGeoIndex(ctx, factory, t.Left, tabID, index, inputCols, getSpanExpr) @@ -320,83 +335,103 @@ func joinGeoIndex( return factory.ConstructOr(leftExpr, rightExpr) case *memo.FunctionExpr: - // Try to extract an inverted join condition from the given function. If - // unsuccessful, try to extract a join condition from an equivalent function - // in which the arguments are commuted. For example: - // - // ST_Intersects(g1, g2) <-> ST_Intersects(g2, g1) - // ST_Covers(g1, g2) <-> ST_CoveredBy(g2, g1) - // - // See joinGeoIndexFromFunction for more details. - fn := joinGeoIndexFromFunction( - factory, t, false /* commuteArgs */, inputCols, tabID, index, - ) - if fn == nil { - fn = joinGeoIndexFromFunction( - factory, t, true /* commuteArgs */, inputCols, tabID, index, - ) + args = t.Args + + case *memo.BBoxCoversExpr, *memo.BBoxIntersectsExpr: + args = memo.ScalarListExpr{ + t.Child(0).(opt.ScalarExpr), t.Child(1).(opt.ScalarExpr), + } + // Cast the arguments to type Geometry if they are type Box2d. + for i := 0; i < len(args); i++ { + if args[i].DataType().Family() == types.Box2DFamily { + args[i] = factory.ConstructCast(args[i], types.Geometry) + } } - return fn default: return nil } + + // Try to extract an inverted join condition from the given filter condition. + // If unsuccessful, try to extract a join condition from an equivalent + // function in which the arguments are commuted. For example: + // + // ST_Intersects(g1, g2) <-> ST_Intersects(g2, g1) + // ST_Covers(g1, g2) <-> ST_CoveredBy(g2, g1) + // g1 && g2 -> ST_Intersects(g2, g1) + // g1 ~ g2 -> ST_CoveredBy(g2, g1) + // + // See joinGeoIndexFromExpr for more details. + fn := joinGeoIndexFromExpr( + factory, filterCond, args, false /* commuteArgs */, inputCols, tabID, index, + ) + if fn == nil { + fn = joinGeoIndexFromExpr( + factory, filterCond, args, true /* commuteArgs */, inputCols, tabID, index, + ) + } + return fn } -// joinGeoIndexFromFunction tries to extract an inverted join condition from the -// given geospatial function. If commuteArgs is true, joinGeoIndexFromFunction -// tries to extract an inverted join condition from an equivalent version of the -// given function in which the first two arguments are swapped. +// joinGeoIndexFromExpr tries to extract an inverted join condition from the +// given expression, which should be either a function or comparison operation. +// If commuteArgs is true, joinGeoIndexFromExpr tries to extract an inverted +// join condition from an equivalent version of the given expression in which +// the first two arguments are swapped. // -// Returns the original function if commuteArgs is false, or a new function -// representing the same relationship but with commuted arguments if -// commuteArgs is true. For example: +// If commuteArgs is false, returns the original function (if the expression +// was a function) or a new function representing the geospatial relationship +// of the comparison operation. If commuteArgs is true, returns a new function +// representing the same relationship but with commuted arguments. For example: // // ST_Intersects(g1, g2) <-> ST_Intersects(g2, g1) // ST_Covers(g1, g2) <-> ST_CoveredBy(g2, g1) +// g1 && g2 -> ST_Intersects(g2, g1) +// g1 ~ g2 -> ST_CoveredBy(g2, g1) // // See geoindex.CommuteRelationshipMap for the full list of mappings. // // Returns nil if a join condition was not successfully extracted. -func joinGeoIndexFromFunction( +func joinGeoIndexFromExpr( factory *norm.Factory, - fn *memo.FunctionExpr, + expr opt.ScalarExpr, + args memo.ScalarListExpr, commuteArgs bool, inputCols opt.ColSet, tabID opt.TableID, index cat.Index, ) opt.ScalarExpr { - if !IsGeoIndexFunction(fn) { + rel, ok := GetGeoIndexRelationship(expr) + if !ok { return nil } // Extract the the inputs to the geospatial function. - if fn.Args.ChildCount() < 2 { + if args.ChildCount() < 2 { panic(errors.AssertionFailedf( "all index-accelerated geospatial functions should have at least two arguments", )) } - arg1, arg2 := fn.Args.Child(0), fn.Args.Child(1) + arg1, arg2 := args.Child(0), args.Child(1) if commuteArgs { arg1, arg2 = arg2, arg1 } // The first argument should either come from the input or be a constant. - variable, ok := arg1.(*memo.VariableExpr) - if ok { - if !inputCols.Contains(variable.Col) { - return nil - } - } else { - if !memo.CanExtractConstDatum(arg1) { + var p props.Shared + memo.BuildSharedProps(arg1, &p) + if !p.OuterCols.Empty() { + if !p.OuterCols.SubsetOf(inputCols) { return nil } + } else if !memo.CanExtractConstDatum(arg1) { + return nil } // The second argument should be a variable corresponding to the index // column. - variable, ok = arg2.(*memo.VariableExpr) + variable, ok := arg2.(*memo.VariableExpr) if !ok { return nil } @@ -406,8 +441,8 @@ func joinGeoIndexFromFunction( } // Any additional params must be constant. - for i := 2; i < fn.Args.ChildCount(); i++ { - if !memo.CanExtractConstDatum(fn.Args.Child(i)) { + for i := 2; i < args.ChildCount(); i++ { + if !memo.CanExtractConstDatum(args.Child(i)) { return nil } } @@ -416,7 +451,6 @@ func joinGeoIndexFromFunction( // Get the geospatial relationship that is equivalent to this one with the // arguments commuted, and construct a new function that represents that // relationship. - rel := geoindex.RelationshipMap[fn.Name] commutedRel, ok := geoindex.CommuteRelationshipMap[rel] if !ok { // It's not possible to commute this relationship. @@ -427,96 +461,131 @@ func joinGeoIndexFromFunction( // Copy the original arguments into a new list, and swap the first two // arguments. - args := make(memo.ScalarListExpr, len(fn.Args)) - copy(args, fn.Args) - args[0], args[1] = args[1], args[0] + commutedArgs := make(memo.ScalarListExpr, len(args)) + copy(commutedArgs, args) + commutedArgs[0], commutedArgs[1] = commutedArgs[1], commutedArgs[0] - props, overload, ok := memo.FindFunction(&args, name) - if !ok { - panic(errors.AssertionFailedf("could not find overload for %s", name)) - } - return factory.ConstructFunction(args, &memo.FunctionPrivate{ - Name: name, - Typ: fn.Typ, - Properties: props, - Overload: overload, - }) + return constructFunction(factory, name, commutedArgs) } - return fn + if _, ok := expr.(*memo.FunctionExpr); !ok { + // This expression was one of the bounding box comparison operators. + // Construct a function that represents the same geospatial relationship. + name := geoindex.RelationshipReverseMap[rel] + return constructFunction(factory, name, args) + } + + return expr +} + +// constructFunction finds a function overload matching the given name and +// argument types, and uses the factory to construct a function. The return +// type of the function must be bool. +func constructFunction( + factory *norm.Factory, name string, args memo.ScalarListExpr, +) opt.ScalarExpr { + props, overload, ok := memo.FindFunction(&args, name) + if !ok { + panic(errors.AssertionFailedf("could not find overload for %s", name)) + } + return factory.ConstructFunction(args, &memo.FunctionPrivate{ + Name: name, + Typ: types.Bool, + Properties: props, + Overload: overload, + }) } // constrainGeoIndex returns an InvertedExpression representing a constraint // of the given geospatial index. func constrainGeoIndex( ctx context.Context, + factory *norm.Factory, expr opt.ScalarExpr, tabID opt.TableID, index cat.Index, getSpanExpr getSpanExprForGeoIndexFn, ) invertedexpr.InvertedExpression { + var args memo.ScalarListExpr switch t := expr.(type) { case *memo.AndExpr: return invertedexpr.And( - constrainGeoIndex(ctx, t.Left, tabID, index, getSpanExpr), - constrainGeoIndex(ctx, t.Right, tabID, index, getSpanExpr), + constrainGeoIndex(ctx, factory, t.Left, tabID, index, getSpanExpr), + constrainGeoIndex(ctx, factory, t.Right, tabID, index, getSpanExpr), ) case *memo.OrExpr: return invertedexpr.Or( - constrainGeoIndex(ctx, t.Left, tabID, index, getSpanExpr), - constrainGeoIndex(ctx, t.Right, tabID, index, getSpanExpr), + constrainGeoIndex(ctx, factory, t.Left, tabID, index, getSpanExpr), + constrainGeoIndex(ctx, factory, t.Right, tabID, index, getSpanExpr), ) case *memo.FunctionExpr: - // Try to constrain the index with the given function. If the resulting - // inverted expression is not a SpanExpression, try constraining the index - // with an equivalent function in which the arguments are commuted. For - // example: - // - // ST_Intersects(g1, g2) <-> ST_Intersects(g2, g1) - // ST_Covers(g1, g2) <-> ST_CoveredBy(g2, g1) - // - // See geoindex.CommuteRelationshipMap for the full list of mappings. - invertedExpr := constrainGeoIndexFromFunction( - ctx, t, false /* commuteArgs */, tabID, index, getSpanExpr, - ) - if _, ok := invertedExpr.(invertedexpr.NonInvertedColExpression); ok { - invertedExpr = constrainGeoIndexFromFunction( - ctx, t, true /* commuteArgs */, tabID, index, getSpanExpr, - ) + args = t.Args + + case *memo.BBoxCoversExpr, *memo.BBoxIntersectsExpr: + args = memo.ScalarListExpr{ + t.Child(0).(opt.ScalarExpr), t.Child(1).(opt.ScalarExpr), + } + // Cast the arguments to type Geometry if they are type Box2d. + for i := 0; i < len(args); i++ { + if args[i].DataType().Family() == types.Box2DFamily { + args[i] = factory.ConstructCast(args[i], types.Geometry) + } } - return invertedExpr default: return invertedexpr.NonInvertedColExpression{} } + + // Try to constrain the index with the given expression. If the resulting + // inverted expression is not a SpanExpression, try constraining the index + // with an equivalent function in which the arguments are commuted. For + // example: + // + // ST_Intersects(g1, g2) <-> ST_Intersects(g2, g1) + // ST_Covers(g1, g2) <-> ST_CoveredBy(g2, g1) + // g1 && g2 -> ST_Intersects(g2, g1) + // g1 ~ g2 -> ST_CoveredBy(g2, g1) + // + // See geoindex.CommuteRelationshipMap for the full list of mappings. + invertedExpr := constrainGeoIndexFromExpr( + ctx, expr, args, false /* commuteArgs */, tabID, index, getSpanExpr, + ) + if _, ok := invertedExpr.(invertedexpr.NonInvertedColExpression); ok { + invertedExpr = constrainGeoIndexFromExpr( + ctx, expr, args, true /* commuteArgs */, tabID, index, getSpanExpr, + ) + } + return invertedExpr } -// constrainGeoIndexFromFunction returns an InvertedExpression representing a -// constraint of the given geospatial index, based on the given function. -// If commuteArgs is true, constrainGeoIndexFromFunction constrains the index -// based on an equivalent version of the given function in which the first two -// arguments are swapped. -func constrainGeoIndexFromFunction( +// constrainGeoIndexFromExpr returns an InvertedExpression representing a +// constraint of the given geospatial index, based on the given expression. +// If commuteArgs is true, constrainGeoIndexFromExpr constrains the index +// based on an equivalent version of the given expression in which the first +// two arguments are swapped. +func constrainGeoIndexFromExpr( ctx context.Context, - fn *memo.FunctionExpr, + expr opt.ScalarExpr, + args memo.ScalarListExpr, commuteArgs bool, tabID opt.TableID, index cat.Index, getSpanExpr getSpanExprForGeoIndexFn, ) invertedexpr.InvertedExpression { - if !IsGeoIndexFunction(fn) { + relationship, ok := GetGeoIndexRelationship(expr) + if !ok { return invertedexpr.NonInvertedColExpression{} } - if fn.Args.ChildCount() < 2 { + if args.ChildCount() < 2 { panic(errors.AssertionFailedf( "all index-accelerated geospatial functions should have at least two arguments", )) } - arg1, arg2 := fn.Args.Child(0), fn.Args.Child(1) + arg1, arg2 := args.Child(0), args.Child(1) if commuteArgs { arg1, arg2 = arg2, arg1 } @@ -540,14 +609,13 @@ func constrainGeoIndexFromFunction( // Any additional params must be constant. var additionalParams []tree.Datum - for i := 2; i < fn.Args.ChildCount(); i++ { - if !memo.CanExtractConstDatum(fn.Args.Child(i)) { + for i := 2; i < args.ChildCount(); i++ { + if !memo.CanExtractConstDatum(args.Child(i)) { return invertedexpr.NonInvertedColExpression{} } - additionalParams = append(additionalParams, memo.ExtractConstDatum(fn.Args.Child(i))) + additionalParams = append(additionalParams, memo.ExtractConstDatum(args.Child(i))) } - relationship := geoindex.RelationshipMap[fn.Name] if commuteArgs { relationship, ok = geoindex.CommuteRelationshipMap[relationship] if !ok { @@ -681,7 +749,7 @@ func NewGeoDatumsToInvertedExpr( // We know that the non-index param is the first param, because the // optimizer already commuted the arguments of any functions where that - // was not the case. See joinGeoIndexFromFunction for details. + // was not the case. See joinGeoIndexFromExpr for details. nonIndexParam := t.Exprs[0].(tree.TypedExpr) var additionalParams []tree.Datum diff --git a/pkg/sql/opt/invertedidx/geo_test.go b/pkg/sql/opt/invertedidx/geo_test.go index 69afc1068c39..c3bb666af335 100644 --- a/pkg/sql/opt/invertedidx/geo_test.go +++ b/pkg/sql/opt/invertedidx/geo_test.go @@ -34,14 +34,14 @@ func TestTryJoinGeoIndex(t *testing.T) { // Create the input table. if _, err := tc.ExecuteDDL( "CREATE TABLE t1 (geom1 GEOMETRY, geog1 GEOGRAPHY, geom11 GEOMETRY, geog11 GEOGRAPHY, " + - "inet1 INET)", + "inet1 INET, bbox1 box2d)", ); err != nil { t.Fatal(err) } // Create the indexed table. if _, err := tc.ExecuteDDL( - "CREATE TABLE t2 (geom2 GEOMETRY, geog2 GEOGRAPHY, inet2 INET, " + + "CREATE TABLE t2 (geom2 GEOMETRY, geog2 GEOGRAPHY, inet2 INET, bbox2 box2d, " + "INVERTED INDEX (geom2), INVERTED INDEX (geog2))", ); err != nil { t.Fatal(err) @@ -200,6 +200,78 @@ func TestTryJoinGeoIndex(t *testing.T) { "st_coveredby('SRID=4326;POINT(-40.23456 70.456772)'::geography, geog2)) AND " + "st_covers('SRID=4326;POINT(-42.89456 75.938299)'::geography, geog2)", }, + + // Bounding box operators. + { + filters: "bbox1 ~ geom2", + indexOrd: geomOrd, + invertedExpr: "st_covers(bbox1::geometry, geom2)", + }, + { + filters: "geom2 ~ bbox1", + indexOrd: geomOrd, + invertedExpr: "st_coveredby(bbox1::geometry, geom2)", + }, + { + filters: "geom1 ~ geom2", + indexOrd: geomOrd, + invertedExpr: "st_covers(geom1, geom2)", + }, + { + filters: "geom2 ~ geom1", + indexOrd: geomOrd, + invertedExpr: "st_coveredby(geom1, geom2)", + }, + { + filters: "bbox1 && geom2", + indexOrd: geomOrd, + invertedExpr: "st_intersects(bbox1::geometry, geom2)", + }, + { + filters: "geom2 && bbox1", + indexOrd: geomOrd, + invertedExpr: "st_intersects(bbox1::geometry, geom2)", + }, + { + filters: "geom1 && geom2", + indexOrd: geomOrd, + invertedExpr: "st_intersects(geom1, geom2)", + }, + { + filters: "geom2 && geom1", + indexOrd: geomOrd, + invertedExpr: "st_intersects(geom1, geom2)", + }, + { + filters: "geom2 && geom1 AND 'BOX(1 2, 3 4)'::box2d ~ geom2", + indexOrd: geomOrd, + invertedExpr: "st_intersects(geom1, geom2) AND " + + "st_covers('BOX(1 2, 3 4)'::box2d::geometry, geom2)", + }, + { + // Wrong index ordinal. + filters: "bbox1 ~ geom2", + indexOrd: geogOrd, + invertedExpr: "", + }, + { + // At least one column from the input is required. + filters: "bbox2 ~ geom2", + indexOrd: geomOrd, + invertedExpr: "", + }, + { + // At least one column from the input is required. + filters: "'BOX(1 2, 3 4)'::box2d ~ geom2", + indexOrd: geomOrd, + invertedExpr: "", + }, + { + // Wrong types. + filters: "geom1::string ~ geom2::string", + indexOrd: geomOrd, + invertedExpr: "", + }, } for _, tc := range testCases { @@ -318,6 +390,54 @@ func TestTryConstrainGeoIndex(t *testing.T) { indexOrd: geogOrd, ok: true, }, + + // Bounding box operators. + { + filters: "'BOX(1 2, 3 4)'::box2d ~ geom", + indexOrd: geomOrd, + ok: true, + }, + { + filters: "geom ~ 'BOX(1 2, 3 4)'::box2d", + indexOrd: geomOrd, + ok: true, + }, + { + filters: "'LINESTRING ( 0 0, 0 2 )'::geometry ~ geom", + indexOrd: geomOrd, + ok: true, + }, + { + filters: "geom ~ 'LINESTRING ( 0 0, 0 2 )'::geometry", + indexOrd: geomOrd, + ok: true, + }, + { + filters: "'BOX(1 2, 3 4)'::box2d && geom", + indexOrd: geomOrd, + ok: true, + }, + { + filters: "geom && 'BOX(1 2, 3 4)'::box2d", + indexOrd: geomOrd, + ok: true, + }, + { + filters: "'LINESTRING ( 0 0, 0 2 )'::geometry && geom", + indexOrd: geomOrd, + ok: true, + }, + { + filters: "geom && 'LINESTRING ( 0 0, 0 2 )'::geometry", + indexOrd: geomOrd, + ok: true, + }, + { + // Wrong index ordinal. + filters: "'BOX(1 2, 3 4)'::box2d ~ geom", + indexOrd: geogOrd, + ok: false, + }, } for _, tc := range testCases { @@ -331,7 +451,7 @@ func TestTryConstrainGeoIndex(t *testing.T) { // that is tested elsewhere. This is just testing that we are constraining // the index when we expect to. _, ok := invertedidx.TryConstrainGeoIndex( - evalCtx.Context, filters, tab, md.Table(tab).Index(tc.indexOrd), + evalCtx.Context, &f, filters, tab, md.Table(tab).Index(tc.indexOrd), ) if tc.ok != ok { t.Fatalf("expected %v, got %v", tc.ok, ok) diff --git a/pkg/sql/opt/operator.go b/pkg/sql/opt/operator.go index 6b6d212c00ae..d68f1bda401d 100644 --- a/pkg/sql/opt/operator.go +++ b/pkg/sql/opt/operator.go @@ -140,6 +140,8 @@ var ComparisonOpReverseMap = map[Operator]tree.ComparisonOperator{ JsonSomeExistsOp: tree.JSONSomeExists, JsonAllExistsOp: tree.JSONAllExists, OverlapsOp: tree.Overlaps, + BBoxCoversOp: tree.RegMatch, + BBoxIntersectsOp: tree.Overlaps, } // BinaryOpReverseMap maps from an optimizer operator type to a semantic tree @@ -259,7 +261,7 @@ func ScalarOperatorTransmitsNulls(op Operator) bool { case BitandOp, BitorOp, BitxorOp, PlusOp, MinusOp, MultOp, DivOp, FloorDivOp, ModOp, PowOp, EqOp, NeOp, LtOp, GtOp, LeOp, GeOp, LikeOp, NotLikeOp, ILikeOp, NotILikeOp, SimilarToOp, NotSimilarToOp, RegMatchOp, NotRegMatchOp, RegIMatchOp, - NotRegIMatchOp, ConstOp: + NotRegIMatchOp, ConstOp, BBoxCoversOp, BBoxIntersectsOp: return true default: @@ -274,7 +276,8 @@ func BoolOperatorRequiresNotNullArgs(op Operator) bool { case EqOp, LtOp, LeOp, GtOp, GeOp, NeOp, LikeOp, NotLikeOp, ILikeOp, NotILikeOp, SimilarToOp, NotSimilarToOp, - RegMatchOp, NotRegMatchOp, RegIMatchOp, NotRegIMatchOp: + RegMatchOp, NotRegMatchOp, RegIMatchOp, NotRegIMatchOp, BBoxCoversOp, + BBoxIntersectsOp: return true } return false diff --git a/pkg/sql/opt/ops/scalar.opt b/pkg/sql/opt/ops/scalar.opt index 4280da79366f..2cb9c8562440 100644 --- a/pkg/sql/opt/ops/scalar.opt +++ b/pkg/sql/opt/ops/scalar.opt @@ -471,6 +471,22 @@ define Overlaps { Right ScalarExpr } +# BBoxCovers is the ~ operator when used with geometry or bounding box +# operands. It maps to tree.RegMatch. +[Scalar, Bool, Comparison] +define BBoxCovers { + Left ScalarExpr + Right ScalarExpr +} + +# BBoxIntersects is the && operator when used with geometry or bounding box +# operands. It maps to tree.Overlaps. +[Scalar, Bool, Comparison] +define BBoxIntersects { + Left ScalarExpr + Right ScalarExpr +} + # AnyScalar is the form of ANY which refers to an ANY operation on a # tuple or array, as opposed to Any which operates on a subquery. [Scalar, Bool] diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index f1483d667787..1e38ee7cab15 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -280,7 +280,7 @@ func (b *Builder) buildScalar( } else { left := b.buildScalar(t.TypedLeft(), inScope, nil, nil, colRefs) right := b.buildScalar(t.TypedRight(), inScope, nil, nil, colRefs) - out = b.constructComparison(t.Operator, left, right) + out = b.constructComparison(t, left, right) } case *tree.DTuple: @@ -635,9 +635,9 @@ func (b *Builder) checkSubqueryOuterCols( } func (b *Builder) constructComparison( - cmp tree.ComparisonOperator, left, right opt.ScalarExpr, + cmp *tree.ComparisonExpr, left, right opt.ScalarExpr, ) opt.ScalarExpr { - switch cmp { + switch cmp.Operator { case tree.EQ: return b.factory.ConstructEq(left, right) case tree.LT: @@ -667,6 +667,13 @@ func (b *Builder) constructComparison( case tree.NotSimilarTo: return b.factory.ConstructNotSimilarTo(left, right) case tree.RegMatch: + leftFam, rightFam := cmp.Fn.LeftType.Family(), cmp.Fn.RightType.Family() + if (leftFam == types.GeometryFamily || leftFam == types.Box2DFamily) && + (rightFam == types.GeometryFamily || rightFam == types.Box2DFamily) { + // The ~ operator means "covers" when used with geometry or bounding box + // operands. + return b.factory.ConstructBBoxCovers(left, right) + } return b.factory.ConstructRegMatch(left, right) case tree.NotRegMatch: return b.factory.ConstructNotRegMatch(left, right) @@ -690,9 +697,16 @@ func (b *Builder) constructComparison( case tree.JSONSomeExists: return b.factory.ConstructJsonSomeExists(left, right) case tree.Overlaps: + leftFam, rightFam := cmp.Fn.LeftType.Family(), cmp.Fn.RightType.Family() + if (leftFam == types.GeometryFamily || leftFam == types.Box2DFamily) && + (rightFam == types.GeometryFamily || rightFam == types.Box2DFamily) { + // The && operator means "intersects" when used with geometry or bounding + // box operands. + return b.factory.ConstructBBoxIntersects(left, right) + } return b.factory.ConstructOverlaps(left, right) } - panic(errors.AssertionFailedf("unhandled comparison operator: %s", log.Safe(cmp))) + panic(errors.AssertionFailedf("unhandled comparison operator: %s", log.Safe(cmp.Operator))) } func (b *Builder) constructBinary( diff --git a/pkg/sql/opt/optbuilder/testdata/scalar b/pkg/sql/opt/optbuilder/testdata/scalar index eda5238313e1..06940dea4dfd 100644 --- a/pkg/sql/opt/optbuilder/testdata/scalar +++ b/pkg/sql/opt/optbuilder/testdata/scalar @@ -1303,3 +1303,97 @@ NOT NULL not [type=bool] └── cast: BOOL [type=bool] └── null [type=unknown] + +build-scalar vars=(geometry, geometry) +@1 ~ @2 +---- +b-box-covers [type=bool] + ├── variable: "@1":1 [type=geometry] + └── variable: "@2":2 [type=geometry] + +build-scalar vars=(geometry, box2d) +@1 ~ @2 +---- +b-box-covers [type=bool] + ├── variable: "@1":1 [type=geometry] + └── variable: "@2":2 [type=box2d] + +build-scalar vars=(box2d, geometry) +@1 ~ @2 +---- +b-box-covers [type=bool] + ├── variable: "@1":1 [type=box2d] + └── variable: "@2":2 [type=geometry] + +build-scalar vars=(box2d, box2d) +@1 ~ @2 +---- +b-box-covers [type=bool] + ├── variable: "@1":1 [type=box2d] + └── variable: "@2":2 [type=box2d] + +build-scalar vars=(geometry, geometry) +@1 && @2 +---- +b-box-intersects [type=bool] + ├── variable: "@1":1 [type=geometry] + └── variable: "@2":2 [type=geometry] + +build-scalar vars=(geometry, box2d) +@1 && @2 +---- +b-box-intersects [type=bool] + ├── variable: "@1":1 [type=geometry] + └── variable: "@2":2 [type=box2d] + +build-scalar vars=(box2d, geometry) +@1 && @2 +---- +b-box-intersects [type=bool] + ├── variable: "@1":1 [type=box2d] + └── variable: "@2":2 [type=geometry] + +build-scalar vars=(box2d, box2d) +@1 && @2 +---- +b-box-intersects [type=bool] + ├── variable: "@1":1 [type=box2d] + └── variable: "@2":2 [type=box2d] + +build-scalar vars=(string, string) +@1 ~ @2 +---- +reg-match [type=bool] + ├── variable: "@1":1 [type=string] + └── variable: "@2":2 [type=string] + +build-scalar vars=(inet, inet) +@1 && @2 +---- +overlaps [type=bool] + ├── variable: "@1":1 [type=inet] + └── variable: "@2":2 [type=inet] + +build-scalar vars=(int[], int[]) +@1 && @2 +---- +overlaps [type=bool] + ├── variable: "@1":1 [type=int[]] + └── variable: "@2":2 [type=int[]] + +build-scalar vars=(geometry[], geometry[]) +@1 && @2 +---- +overlaps [type=bool] + ├── variable: "@1":1 [type=geometry[]] + └── variable: "@2":2 [type=geometry[]] + +build-scalar vars=(string, geometry) +@1 ~ @2 +---- +error: unsupported comparison operator: ~ + +build-scalar vars=(geometry[], geometry) +@1 && @2 +---- +error: unsupported comparison operator: && diff --git a/pkg/sql/opt/xform/custom_funcs.go b/pkg/sql/opt/xform/custom_funcs.go index 0168f32d49b7..f0599dd02aef 100644 --- a/pkg/sql/opt/xform/custom_funcs.go +++ b/pkg/sql/opt/xform/custom_funcs.go @@ -1004,7 +1004,7 @@ func (c *CustomFuncs) GenerateInvertedIndexScans( // Check whether the filter can constrain the index. // TODO(rytaft): Unify these two cases so both return a spanExpr. spanExpr, geoOk = invertedidx.TryConstrainGeoIndex( - c.e.evalCtx.Context, filters, scanPrivate.Table, iter.Index(), + c.e.evalCtx.Context, c.e.f, filters, scanPrivate.Table, iter.Index(), ) if geoOk { // Geo index scans can never be tight, so remaining filters is always the diff --git a/pkg/sql/opt/xform/testdata/rules/join b/pkg/sql/opt/xform/testdata/rules/join index 36dd804d21f5..1c9c40f2ea4d 100644 --- a/pkg/sql/opt/xform/testdata/rules/join +++ b/pkg/sql/opt/xform/testdata/rules/join @@ -3457,6 +3457,107 @@ anti-join (cross) └── filters └── st_covers(c.geom:10, n.geom:16) [outer=(10,16), immutable] +# Bounding box operations. +opt expect=GenerateInvertedJoins +SELECT + n.name, c.boroname +FROM nyc_census_blocks AS c +JOIN nyc_neighborhoods@nyc_neighborhoods_geo_idx AS n +ON c.geom::box2d && n.geom +---- +project + ├── columns: name:15 boroname:9 + ├── immutable + └── inner-join (lookup nyc_neighborhoods) + ├── columns: c.boroname:9 c.geom:10 name:15 n.geom:16 + ├── key columns: [13] = [13] + ├── lookup columns are key + ├── immutable + ├── inner-join (inverted-lookup nyc_neighborhoods@nyc_neighborhoods_geo_idx) + │ ├── columns: c.boroname:9 c.geom:10 n.gid:13!null + │ ├── inverted-expr + │ │ └── st_intersects(c.geom:10::BOX2D::GEOMETRY, n.geom:16) + │ ├── scan c + │ │ └── columns: c.boroname:9 c.geom:10 + │ └── filters (true) + └── filters + └── c.geom:10::BOX2D && n.geom:16 [outer=(10,16), immutable] + +opt expect=GenerateInvertedJoins +SELECT + n.name, c.boroname +FROM nyc_census_blocks AS c +JOIN nyc_neighborhoods@nyc_neighborhoods_geo_idx AS n +ON c.geom::box2d ~ n.geom +---- +project + ├── columns: name:15 boroname:9 + ├── immutable + └── inner-join (lookup nyc_neighborhoods) + ├── columns: c.boroname:9 c.geom:10 name:15 n.geom:16 + ├── key columns: [13] = [13] + ├── lookup columns are key + ├── immutable + ├── inner-join (inverted-lookup nyc_neighborhoods@nyc_neighborhoods_geo_idx) + │ ├── columns: c.boroname:9 c.geom:10 n.gid:13!null + │ ├── inverted-expr + │ │ └── st_covers(c.geom:10::BOX2D::GEOMETRY, n.geom:16) + │ ├── scan c + │ │ └── columns: c.boroname:9 c.geom:10 + │ └── filters (true) + └── filters + └── c.geom:10::BOX2D ~ n.geom:16 [outer=(10,16), immutable] + +opt expect=GenerateInvertedJoins +SELECT + n.name, c.boroname +FROM nyc_census_blocks AS c +JOIN nyc_neighborhoods@nyc_neighborhoods_geo_idx AS n +ON n.geom ~ c.geom::box2d +---- +project + ├── columns: name:15 boroname:9 + ├── immutable + └── inner-join (lookup nyc_neighborhoods) + ├── columns: c.boroname:9 c.geom:10 name:15 n.geom:16!null + ├── key columns: [13] = [13] + ├── lookup columns are key + ├── immutable + ├── inner-join (inverted-lookup nyc_neighborhoods@nyc_neighborhoods_geo_idx) + │ ├── columns: c.boroname:9 c.geom:10 n.gid:13!null + │ ├── inverted-expr + │ │ └── st_coveredby(c.geom:10::BOX2D::GEOMETRY, n.geom:16) + │ ├── scan c + │ │ └── columns: c.boroname:9 c.geom:10 + │ └── filters (true) + └── filters + └── n.geom:16 ~ c.geom:10::BOX2D [outer=(10,16), immutable, constraints=(/16: (/NULL - ])] + +opt expect=GenerateInvertedJoins +SELECT + n.name, c.boroname +FROM nyc_census_blocks AS c +JOIN nyc_neighborhoods@nyc_neighborhoods_geo_idx AS n +ON n.geom ~ c.geom +---- +project + ├── columns: name:15 boroname:9 + ├── immutable + └── inner-join (lookup nyc_neighborhoods) + ├── columns: c.boroname:9 c.geom:10!null name:15 n.geom:16!null + ├── key columns: [13] = [13] + ├── lookup columns are key + ├── immutable + ├── inner-join (inverted-lookup nyc_neighborhoods@nyc_neighborhoods_geo_idx) + │ ├── columns: c.boroname:9 c.geom:10 n.gid:13!null + │ ├── inverted-expr + │ │ └── st_coveredby(c.geom:10, n.geom:16) + │ ├── scan c + │ │ └── columns: c.boroname:9 c.geom:10 + │ └── filters (true) + └── filters + └── n.geom:16 ~ c.geom:10 [outer=(10,16), immutable, constraints=(/10: (/NULL - ]; /16: (/NULL - ])] + # -------------------------------------------------- # GenerateZigZagJoins # -------------------------------------------------- diff --git a/pkg/sql/opt/xform/testdata/rules/select b/pkg/sql/opt/xform/testdata/rules/select index b9bad0d186e0..3cf42776fb83 100644 --- a/pkg/sql/opt/xform/testdata/rules/select +++ b/pkg/sql/opt/xform/testdata/rules/select @@ -2146,6 +2146,120 @@ project ├── st_covers('0101000020E61000009279E40F069E45C0BEE36FD63B1D5240', geog:4) [outer=(4), immutable] └── v:2 = 3 [outer=(2), constraints=(/2: [/3 - /3]; tight), fd=()-->(2)] +# Bounding box operations. +opt +SELECT k FROM g WHERE 'BOX(1 2, 3 4)'::box2d ~ geom +---- +project + ├── columns: k:1!null + ├── immutable + ├── key: (1) + └── select + ├── columns: k:1!null geom:3 + ├── immutable + ├── key: (1) + ├── fd: (1)-->(3) + ├── index-join g + │ ├── columns: k:1!null geom:3 + │ ├── key: (1) + │ ├── fd: (1)-->(3) + │ └── inverted-filter + │ ├── columns: k:1!null + │ ├── inverted expression: /6 + │ │ ├── tight: false + │ │ └── union spans + │ │ ├── ["\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "\xfd\x10\x00\x00\x00\x00\x00\x00\x00"] + │ │ └── ["\xfd\x10\x00\x00\x00\x00\x00\x00\x01", "\xfd\x18") + │ ├── key: (1) + │ └── scan g@geom_idx + │ ├── columns: k:1!null geom_inverted_key:6!null + │ ├── inverted constraint: /6/1 + │ │ └── spans + │ │ ├── ["\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "\xfd\x10\x00\x00\x00\x00\x00\x00\x00"] + │ │ └── ["\xfd\x10\x00\x00\x00\x00\x00\x00\x01", "\xfd\x18") + │ ├── key: (1) + │ └── fd: (1)-->(6) + └── filters + └── 'BOX(1 2,3 4)' ~ geom:3 [outer=(3), immutable] + +opt +SELECT k FROM g WHERE geom ~ 'BOX(1 2, 3 4)'::box2d +---- +project + ├── columns: k:1!null + ├── immutable + ├── key: (1) + └── select + ├── columns: k:1!null geom:3!null + ├── immutable + ├── key: (1) + ├── fd: (1)-->(3) + ├── index-join g + │ ├── columns: k:1!null geom:3 + │ ├── key: (1) + │ ├── fd: (1)-->(3) + │ └── inverted-filter + │ ├── columns: k:1!null + │ ├── inverted expression: /6 + │ │ ├── tight: false + │ │ └── union spans + │ │ ├── ["\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "\xfd\x10\x00\x00\x00\x00\x00\x00\x00"] + │ │ └── ["\xfd\x14\x00\x00\x00\x00\x00\x00\x00", "\xfd\x14\x00\x00\x00\x00\x00\x00\x00"] + │ ├── key: (1) + │ └── scan g@geom_idx + │ ├── columns: k:1!null geom_inverted_key:6!null + │ ├── inverted constraint: /6/1 + │ │ └── spans + │ │ ├── ["\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "\xfd\x10\x00\x00\x00\x00\x00\x00\x00"] + │ │ └── ["\xfd\x14\x00\x00\x00\x00\x00\x00\x00", "\xfd\x14\x00\x00\x00\x00\x00\x00\x00"] + │ ├── key: (1) + │ └── fd: (1)-->(6) + └── filters + └── geom:3 ~ 'BOX(1 2,3 4)' [outer=(3), immutable, constraints=(/3: (/NULL - ])] + +opt +SELECT k FROM g WHERE geom ~ 'MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry +---- +project + ├── columns: k:1!null + ├── immutable + ├── key: (1) + └── select + ├── columns: k:1!null geom:3!null + ├── immutable + ├── key: (1) + ├── fd: (1)-->(3) + ├── index-join g + │ ├── columns: k:1!null geom:3 + │ ├── key: (1) + │ ├── fd: (1)-->(3) + │ └── inverted-filter + │ ├── columns: k:1!null + │ ├── inverted expression: /6 + │ │ ├── tight: false + │ │ ├── union spans + │ │ │ ├── ["\xfd\x14\x00\x00\x00\x00\x00\x00\x00", "\xfd\x14\x00\x00\x00\x00\x00\x00\x00"] + │ │ │ └── ["\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "\xfd\x10\x00\x00\x00\x00\x00\x00\x00"] + │ │ └── INTERSECTION + │ │ ├── span expression + │ │ │ ├── tight: false + │ │ │ └── union spans: ["\xfd\x15\x00\x00\x00\x00\x00\x00\x00", "\xfd\x15\x00\x00\x00\x00\x00\x00\x00"] + │ │ └── span expression + │ │ ├── tight: false + │ │ └── union spans: ["\xfd\x11\x00\x00\x00\x00\x00\x00\x00", "\xfd\x11\x00\x00\x00\x00\x00\x00\x00"] + │ ├── key: (1) + │ └── scan g@geom_idx + │ ├── columns: k:1!null geom_inverted_key:6!null + │ ├── inverted constraint: /6/1 + │ │ └── spans + │ │ ├── ["\xfd\x10\x00\x00\x00\x00\x00\x00\x00", "\xfd\x10\x00\x00\x00\x00\x00\x00\x00"] + │ │ ├── ["\xfd\x14\x00\x00\x00\x00\x00\x00\x00", "\xfd\x14\x00\x00\x00\x00\x00\x00\x00"] + │ │ ├── ["\xfd\x11\x00\x00\x00\x00\x00\x00\x00", "\xfd\x11\x00\x00\x00\x00\x00\x00\x00"] + │ │ └── ["\xfd\x15\x00\x00\x00\x00\x00\x00\x00", "\xfd\x15\x00\x00\x00\x00\x00\x00\x00"] + │ ├── key: (1) + │ └── fd: (1)-->(6) + └── filters + └── geom:3 ~ '01040000000200000001010000009A999999999901409A99999999990140010100000000000000000008400000000000000840' [outer=(3), immutable, constraints=(/3: (/NULL - ])] # -------------------------------------------------- # SplitDisjunction From 471e856cf122f1094800b134041d0c0bf1e9a049 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Sun, 14 Jun 2020 19:53:38 -0400 Subject: [PATCH 3/9] sql/opt: add implicit SELECT FOR UPDATE support for UPSERT statements Fixes #50180. This commit adds support for implicit SELECT FOR UPDATE support for UPSERT statements with a VALUES clause. This should improve throughput and latency for contended UPSERT statements in much the same way that 435fa43 did so for UPDATE statements. However, this only has an effect on UPSERT statements into tables with multiple indexes because UPSERT statements into single-index tables hit a fast-path where they perform a blind-write without doing an initial row scan. Conceptually, if we picture an UPSERT statement as the composition of a SELECT statement and an INSERT statement (with loosened semantics around existing rows) then this change performs the following transformation: ``` UPSERT t = SELECT FROM t + INSERT INTO t => UPSERT t = SELECT FROM t FOR UPDATE + INSERT INTO t ``` I plan to test this out on a contended `indexes` workload at some point in the future. Release note (sql change): UPSERT statements now acquire locks using the FOR UPDATE locking mode during their initial row scan, which improves performance for contended workloads. This behavior is configurable using the enable_implicit_select_for_update session variable and the sql.defaults.implicit_select_for_update.enabled cluster setting. --- pkg/sql/distsql_physical_planner.go | 8 ++++ pkg/sql/distsql_spec_exec_factory.go | 1 + pkg/sql/opt/exec/execbuilder/mutation.go | 31 +++++++++++++- pkg/sql/opt/exec/execbuilder/relational.go | 6 +++ pkg/sql/opt/exec/execbuilder/testdata/upsert | 43 ++++++++++++++++++++ pkg/sql/opt/exec/explain/emit.go | 1 + pkg/sql/opt/exec/factory.opt | 1 + pkg/sql/opt_exec_factory.go | 5 +++ 8 files changed, 95 insertions(+), 1 deletion(-) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 67e6050db696..bb9c685c59a1 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -436,6 +436,14 @@ func checkSupportForPlanNode(node planNode) (distRecommendation, error) { return checkSupportForPlanNode(n.plan) case *lookupJoinNode: + if n.table.lockingStrength != descpb.ScanLockingStrength_FOR_NONE { + // Lookup joins that are performing row-level locking cannot + // currently be distributed because their locks would not be + // propagated back to the root transaction coordinator. + // TODO(nvanbenschoten): lift this restriction. + return cannotDistribute, cannotDistributeRowLevelLockingErr + } + if err := checkExpr(n.onCond); err != nil { return cannotDistribute, err } diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index 18f2beddee25..aa23b6fea2c1 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -644,6 +644,7 @@ func (e *distSQLSpecExecFactory) ConstructLookupJoin( lookupCols exec.TableColumnOrdinalSet, onCond tree.TypedExpr, reqOrdering exec.OutputOrdering, + locking *tree.LockingItem, ) (exec.Node, error) { // TODO (rohany): Implement production of system columns by the underlying scan here. return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning: lookup join") diff --git a/pkg/sql/opt/exec/execbuilder/mutation.go b/pkg/sql/opt/exec/execbuilder/mutation.go index 29637a667bfd..487a47947456 100644 --- a/pkg/sql/opt/exec/execbuilder/mutation.go +++ b/pkg/sql/opt/exec/execbuilder/mutation.go @@ -996,7 +996,36 @@ func (b *Builder) shouldApplyImplicitLockingToUpdateInput(upd *memo.UpdateExpr) // TODO(nvanbenschoten): implement this method to match on appropriate Upsert // expression trees and apply a row-level locking mode. func (b *Builder) shouldApplyImplicitLockingToUpsertInput(ups *memo.UpsertExpr) bool { - return false + if !b.evalCtx.SessionData.ImplicitSelectForUpdate { + return false + } + + // Try to match the Upsert's input expression against the pattern: + // + // [Project] (LeftJoin Scan | LookupJoin) [Project] Values + // + input := ups.Input + if proj, ok := input.(*memo.ProjectExpr); ok { + input = proj.Input + } + switch join := input.(type) { + case *memo.LeftJoinExpr: + if _, ok := join.Right.(*memo.ScanExpr); !ok { + return false + } + input = join.Left + + case *memo.LookupJoinExpr: + input = join.Input + + default: + return false + } + if proj, ok := input.(*memo.ProjectExpr); ok { + input = proj.Input + } + _, ok := input.(*memo.ValuesExpr) + return ok } // tryApplyImplicitLockingToDeleteInput determines whether or not the builder diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 86c396ded0c6..026f03bf0533 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -1560,6 +1560,11 @@ func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { tab := md.Table(join.Table) idx := tab.Index(join.Index) + var locking *tree.LockingItem + if b.forceForUpdateLocking { + locking = forUpdateLocking + } + res.root, err = b.factory.ConstructLookupJoin( joinOpToJoinType(join.JoinType), input.root, @@ -1570,6 +1575,7 @@ func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { lookupOrdinals, onExpr, res.reqOrdering(join), + locking, ) if err != nil { return execPlan{}, err diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert b/pkg/sql/opt/exec/execbuilder/testdata/upsert index da904462d2ab..cae98a9955ed 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert @@ -164,6 +164,49 @@ upsert · · · estimated row count 1 (missing stats) · table indexed@primary · spans /1-/1/# +· locking strength for update + +query TTT +SELECT tree, field, description FROM [ +EXPLAIN (VERBOSE) UPSERT INTO indexed VALUES (1), (2), (3), (4) +] +---- +· distribution local +· vectorized false +upsert · · + │ estimated row count 0 (missing stats) + │ into indexed(a, b, c, d) + │ auto commit · + └── project · · + └── render · · + │ estimated row count 4 (missing stats) + │ render 0 column8 > 0 + │ render 1 column1 + │ render 2 column7 + │ render 3 column8 + │ render 4 column9 + │ render 5 a + │ render 6 b + │ render 7 c + │ render 8 d + └── lookup join (left outer) · · + │ estimated row count 4 (missing stats) + │ table indexed@primary + │ equality (column1) = (a) + │ equality cols are key · + │ locking strength for update + └── render · · + │ estimated row count 4 + │ render 0 column1 + 10 + │ render 1 CAST(NULL AS INT8) + │ render 2 10 + │ render 3 column1 + └── values · · +· size 1 column, 4 rows +· row 0, expr 0 1 +· row 1, expr 0 2 +· row 2, expr 0 3 +· row 3, expr 0 4 # Drop index and verify that existing values no longer need to be fetched. statement ok diff --git a/pkg/sql/opt/exec/explain/emit.go b/pkg/sql/opt/exec/explain/emit.go index e5cd81e8c0cd..d356a9102e9f 100644 --- a/pkg/sql/opt/exec/explain/emit.go +++ b/pkg/sql/opt/exec/explain/emit.go @@ -497,6 +497,7 @@ func (e *emitter) emitNodeAttributes(n *Node) error { ob.Attr("equality cols are key", "") } ob.Expr("pred", a.OnCond, appendColumns(inputCols, tableColumns(a.Table, a.LookupCols)...)) + e.emitLockingPolicy(a.Locking) case interleavedJoinOp: a := n.args.(*interleavedJoinArgs) diff --git a/pkg/sql/opt/exec/factory.opt b/pkg/sql/opt/exec/factory.opt index 4728e1e60eff..6d94ddcdf0b4 100644 --- a/pkg/sql/opt/exec/factory.opt +++ b/pkg/sql/opt/exec/factory.opt @@ -244,6 +244,7 @@ define LookupJoin { LookupCols exec.TableColumnOrdinalSet OnCond tree.TypedExpr ReqOrdering exec.OutputOrdering + Locking *tree.LockingItem } # InvertedJoin performs a lookup join into an inverted index. diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 81a8d1ecce5b..d43944698a0c 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -624,6 +624,7 @@ func (ef *execFactory) ConstructLookupJoin( lookupCols exec.TableColumnOrdinalSet, onCond tree.TypedExpr, reqOrdering exec.OutputOrdering, + locking *tree.LockingItem, ) (exec.Node, error) { if table.IsVirtualTable() { return ef.constructVirtualTableLookupJoin(joinType, input, table, index, eqCols, lookupCols, onCond) @@ -638,6 +639,10 @@ func (ef *execFactory) ConstructLookupJoin( } tableScan.index = indexDesc + if locking != nil { + tableScan.lockingStrength = descpb.ToScanLockingStrength(locking.Strength) + tableScan.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(locking.WaitPolicy) + } n := &lookupJoinNode{ input: input.(planNode), From 7a799222e39a5296c86d669cc3fcd72b2656f0b4 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 20 Aug 2020 13:45:38 -0400 Subject: [PATCH 4/9] sql/opt/exec: re-enable locking policy EXPLAIN non-verbose attributes Reverts a portion of #53003. These attributes are not included when they are not interesting, but when they are included, they are very interesting and deserve to be surfaced. --- pkg/sql/opt/exec/execbuilder/testdata/fk | 77 ++++---- .../exec/execbuilder/testdata/select_index | 23 +-- pkg/sql/opt/exec/execbuilder/testdata/spool | 86 ++++----- pkg/sql/opt/exec/execbuilder/testdata/update | 165 +++++++++--------- pkg/sql/opt/exec/explain/emit.go | 4 +- 5 files changed, 188 insertions(+), 167 deletions(-) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/fk b/pkg/sql/opt/exec/execbuilder/testdata/fk index ebc13ed614b4..1c1c42b4a431 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/fk +++ b/pkg/sql/opt/exec/execbuilder/testdata/fk @@ -309,6 +309,7 @@ root · · │ missing stats · │ table child@primary │ spans FULL SCAN + │ locking strength for update └── fk-check · · └── error if rows · · └── hash join (anti) · · @@ -337,6 +338,7 @@ root · · │ missing stats · │ table child@primary │ spans [/10 - /10] + │ locking strength for update └── fk-check · · └── error if rows · · └── lookup join (anti) · · @@ -362,6 +364,7 @@ root · · │ missing stats · │ table parent@primary │ spans FULL SCAN + │ locking strength for update ├── fk-check · · │ └── error if rows · · │ └── hash join · · @@ -402,39 +405,40 @@ root · · query TTT EXPLAIN UPDATE parent SET p = p+1 WHERE other = 10 ---- -· distribution local -· vectorized false -root · · - ├── update · · - │ │ table parent - │ │ set p - │ └── buffer · · - │ │ label buffer 1 - │ └── render · · - │ └── scan · · - │ missing stats · - │ table parent@parent_other_key - │ spans [/10 - /10] - ├── fk-check · · - │ └── error if rows · · - │ └── lookup join (semi) · · - │ │ table child@child_p_idx - │ │ equality (p) = (p) - │ └── except · · - │ ├── scan buffer · · - │ │ label buffer 1 - │ └── scan buffer · · - │ label buffer 1 - └── fk-check · · - └── error if rows · · - └── lookup join (semi) · · - │ table child_nullable@child_nullable_p_idx - │ equality (p) = (p) - └── except · · - ├── scan buffer · · - │ label buffer 1 - └── scan buffer · · -· label buffer 1 +· distribution local +· vectorized false +root · · + ├── update · · + │ │ table parent + │ │ set p + │ └── buffer · · + │ │ label buffer 1 + │ └── render · · + │ └── scan · · + │ missing stats · + │ table parent@parent_other_key + │ spans [/10 - /10] + │ locking strength for update + ├── fk-check · · + │ └── error if rows · · + │ └── lookup join (semi) · · + │ │ table child@child_p_idx + │ │ equality (p) = (p) + │ └── except · · + │ ├── scan buffer · · + │ │ label buffer 1 + │ └── scan buffer · · + │ label buffer 1 + └── fk-check · · + └── error if rows · · + └── lookup join (semi) · · + │ table child_nullable@child_nullable_p_idx + │ equality (p) = (p) + └── except · · + ├── scan buffer · · + │ label buffer 1 + └── scan buffer · · +· label buffer 1 statement ok CREATE TABLE grandchild (g INT PRIMARY KEY, c INT NOT NULL REFERENCES child(c)) @@ -455,6 +459,7 @@ root · · │ missing stats · │ table child@primary │ spans FULL SCAN + │ locking strength for update └── fk-check · · └── error if rows · · └── hash join · · @@ -490,6 +495,7 @@ root · · │ missing stats · │ table child@primary │ spans FULL SCAN + │ locking strength for update └── fk-check · · └── error if rows · · └── hash join (anti) · · @@ -517,6 +523,7 @@ root · · │ missing stats · │ table child@primary │ spans FULL SCAN + │ locking strength for update └── fk-check · · └── error if rows · · └── hash join (anti) · · @@ -545,6 +552,7 @@ root · · │ missing stats · │ table child@primary │ spans FULL SCAN + │ locking strength for update ├── fk-check · · │ └── error if rows · · │ └── hash join (anti) · · @@ -594,6 +602,7 @@ root · · │ missing stats · │ table child@primary │ spans FULL SCAN + │ locking strength for update └── fk-check · · └── error if rows · · └── hash join (anti) · · @@ -625,6 +634,7 @@ root · · │ missing stats · │ table self@primary │ spans FULL SCAN + │ locking strength for update └── fk-check · · └── error if rows · · └── hash join (anti) · · @@ -653,6 +663,7 @@ root · · │ missing stats · │ table self@primary │ spans FULL SCAN + │ locking strength for update └── fk-check · · └── error if rows · · └── hash join · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/select_index b/pkg/sql/opt/exec/execbuilder/testdata/select_index index 702bb4114dcd..be1fce24b1fa 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/select_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/select_index @@ -1613,17 +1613,18 @@ scan · · query TTT EXPLAIN UPDATE t4 SET c = 30 WHERE a = 10 and b = 20 ---- -· distribution local -· vectorized false -update · · - │ table t4 - │ set c - │ auto commit · - └── render · · - └── scan · · -· missing stats · -· table t4@primary -· spans [/10/20 - /10/20] +· distribution local +· vectorized false +update · · + │ table t4 + │ set c + │ auto commit · + └── render · · + └── scan · · +· missing stats · +· table t4@primary +· spans [/10/20 - /10/20] +· locking strength for update # Optimization should not be applied for deletes. query TTT diff --git a/pkg/sql/opt/exec/execbuilder/testdata/spool b/pkg/sql/opt/exec/execbuilder/testdata/spool index 649dbfc2380a..88951aea4c85 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/spool +++ b/pkg/sql/opt/exec/execbuilder/testdata/spool @@ -60,27 +60,28 @@ query TTT EXPLAIN WITH a AS (UPDATE t SET x = x + 1 RETURNING x) SELECT * FROM a LIMIT 1 ---- -· distribution local -· vectorized false -root · · - ├── limit · · - │ │ count 1 - │ └── scan buffer · · - │ label buffer 1 (a) - └── subquery · · - │ id @S1 - │ original sql UPDATE t SET x = x + 1 RETURNING x - │ exec mode all rows - └── buffer · · - │ label buffer 1 (a) - └── update · · - │ table t - │ set x - └── render · · - └── scan · · -· missing stats · -· table t@primary -· spans FULL SCAN +· distribution local +· vectorized false +root · · + ├── limit · · + │ │ count 1 + │ └── scan buffer · · + │ label buffer 1 (a) + └── subquery · · + │ id @S1 + │ original sql UPDATE t SET x = x + 1 RETURNING x + │ exec mode all rows + └── buffer · · + │ label buffer 1 (a) + └── update · · + │ table t + │ set x + └── render · · + └── scan · · +· missing stats · +· table t@primary +· spans FULL SCAN +· locking strength for update query TTT EXPLAIN WITH a AS (UPSERT INTO t VALUES (2), (3) RETURNING x) @@ -154,27 +155,28 @@ root · · query TTT EXPLAIN SELECT * FROM [UPDATE t SET x = x + 1 RETURNING x] LIMIT 1 ---- -· distribution local -· vectorized false -root · · - ├── limit · · - │ │ count 1 - │ └── scan buffer · · - │ label buffer 1 - └── subquery · · - │ id @S1 - │ original sql UPDATE t SET x = x + 1 RETURNING x - │ exec mode all rows - └── buffer · · - │ label buffer 1 - └── update · · - │ table t - │ set x - └── render · · - └── scan · · -· missing stats · -· table t@primary -· spans FULL SCAN +· distribution local +· vectorized false +root · · + ├── limit · · + │ │ count 1 + │ └── scan buffer · · + │ label buffer 1 + └── subquery · · + │ id @S1 + │ original sql UPDATE t SET x = x + 1 RETURNING x + │ exec mode all rows + └── buffer · · + │ label buffer 1 + └── update · · + │ table t + │ set x + └── render · · + └── scan · · +· missing stats · +· table t@primary +· spans FULL SCAN +· locking strength for update query TTT EXPLAIN SELECT * FROM [UPSERT INTO t VALUES (2), (3) RETURNING x] LIMIT 1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/update b/pkg/sql/opt/exec/execbuilder/testdata/update index f5543407f8b5..252af412bb5c 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/update +++ b/pkg/sql/opt/exec/execbuilder/testdata/update @@ -101,16 +101,17 @@ CREATE TABLE xyz ( query TTT EXPLAIN UPDATE xyz SET y = x ---- -· distribution local -· vectorized false -update · · - │ table xyz - │ set y - │ auto commit · - └── scan · · -· missing stats · -· table xyz@primary -· spans FULL SCAN +· distribution local +· vectorized false +update · · + │ table xyz + │ set y + │ auto commit · + └── scan · · +· missing stats · +· table xyz@primary +· spans FULL SCAN +· locking strength for update query TTTTT EXPLAIN (VERBOSE) UPDATE xyz SET (x, y) = (1, 2) @@ -241,18 +242,19 @@ update · · query TTT EXPLAIN UPDATE kv SET v = v - 1 WHERE k < 3 LIMIT 1 ---- -· distribution local -· vectorized false -update · · - │ table kv - │ set v - │ auto commit · - └── render · · - └── scan · · -· missing stats · -· table kv@primary -· spans [ - /2] -· limit 1 +· distribution local +· vectorized false +update · · + │ table kv + │ set v + │ auto commit · + └── render · · + └── scan · · +· missing stats · +· table kv@primary +· spans [ - /2] +· limit 1 +· locking strength for update # Check that updates on tables with multiple column families behave as # they should. @@ -418,46 +420,49 @@ update · · () query TTT EXPLAIN UPDATE kv SET v = 10 WHERE k = 3 ---- -· distribution local -· vectorized false -update · · - │ table kv - │ set v - │ auto commit · - └── render · · - └── scan · · -· missing stats · -· table kv@primary -· spans [/3 - /3] +· distribution local +· vectorized false +update · · + │ table kv + │ set v + │ auto commit · + └── render · · + └── scan · · +· missing stats · +· table kv@primary +· spans [/3 - /3] +· locking strength for update query TTT EXPLAIN UPDATE kv SET v = k WHERE k > 1 AND k < 10 ---- -· distribution local -· vectorized false -update · · - │ table kv - │ set v - │ auto commit · - └── scan · · -· missing stats · -· table kv@primary -· spans [/2 - /9] +· distribution local +· vectorized false +update · · + │ table kv + │ set v + │ auto commit · + └── scan · · +· missing stats · +· table kv@primary +· spans [/2 - /9] +· locking strength for update query TTT EXPLAIN UPDATE kv SET v = 10 ---- -· distribution local -· vectorized false -update · · - │ table kv - │ set v - │ auto commit · - └── render · · - └── scan · · -· missing stats · -· table kv@primary -· spans FULL SCAN +· distribution local +· vectorized false +update · · + │ table kv + │ set v + │ auto commit · + └── render · · + └── scan · · +· missing stats · +· table kv@primary +· spans FULL SCAN +· locking strength for update statement ok CREATE TABLE kv3 ( @@ -471,35 +476,37 @@ CREATE TABLE kv3 ( query TTT EXPLAIN UPDATE kv3 SET k = 3 WHERE v = 10 ---- -· distribution local -· vectorized false -update · · - │ table kv3 - │ set k - │ auto commit · - └── render · · - └── index join · · - │ table kv3@primary - └── scan · · -· missing stats · -· table kv3@kv3_v_idx -· spans [/10 - /10] +· distribution local +· vectorized false +update · · + │ table kv3 + │ set k + │ auto commit · + └── render · · + └── index join · · + │ table kv3@primary + └── scan · · +· missing stats · +· table kv3@kv3_v_idx +· spans [/10 - /10] +· locking strength for update query TTT EXPLAIN UPDATE kv3 SET k = v WHERE v > 1 AND v < 10 ---- -· distribution local -· vectorized false -update · · - │ table kv3 - │ set k - │ auto commit · - └── index join · · - │ table kv3@primary - └── scan · · -· missing stats · -· table kv3@kv3_v_idx -· spans [/2 - /9] +· distribution local +· vectorized false +update · · + │ table kv3 + │ set k + │ auto commit · + └── index join · · + │ table kv3@primary + └── scan · · +· missing stats · +· table kv3@kv3_v_idx +· spans [/2 - /9] +· locking strength for update statement ok SET enable_implicit_select_for_update = false diff --git a/pkg/sql/opt/exec/explain/emit.go b/pkg/sql/opt/exec/explain/emit.go index d356a9102e9f..ccbcade857c1 100644 --- a/pkg/sql/opt/exec/explain/emit.go +++ b/pkg/sql/opt/exec/explain/emit.go @@ -737,10 +737,10 @@ func (e *emitter) emitLockingPolicy(locking *tree.LockingItem) { strength := descpb.ToScanLockingStrength(locking.Strength) waitPolicy := descpb.ToScanLockingWaitPolicy(locking.WaitPolicy) if strength != descpb.ScanLockingStrength_FOR_NONE { - e.ob.VAttr("locking strength", strength.PrettyString()) + e.ob.Attr("locking strength", strength.PrettyString()) } if waitPolicy != descpb.ScanLockingWaitPolicy_BLOCK { - e.ob.VAttr("locking wait policy", waitPolicy.PrettyString()) + e.ob.Attr("locking wait policy", waitPolicy.PrettyString()) } } From d67890ccb168219b8b0910d00e8a44523522ba87 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 20 Aug 2020 13:40:42 -0400 Subject: [PATCH 5/9] workload/indexes: add --cycle-length flag Controls the number of keys repeatedly accessed by each writer through upserts. Mirrors the same flag in `kv`. --- pkg/workload/indexes/indexes.go | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/pkg/workload/indexes/indexes.go b/pkg/workload/indexes/indexes.go index 93e184c7ddea..d6c0556b9996 100644 --- a/pkg/workload/indexes/indexes.go +++ b/pkg/workload/indexes/indexes.go @@ -47,10 +47,11 @@ type indexes struct { flags workload.Flags connFlags *workload.ConnFlags - seed int64 - idxs int - unique bool - payload int + seed int64 + idxs int + unique bool + payload int + cycleLength uint64 } func init() { @@ -68,6 +69,8 @@ var indexesMeta = workload.Meta{ g.flags.IntVar(&g.idxs, `secondary-indexes`, 1, `Number of indexes to add to the table.`) g.flags.BoolVar(&g.unique, `unique-indexes`, false, `Use UNIQUE secondary indexes.`) g.flags.IntVar(&g.payload, `payload`, 64, `Size of the unindexed payload column.`) + g.flags.Uint64Var(&g.cycleLength, `cycle-length`, math.MaxUint64, + `Number of keys repeatedly accessed by each writer through upserts.`) g.connFlags = workload.NewConnFlags(&g.flags) return g }, @@ -184,10 +187,10 @@ type indexesOp struct { } func (o *indexesOp) run(ctx context.Context) error { - keyHi, keyLo := o.rand.Uint64(), o.rand.Uint64() + keyLo := o.rand.Uint64() % o.config.cycleLength _, _ = o.rand.Read(o.buf[:]) args := []interface{}{ - uuid.FromUint128(uint128.FromInts(keyHi, keyLo)).String(), // key + uuid.FromUint128(uint128.FromInts(0, keyLo)).String(), // key int64(keyLo + 0), // col0 int64(keyLo + 1), // col1 int64(keyLo + 2), // col2 From bc775f9e2b16ee9ca1124bb1ebe07255e6a46920 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Mon, 17 Aug 2020 11:48:54 -0400 Subject: [PATCH 6/9] bulkio: Fix `SHOW JOBS FOR SCHEDULE` Fix show jobs delegation code to correclty handle `FOR SCHEDUL(s)` clause. Release Notes: None --- pkg/sql/delegate/show_jobs.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pkg/sql/delegate/show_jobs.go b/pkg/sql/delegate/show_jobs.go index d5226c6cee2d..dbd7103dfce8 100644 --- a/pkg/sql/delegate/show_jobs.go +++ b/pkg/sql/delegate/show_jobs.go @@ -20,6 +20,14 @@ import ( ) func (d *delegator) delegateShowJobs(n *tree.ShowJobs) (tree.Statement, error) { + if n.Schedules != nil { + // Limit the jobs displayed to the ones started by specified schedules. + return parse(fmt.Sprintf(` +SHOW JOBS SELECT id FROM system.jobs WHERE created_by_type='%s' and created_by_id IN (%s) +`, jobs.CreatedByScheduledJobs, n.Schedules.String()), + ) + } + sqltelemetry.IncrementShowCounter(sqltelemetry.Jobs) if n.Schedules != nil { From aab9608e982895613ce13a028151a2ac9fe13e76 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Wed, 19 Aug 2020 11:16:05 -0400 Subject: [PATCH 7/9] bulkio: Fix transaction semantics in job scheduler. Use transaction when querying for the schedules to run. In addition, ensure that a single bad schedule does not cause all of the previous work to be wasted by using transaction savepoints. Release Notes: None --- pkg/jobs/job_scheduler.go | 22 +++++++-- pkg/jobs/job_scheduler_test.go | 82 +++++++++++++++++++++++++++++++--- 2 files changed, 94 insertions(+), 10 deletions(-) diff --git a/pkg/jobs/job_scheduler.go b/pkg/jobs/job_scheduler.go index 6ecc863c05fe..e1752d2c1d0d 100644 --- a/pkg/jobs/job_scheduler.go +++ b/pkg/jobs/job_scheduler.go @@ -72,7 +72,7 @@ const allSchedules = 0 // scheduled jobs that should be started. func getFindSchedulesStatement(env scheduledjobs.JobSchedulerEnv, maxSchedules int64) string { limitClause := "" - if maxSchedules > 0 { + if maxSchedules != allSchedules { limitClause = fmt.Sprintf("LIMIT %d", maxSchedules) } @@ -236,8 +236,10 @@ func (s *jobScheduler) executeSchedules( defer stats.updateMetrics(&s.metrics) findSchedulesStmt := getFindSchedulesStatement(s.env, maxSchedules) - rows, cols, err := s.InternalExecutor.QueryWithCols(ctx, "find-scheduled-jobs", nil, - sqlbase.InternalExecutorSessionDataOverride{User: security.RootUser}, + rows, cols, err := s.InternalExecutor.QueryWithCols( + ctx, "find-scheduled-jobs", + txn, + sqlbase.InternalExecutorSessionDataOverride{User: security.NodeUser}, findSchedulesStmt) if err != nil { @@ -252,8 +254,20 @@ func (s *jobScheduler) executeSchedules( continue } + sp, err := txn.CreateSavepoint(ctx) + if err != nil { + return err + } + if err := s.processSchedule(ctx, schedule, numRunning, stats, txn); err != nil { - // We don't know if txn is good at this point, so bail out. + log.Errorf(ctx, "error processing schedule %d: %+v", schedule.ScheduleID(), err) + + if err := txn.RollbackToSavepoint(ctx, sp); err != nil { + return errors.Wrapf(err, "failed to rollback savepoint for schedule %d", schedule.ScheduleID()) + } + } + + if err := txn.ReleaseSavepoint(ctx, sp); err != nil { return err } } diff --git a/pkg/jobs/job_scheduler_test.go b/pkg/jobs/job_scheduler_test.go index d4dee5974d9c..1c8582c23a4c 100644 --- a/pkg/jobs/job_scheduler_test.go +++ b/pkg/jobs/job_scheduler_test.go @@ -26,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" - "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -43,8 +42,6 @@ func TestJobSchedulerReschedulesRunning(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - skip.WithIssue(t, 52959) - h, cleanup := newTestHelper(t) defer cleanup() @@ -84,7 +81,10 @@ func TestJobSchedulerReschedulesRunning(t *testing.T) { // The job should not run -- it should be rescheduled `recheckJobAfter` time in the // future. s := newJobScheduler(h.cfg, h.env, metric.NewRegistry()) - require.NoError(t, s.executeSchedules(ctx, allSchedules, nil)) + require.NoError(t, + h.cfg.DB.Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { + return s.executeSchedules(ctx, allSchedules, txn) + })) if wait == jobspb.ScheduleDetails_WAIT { expectedRunTime = h.env.Now().Add(recheckRunningAfter) @@ -138,7 +138,10 @@ func TestJobSchedulerExecutesAfterTerminal(t *testing.T) { // Execute the job and verify it has the next run scheduled. s := newJobScheduler(h.cfg, h.env, metric.NewRegistry()) - require.NoError(t, s.executeSchedules(ctx, allSchedules, nil)) + require.NoError(t, + h.cfg.DB.Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { + return s.executeSchedules(ctx, allSchedules, txn) + })) expectedRunTime = cronexpr.MustParse("@hourly").Next(h.env.Now()) loaded = h.loadSchedule(t, j.ScheduleID()) @@ -175,7 +178,10 @@ func TestJobSchedulerExecutesAndSchedulesNextRun(t *testing.T) { // Execute the job and verify it has the next run scheduled. s := newJobScheduler(h.cfg, h.env, metric.NewRegistry()) - require.NoError(t, s.executeSchedules(ctx, allSchedules, nil)) + require.NoError(t, + h.cfg.DB.Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { + return s.executeSchedules(ctx, allSchedules, txn) + })) expectedRunTime = cronexpr.MustParse("@hourly").Next(h.env.Now()) loaded = h.loadSchedule(t, j.ScheduleID()) @@ -431,6 +437,70 @@ func TestJobSchedulerDaemonHonorsMaxJobsLimit(t *testing.T) { stopper.Stop(ctx) } +// returnErrorExecutor counts the number of times it is +// called, and always returns an error. +type returnErrorExecutor struct { + numCalls int +} + +func (e *returnErrorExecutor) ExecuteJob( + _ context.Context, + _ *scheduledjobs.JobExecutionConfig, + _ scheduledjobs.JobSchedulerEnv, + schedule *ScheduledJob, + _ *kv.Txn, +) error { + e.numCalls++ + return errors.Newf("error for schedule %d", schedule.ScheduleID()) +} + +func (e *returnErrorExecutor) NotifyJobTermination( + _ context.Context, + _ int64, + _ Status, + _ scheduledjobs.JobSchedulerEnv, + _ *ScheduledJob, + _ sqlutil.InternalExecutor, + _ *kv.Txn, +) error { + return nil +} + +func (e *returnErrorExecutor) Metrics() metric.Struct { + return nil +} + +var _ ScheduledJobExecutor = &returnErrorExecutor{} + +func TestJobSchedulerToleratesBadSchedules(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + h, cleanup := newTestHelper(t) + defer cleanup() + + ctx := context.Background() + + const executorName = "return_error" + ex := &returnErrorExecutor{} + defer registerScopedScheduledJobExecutor(executorName, ex)() + + // Create few one-off schedules. + const numJobs = 5 + scheduleRunTime := h.env.Now().Add(time.Hour) + for i := 0; i < numJobs; i++ { + s := h.newScheduledJobForExecutor("schedule", executorName, nil) + s.SetNextRun(scheduleRunTime) + require.NoError(t, s.Create(ctx, h.cfg.InternalExecutor, nil)) + } + h.env.SetTime(scheduleRunTime.Add(time.Second)) + daemon := newJobScheduler(h.cfg, h.env, metric.NewRegistry()) + require.NoError(t, + h.cfg.DB.Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { + return daemon.executeSchedules(ctx, numJobs, txn) + })) + require.Equal(t, numJobs, ex.numCalls) +} + func TestJobSchedulerDaemonUsesSystemTables(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) From ed34965b3622d551cd56403838ebe3024036d848 Mon Sep 17 00:00:00 2001 From: Alex Lunev Date: Fri, 17 Jul 2020 13:00:21 -0700 Subject: [PATCH 8/9] kvserver: Allow rebalances between stores on the same nodes. Closes #6782 This change modifies the replica_queue to allow rebalances between multiple stores within a single node. This is possible because we previously introduced atomic rebalances in #12768. The first step was to remove the constraints in the allocator that prevented same node rebalances and update the validation in the replica_queue to accept these rebalance proposals. There is one caveat that with 1x replication an atomic rebalance is not possible, so we now support adding multiple replicas of the range to the same node under this condition. With the constraints removed there would be nothing in the allocator to prevent it from placing multiple replicas of a range on the same node across multiple stores. This is not desired because in a simple 3x replication scenario a single node crash may result in a whole range becoming unavailable. Allocator uses locality tags to model failure domains, but a node was not considered to be a locality. It is thus natural to extend the failure domain definition to the node and model it as a locality tier. Now stores on the same node would be factored into the diversity_score and repel each other, just like nodes in the same datacenter do in a multi-region setup. Release note (performance improvement): This change removes the last roadblock to running CockroachDB with multiple stores (i.e. disks) per node. The allocation algorithm now supports intra-node rebalances, which means CRDB can fully utilize the additional stores on the same node. --- pkg/kv/kvserver/allocator.go | 11 +- pkg/kv/kvserver/allocator_scorer.go | 132 +++++++++------------ pkg/kv/kvserver/allocator_scorer_test.go | 79 ++++++------ pkg/kv/kvserver/allocator_test.go | 145 ++++++++++++++++++++--- pkg/kv/kvserver/replica_command.go | 101 +++++++++++----- pkg/kv/kvserver/replica_command_test.go | 139 ++++++++++++++++++++++ pkg/kv/kvserver/replica_test.go | 6 + pkg/kv/kvserver/store_pool.go | 28 ++++- pkg/kv/kvserver/store_pool_test.go | 31 +++-- pkg/kv/kvserver/store_rebalancer.go | 4 +- pkg/roachpb/metadata.go | 18 +++ pkg/roachpb/metadata_test.go | 12 ++ pkg/server/status_test.go | 7 ++ 13 files changed, 530 insertions(+), 183 deletions(-) diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index b2d38985bdb6..a8167345ed12 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -502,8 +502,8 @@ func (a *Allocator) allocateTargetFromList( analyzedConstraints := constraint.AnalyzeConstraints( ctx, a.storePool.getStoreDescriptor, candidateReplicas, zone) candidates := allocateCandidates( - sl, analyzedConstraints, candidateReplicas, a.storePool.getLocalities(candidateReplicas), - options, + sl, analyzedConstraints, candidateReplicas, + a.storePool.getLocalitiesByStore(candidateReplicas), options, ) log.VEventf(ctx, 3, "allocate candidates: %s", candidates) if target := candidates.selectGood(a.randGen); target != nil { @@ -568,7 +568,7 @@ func (a Allocator) RemoveTarget( rankedCandidates := removeCandidates( sl, analyzedConstraints, - a.storePool.getLocalities(existingReplicas), + a.storePool.getLocalitiesByStore(existingReplicas), options, ) log.VEventf(ctx, 3, "remove candidates: %s", rankedCandidates) @@ -663,8 +663,7 @@ func (a Allocator) RebalanceTarget( sl, analyzedConstraints, existingReplicas, - a.storePool.getLocalities(existingReplicas), - a.storePool.getNodeLocalityString, + a.storePool.getLocalitiesByStore(existingReplicas), options, ) @@ -1005,7 +1004,7 @@ func (a Allocator) shouldTransferLeaseUsingStats( if stats == nil || !enableLoadBasedLeaseRebalancing.Get(&a.storePool.st.SV) { return decideWithoutStats, roachpb.ReplicaDescriptor{} } - replicaLocalities := a.storePool.getLocalities(existing) + replicaLocalities := a.storePool.getLocalitiesByNode(existing) for _, locality := range replicaLocalities { if len(locality.Tiers) == 0 { return decideWithoutStats, roachpb.ReplicaDescriptor{} diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index a13b65eab46c..bf882745acc1 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -410,7 +410,7 @@ func allocateCandidates( sl StoreList, constraints constraint.AnalyzedConstraints, existing []roachpb.ReplicaDescriptor, - existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, + existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, options scorerOptions, ) candidateList { var candidates candidateList @@ -425,7 +425,7 @@ func allocateCandidates( if !maxCapacityCheck(s) { continue } - diversityScore := diversityAllocateScore(s, existingNodeLocalities) + diversityScore := diversityAllocateScore(s, existingStoreLocalities) balanceScore := balanceScore(sl, s.Capacity, options) var convergesScore int if options.qpsRebalanceThreshold > 0 { @@ -463,7 +463,7 @@ func allocateCandidates( func removeCandidates( sl StoreList, constraints constraint.AnalyzedConstraints, - existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, + existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, options scorerOptions, ) candidateList { var candidates candidateList @@ -478,7 +478,7 @@ func removeCandidates( }) continue } - diversityScore := diversityRemovalScore(s.Node.NodeID, existingNodeLocalities) + diversityScore := diversityRemovalScore(s.StoreID, existingStoreLocalities) balanceScore := balanceScore(sl, s.Capacity, options) var convergesScore int if !rebalanceFromConvergesOnMean(sl, s.Capacity) { @@ -522,18 +522,13 @@ func rebalanceCandidates( allStores StoreList, constraints constraint.AnalyzedConstraints, existingReplicas []roachpb.ReplicaDescriptor, - existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, - localityLookupFn func(roachpb.NodeID) string, + existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, options scorerOptions, ) []rebalanceOptions { // 1. Determine whether existing replicas are valid and/or necessary. - type existingStore struct { - cand candidate - localityStr string - } - existingStores := make(map[roachpb.StoreID]existingStore) + existingStores := make(map[roachpb.StoreID]candidate) var needRebalanceFrom bool - curDiversityScore := rangeDiversityScore(existingNodeLocalities) + curDiversityScore := rangeDiversityScore(existingStoreLocalities) for _, store := range allStores.stores { for _, repl := range existingReplicas { if store.StoreID != repl.StoreID { @@ -544,7 +539,7 @@ func rebalanceCandidates( if !valid { if !needRebalanceFrom { log.VEventf(ctx, 2, "s%d: should-rebalance(invalid): locality:%q", - store.StoreID, store.Node.Locality) + store.StoreID, store.Locality()) } needRebalanceFrom = true } @@ -555,15 +550,12 @@ func rebalanceCandidates( } needRebalanceFrom = true } - existingStores[store.StoreID] = existingStore{ - cand: candidate{ - store: store, - valid: valid, - necessary: necessary, - fullDisk: fullDisk, - diversityScore: curDiversityScore, - }, - localityStr: localityLookupFn(store.Node.NodeID), + existingStores[store.StoreID] = candidate{ + store: store, + valid: valid, + necessary: necessary, + fullDisk: fullDisk, + diversityScore: curDiversityScore, } } } @@ -599,8 +591,8 @@ func rebalanceCandidates( // include Node/Store Attributes because they affect constraints. var matchedOtherExisting bool for i, stores := range comparableStores { - if sameLocalityAndAttrs(stores.existing[0], existing.cand.store) { - comparableStores[i].existing = append(comparableStores[i].existing, existing.cand.store) + if sameLocalityAndAttrs(stores.existing[0], existing.store) { + comparableStores[i].existing = append(comparableStores[i].existing, existing.store) matchedOtherExisting = true break } @@ -610,21 +602,11 @@ func rebalanceCandidates( } var comparableCands candidateList for _, store := range allStores.stores { - // Nodes that already have a replica on one of their stores aren't valid - // rebalance targets. We do include stores that currently have a replica - // because we want them to be considered as valid stores in the - // ConvergesOnMean calculations below. This is subtle but important. - if nodeHasReplica(store.Node.NodeID, existingReplicas) && - !storeHasReplica(store.StoreID, existingReplicas) { - log.VEventf(ctx, 2, "nodeHasReplica(n%d, %v)=true", - store.Node.NodeID, existingReplicas) - continue - } constraintsOK, necessary := rebalanceFromConstraintsCheck( - store, existing.cand.store.StoreID, constraints) + store, existing.store.StoreID, constraints) maxCapacityOK := maxCapacityCheck(store) diversityScore := diversityRebalanceFromScore( - store, existing.cand.store.Node.NodeID, existingNodeLocalities) + store, existing.store.StoreID, existingStoreLocalities) cand := candidate{ store: store, valid: constraintsOK, @@ -632,15 +614,15 @@ func rebalanceCandidates( fullDisk: !maxCapacityOK, diversityScore: diversityScore, } - if !cand.less(existing.cand) { + if !cand.less(existing) { comparableCands = append(comparableCands, cand) - if !needRebalanceFrom && !needRebalanceTo && existing.cand.less(cand) { + if !needRebalanceFrom && !needRebalanceTo && existing.less(cand) { needRebalanceTo = true log.VEventf(ctx, 2, "s%d: should-rebalance(necessary/diversity=s%d): oldNecessary:%t, newNecessary:%t, "+ "oldDiversity:%f, newDiversity:%f, locality:%q", - existing.cand.store.StoreID, store.StoreID, existing.cand.necessary, cand.necessary, - existing.cand.diversityScore, cand.diversityScore, store.Node.Locality) + existing.store.StoreID, store.StoreID, existing.necessary, cand.necessary, + existing.diversityScore, cand.diversityScore, store.Locality()) } } } @@ -655,7 +637,7 @@ func rebalanceCandidates( bestStores[i] = bestCands[i].store } comparableStores = append(comparableStores, comparableStoreList{ - existing: []roachpb.StoreDescriptor{existing.cand.store}, + existing: []roachpb.StoreDescriptor{existing.store}, sl: makeStoreList(bestStores), candidates: bestCands, }) @@ -673,7 +655,7 @@ func rebalanceCandidates( outer: for _, comparable := range comparableStores { for _, existingCand := range comparable.existing { - if existing.cand.store.StoreID == existingCand.StoreID { + if existing.store.StoreID == existingCand.StoreID { sl = comparable.sl break outer } @@ -681,7 +663,7 @@ func rebalanceCandidates( } // TODO(a-robinson): Some moderate refactoring could extract this logic out // into the loop below, avoiding duplicate balanceScore calculations. - if shouldRebalance(ctx, existing.cand.store, sl, options) { + if shouldRebalance(ctx, existing.store, sl, options) { shouldRebalanceCheck = true break } @@ -705,24 +687,24 @@ func rebalanceCandidates( existingDesc, existingStores) continue } - if !existing.cand.valid { - existing.cand.details = "constraint check fail" - existingCandidates = append(existingCandidates, existing.cand) + if !existing.valid { + existing.details = "constraint check fail" + existingCandidates = append(existingCandidates, existing) continue } - balanceScore := balanceScore(comparable.sl, existing.cand.store.Capacity, options) + balanceScore := balanceScore(comparable.sl, existing.store.Capacity, options) var convergesScore int - if !rebalanceFromConvergesOnMean(comparable.sl, existing.cand.store.Capacity) { + if !rebalanceFromConvergesOnMean(comparable.sl, existing.store.Capacity) { // Similarly to in removeCandidates, any replica whose removal // would not converge the range stats to their means is given a // constraint score boost of 1 to make it less attractive for // removal. convergesScore = 1 } - existing.cand.convergesScore = convergesScore - existing.cand.balanceScore = balanceScore - existing.cand.rangeCount = int(existing.cand.store.Capacity.RangeCount) - existingCandidates = append(existingCandidates, existing.cand) + existing.convergesScore = convergesScore + existing.balanceScore = balanceScore + existing.rangeCount = int(existing.store.Capacity.RangeCount) + existingCandidates = append(existingCandidates, existing) } for _, cand := range comparable.candidates { @@ -898,7 +880,7 @@ func storeHasReplica(storeID roachpb.StoreID, existing []roachpb.ReplicaDescript } func sameLocalityAndAttrs(s1, s2 roachpb.StoreDescriptor) bool { - if !s1.Node.Locality.Equals(s2.Node.Locality) { + if !s1.Locality().Equals(s2.Locality()) { return false } if !s1.Node.Attrs.Equals(s2.Node.Attrs) { @@ -1058,14 +1040,14 @@ func constraintsCheck( // given range is. A higher score means the range is more diverse. // All below diversity-scoring methods should in theory be implemented by // calling into this one, but they aren't to avoid allocations. -func rangeDiversityScore(existingNodeLocalities map[roachpb.NodeID]roachpb.Locality) float64 { +func rangeDiversityScore(existingStoreLocalities map[roachpb.StoreID]roachpb.Locality) float64 { var sumScore float64 var numSamples int - for n1, l1 := range existingNodeLocalities { - for n2, l2 := range existingNodeLocalities { + for s1, l1 := range existingStoreLocalities { + for s2, l2 := range existingStoreLocalities { // Only compare pairs of replicas where s2 > s1 to avoid computing the // diversity score between each pair of localities twice. - if n2 <= n1 { + if s2 <= s1 { continue } sumScore += l1.DiversityScore(l2) @@ -1082,7 +1064,7 @@ func rangeDiversityScore(existingNodeLocalities map[roachpb.NodeID]roachpb.Local // desirable it would be to add a replica to store. A higher score means the // store is a better fit. func diversityAllocateScore( - store roachpb.StoreDescriptor, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, + store roachpb.StoreDescriptor, existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, ) float64 { var sumScore float64 var numSamples int @@ -1090,8 +1072,8 @@ func diversityAllocateScore( // how well the new store would fit, because for any store that we might // consider adding the pairwise average diversity of the existing replicas // is the same. - for _, locality := range existingNodeLocalities { - newScore := store.Node.Locality.DiversityScore(locality) + for _, locality := range existingStoreLocalities { + newScore := store.Locality().DiversityScore(locality) sumScore += newScore numSamples++ } @@ -1106,15 +1088,15 @@ func diversityAllocateScore( // it would be to remove a node's replica of a range. A higher score indicates // that the node is a better fit (i.e. keeping it around is good for diversity). func diversityRemovalScore( - nodeID roachpb.NodeID, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, + storeID roachpb.StoreID, existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, ) float64 { var sumScore float64 var numSamples int - locality := existingNodeLocalities[nodeID] + locality := existingStoreLocalities[storeID] // We don't need to calculate the overall diversityScore for the range, because the original overall diversityScore // of this range is always the same. - for otherNodeID, otherLocality := range existingNodeLocalities { - if otherNodeID == nodeID { + for otherStoreID, otherLocality := range existingStoreLocalities { + if otherStoreID == storeID { continue } newScore := locality.DiversityScore(otherLocality) @@ -1134,16 +1116,16 @@ func diversityRemovalScore( // higher score indicates that the provided store is a better fit for the // range. func diversityRebalanceScore( - store roachpb.StoreDescriptor, existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, + store roachpb.StoreDescriptor, existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, ) float64 { - if len(existingNodeLocalities) == 0 { + if len(existingStoreLocalities) == 0 { return roachpb.MaxDiversityScore } var maxScore float64 // For every existing node, calculate what the diversity score would be if we // remove that node's replica to replace it with one on the provided store. - for removedNodeID := range existingNodeLocalities { - score := diversityRebalanceFromScore(store, removedNodeID, existingNodeLocalities) + for removedStoreID := range existingStoreLocalities { + score := diversityRebalanceFromScore(store, removedStoreID, existingStoreLocalities) if score > maxScore { maxScore = score } @@ -1159,24 +1141,24 @@ func diversityRebalanceScore( // range. func diversityRebalanceFromScore( store roachpb.StoreDescriptor, - fromNodeID roachpb.NodeID, - existingNodeLocalities map[roachpb.NodeID]roachpb.Locality, + fromStoreID roachpb.StoreID, + existingNodeLocalities map[roachpb.StoreID]roachpb.Locality, ) float64 { // Compute the pairwise diversity score of all replicas that will exist // after adding store and removing fromNodeID. var sumScore float64 var numSamples int - for nodeID, locality := range existingNodeLocalities { - if nodeID == fromNodeID { + for storeID, locality := range existingNodeLocalities { + if storeID == fromStoreID { continue } - newScore := store.Node.Locality.DiversityScore(locality) + newScore := store.Locality().DiversityScore(locality) sumScore += newScore numSamples++ - for otherNodeID, otherLocality := range existingNodeLocalities { + for otherStoreID, otherLocality := range existingNodeLocalities { // Only compare pairs of replicas where otherNodeID > nodeID to avoid // computing the diversity score between each pair of localities twice. - if otherNodeID <= nodeID || otherNodeID == fromNodeID { + if otherStoreID <= storeID || otherStoreID == fromStoreID { continue } newScore := locality.DiversityScore(otherLocality) diff --git a/pkg/kv/kvserver/allocator_scorer_test.go b/pkg/kv/kvserver/allocator_scorer_test.go index a59c022f1b83..16ecb942139c 100644 --- a/pkg/kv/kvserver/allocator_scorer_test.go +++ b/pkg/kv/kvserver/allocator_scorer_test.go @@ -482,7 +482,7 @@ var ( Attrs: []string{"a"}, }, Node: roachpb.NodeDescriptor{ - NodeID: roachpb.NodeID(testStoreUSa15Dupe), + NodeID: roachpb.NodeID(testStoreUSa15), Locality: roachpb.Locality{ Tiers: testStoreTierSetup("us", "a", "1", "5"), }, @@ -1082,10 +1082,10 @@ func TestShouldRebalanceDiversity(t *testing.T) { }, } } - localityForNodeID := func(sl StoreList, id roachpb.NodeID) roachpb.Locality { + localityForStoreID := func(sl StoreList, id roachpb.StoreID) roachpb.Locality { for _, store := range sl.stores { - if store.Node.NodeID == id { - return store.Node.Locality + if store.StoreID == id { + return store.Locality() } } t.Fatalf("no locality for n%d in StoreList %+v", id, sl) @@ -1191,14 +1191,15 @@ func TestShouldRebalanceDiversity(t *testing.T) { } filteredSL := tc.sl filteredSL.stores = append([]roachpb.StoreDescriptor(nil), filteredSL.stores...) - existingNodeLocalities := make(map[roachpb.NodeID]roachpb.Locality) + existingStoreLocalities := make(map[roachpb.StoreID]roachpb.Locality) var replicas []roachpb.ReplicaDescriptor for _, nodeID := range tc.existingNodeIDs { + storeID := roachpb.StoreID(nodeID) replicas = append(replicas, roachpb.ReplicaDescriptor{ NodeID: nodeID, - StoreID: roachpb.StoreID(nodeID), + StoreID: storeID, }) - existingNodeLocalities[nodeID] = localityForNodeID(tc.sl, nodeID) + existingStoreLocalities[storeID] = localityForStoreID(tc.sl, storeID) // For the sake of testing, remove all other existing stores from the // store list to only test whether we want to remove the replica on tc.s. if nodeID != tc.s.Node.NodeID { @@ -1211,11 +1212,7 @@ func TestShouldRebalanceDiversity(t *testing.T) { filteredSL, constraint.AnalyzedConstraints{}, replicas, - existingNodeLocalities, - func(nodeID roachpb.NodeID) string { - locality := localityForNodeID(tc.sl, nodeID) - return locality.String() - }, + existingStoreLocalities, options) actual := len(targets) > 0 if actual != tc.expected { @@ -1254,17 +1251,17 @@ func TestAllocateDiversityScore(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - existingNodeLocalities := make(map[roachpb.NodeID]roachpb.Locality) + existingStoreLocalities := make(map[roachpb.StoreID]roachpb.Locality) for _, s := range tc.stores { - existingNodeLocalities[testStores[s].Node.NodeID] = testStores[s].Node.Locality + existingStoreLocalities[testStores[s].StoreID] = testStores[s].Locality() } var scores storeScores for _, s := range testStores { - if _, ok := existingNodeLocalities[s.Node.NodeID]; ok { + if _, ok := existingStoreLocalities[s.StoreID]; ok { continue } var score storeScore - actualScore := diversityAllocateScore(s, existingNodeLocalities) + actualScore := diversityAllocateScore(s, existingStoreLocalities) score.storeID = s.StoreID score.score = actualScore scores = append(scores, score) @@ -1329,17 +1326,17 @@ func TestRebalanceToDiversityScore(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - existingNodeLocalities := make(map[roachpb.NodeID]roachpb.Locality) + existingStoreLocalities := make(map[roachpb.StoreID]roachpb.Locality) for _, s := range tc.stores { - existingNodeLocalities[testStores[s].Node.NodeID] = testStores[s].Node.Locality + existingStoreLocalities[testStores[s].StoreID] = testStores[s].Locality() } var scores storeScores for _, s := range testStores { - if _, ok := existingNodeLocalities[s.Node.NodeID]; ok { + if _, ok := existingStoreLocalities[s.StoreID]; ok { continue } var score storeScore - actualScore := diversityRebalanceScore(s, existingNodeLocalities) + actualScore := diversityRebalanceScore(s, existingStoreLocalities) score.storeID = s.StoreID score.score = actualScore scores = append(scores, score) @@ -1400,15 +1397,15 @@ func TestRemovalDiversityScore(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - existingNodeLocalities := make(map[roachpb.NodeID]roachpb.Locality) + existingStoreLocalities := make(map[roachpb.StoreID]roachpb.Locality) for _, s := range tc.stores { - existingNodeLocalities[testStores[s].Node.NodeID] = testStores[s].Node.Locality + existingStoreLocalities[testStores[s].StoreID] = testStores[s].Locality() } var scores storeScores for _, storeID := range tc.stores { s := testStores[storeID] var score storeScore - actualScore := diversityRemovalScore(s.Node.NodeID, existingNodeLocalities) + actualScore := diversityRemovalScore(s.StoreID, existingStoreLocalities) score.storeID = s.StoreID score.score = actualScore scores = append(scores, score) @@ -1436,19 +1433,19 @@ func TestDiversityScoreEquivalence(t *testing.T) { {[]roachpb.StoreID{testStoreUSa15}, 1.0}, {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe}, 0.0}, {[]roachpb.StoreID{testStoreUSa15, testStoreUSa1}, 0.25}, - {[]roachpb.StoreID{testStoreUSa15, testStoreUSb}, 0.5}, + {[]roachpb.StoreID{testStoreUSa15, testStoreUSb}, 2.0 / 3.0}, {[]roachpb.StoreID{testStoreUSa15, testStoreEurope}, 1.0}, {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreUSa1}, 1.0 / 6.0}, - {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreUSb}, 1.0 / 3.0}, + {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreUSb}, 4.0 / 9.0}, {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreEurope}, 2.0 / 3.0}, - {[]roachpb.StoreID{testStoreUSa15, testStoreUSa1, testStoreUSb}, 5.0 / 12.0}, + {[]roachpb.StoreID{testStoreUSa15, testStoreUSa1, testStoreUSb}, 19.0 / 36.0}, {[]roachpb.StoreID{testStoreUSa15, testStoreUSa1, testStoreEurope}, 3.0 / 4.0}, - {[]roachpb.StoreID{testStoreUSa1, testStoreUSb, testStoreEurope}, 5.0 / 6.0}, - {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreUSa1, testStoreUSb}, 1.0 / 3.0}, + {[]roachpb.StoreID{testStoreUSa1, testStoreUSb, testStoreEurope}, 8.0 / 9.0}, + {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreUSa1, testStoreUSb}, 5.0 / 12.0}, {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreUSa1, testStoreEurope}, 7.0 / 12.0}, - {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreUSb, testStoreEurope}, 2.0 / 3.0}, - {[]roachpb.StoreID{testStoreUSa15, testStoreUSa1, testStoreUSb, testStoreEurope}, 17.0 / 24.0}, - {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreUSa1, testStoreUSb, testStoreEurope}, 3.0 / 5.0}, + {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreUSb, testStoreEurope}, 26.0 / 36.0}, + {[]roachpb.StoreID{testStoreUSa15, testStoreUSa1, testStoreUSb, testStoreEurope}, 55.0 / 72.0}, + {[]roachpb.StoreID{testStoreUSa15, testStoreUSa15Dupe, testStoreUSa1, testStoreUSb, testStoreEurope}, 13.0 / 20.0}, } // Ensure that rangeDiversityScore and diversityRebalanceFromScore return @@ -1457,27 +1454,27 @@ func TestDiversityScoreEquivalence(t *testing.T) { // diversityAllocateScore and diversityRemovalScore as of their initial // creation or else we would test them here as well. for _, tc := range testCases { - existingLocalities := make(map[roachpb.NodeID]roachpb.Locality) + existingLocalities := make(map[roachpb.StoreID]roachpb.Locality) for _, storeID := range tc.stores { s := testStores[storeID] - existingLocalities[s.Node.NodeID] = s.Node.Locality + existingLocalities[s.StoreID] = s.Locality() } rangeScore := rangeDiversityScore(existingLocalities) - if a, e := rangeScore, tc.expected; a != e { + if a, e := rangeScore, tc.expected; !scoresAlmostEqual(a, e) { t.Errorf("rangeDiversityScore(%v) got %f, want %f", existingLocalities, a, e) } for _, storeID := range tc.stores { s := testStores[storeID] - fromNodeID := s.Node.NodeID - s.Node.NodeID = 99 - rebalanceScore := diversityRebalanceFromScore(s, fromNodeID, existingLocalities) - if a, e := rebalanceScore, tc.expected; a != e { + fromStoreID := s.StoreID + s.StoreID = 99 + rebalanceScore := diversityRebalanceFromScore(s, fromStoreID, existingLocalities) + if a, e := rebalanceScore, tc.expected; !scoresAlmostEqual(a, e) { t.Errorf("diversityRebalanceFromScore(%v, %d, %v) got %f, want %f", - s, fromNodeID, existingLocalities, a, e) + s, fromStoreID, existingLocalities, a, e) } - if a, e := rebalanceScore, rangeScore; a != e { + if a, e := rebalanceScore, rangeScore; !scoresAlmostEqual(a, e) { t.Errorf("diversityRebalanceFromScore(%v, %d, %v)=%f not equal to rangeDiversityScore(%v)=%f", - s, fromNodeID, existingLocalities, a, existingLocalities, e) + s, fromStoreID, existingLocalities, a, existingLocalities, e) } } } diff --git a/pkg/kv/kvserver/allocator_test.go b/pkg/kv/kvserver/allocator_test.go index 7b5b40aa316a..c793adf8de63 100644 --- a/pkg/kv/kvserver/allocator_test.go +++ b/pkg/kv/kvserver/allocator_test.go @@ -550,28 +550,32 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { gossiputil.NewStoreGossiper(g).GossipStores(stores, t) testCases := []struct { - existing []roachpb.ReplicaDescriptor - expectTarget bool + existing []roachpb.ReplicaDescriptor + expectTargetAllocate bool + expectTargetRebalance bool }{ { existing: []roachpb.ReplicaDescriptor{ {NodeID: 1, StoreID: 1}, }, - expectTarget: true, + expectTargetAllocate: true, + expectTargetRebalance: true, }, { existing: []roachpb.ReplicaDescriptor{ {NodeID: 1, StoreID: 2}, {NodeID: 2, StoreID: 3}, }, - expectTarget: true, + expectTargetAllocate: true, + expectTargetRebalance: true, }, { existing: []roachpb.ReplicaDescriptor{ {NodeID: 1, StoreID: 2}, {NodeID: 3, StoreID: 6}, }, - expectTarget: true, + expectTargetAllocate: true, + expectTargetRebalance: true, }, { existing: []roachpb.ReplicaDescriptor{ @@ -579,7 +583,8 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { {NodeID: 2, StoreID: 3}, {NodeID: 3, StoreID: 5}, }, - expectTarget: false, + expectTargetAllocate: false, + expectTargetRebalance: true, }, { existing: []roachpb.ReplicaDescriptor{ @@ -587,7 +592,8 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { {NodeID: 2, StoreID: 4}, {NodeID: 3, StoreID: 6}, }, - expectTarget: false, + expectTargetAllocate: false, + expectTargetRebalance: false, }, } @@ -598,9 +604,9 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { zonepb.EmptyCompleteZoneConfig(), tc.existing, ) - if e, a := tc.expectTarget, result != nil; e != a { + if e, a := tc.expectTargetAllocate, result != nil; e != a { t.Errorf("AllocateTarget(%v) got target %v, err %v; expectTarget=%v", - tc.existing, result, err, tc.expectTarget) + tc.existing, result, err, tc.expectTargetAllocate) } } @@ -614,14 +620,118 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { rangeUsageInfo, storeFilterThrottled, ) - if e, a := tc.expectTarget, ok; e != a { + if e, a := tc.expectTargetRebalance, ok; e != a { t.Errorf("RebalanceTarget(%v) got target %v, details %v; expectTarget=%v", - tc.existing, target, details, tc.expectTarget) + tc.existing, target, details, tc.expectTargetRebalance) } } } } +func TestAllocatorMultipleStoresPerNodeLopsided(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + store1 := roachpb.StoreDescriptor{ + StoreID: 1, + Node: roachpb.NodeDescriptor{NodeID: 1}, + Capacity: roachpb.StoreCapacity{Capacity: 200, Available: 100, RangeCount: 40}, + } + store2 := roachpb.StoreDescriptor{ + StoreID: 2, + Node: roachpb.NodeDescriptor{NodeID: 1}, + Capacity: roachpb.StoreCapacity{Capacity: 200, Available: 100, RangeCount: 0}, + } + + // We start out with 40 ranges on 3 nodes and 3 stores, we then add a new store + // on Node 1 and try to rebalance all the ranges. What we want to see happen + // is an equilibrium where 20 ranges move from Store 1 to Store 2. + stores := []*roachpb.StoreDescriptor{ + &store1, + &store2, + { + StoreID: 3, + Node: roachpb.NodeDescriptor{NodeID: 2}, + Capacity: roachpb.StoreCapacity{Capacity: 200, Available: 100, RangeCount: 40}, + }, + { + StoreID: 4, + Node: roachpb.NodeDescriptor{NodeID: 3}, + Capacity: roachpb.StoreCapacity{Capacity: 200, Available: 100, RangeCount: 40}, + }, + } + + ranges := make([]roachpb.RangeDescriptor, 40) + for i := 0; i < 40; i++ { + ranges[i] = roachpb.RangeDescriptor{ + InternalReplicas: []roachpb.ReplicaDescriptor{ + {NodeID: 1, StoreID: 1}, + {NodeID: 2, StoreID: 3}, + {NodeID: 3, StoreID: 4}, + }, + } + } + + stopper, g, _, a, _ := createTestAllocator(10, false /* deterministic */) + defer stopper.Stop(context.Background()) + storeGossiper := gossiputil.NewStoreGossiper(g) + storeGossiper.GossipStores(stores, t) + + // We run through all the ranges once to get the cluster to balance. + // After that we should not be seeing replicas move. + var rangeUsageInfo RangeUsageInfo + for i := 1; i < 40; i++ { + add, remove, _, ok := a.RebalanceTarget( + context.Background(), + zonepb.EmptyCompleteZoneConfig(), + nil, /* raftStatus */ + ranges[i].InternalReplicas, + rangeUsageInfo, + storeFilterThrottled, + ) + if ok { + // Update the descriptor. + newReplicas := make([]roachpb.ReplicaDescriptor, 0, len(ranges[i].InternalReplicas)) + for _, repl := range ranges[i].InternalReplicas { + if remove.StoreID != repl.StoreID { + newReplicas = append(newReplicas, repl) + } + } + newReplicas = append(newReplicas, roachpb.ReplicaDescriptor{ + StoreID: add.StoreID, + NodeID: add.NodeID, + }) + ranges[i].InternalReplicas = newReplicas + + for _, store := range stores { + if store.StoreID == add.StoreID { + store.Capacity.RangeCount = store.Capacity.RangeCount + 1 + } else if store.StoreID == remove.StoreID { + store.Capacity.RangeCount = store.Capacity.RangeCount - 1 + } + } + storeGossiper.GossipStores(stores, t) + } + } + + // Verify that the stores are reasonably balanced. + require.True(t, math.Abs(float64( + store1.Capacity.RangeCount-store2.Capacity.RangeCount)) <= minRangeRebalanceThreshold*2) + // We dont expect any range wanting to move since the system should have + // reached a stable state at this point. + for i := 1; i < 40; i++ { + _, _, _, ok := a.RebalanceTarget( + context.Background(), + zonepb.EmptyCompleteZoneConfig(), + nil, /* raftStatus */ + ranges[i].InternalReplicas, + rangeUsageInfo, + storeFilterThrottled, + ) + require.False(t, ok) + } +} + // TestAllocatorRebalance verifies that rebalance targets are chosen // randomly from amongst stores under the maxFractionUsedThreshold. func TestAllocatorRebalance(t *testing.T) { @@ -2559,7 +2669,7 @@ func TestAllocateCandidatesNumReplicasConstraints(t *testing.T) { sl, analyzed, existingRepls, - a.storePool.getLocalities(existingRepls), + a.storePool.getLocalitiesByStore(existingRepls), a.scorerOptions(), ) best := candidates.best() @@ -2782,7 +2892,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { candidates := removeCandidates( sl, analyzed, - a.storePool.getLocalities(existingRepls), + a.storePool.getLocalitiesByStore(existingRepls), a.scorerOptions(), ) if !expectedStoreIDsMatch(tc.expected, candidates.worst()) { @@ -3579,8 +3689,7 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { sl, analyzed, existingRepls, - a.storePool.getLocalities(existingRepls), - a.storePool.getNodeLocalityString, + a.storePool.getLocalitiesByStore(existingRepls), a.scorerOptions(), ) match := true @@ -5477,9 +5586,9 @@ func TestAllocatorRebalanceAway(t *testing.T) { } existingReplicas := []roachpb.ReplicaDescriptor{ - {StoreID: stores[0].StoreID}, - {StoreID: stores[1].StoreID}, - {StoreID: stores[2].StoreID}, + {StoreID: stores[0].StoreID, NodeID: stores[0].Node.NodeID}, + {StoreID: stores[1].StoreID, NodeID: stores[1].Node.NodeID}, + {StoreID: stores[2].StoreID, NodeID: stores[2].Node.NodeID}, } testCases := []struct { constraint zonepb.Constraint diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 22d13a2ccfe1..2e2780a86162 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -1125,50 +1125,95 @@ func validateReplicationChanges( ) error { // First make sure that the changes don't self-overlap (i.e. we're not adding // a replica twice, or removing and immediately re-adding it). - byNodeID := make(map[roachpb.NodeID]roachpb.ReplicationChange, len(chgs)) + byNodeAndStoreID := make(map[roachpb.NodeID]map[roachpb.StoreID]roachpb.ReplicationChange, len(chgs)) for _, chg := range chgs { - if _, ok := byNodeID[chg.Target.NodeID]; ok { - return fmt.Errorf("changes %+v refer to n%d twice", chgs, chg.Target.NodeID) + byStoreID, ok := byNodeAndStoreID[chg.Target.NodeID] + if !ok { + byStoreID = make(map[roachpb.StoreID]roachpb.ReplicationChange) + byNodeAndStoreID[chg.Target.NodeID] = byStoreID + } else { + // The only operation that is allowed within a node is an Add/Remove. + for _, prevChg := range byStoreID { + if prevChg.ChangeType == chg.ChangeType { + return fmt.Errorf("changes %+v refer to n%d twice for change %v", + chgs, chg.Target.NodeID, chg.ChangeType) + } + if prevChg.ChangeType != roachpb.ADD_REPLICA { + return fmt.Errorf("can only add-remove a replica within a node, but got %+v", chgs) + } + } } - byNodeID[chg.Target.NodeID] = chg + if _, ok := byStoreID[chg.Target.StoreID]; ok { + return fmt.Errorf("changes %+v refer to n%d and s%d twice", chgs, + chg.Target.NodeID, chg.Target.StoreID) + } + byStoreID[chg.Target.StoreID] = chg } // Then, check that we're not adding a second replica on nodes that already - // have one, or "re-add" an existing replica. We delete from byNodeID so that - // after this loop, it contains only StoreIDs that we haven't seen in desc. + // have one, or "re-add" an existing replica. We delete from byNodeAndStoreID so that + // after this loop, it contains only Nodes that we haven't seen in desc. for _, rDesc := range desc.Replicas().All() { - chg, ok := byNodeID[rDesc.NodeID] - delete(byNodeID, rDesc.NodeID) - if !ok || chg.ChangeType != roachpb.ADD_REPLICA { + byStoreID, ok := byNodeAndStoreID[rDesc.NodeID] + if !ok { continue } - // We're adding a replica that's already there. This isn't allowed, even - // when the newly added one would be on a different store. - if rDesc.StoreID != chg.Target.StoreID { - return errors.Errorf("unable to add replica %v; node already has a replica in %s", chg.Target.StoreID, desc) + delete(byNodeAndStoreID, rDesc.NodeID) + if len(byStoreID) == 2 { + chg, k := byStoreID[rDesc.StoreID] + // We should be removing the replica from the existing store during a + // rebalance within the node. + if !k || chg.ChangeType != roachpb.REMOVE_REPLICA { + return errors.Errorf( + "Expected replica to be removed from %v during a lateral rebalance %v within the node.", rDesc, chgs) + } + continue } + chg, ok := byStoreID[rDesc.StoreID] + // There are two valid conditions here: + // (1) removal of an existing store. + // (2) add on the node, when we only have one replica. + // See https://github.com/cockroachdb/cockroach/issues/40333. + if ok { + if chg.ChangeType == roachpb.REMOVE_REPLICA { + continue + } + // Looks like we found a replica with the same store and node id. If the + // replica is already a learner, then either some previous leaseholder was + // trying to add it with the learner+snapshot+voter cycle and got + // interrupted or else we hit a race between the replicate queue and + // AdminChangeReplicas. + if rDesc.GetType() == roachpb.LEARNER { + return errors.Errorf( + "unable to add replica %v which is already present as a learner in %s", chg.Target, desc) + } - // Looks like we found a replica with the same store and node id. If the - // replica is already a learner, then either some previous leaseholder was - // trying to add it with the learner+snapshot+voter cycle and got - // interrupted or else we hit a race between the replicate queue and - // AdminChangeReplicas. - if rDesc.GetType() == roachpb.LEARNER { - return errors.Errorf( - "unable to add replica %v which is already present as a learner in %s", chg.Target, desc) + // Otherwise, we already had a full voter replica. Can't add another to + // this store. + return errors.Errorf("unable to add replica %v which is already present in %s", chg.Target, desc) } - // Otherwise, we already had a full voter replica. Can't add another to - // this store. - return errors.Errorf("unable to add replica %v which is already present in %s", chg.Target, desc) + for _, c := range byStoreID { + // We're adding a replica that's already there. This isn't allowed, even + // when the newly added one would be on a different store. + if c.ChangeType == roachpb.ADD_REPLICA { + if len(desc.Replicas().All()) > 1 { + return errors.Errorf("unable to add replica %v; node already has a replica in %s", c.Target.StoreID, desc) + } + } else { + return errors.Errorf("removing %v which is not in %s", c.Target, desc) + } + } } // Any removals left in the map now refer to nonexisting replicas, and we refuse them. - for _, chg := range byNodeID { - if chg.ChangeType != roachpb.REMOVE_REPLICA { - continue + for _, byStoreID := range byNodeAndStoreID { + for _, chg := range byStoreID { + if chg.ChangeType != roachpb.REMOVE_REPLICA { + continue + } + return errors.Errorf("removing %v which is not in %s", chg.Target, desc) } - return errors.Errorf("removing %v which is not in %s", chg.Target, desc) } return nil } diff --git a/pkg/kv/kvserver/replica_command_test.go b/pkg/kv/kvserver/replica_command_test.go index 75bb0c030787..48328d6ea408 100644 --- a/pkg/kv/kvserver/replica_command_test.go +++ b/pkg/kv/kvserver/replica_command_test.go @@ -91,3 +91,142 @@ func TestRangeDescriptorUpdateProtoChangedAcrossVersions(t *testing.T) { t.Fatal(err) } } + +func TestValidateReplicationChanges(t *testing.T) { + defer leaktest.AfterTest(t)() + + learnerType := roachpb.LEARNER + desc := &roachpb.RangeDescriptor{ + InternalReplicas: []roachpb.ReplicaDescriptor{ + {NodeID: 1, StoreID: 1}, + {NodeID: 3, StoreID: 3}, + {NodeID: 4, StoreID: 4, Type: &learnerType}, + }, + } + + // Test Case 1: Add a new replica to another node. + err := validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, + }) + require.NoError(t, err) + + // Test Case 2: Remove a replica from an existing node. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + }) + require.NoError(t, err) + + // Test Case 3: Remove a replica from wrong node. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, + }) + require.Regexp(t, "removing n2,s2 which is not in", err) + + // Test Case 4: Remove a replica from wrong store. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, + }) + require.Regexp(t, "removing n1,s2 which is not in", err) + + // Test Case 5: Re-balance a replica within a store. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + }) + require.NoError(t, err) + + // Test Case 6: Re-balance a replica within a store, but attempt remove from + // the wrong one. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 3}}, + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, + }) + require.Regexp(t, "Expected replica to be removed from", err) + + // Test Case 7: Add replica to same node and store. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + }) + require.Regexp(t, "unable to add replica n1,s1 which is already present", err) + + // Test Case 8: Add replica to same node and different store. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, + }) + require.Regexp(t, "unable to add replica 2", err) + + // Test Case 9: Try to rebalance a replica on the same node, but also add an extra. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 3}}, + }) + require.Regexp(t, "can only add-remove a replica within a node", err) + + // Test Case 10: Try to add twice to the same node. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 4, StoreID: 4}}, + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 4, StoreID: 5}}, + }) + require.Regexp(t, "refer to n4 twice for change ADD_REPLICA", err) + + // Test Case 11: Try to remove twice to the same node. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, + }) + require.Regexp(t, "refer to n1 twice for change REMOVE_REPLICA", err) + + // Test Case 12: Try to add where there is already a learner. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 4, StoreID: 5}}, + }) + require.Error(t, err) + + // Test Case 13: Add/Remove multiple replicas. + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 5, StoreID: 5}}, + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 6, StoreID: 6}}, + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 3, StoreID: 3}}, + }) + require.NoError(t, err) + + // Test Case 14: We are rebalancing within a node and do a remove. + descRebalancing := &roachpb.RangeDescriptor{ + InternalReplicas: []roachpb.ReplicaDescriptor{ + {NodeID: 1, StoreID: 1}, + {NodeID: 2, StoreID: 2}, + {NodeID: 1, StoreID: 2, Type: &learnerType}, + }, + } + err = validateReplicationChanges(descRebalancing, roachpb.ReplicationChanges{ + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + }) + require.NoError(t, err) + + // Test Case 15: Do an add while rebalancing within a node + err = validateReplicationChanges(descRebalancing, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 3, StoreID: 3}}, + }) + require.NoError(t, err) + + // Test Case 16: Remove/Add within a node is not allowed, since we expect Add/Remove + err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ + {ChangeType: roachpb.REMOVE_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, + }) + require.Regexp(t, "can only add-remove a replica within a node, but got ", err) + + // Test Case 17: We are rebalancing within a node and have only one replica + descSingle := &roachpb.RangeDescriptor{ + InternalReplicas: []roachpb.ReplicaDescriptor{ + {NodeID: 1, StoreID: 1}, + }, + } + err = validateReplicationChanges(descSingle, roachpb.ReplicationChanges{ + {ChangeType: roachpb.ADD_REPLICA, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, + }) + require.NoError(t, err) +} diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index d4582205e695..bc8b433b0fc2 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -6432,6 +6432,12 @@ func TestChangeReplicasDuplicateError(t *testing.T) { defer stopper.Stop(context.Background()) tc.Start(t, stopper) + // We now allow adding a replica to the same node, to support rebalances + // within the same node when replication is 1x, so add another replica to the + // range descriptor to avoid this case. + if _, err := tc.addBogusReplicaToRangeDesc(context.Background()); err != nil { + t.Fatalf("Unexpected error %v", err) + } chgs := roachpb.MakeReplicationChanges(roachpb.ADD_REPLICA, roachpb.ReplicationTarget{ NodeID: tc.store.Ident.NodeID, StoreID: 9999, diff --git a/pkg/kv/kvserver/store_pool.go b/pkg/kv/kvserver/store_pool.go index ccf3e133edc8..4a3b25b79a7e 100644 --- a/pkg/kv/kvserver/store_pool.go +++ b/pkg/kv/kvserver/store_pool.go @@ -757,10 +757,34 @@ func (sp *StorePool) throttle(reason throttleReason, why string, storeID roachpb } } -// getLocalities returns the localities for the provided replicas. +// getLocalitiesByStore returns the localities for the provided replicas. In +// this case we consider the node part of the failure domain and add it to +// the locality data. +func (sp *StorePool) getLocalitiesByStore( + replicas []roachpb.ReplicaDescriptor, +) map[roachpb.StoreID]roachpb.Locality { + sp.localitiesMu.RLock() + defer sp.localitiesMu.RUnlock() + localities := make(map[roachpb.StoreID]roachpb.Locality) + for _, replica := range replicas { + nodeTier := roachpb.Tier{Key: "node", Value: replica.NodeID.String()} + if locality, ok := sp.localitiesMu.nodeLocalities[replica.NodeID]; ok { + localities[replica.StoreID] = locality.locality.AddTier(nodeTier) + } else { + localities[replica.StoreID] = roachpb.Locality{ + Tiers: []roachpb.Tier{nodeTier}, + } + } + } + return localities +} + +// getLocalitiesByNode returns the localities for the provided replicas. In this +// case we only consider the locality by node, where the node itself is not +// part of the failure domain. // TODO(bram): consider storing a full list of all node to node diversity // scores for faster lookups. -func (sp *StorePool) getLocalities( +func (sp *StorePool) getLocalitiesByNode( replicas []roachpb.ReplicaDescriptor, ) map[roachpb.NodeID]roachpb.Locality { sp.localitiesMu.RLock() diff --git a/pkg/kv/kvserver/store_pool_test.go b/pkg/kv/kvserver/store_pool_test.go index adaf281e18c0..ca0ac4cf6258 100644 --- a/pkg/kv/kvserver/store_pool_test.go +++ b/pkg/kv/kvserver/store_pool_test.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/kr/pretty" + "github.com/stretchr/testify/require" ) var uniqueStore = []*roachpb.StoreDescriptor{ @@ -837,7 +838,7 @@ func TestGetLocalities(t *testing.T) { createDescWithLocality := func(tierCount int) roachpb.NodeDescriptor { return roachpb.NodeDescriptor{ NodeID: roachpb.NodeID(tierCount), - Locality: createLocality(tierCount), + Locality: createLocality(tierCount - 1), } } @@ -864,22 +865,30 @@ func TestGetLocalities(t *testing.T) { var existingReplicas []roachpb.ReplicaDescriptor for _, store := range stores { - existingReplicas = append(existingReplicas, roachpb.ReplicaDescriptor{NodeID: store.Node.NodeID}) + existingReplicas = append(existingReplicas, + roachpb.ReplicaDescriptor{ + NodeID: store.Node.NodeID, + StoreID: store.StoreID, + }, + ) } - localities := sp.getLocalities(existingReplicas) + localitiesByStore := sp.getLocalitiesByStore(existingReplicas) + localitiesByNode := sp.getLocalitiesByNode(existingReplicas) for _, store := range stores { + storeID := store.StoreID nodeID := store.Node.NodeID - locality, ok := localities[nodeID] + localityByStore, ok := localitiesByStore[storeID] if !ok { - t.Fatalf("could not find locality for node %d", nodeID) - } - if e, a := int(nodeID), len(locality.Tiers); e != a { - t.Fatalf("for node %d, expected %d tiers, only got %d", nodeID, e, a) - } - if e, a := createLocality(int(nodeID)).String(), sp.getNodeLocalityString(nodeID); e != a { - t.Fatalf("for getNodeLocalityString(%d), expected %q, got %q", nodeID, e, a) + t.Fatalf("could not find locality for store %d", storeID) } + localityByNode, ok := localitiesByNode[nodeID] + require.Truef(t, ok, "could not find locality for node %d", nodeID) + require.Equal(t, int(nodeID), len(localityByStore.Tiers)) + require.Equal(t, localityByStore.Tiers[len(localityByStore.Tiers)-1], + roachpb.Tier{Key: "node", Value: nodeID.String()}) + require.Equal(t, int(nodeID)-1, len(localityByNode.Tiers)) + require.Equal(t, createLocality(int(nodeID)-1).String(), sp.getNodeLocalityString(nodeID)) } } diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index 0fc25b15b30b..1219d8c1024b 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -509,7 +509,7 @@ func (sr *StoreRebalancer) chooseReplicaToRebalance( // Check the range's existing diversity score, since we want to ensure we // don't hurt locality diversity just to improve QPS. curDiversity := rangeDiversityScore( - sr.rq.allocator.storePool.getLocalities(currentReplicas)) + sr.rq.allocator.storePool.getLocalitiesByStore(currentReplicas)) // Check the existing replicas, keeping around those that aren't overloaded. for i := range currentReplicas { @@ -579,7 +579,7 @@ func (sr *StoreRebalancer) chooseReplicaToRebalance( desc.RangeID, len(targets), desiredReplicas) continue } - newDiversity := rangeDiversityScore(sr.rq.allocator.storePool.getLocalities(targetReplicas)) + newDiversity := rangeDiversityScore(sr.rq.allocator.storePool.getLocalitiesByStore(targetReplicas)) if newDiversity < curDiversity { log.VEventf(ctx, 3, "new diversity %.2f for r%d worse than current diversity %.2f; not rebalancing", diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index 638de8524814..167d0ad9e2e0 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -659,3 +659,21 @@ var DefaultLocationInformation = []struct { Longitude: "3.81886", }, } + +// Locality returns the locality of the Store, which is the Locality of the node +// plus an extra tier for the node itself. +func (s StoreDescriptor) Locality() Locality { + return s.Node.Locality.AddTier( + Tier{Key: "node", Value: s.Node.NodeID.String()}) +} + +// AddTier creates a new Locality with a Tier at the end. +func (l Locality) AddTier(tier Tier) Locality { + if len(l.Tiers) > 0 { + tiers := make([]Tier, len(l.Tiers), len(l.Tiers)+1) + copy(tiers, l.Tiers) + tiers = append(tiers, tier) + return Locality{Tiers: tiers} + } + return Locality{Tiers: []Tier{tier}} +} diff --git a/pkg/roachpb/metadata_test.go b/pkg/roachpb/metadata_test.go index 76f12b182f95..dc9300fac542 100644 --- a/pkg/roachpb/metadata_test.go +++ b/pkg/roachpb/metadata_test.go @@ -268,3 +268,15 @@ func TestDiversityScore(t *testing.T) { }) } } + +func TestAddTier(t *testing.T) { + l1 := Locality{} + l2 := Locality{ + Tiers: []Tier{{Key: "foo", Value: "bar"}}, + } + l3 := Locality{ + Tiers: []Tier{{Key: "foo", Value: "bar"}, {Key: "bar", Value: "foo"}}, + } + require.Equal(t, l2, l1.AddTier(Tier{Key: "foo", Value: "bar"})) + require.Equal(t, l3, l2.AddTier(Tier{Key: "bar", Value: "foo"})) +} diff --git a/pkg/server/status_test.go b/pkg/server/status_test.go index d52ab46d5f0b..fb5b1b4a3b61 100644 --- a/pkg/server/status_test.go +++ b/pkg/server/status_test.go @@ -329,6 +329,13 @@ func startServer(t *testing.T) *TestServer { base.DefaultTestStoreSpec, base.DefaultTestStoreSpec, }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + // Now that we allow same node rebalances, disable it in these tests, + // as they dont expect replicas to move. + DisableReplicaRebalancing: true, + }, + }, }) ts := tsI.(*TestServer) From d73710d1cbfba7b1326aa2880782d1d17e2b3e69 Mon Sep 17 00:00:00 2001 From: "James H. Linder" Date: Wed, 10 Jun 2020 10:50:13 -0400 Subject: [PATCH 9/9] Script for the PublishRelease TC build configuration Before: the script wasn't implemented. Now: Part of the new release process, this script - tags the selected SHA to the provided name - compiles the binaries and archive and uploads them to S3 as the versioned name - uploads the docker image to docker.io/cockroachdb/cockroach - pushes the tag to github.com/cockroachdb/cockroach - push all the artificats to their respective `latest` locations as appropriate Release note: None --- build/release/teamcity-publish-release.sh | 151 +++++++++++++++++++++- build/release/teamcity-support.sh | 2 +- 2 files changed, 151 insertions(+), 2 deletions(-) diff --git a/build/release/teamcity-publish-release.sh b/build/release/teamcity-publish-release.sh index e1782d4ca0f8..d2f1c94fa45f 100755 --- a/build/release/teamcity-publish-release.sh +++ b/build/release/teamcity-publish-release.sh @@ -1,4 +1,153 @@ #!/usr/bin/env bash -echo "Implement me!" +set -euxo pipefail +source "$(dirname "${0}")/teamcity-support.sh" + + +if [[ -n "${PUBLISH_LATEST}" && -n "$PRE_RELEASE" ]]; then + echo "Invalid parameter combination: PUBLISH_LATEST and PRE_RELEASE can't both be set." + exit 6 +fi + + +tc_start_block "Variable Setup" +export BUILDER_HIDE_GOPATH_SRC=1 + +# Matching the version name regex from within the cockroach code except +# for the `metadata` part at the end because Docker tags don't support +# `+` in the tag name. +# https://github.com/cockroachdb/cockroach/blob/4c6864b44b9044874488cfedee3a31e6b23a6790/pkg/util/version/version.go#L75 +build_name="$(echo "${NAME}" | grep -E -o '^v(0|[1-9][0-9]*)\.(0|[1-9][0-9]*)\.(0|[1-9][0-9]*)(-[-.0-9A-Za-z]+)?$')" +# ^major ^minor ^patch ^preRelease + +if [[ -z "$build_name" ]] ; then + echo "Invalid NAME \"${NAME}\". Must be of the format \"vMAJOR.MINOR.PATCH(-PRERELEASE)?\"." + exit 1 +fi + +release_branch=$(echo ${build_name} | grep -E -o '^v[0-9]+\.[0-9]+') + +if [[ -z "${DRY_RUN}" ]] ; then + bucket="${BUCKET:-binaries.cockroachdb.com}" + google_credentials="$GOOGLE_COCKROACH_CLOUD_IMAGES_CREDENTIALS" + dockerhub_repository="docker.io/cockroachdb/cockroach" + gcr_repository="us.gcr.io/cockroach-cloud-images/cockroach" + s3_download_hostname="${bucket}" + git_repo_for_tag="cockroachdb/cockroach" +else + bucket="${BUCKET:-cockroach-builds-test}" + google_credentials="$GOOGLE_COCKROACH_RELEASE_CREDENTIALS" + dockerhub_repository="docker.io/cockroachdb/cockroach-misc" + gcr_repository="us.gcr.io/cockroach-release/cockroach-test" + s3_download_hostname="${bucket}.s3.amazonaws.com" + git_repo_for_tag="cockroachlabs/release-staging" + if [[ -z "$(echo ${build_name} | grep -E -o '^v[0-9]+\.[0-9]+\.[0-9]+$')" ]] ; then + # Using `.` to match how we usually format the pre-release portion of the + # version string using '.' separators. + # ex: v20.2.0-rc.2.dryrun + build_name="${build_name}.dryrun" + else + # Using `-` to put dryrun in the pre-release portion of the version string. + # ex: v20.2.0-dryrun + build_name="${build_name}-dryrun" + fi +fi + +# Used for docker login for gcloud +gcr_hostname="us.gcr.io" + +tc_end_block "Variable Setup" + + +tc_start_block "Tag the release" +git tag "${build_name}" +tc_end_block "Tag the release" + + +tc_start_block "Compile publish-artifacts" +build/builder.sh go install ./pkg/cmd/publish-artifacts +tc_end_block "Compile publish-artifacts" + + +tc_start_block "Compile publish-provisional-artifacts" +build/builder.sh go install ./pkg/cmd/publish-provisional-artifacts +tc_end_block "Compile publish-provisional-artifacts" + + +tc_start_block "Make and publish release S3 artifacts" +build/builder.sh env \ + AWS_ACCESS_KEY_ID="$AWS_ACCESS_KEY_ID" \ + AWS_SECRET_ACCESS_KEY="$AWS_SECRET_ACCESS_KEY" \ + TC_BUILD_BRANCH="$build_name" \ + publish-artifacts -release -bucket "$bucket" +tc_end_block "Make and publish release S3 artifacts" + + +tc_start_block "Make and push docker images" +configure_docker_creds +docker_login_with_google +docker_login + +# TODO: update publish-artifacts with option to leave one or more cockroach binaries in the local filesystem +curl -f -s -S -o- "https://${s3_download_hostname}/cockroach-${build_name}.linux-amd64.tgz" | tar ixfz - --strip-components 1 +cp cockroach build/deploy + +docker build --no-cache --tag=${dockerhub_repository}:{"$build_name",latest,latest-"${release_branch}"} --tag=${gcr_repository}:${build_name} build/deploy + +docker push "${dockerhub_repository}:${build_name}" +docker push "${gcr_repository}:${build_name}" +tc_end_block "Make and push docker images" + + +tc_start_block "Push release tag to GitHub" +github_ssh_key="${GITHUB_COCKROACH_TEAMCITY_PRIVATE_SSH_KEY}" +configure_git_ssh_key +push_to_git "ssh://git@github.com/${git_repo_for_tag}.git" "$build_name" +tc_end_block "Push release tag to GitHub" + + +tc_start_block "Publish S3 binaries and archive as latest-RELEASE_BRANCH" +# example: v20.1-latest +if [[ -z "$PRE_RELEASE" ]]; then + #TODO: implement me! + echo "Pushing latest-RELEASE_BRANCH S3 binaries and archive is not implemented." +else + echo "Pushing latest-RELEASE_BRANCH S3 binaries and archive is not implemented." +fi +tc_end_block "Publish S3 binaries and archive as latest-RELEASE_BRANCH" + + +tc_start_block "Publish S3 binaries and archive as latest" +# Only push the "latest" for our most recent release branch. +# https://github.com/cockroachdb/cockroach/issues/41067 +if [[ -n "${PUBLISH_LATEST}" && -z "${PRE_RELEASE}" ]]; then + build/builder.sh env \ + AWS_ACCESS_KEY_ID="$AWS_ACCESS_KEY_ID" \ + AWS_SECRET_ACCESS_KEY="$AWS_SECRET_ACCESS_KEY" \ + TC_BUILD_BRANCH="$build_name" \ + publish-provisional-artifacts -bless -release -bucket "${bucket}" +else + echo "The latest S3 binaries and archive were _not_ updated." +fi +tc_end_block "Publish S3 binaries and archive as latest" + + +tc_start_block "Tag docker image as latest-RELEASE_BRANCH" +if [[ -z "$PRE_RELEASE" ]]; then + docker push "${dockerhub_repository}:latest-${release_branch}" +else + echo "The ${dockerhub_repository}:latest-${release_branch} docker image tag was _not_ pushed." +fi +tc_end_block "Tag docker image as latest-RELEASE_BRANCH" + + +tc_start_block "Tag docker image as latest" +# Only push the "latest" tag for our most recent release branch. +# https://github.com/cockroachdb/cockroach/issues/41067 +if [[ -n "${PUBLISH_LATEST}" && -z "$PRE_RELEASE" ]]; then + docker push "${dockerhub_repository}:latest" +else + echo "The ${dockerhub_repository}:latest docker image tag was _not_ pushed." +fi +tc_end_block "Tag docker image as latest" diff --git a/build/release/teamcity-support.sh b/build/release/teamcity-support.sh index 39e3506b1c68..5b5b464966dd 100644 --- a/build/release/teamcity-support.sh +++ b/build/release/teamcity-support.sh @@ -29,7 +29,7 @@ docker_login_with_google() { } docker_login() { - echo "${DOCKER_AUTH}" | docker login --username "${DOCKER_ID}" --password-stdin + echo "${DOCKER_ACCESS_TOKEN}" | docker login --username "${DOCKER_ID}" --password-stdin } configure_docker_creds() {