From de5f8e7e76a5d5632a6fce6a4a2f4af4345ce34b Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:02 -0500 Subject: [PATCH 01/15] sql/catalog: add a DescriptorIDSet in the image of ColumnIDSet Release note: None --- pkg/sql/catalog/BUILD.bazel | 1 + pkg/sql/catalog/descriptor_id_set.go | 69 ++++++++++++++++++++++++++++ 2 files changed, 70 insertions(+) create mode 100644 pkg/sql/catalog/descriptor_id_set.go diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index 36575a9cefe6..b352aba2aca8 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -7,6 +7,7 @@ go_library( "catalog.go", "desc_getter.go", "descriptor.go", + "descriptor_id_set.go", "errors.go", "table_col_map.go", "table_col_set.go", diff --git a/pkg/sql/catalog/descriptor_id_set.go b/pkg/sql/catalog/descriptor_id_set.go new file mode 100644 index 000000000000..4fbb91210e13 --- /dev/null +++ b/pkg/sql/catalog/descriptor_id_set.go @@ -0,0 +1,69 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package catalog + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util" +) + +// DescriptorIDSet efficiently stores an unordered set of descriptor ids. +type DescriptorIDSet struct { + set util.FastIntSet +} + +// MakeDescriptorIDSet returns a set initialized with the given values. +func MakeDescriptorIDSet(ids ...descpb.ID) DescriptorIDSet { + s := DescriptorIDSet{} + for _, id := range ids { + s.Add(id) + } + return s +} + +// Suppress the linter. +var _ = MakeDescriptorIDSet + +// Add adds an id to the set. No-op if the id is already in the set. +func (d *DescriptorIDSet) Add(id descpb.ID) { + d.set.Add(int(id)) +} + +// Len returns the number of the ids in the set. +func (d DescriptorIDSet) Len() int { + return d.set.Len() +} + +// Contains returns true if the set contains the column. +func (d DescriptorIDSet) Contains(id descpb.ID) bool { + return d.set.Contains(int(id)) +} + +// ForEach calls a function for each column in the set (in increasing order). +func (d DescriptorIDSet) ForEach(f func(id descpb.ID)) { + d.set.ForEach(func(i int) { f(descpb.ID(i)) }) +} + +// Empty returns true if the set is empty. +func (d DescriptorIDSet) Empty() bool { return d.set.Empty() } + +// Ordered returns a slice with all the descpb.IDs in the set, in +// increasing order. +func (d DescriptorIDSet) Ordered() []descpb.ID { + if d.Empty() { + return nil + } + result := make([]descpb.ID, 0, d.Len()) + d.ForEach(func(i descpb.ID) { + result = append(result, i) + }) + return result +} From a1decc6092d5c1dcd4897f22f56595ea1198b1da Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:02 -0500 Subject: [PATCH 02/15] schemachanger/scpb: introduce protobufs for schemachanger state `scpb` defines the building blocks of the schema changer's plan-related state: `Element`, `Target`, `State`, `Node`. We'll need protobuf serialization for the job record. Release note: None --- pkg/sql/schemachanger/scpb/BUILD.bazel | 53 + pkg/sql/schemachanger/scpb/elements.go | 73 + pkg/sql/schemachanger/scpb/elements_test.go | 27 + pkg/sql/schemachanger/scpb/scpb.pb.go | 2803 +++++++++++++++++++ pkg/sql/schemachanger/scpb/scpb.proto | 103 + 5 files changed, 3059 insertions(+) create mode 100644 pkg/sql/schemachanger/scpb/BUILD.bazel create mode 100644 pkg/sql/schemachanger/scpb/elements.go create mode 100644 pkg/sql/schemachanger/scpb/elements_test.go create mode 100644 pkg/sql/schemachanger/scpb/scpb.pb.go create mode 100644 pkg/sql/schemachanger/scpb/scpb.proto diff --git a/pkg/sql/schemachanger/scpb/BUILD.bazel b/pkg/sql/schemachanger/scpb/BUILD.bazel new file mode 100644 index 000000000000..ab74caa591a4 --- /dev/null +++ b/pkg/sql/schemachanger/scpb/BUILD.bazel @@ -0,0 +1,53 @@ +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "targets", + srcs = ["targets.go"], + embed = [":targets_go_proto"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/targets", + visibility = ["//visibility:public"], +) + +go_library( + name = "scpb", + srcs = ["elements.go"], + embed = [":scpb_go_proto"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/catalog/descpb", + "//pkg/util/protoutil", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_proto_library( + name = "scpb_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb", + proto = ":scpb_proto", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/catalog/descpb", + "@com_github_gogo_protobuf//gogoproto", + ], +) + +go_test( + name = "scpb_test", + srcs = ["elements_test.go"], + embed = [":scpb"], +) + +proto_library( + name = "scpb_proto", + srcs = ["scpb.proto"], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/catalog/descpb:descpb_proto", + "@com_github_gogo_protobuf//gogoproto:gogo_proto", + ], +) diff --git a/pkg/sql/schemachanger/scpb/elements.go b/pkg/sql/schemachanger/scpb/elements.go new file mode 100644 index 000000000000..030b76e3e7db --- /dev/null +++ b/pkg/sql/schemachanger/scpb/elements.go @@ -0,0 +1,73 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scpb + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +// NumStates is the number of values which State may take on. +var NumStates = len(State_name) + +// Node represents a Target in a given state. +type Node struct { + Target *Target + State State +} + +// Element returns the target's element. +func (n *Node) Element() Element { + return n.Target.Element() +} + +// Element represents a logical component of a catalog entry's schema (e.g., an +// index or column in a table). +type Element interface { + protoutil.Message + DescriptorID() descpb.ID +} + +// Element returns an Element from its wrapper for serialization. +func (e *ElementProto) Element() Element { + return e.GetValue().(Element) +} + +// NewTarget constructs a new Target. The passed elem must be one of the oneOf +// members of Element. If not, this call will panic. +func NewTarget(dir Target_Direction, elem Element) *Target { + t := Target{ + Direction: dir, + } + if !t.SetValue(elem) { + panic(errors.Errorf("unknown element type %T", elem)) + } + return &t +} + +// DescriptorID implements the Element interface. +func (e *Column) DescriptorID() descpb.ID { return e.TableID } + +// DescriptorID implements the Element interface. +func (e *PrimaryIndex) DescriptorID() descpb.ID { return e.TableID } + +// DescriptorID implements the Element interface. +func (e *SecondaryIndex) DescriptorID() descpb.ID { return e.TableID } + +// DescriptorID implements the Element interface. +func (e *SequenceDependency) DescriptorID() descpb.ID { return e.SequenceID } + +// DescriptorID implements the Element interface. +func (e *UniqueConstraint) DescriptorID() descpb.ID { return e.TableID } + +// DescriptorID implements the Element interface. +func (e *CheckConstraint) DescriptorID() descpb.ID { return e.TableID } diff --git a/pkg/sql/schemachanger/scpb/elements_test.go b/pkg/sql/schemachanger/scpb/elements_test.go new file mode 100644 index 000000000000..3cb8af4636a2 --- /dev/null +++ b/pkg/sql/schemachanger/scpb/elements_test.go @@ -0,0 +1,27 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scpb + +import ( + "reflect" + "testing" +) + +func TestGetElement(t *testing.T) { + typ := reflect.TypeOf((*ElementProto)(nil)).Elem() + elementInterfaceType := reflect.TypeOf((*Element)(nil)).Elem() + for i := 0; i < typ.NumField(); i++ { + f := typ.Field(i) + if !f.Type.Implements(elementInterfaceType) { + t.Errorf("%v does not implement %v", f.Type, elementInterfaceType) + } + } +} diff --git a/pkg/sql/schemachanger/scpb/scpb.pb.go b/pkg/sql/schemachanger/scpb/scpb.pb.go new file mode 100644 index 000000000000..b19bac262b4b --- /dev/null +++ b/pkg/sql/schemachanger/scpb/scpb.pb.go @@ -0,0 +1,2803 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: sql/schemachanger/scpb/scpb.proto + +package scpb + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" +import descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + +import github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + +import io "io" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + +type State int32 + +const ( + State_UNKNOWN State = 0 + State_ABSENT State = 1 + State_DELETE_ONLY State = 2 + State_DELETE_AND_WRITE_ONLY State = 3 + State_BACKFILLED State = 4 + State_VALIDATED State = 5 + State_PUBLIC State = 6 +) + +var State_name = map[int32]string{ + 0: "UNKNOWN", + 1: "ABSENT", + 2: "DELETE_ONLY", + 3: "DELETE_AND_WRITE_ONLY", + 4: "BACKFILLED", + 5: "VALIDATED", + 6: "PUBLIC", +} +var State_value = map[string]int32{ + "UNKNOWN": 0, + "ABSENT": 1, + "DELETE_ONLY": 2, + "DELETE_AND_WRITE_ONLY": 3, + "BACKFILLED": 4, + "VALIDATED": 5, + "PUBLIC": 6, +} + +func (x State) String() string { + return proto.EnumName(State_name, int32(x)) +} +func (State) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{0} +} + +type Target_Direction int32 + +const ( + Target_UNKNOWN Target_Direction = 0 + Target_ADD Target_Direction = 1 + Target_DROP Target_Direction = 2 +) + +var Target_Direction_name = map[int32]string{ + 0: "UNKNOWN", + 1: "ADD", + 2: "DROP", +} +var Target_Direction_value = map[string]int32{ + "UNKNOWN": 0, + "ADD": 1, + "DROP": 2, +} + +func (x Target_Direction) String() string { + return proto.EnumName(Target_Direction_name, int32(x)) +} +func (Target_Direction) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{1, 0} +} + +type SequenceDependency_Type int32 + +const ( + SequenceDependency_UNKNOWN SequenceDependency_Type = 0 + SequenceDependency_USES SequenceDependency_Type = 1 + SequenceDependency_OWNS SequenceDependency_Type = 2 +) + +var SequenceDependency_Type_name = map[int32]string{ + 0: "UNKNOWN", + 1: "USES", + 2: "OWNS", +} +var SequenceDependency_Type_value = map[string]int32{ + "UNKNOWN": 0, + "USES": 1, + "OWNS": 2, +} + +func (x SequenceDependency_Type) String() string { + return proto.EnumName(SequenceDependency_Type_name, int32(x)) +} +func (SequenceDependency_Type) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{5, 0} +} + +type ElementProto struct { + Column *Column `protobuf:"bytes,1,opt,name=column,proto3" json:"column,omitempty"` + PrimaryIndex *PrimaryIndex `protobuf:"bytes,2,opt,name=primary_index,json=primaryIndex,proto3" json:"primary_index,omitempty"` + SecondaryIndex *SecondaryIndex `protobuf:"bytes,3,opt,name=secondary_index,json=secondaryIndex,proto3" json:"secondary_index,omitempty"` + SequenceDependency *SequenceDependency `protobuf:"bytes,4,opt,name=sequence_dependency,json=sequenceDependency,proto3" json:"sequence_dependency,omitempty"` + UniqueConstraint *UniqueConstraint `protobuf:"bytes,5,opt,name=unique_constraint,json=uniqueConstraint,proto3" json:"unique_constraint,omitempty"` + CheckConstraint *CheckConstraint `protobuf:"bytes,6,opt,name=check_constraint,json=checkConstraint,proto3" json:"check_constraint,omitempty"` +} + +func (m *ElementProto) Reset() { *m = ElementProto{} } +func (m *ElementProto) String() string { return proto.CompactTextString(m) } +func (*ElementProto) ProtoMessage() {} +func (*ElementProto) Descriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{0} +} +func (m *ElementProto) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ElementProto) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ElementProto) XXX_Merge(src proto.Message) { + xxx_messageInfo_ElementProto.Merge(dst, src) +} +func (m *ElementProto) XXX_Size() int { + return m.Size() +} +func (m *ElementProto) XXX_DiscardUnknown() { + xxx_messageInfo_ElementProto.DiscardUnknown(m) +} + +var xxx_messageInfo_ElementProto proto.InternalMessageInfo + +type Target struct { + ElementProto `protobuf:"bytes,1,opt,name=element_proto,json=elementProto,proto3,embedded=element_proto" json:"element_proto"` + Direction Target_Direction `protobuf:"varint,2,opt,name=direction,proto3,enum=cockroach.sql.schemachanger.scpb.Target_Direction" json:"direction,omitempty"` +} + +func (m *Target) Reset() { *m = Target{} } +func (m *Target) String() string { return proto.CompactTextString(m) } +func (*Target) ProtoMessage() {} +func (*Target) Descriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{1} +} +func (m *Target) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Target) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Target) XXX_Merge(src proto.Message) { + xxx_messageInfo_Target.Merge(dst, src) +} +func (m *Target) XXX_Size() int { + return m.Size() +} +func (m *Target) XXX_DiscardUnknown() { + xxx_messageInfo_Target.DiscardUnknown(m) +} + +var xxx_messageInfo_Target proto.InternalMessageInfo + +type Column struct { + TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` + FamilyID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.FamilyID `protobuf:"varint,2,opt,name=family_id,json=familyId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.FamilyID" json:"family_id,omitempty"` + FamilyName string `protobuf:"bytes,3,opt,name=family_name,json=familyName,proto3" json:"family_name,omitempty"` + Column descpb.ColumnDescriptor `protobuf:"bytes,4,opt,name=column,proto3" json:"column"` +} + +func (m *Column) Reset() { *m = Column{} } +func (m *Column) String() string { return proto.CompactTextString(m) } +func (*Column) ProtoMessage() {} +func (*Column) Descriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{2} +} +func (m *Column) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Column) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Column) XXX_Merge(src proto.Message) { + xxx_messageInfo_Column.Merge(dst, src) +} +func (m *Column) XXX_Size() int { + return m.Size() +} +func (m *Column) XXX_DiscardUnknown() { + xxx_messageInfo_Column.DiscardUnknown(m) +} + +var xxx_messageInfo_Column proto.InternalMessageInfo + +type PrimaryIndex struct { + TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` + Index descpb.IndexDescriptor `protobuf:"bytes,2,opt,name=index,proto3" json:"index"` + OtherPrimaryIndexID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID `protobuf:"varint,3,opt,name=other_primary_index_id,json=otherPrimaryIndexId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID" json:"other_primary_index_id,omitempty"` + StoreColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,4,rep,packed,name=store_column_ids,json=storeColumnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"store_column_ids,omitempty"` + StoreColumnNames []string `protobuf:"bytes,5,rep,name=store_column_names,json=storeColumnNames,proto3" json:"store_column_names,omitempty"` +} + +func (m *PrimaryIndex) Reset() { *m = PrimaryIndex{} } +func (m *PrimaryIndex) String() string { return proto.CompactTextString(m) } +func (*PrimaryIndex) ProtoMessage() {} +func (*PrimaryIndex) Descriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{3} +} +func (m *PrimaryIndex) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PrimaryIndex) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PrimaryIndex) XXX_Merge(src proto.Message) { + xxx_messageInfo_PrimaryIndex.Merge(dst, src) +} +func (m *PrimaryIndex) XXX_Size() int { + return m.Size() +} +func (m *PrimaryIndex) XXX_DiscardUnknown() { + xxx_messageInfo_PrimaryIndex.DiscardUnknown(m) +} + +var xxx_messageInfo_PrimaryIndex proto.InternalMessageInfo + +type SecondaryIndex struct { + TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` + Index descpb.IndexDescriptor `protobuf:"bytes,2,opt,name=index,proto3" json:"index"` + PrimaryIndex github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID `protobuf:"varint,3,opt,name=primary_index,json=primaryIndex,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID" json:"primary_index,omitempty"` +} + +func (m *SecondaryIndex) Reset() { *m = SecondaryIndex{} } +func (m *SecondaryIndex) String() string { return proto.CompactTextString(m) } +func (*SecondaryIndex) ProtoMessage() {} +func (*SecondaryIndex) Descriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{4} +} +func (m *SecondaryIndex) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SecondaryIndex) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SecondaryIndex) XXX_Merge(src proto.Message) { + xxx_messageInfo_SecondaryIndex.Merge(dst, src) +} +func (m *SecondaryIndex) XXX_Size() int { + return m.Size() +} +func (m *SecondaryIndex) XXX_DiscardUnknown() { + xxx_messageInfo_SecondaryIndex.DiscardUnknown(m) +} + +var xxx_messageInfo_SecondaryIndex proto.InternalMessageInfo + +type SequenceDependency struct { + TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` + ColumnID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,2,opt,name=column_id,json=columnId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"column_id,omitempty"` + SequenceID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,3,opt,name=sequence_id,json=sequenceId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"sequence_id,omitempty"` + Type SequenceDependency_Type `protobuf:"varint,4,opt,name=type,proto3,enum=cockroach.sql.schemachanger.scpb.SequenceDependency_Type" json:"type,omitempty"` +} + +func (m *SequenceDependency) Reset() { *m = SequenceDependency{} } +func (m *SequenceDependency) String() string { return proto.CompactTextString(m) } +func (*SequenceDependency) ProtoMessage() {} +func (*SequenceDependency) Descriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{5} +} +func (m *SequenceDependency) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SequenceDependency) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SequenceDependency) XXX_Merge(src proto.Message) { + xxx_messageInfo_SequenceDependency.Merge(dst, src) +} +func (m *SequenceDependency) XXX_Size() int { + return m.Size() +} +func (m *SequenceDependency) XXX_DiscardUnknown() { + xxx_messageInfo_SequenceDependency.DiscardUnknown(m) +} + +var xxx_messageInfo_SequenceDependency proto.InternalMessageInfo + +type UniqueConstraint struct { + TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` + IndexID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID `protobuf:"varint,2,opt,name=index_id,json=indexId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID" json:"index_id,omitempty"` + ColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,3,rep,packed,name=column_ids,json=columnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"column_ids,omitempty"` +} + +func (m *UniqueConstraint) Reset() { *m = UniqueConstraint{} } +func (m *UniqueConstraint) String() string { return proto.CompactTextString(m) } +func (*UniqueConstraint) ProtoMessage() {} +func (*UniqueConstraint) Descriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{6} +} +func (m *UniqueConstraint) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UniqueConstraint) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UniqueConstraint) XXX_Merge(src proto.Message) { + xxx_messageInfo_UniqueConstraint.Merge(dst, src) +} +func (m *UniqueConstraint) XXX_Size() int { + return m.Size() +} +func (m *UniqueConstraint) XXX_DiscardUnknown() { + xxx_messageInfo_UniqueConstraint.DiscardUnknown(m) +} + +var xxx_messageInfo_UniqueConstraint proto.InternalMessageInfo + +type CheckConstraint struct { + TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Expr string `protobuf:"bytes,3,opt,name=expr,proto3" json:"expr,omitempty"` + ColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,4,rep,packed,name=column_ids,json=columnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"column_ids,omitempty"` + Validated bool `protobuf:"varint,5,opt,name=validated,proto3" json:"validated,omitempty"` +} + +func (m *CheckConstraint) Reset() { *m = CheckConstraint{} } +func (m *CheckConstraint) String() string { return proto.CompactTextString(m) } +func (*CheckConstraint) ProtoMessage() {} +func (*CheckConstraint) Descriptor() ([]byte, []int) { + return fileDescriptor_scpb_0485450e9798247c, []int{7} +} +func (m *CheckConstraint) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CheckConstraint) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CheckConstraint) XXX_Merge(src proto.Message) { + xxx_messageInfo_CheckConstraint.Merge(dst, src) +} +func (m *CheckConstraint) XXX_Size() int { + return m.Size() +} +func (m *CheckConstraint) XXX_DiscardUnknown() { + xxx_messageInfo_CheckConstraint.DiscardUnknown(m) +} + +var xxx_messageInfo_CheckConstraint proto.InternalMessageInfo + +func init() { + proto.RegisterType((*ElementProto)(nil), "cockroach.sql.schemachanger.scpb.ElementProto") + proto.RegisterType((*Target)(nil), "cockroach.sql.schemachanger.scpb.Target") + proto.RegisterType((*Column)(nil), "cockroach.sql.schemachanger.scpb.Column") + proto.RegisterType((*PrimaryIndex)(nil), "cockroach.sql.schemachanger.scpb.PrimaryIndex") + proto.RegisterType((*SecondaryIndex)(nil), "cockroach.sql.schemachanger.scpb.SecondaryIndex") + proto.RegisterType((*SequenceDependency)(nil), "cockroach.sql.schemachanger.scpb.SequenceDependency") + proto.RegisterType((*UniqueConstraint)(nil), "cockroach.sql.schemachanger.scpb.UniqueConstraint") + proto.RegisterType((*CheckConstraint)(nil), "cockroach.sql.schemachanger.scpb.CheckConstraint") + proto.RegisterEnum("cockroach.sql.schemachanger.scpb.State", State_name, State_value) + proto.RegisterEnum("cockroach.sql.schemachanger.scpb.Target_Direction", Target_Direction_name, Target_Direction_value) + proto.RegisterEnum("cockroach.sql.schemachanger.scpb.SequenceDependency_Type", SequenceDependency_Type_name, SequenceDependency_Type_value) +} +func (this *Column) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Column) + if !ok { + that2, ok := that.(Column) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.TableID != that1.TableID { + return false + } + if this.FamilyID != that1.FamilyID { + return false + } + if this.FamilyName != that1.FamilyName { + return false + } + if !this.Column.Equal(&that1.Column) { + return false + } + return true +} +func (this *PrimaryIndex) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PrimaryIndex) + if !ok { + that2, ok := that.(PrimaryIndex) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.TableID != that1.TableID { + return false + } + if !this.Index.Equal(&that1.Index) { + return false + } + if this.OtherPrimaryIndexID != that1.OtherPrimaryIndexID { + return false + } + if len(this.StoreColumnIDs) != len(that1.StoreColumnIDs) { + return false + } + for i := range this.StoreColumnIDs { + if this.StoreColumnIDs[i] != that1.StoreColumnIDs[i] { + return false + } + } + if len(this.StoreColumnNames) != len(that1.StoreColumnNames) { + return false + } + for i := range this.StoreColumnNames { + if this.StoreColumnNames[i] != that1.StoreColumnNames[i] { + return false + } + } + return true +} +func (this *SecondaryIndex) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SecondaryIndex) + if !ok { + that2, ok := that.(SecondaryIndex) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.TableID != that1.TableID { + return false + } + if !this.Index.Equal(&that1.Index) { + return false + } + if this.PrimaryIndex != that1.PrimaryIndex { + return false + } + return true +} +func (this *SequenceDependency) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SequenceDependency) + if !ok { + that2, ok := that.(SequenceDependency) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.TableID != that1.TableID { + return false + } + if this.ColumnID != that1.ColumnID { + return false + } + if this.SequenceID != that1.SequenceID { + return false + } + if this.Type != that1.Type { + return false + } + return true +} +func (this *UniqueConstraint) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*UniqueConstraint) + if !ok { + that2, ok := that.(UniqueConstraint) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.TableID != that1.TableID { + return false + } + if this.IndexID != that1.IndexID { + return false + } + if len(this.ColumnIDs) != len(that1.ColumnIDs) { + return false + } + for i := range this.ColumnIDs { + if this.ColumnIDs[i] != that1.ColumnIDs[i] { + return false + } + } + return true +} +func (this *CheckConstraint) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*CheckConstraint) + if !ok { + that2, ok := that.(CheckConstraint) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.TableID != that1.TableID { + return false + } + if this.Name != that1.Name { + return false + } + if this.Expr != that1.Expr { + return false + } + if len(this.ColumnIDs) != len(that1.ColumnIDs) { + return false + } + for i := range this.ColumnIDs { + if this.ColumnIDs[i] != that1.ColumnIDs[i] { + return false + } + } + if this.Validated != that1.Validated { + return false + } + return true +} +func (m *ElementProto) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ElementProto) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Column != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.Column.Size())) + n1, err := m.Column.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n1 + } + if m.PrimaryIndex != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.PrimaryIndex.Size())) + n2, err := m.PrimaryIndex.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n2 + } + if m.SecondaryIndex != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.SecondaryIndex.Size())) + n3, err := m.SecondaryIndex.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n3 + } + if m.SequenceDependency != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.SequenceDependency.Size())) + n4, err := m.SequenceDependency.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n4 + } + if m.UniqueConstraint != nil { + dAtA[i] = 0x2a + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.UniqueConstraint.Size())) + n5, err := m.UniqueConstraint.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n5 + } + if m.CheckConstraint != nil { + dAtA[i] = 0x32 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.CheckConstraint.Size())) + n6, err := m.CheckConstraint.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n6 + } + return i, nil +} + +func (m *Target) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Target) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.ElementProto.Size())) + n7, err := m.ElementProto.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n7 + if m.Direction != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.Direction)) + } + return i, nil +} + +func (m *Column) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Column) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.TableID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) + } + if m.FamilyID != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.FamilyID)) + } + if len(m.FamilyName) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintScpb(dAtA, i, uint64(len(m.FamilyName))) + i += copy(dAtA[i:], m.FamilyName) + } + dAtA[i] = 0x22 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.Column.Size())) + n8, err := m.Column.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n8 + return i, nil +} + +func (m *PrimaryIndex) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *PrimaryIndex) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.TableID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) + } + dAtA[i] = 0x12 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.Index.Size())) + n9, err := m.Index.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n9 + if m.OtherPrimaryIndexID != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.OtherPrimaryIndexID)) + } + if len(m.StoreColumnIDs) > 0 { + dAtA11 := make([]byte, len(m.StoreColumnIDs)*10) + var j10 int + for _, num := range m.StoreColumnIDs { + for num >= 1<<7 { + dAtA11[j10] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j10++ + } + dAtA11[j10] = uint8(num) + j10++ + } + dAtA[i] = 0x22 + i++ + i = encodeVarintScpb(dAtA, i, uint64(j10)) + i += copy(dAtA[i:], dAtA11[:j10]) + } + if len(m.StoreColumnNames) > 0 { + for _, s := range m.StoreColumnNames { + dAtA[i] = 0x2a + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + return i, nil +} + +func (m *SecondaryIndex) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SecondaryIndex) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.TableID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) + } + dAtA[i] = 0x12 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.Index.Size())) + n12, err := m.Index.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n12 + if m.PrimaryIndex != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.PrimaryIndex)) + } + return i, nil +} + +func (m *SequenceDependency) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SequenceDependency) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.TableID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) + } + if m.ColumnID != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.ColumnID)) + } + if m.SequenceID != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.SequenceID)) + } + if m.Type != 0 { + dAtA[i] = 0x20 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.Type)) + } + return i, nil +} + +func (m *UniqueConstraint) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *UniqueConstraint) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.TableID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) + } + if m.IndexID != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.IndexID)) + } + if len(m.ColumnIDs) > 0 { + dAtA14 := make([]byte, len(m.ColumnIDs)*10) + var j13 int + for _, num := range m.ColumnIDs { + for num >= 1<<7 { + dAtA14[j13] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j13++ + } + dAtA14[j13] = uint8(num) + j13++ + } + dAtA[i] = 0x1a + i++ + i = encodeVarintScpb(dAtA, i, uint64(j13)) + i += copy(dAtA[i:], dAtA14[:j13]) + } + return i, nil +} + +func (m *CheckConstraint) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CheckConstraint) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.TableID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) + } + if len(m.Name) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintScpb(dAtA, i, uint64(len(m.Name))) + i += copy(dAtA[i:], m.Name) + } + if len(m.Expr) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintScpb(dAtA, i, uint64(len(m.Expr))) + i += copy(dAtA[i:], m.Expr) + } + if len(m.ColumnIDs) > 0 { + dAtA16 := make([]byte, len(m.ColumnIDs)*10) + var j15 int + for _, num := range m.ColumnIDs { + for num >= 1<<7 { + dAtA16[j15] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j15++ + } + dAtA16[j15] = uint8(num) + j15++ + } + dAtA[i] = 0x22 + i++ + i = encodeVarintScpb(dAtA, i, uint64(j15)) + i += copy(dAtA[i:], dAtA16[:j15]) + } + if m.Validated { + dAtA[i] = 0x28 + i++ + if m.Validated { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + return i, nil +} + +func encodeVarintScpb(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *ElementProto) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Column != nil { + l = m.Column.Size() + n += 1 + l + sovScpb(uint64(l)) + } + if m.PrimaryIndex != nil { + l = m.PrimaryIndex.Size() + n += 1 + l + sovScpb(uint64(l)) + } + if m.SecondaryIndex != nil { + l = m.SecondaryIndex.Size() + n += 1 + l + sovScpb(uint64(l)) + } + if m.SequenceDependency != nil { + l = m.SequenceDependency.Size() + n += 1 + l + sovScpb(uint64(l)) + } + if m.UniqueConstraint != nil { + l = m.UniqueConstraint.Size() + n += 1 + l + sovScpb(uint64(l)) + } + if m.CheckConstraint != nil { + l = m.CheckConstraint.Size() + n += 1 + l + sovScpb(uint64(l)) + } + return n +} + +func (m *Target) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.ElementProto.Size() + n += 1 + l + sovScpb(uint64(l)) + if m.Direction != 0 { + n += 1 + sovScpb(uint64(m.Direction)) + } + return n +} + +func (m *Column) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TableID != 0 { + n += 1 + sovScpb(uint64(m.TableID)) + } + if m.FamilyID != 0 { + n += 1 + sovScpb(uint64(m.FamilyID)) + } + l = len(m.FamilyName) + if l > 0 { + n += 1 + l + sovScpb(uint64(l)) + } + l = m.Column.Size() + n += 1 + l + sovScpb(uint64(l)) + return n +} + +func (m *PrimaryIndex) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TableID != 0 { + n += 1 + sovScpb(uint64(m.TableID)) + } + l = m.Index.Size() + n += 1 + l + sovScpb(uint64(l)) + if m.OtherPrimaryIndexID != 0 { + n += 1 + sovScpb(uint64(m.OtherPrimaryIndexID)) + } + if len(m.StoreColumnIDs) > 0 { + l = 0 + for _, e := range m.StoreColumnIDs { + l += sovScpb(uint64(e)) + } + n += 1 + sovScpb(uint64(l)) + l + } + if len(m.StoreColumnNames) > 0 { + for _, s := range m.StoreColumnNames { + l = len(s) + n += 1 + l + sovScpb(uint64(l)) + } + } + return n +} + +func (m *SecondaryIndex) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TableID != 0 { + n += 1 + sovScpb(uint64(m.TableID)) + } + l = m.Index.Size() + n += 1 + l + sovScpb(uint64(l)) + if m.PrimaryIndex != 0 { + n += 1 + sovScpb(uint64(m.PrimaryIndex)) + } + return n +} + +func (m *SequenceDependency) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TableID != 0 { + n += 1 + sovScpb(uint64(m.TableID)) + } + if m.ColumnID != 0 { + n += 1 + sovScpb(uint64(m.ColumnID)) + } + if m.SequenceID != 0 { + n += 1 + sovScpb(uint64(m.SequenceID)) + } + if m.Type != 0 { + n += 1 + sovScpb(uint64(m.Type)) + } + return n +} + +func (m *UniqueConstraint) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TableID != 0 { + n += 1 + sovScpb(uint64(m.TableID)) + } + if m.IndexID != 0 { + n += 1 + sovScpb(uint64(m.IndexID)) + } + if len(m.ColumnIDs) > 0 { + l = 0 + for _, e := range m.ColumnIDs { + l += sovScpb(uint64(e)) + } + n += 1 + sovScpb(uint64(l)) + l + } + return n +} + +func (m *CheckConstraint) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TableID != 0 { + n += 1 + sovScpb(uint64(m.TableID)) + } + l = len(m.Name) + if l > 0 { + n += 1 + l + sovScpb(uint64(l)) + } + l = len(m.Expr) + if l > 0 { + n += 1 + l + sovScpb(uint64(l)) + } + if len(m.ColumnIDs) > 0 { + l = 0 + for _, e := range m.ColumnIDs { + l += sovScpb(uint64(e)) + } + n += 1 + sovScpb(uint64(l)) + l + } + if m.Validated { + n += 2 + } + return n +} + +func sovScpb(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozScpb(x uint64) (n int) { + return sovScpb(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *ElementProto) GetValue() interface{} { + if this.Column != nil { + return this.Column + } + if this.PrimaryIndex != nil { + return this.PrimaryIndex + } + if this.SecondaryIndex != nil { + return this.SecondaryIndex + } + if this.SequenceDependency != nil { + return this.SequenceDependency + } + if this.UniqueConstraint != nil { + return this.UniqueConstraint + } + if this.CheckConstraint != nil { + return this.CheckConstraint + } + return nil +} + +func (this *ElementProto) SetValue(value interface{}) bool { + switch vt := value.(type) { + case *Column: + this.Column = vt + case *PrimaryIndex: + this.PrimaryIndex = vt + case *SecondaryIndex: + this.SecondaryIndex = vt + case *SequenceDependency: + this.SequenceDependency = vt + case *UniqueConstraint: + this.UniqueConstraint = vt + case *CheckConstraint: + this.CheckConstraint = vt + default: + return false + } + return true +} +func (m *ElementProto) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ElementProto: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ElementProto: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Column", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Column == nil { + m.Column = &Column{} + } + if err := m.Column.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PrimaryIndex == nil { + m.PrimaryIndex = &PrimaryIndex{} + } + if err := m.PrimaryIndex.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SecondaryIndex", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SecondaryIndex == nil { + m.SecondaryIndex = &SecondaryIndex{} + } + if err := m.SecondaryIndex.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SequenceDependency", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SequenceDependency == nil { + m.SequenceDependency = &SequenceDependency{} + } + if err := m.SequenceDependency.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UniqueConstraint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.UniqueConstraint == nil { + m.UniqueConstraint = &UniqueConstraint{} + } + if err := m.UniqueConstraint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CheckConstraint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CheckConstraint == nil { + m.CheckConstraint = &CheckConstraint{} + } + if err := m.CheckConstraint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipScpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthScpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Target) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Target: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Target: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ElementProto", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ElementProto.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Direction", wireType) + } + m.Direction = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Direction |= (Target_Direction(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipScpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthScpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Column) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Column: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Column: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + } + m.TableID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TableID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FamilyID", wireType) + } + m.FamilyID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.FamilyID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.FamilyID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FamilyName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FamilyName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Column", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Column.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipScpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthScpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *PrimaryIndex) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: PrimaryIndex: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PrimaryIndex: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + } + m.TableID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TableID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Index.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field OtherPrimaryIndexID", wireType) + } + m.OtherPrimaryIndexID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.OtherPrimaryIndexID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.StoreColumnIDs = append(m.StoreColumnIDs, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.StoreColumnIDs) == 0 { + m.StoreColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) + } + for iNdEx < postIndex { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.StoreColumnIDs = append(m.StoreColumnIDs, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field StoreColumnIDs", wireType) + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StoreColumnNames", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StoreColumnNames = append(m.StoreColumnNames, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipScpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthScpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SecondaryIndex) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SecondaryIndex: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SecondaryIndex: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + } + m.TableID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TableID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Index.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryIndex", wireType) + } + m.PrimaryIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PrimaryIndex |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipScpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthScpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SequenceDependency) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SequenceDependency: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SequenceDependency: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + } + m.TableID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TableID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ColumnID", wireType) + } + m.ColumnID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ColumnID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SequenceID", wireType) + } + m.SequenceID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SequenceID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= (SequenceDependency_Type(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipScpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthScpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *UniqueConstraint) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: UniqueConstraint: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UniqueConstraint: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + } + m.TableID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TableID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IndexID", wireType) + } + m.IndexID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.IndexID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.ColumnIDs = append(m.ColumnIDs, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ColumnIDs) == 0 { + m.ColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) + } + for iNdEx < postIndex { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.ColumnIDs = append(m.ColumnIDs, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field ColumnIDs", wireType) + } + default: + iNdEx = preIndex + skippy, err := skipScpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthScpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CheckConstraint) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CheckConstraint: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CheckConstraint: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + } + m.TableID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TableID |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Expr = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.ColumnIDs = append(m.ColumnIDs, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ColumnIDs) == 0 { + m.ColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) + } + for iNdEx < postIndex { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.ColumnIDs = append(m.ColumnIDs, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field ColumnIDs", wireType) + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Validated", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Validated = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipScpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthScpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipScpb(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowScpb + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowScpb + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowScpb + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthScpb + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowScpb + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipScpb(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthScpb = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowScpb = fmt.Errorf("proto: integer overflow") +) + +func init() { + proto.RegisterFile("sql/schemachanger/scpb/scpb.proto", fileDescriptor_scpb_0485450e9798247c) +} + +var fileDescriptor_scpb_0485450e9798247c = []byte{ + // 1069 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x56, 0xcd, 0x6e, 0xe3, 0x54, + 0x14, 0x8e, 0x13, 0x37, 0x4d, 0x4e, 0xdb, 0xd4, 0xdc, 0x0e, 0x28, 0x8c, 0x50, 0x5c, 0x8a, 0x34, + 0x53, 0x81, 0xe4, 0x40, 0x61, 0x43, 0x37, 0x4c, 0x5c, 0x67, 0x24, 0x6b, 0x42, 0x1a, 0x9c, 0x74, + 0xaa, 0x19, 0x81, 0x2c, 0xe7, 0xfa, 0x92, 0x58, 0x8d, 0x7f, 0x6a, 0x3b, 0x30, 0xdd, 0xf1, 0x06, + 0xb0, 0x63, 0xcb, 0x1a, 0xb1, 0xe5, 0x1d, 0x2a, 0xb1, 0xe9, 0x0a, 0x8d, 0x84, 0x64, 0x81, 0x2b, + 0x21, 0x9e, 0x81, 0x15, 0xba, 0xd7, 0x76, 0xea, 0xa4, 0x42, 0xad, 0x3a, 0xa5, 0x12, 0x9b, 0xe8, + 0xfa, 0xc4, 0xe7, 0x3b, 0x7f, 0xdf, 0xfd, 0x8e, 0xe1, 0xed, 0xe0, 0x78, 0xd2, 0x0c, 0xf0, 0x98, + 0xd8, 0x06, 0x1e, 0x1b, 0xce, 0x88, 0xf8, 0xcd, 0x00, 0x7b, 0x43, 0xf6, 0x23, 0x79, 0xbe, 0x1b, + 0xba, 0x68, 0x13, 0xbb, 0xf8, 0xc8, 0x77, 0x0d, 0x3c, 0x96, 0x82, 0xe3, 0x89, 0x34, 0xf7, 0xb2, + 0x44, 0xdf, 0xbb, 0xff, 0x0e, 0x05, 0xc1, 0x46, 0x68, 0x4c, 0xdc, 0x51, 0xd3, 0x24, 0x09, 0x40, + 0xe8, 0x4f, 0x71, 0x38, 0xf5, 0x89, 0x99, 0xc0, 0xdc, 0xbf, 0x37, 0x72, 0x47, 0x2e, 0x3b, 0x36, + 0xe9, 0x29, 0xb1, 0x6e, 0xfd, 0xc4, 0xc3, 0x6a, 0x7b, 0x42, 0x6c, 0xe2, 0x84, 0x3d, 0x16, 0xed, + 0x11, 0x94, 0xb1, 0x3b, 0x99, 0xda, 0x4e, 0x9d, 0xdb, 0xe4, 0xb6, 0x57, 0x76, 0xb6, 0xa5, 0xab, + 0xc2, 0x4b, 0x7b, 0xec, 0x7d, 0x2d, 0xf5, 0x43, 0x7d, 0x58, 0xf3, 0x7c, 0xcb, 0x36, 0xfc, 0x13, + 0xdd, 0x72, 0x4c, 0xf2, 0xa2, 0x5e, 0x64, 0x40, 0xd2, 0xd5, 0x40, 0xbd, 0xc4, 0x4d, 0xa5, 0x5e, + 0xda, 0xaa, 0x97, 0x7b, 0x42, 0xcf, 0x60, 0x3d, 0x20, 0xd8, 0x75, 0xcc, 0x0b, 0xd8, 0x12, 0x83, + 0x7d, 0xff, 0x6a, 0xd8, 0x7e, 0xe6, 0x98, 0x00, 0xd7, 0x82, 0xb9, 0x67, 0x44, 0x60, 0x23, 0x20, + 0xc7, 0x53, 0xe2, 0x60, 0xa2, 0x9b, 0xc4, 0x23, 0x8e, 0x49, 0x1c, 0x7c, 0x52, 0xe7, 0x19, 0xfc, + 0x47, 0xd7, 0x81, 0x4f, 0x9c, 0x95, 0x99, 0xaf, 0x86, 0x82, 0x4b, 0x36, 0xa4, 0xc3, 0x6b, 0x53, + 0xc7, 0x3a, 0x9e, 0x12, 0x1d, 0xbb, 0x4e, 0x10, 0xfa, 0x86, 0xe5, 0x84, 0xf5, 0x25, 0x16, 0x64, + 0xe7, 0xea, 0x20, 0x07, 0xcc, 0x75, 0x6f, 0xe6, 0xa9, 0x09, 0xd3, 0x05, 0x0b, 0xfa, 0x1c, 0x04, + 0x3c, 0x26, 0xf8, 0x28, 0x8f, 0x5f, 0x66, 0xf8, 0x1f, 0x5c, 0x63, 0x86, 0xd4, 0x33, 0x07, 0xbf, + 0x8e, 0xe7, 0x0d, 0xbb, 0xfc, 0xe9, 0x0f, 0x22, 0xb7, 0xf5, 0x27, 0x07, 0xe5, 0x81, 0xe1, 0x8f, + 0x48, 0x88, 0xbe, 0x80, 0x35, 0x92, 0x10, 0x47, 0x67, 0x54, 0x4a, 0xf9, 0x72, 0x8d, 0x31, 0xe7, + 0xf9, 0x26, 0x57, 0x4e, 0x23, 0xb1, 0x70, 0x16, 0x89, 0x9c, 0xb6, 0x4a, 0xf2, 0x3c, 0xec, 0x41, + 0xd5, 0xb4, 0x7c, 0x82, 0x43, 0xcb, 0x75, 0x18, 0x83, 0x6a, 0xd7, 0x69, 0x53, 0x92, 0x9b, 0xa4, + 0x64, 0x9e, 0xda, 0x05, 0xc8, 0xd6, 0x7b, 0x50, 0x9d, 0xd9, 0xd1, 0x0a, 0x2c, 0x1f, 0x74, 0x9f, + 0x74, 0xf7, 0x0f, 0xbb, 0x42, 0x01, 0x2d, 0x43, 0xa9, 0xa5, 0x28, 0x02, 0x87, 0x2a, 0xc0, 0x2b, + 0xda, 0x7e, 0x4f, 0x28, 0x6e, 0xfd, 0x52, 0x84, 0x72, 0xc2, 0x6b, 0x64, 0x42, 0x25, 0x34, 0x86, + 0x13, 0xa2, 0x5b, 0x26, 0xab, 0x71, 0x4d, 0x56, 0xe3, 0x48, 0x5c, 0x1e, 0x50, 0x9b, 0xaa, 0xfc, + 0x1d, 0x89, 0xbb, 0x23, 0x2b, 0x1c, 0x4f, 0x87, 0x12, 0x76, 0xed, 0xe6, 0x2c, 0x43, 0x73, 0x78, + 0x71, 0x6e, 0x7a, 0x47, 0xa3, 0xe6, 0xe5, 0x3b, 0x2a, 0xa9, 0x8a, 0xb6, 0xcc, 0xa0, 0x55, 0x13, + 0xd9, 0x50, 0xfd, 0xd2, 0xb0, 0xad, 0xc9, 0x09, 0x0d, 0x53, 0x64, 0x61, 0x7a, 0x71, 0x24, 0x56, + 0x1e, 0x33, 0x23, 0x8b, 0xf3, 0xe8, 0xa6, 0x71, 0x32, 0x0c, 0xad, 0x92, 0x84, 0x50, 0x4d, 0x24, + 0xc2, 0x4a, 0x1a, 0xce, 0x31, 0x6c, 0xc2, 0xee, 0x52, 0x55, 0x83, 0xc4, 0xd4, 0x35, 0x6c, 0x82, + 0xda, 0x33, 0x1d, 0x48, 0x2e, 0xc2, 0xc3, 0xc5, 0xe6, 0x1f, 0x4f, 0x86, 0x46, 0x40, 0xd2, 0xcb, + 0xaf, 0x90, 0x00, 0xfb, 0x96, 0x17, 0xba, 0xbe, 0xcc, 0xd3, 0x81, 0x66, 0x62, 0xb0, 0xcb, 0xff, + 0x45, 0x69, 0xf3, 0x3d, 0x0f, 0xab, 0xf9, 0xcb, 0x7d, 0x47, 0x3d, 0x95, 0x61, 0x29, 0xaf, 0x40, + 0x0f, 0xfe, 0xa5, 0x04, 0x96, 0xd2, 0xa5, 0x0a, 0x12, 0x57, 0xf4, 0x2d, 0x07, 0x6f, 0xb8, 0xe1, + 0x98, 0xf8, 0xfa, 0x9c, 0xa8, 0xd1, 0xc4, 0x4b, 0x2c, 0xf1, 0xe7, 0x71, 0x24, 0x6e, 0xec, 0xd3, + 0x37, 0xf2, 0x15, 0xb2, 0x22, 0x3e, 0xb9, 0x71, 0x11, 0x09, 0x84, 0xb6, 0xe1, 0x5e, 0xc2, 0x35, + 0xd1, 0x37, 0x1c, 0x08, 0x41, 0xe8, 0xfa, 0x54, 0x48, 0x68, 0x8f, 0x75, 0xcb, 0x0c, 0xea, 0xfc, + 0x66, 0x69, 0x7b, 0x4d, 0x7e, 0x1a, 0x47, 0x62, 0xad, 0x4f, 0xff, 0x4b, 0xc6, 0xa2, 0x2a, 0xc1, + 0xab, 0xf0, 0x26, 0x03, 0xd1, 0x6a, 0x41, 0x0e, 0xd3, 0x0c, 0x90, 0x0c, 0x68, 0x2e, 0x03, 0xca, + 0xa1, 0xa0, 0xbe, 0xb4, 0x59, 0xda, 0xae, 0xca, 0xf7, 0xe2, 0x48, 0x14, 0x72, 0x39, 0x50, 0x36, + 0x05, 0x9a, 0x10, 0x2c, 0x58, 0x52, 0x66, 0xfc, 0x58, 0x84, 0xda, 0xbc, 0x3e, 0xff, 0x8f, 0xb8, + 0x31, 0x5e, 0xdc, 0x74, 0x09, 0x23, 0xf6, 0x6e, 0x63, 0xf4, 0x73, 0xeb, 0x2f, 0x6d, 0xd6, 0xaf, + 0x25, 0x40, 0x97, 0xb7, 0xcd, 0xdd, 0x09, 0xd4, 0x8c, 0x6f, 0x79, 0x81, 0xca, 0x48, 0x72, 0x2b, + 0x44, 0xab, 0xe0, 0x94, 0x63, 0xc8, 0x86, 0x95, 0xd9, 0x56, 0x9e, 0xdd, 0xb5, 0x4e, 0x1c, 0x89, + 0x90, 0x75, 0xe0, 0x95, 0x4b, 0x83, 0x2c, 0x80, 0x6a, 0xa2, 0x4f, 0x81, 0x0f, 0x4f, 0x3c, 0xc2, + 0xc4, 0xae, 0xb6, 0xf3, 0xf1, 0x4d, 0xb6, 0xbe, 0x34, 0x38, 0xf1, 0x88, 0xc6, 0x60, 0xb6, 0x1e, + 0x02, 0x4f, 0x9f, 0xe6, 0xd7, 0x4c, 0x05, 0xf8, 0x83, 0x7e, 0xbb, 0x9f, 0xec, 0x99, 0xfd, 0xc3, + 0x6e, 0x5f, 0x28, 0xa6, 0x83, 0xfd, 0xad, 0x08, 0xc2, 0xe2, 0x86, 0xbf, 0xa3, 0xb1, 0x5a, 0x50, + 0x99, 0x09, 0x5a, 0x32, 0xd5, 0x2e, 0x8d, 0x72, 0x8b, 0x22, 0xb6, 0x6c, 0xa5, 0xc2, 0xe5, 0x01, + 0xe4, 0x14, 0xab, 0xc4, 0x14, 0xeb, 0xb3, 0x38, 0x12, 0xab, 0xb7, 0x2b, 0x56, 0xd5, 0x8c, 0x43, + 0x99, 0xc6, 0xfc, 0x5c, 0x84, 0xf5, 0x85, 0xef, 0x9b, 0x3b, 0x6a, 0x2e, 0x02, 0x9e, 0xad, 0xd7, + 0x22, 0x5b, 0xaf, 0xec, 0x4c, 0x6d, 0xe4, 0x85, 0xe7, 0xa7, 0x2b, 0x97, 0x9d, 0x17, 0x3a, 0xc3, + 0xff, 0xf7, 0x9d, 0x41, 0x6f, 0x41, 0xf5, 0x2b, 0x63, 0x62, 0x99, 0x46, 0x48, 0x4c, 0xf6, 0x15, + 0x5a, 0xd1, 0x2e, 0x0c, 0x49, 0xdf, 0xde, 0xfd, 0x1a, 0x96, 0xfa, 0xa1, 0x11, 0x2e, 0xb0, 0x18, + 0xa0, 0xdc, 0x92, 0xfb, 0xed, 0xee, 0x40, 0xe0, 0xd0, 0x3a, 0xac, 0x28, 0xed, 0x4e, 0x7b, 0xd0, + 0xd6, 0xf7, 0xbb, 0x9d, 0x67, 0x42, 0x11, 0xbd, 0x09, 0xaf, 0xa7, 0x86, 0x56, 0x57, 0xd1, 0x0f, + 0x35, 0x35, 0xfb, 0xab, 0x84, 0x6a, 0x00, 0x72, 0x6b, 0xef, 0xc9, 0x63, 0xb5, 0xd3, 0x69, 0x2b, + 0x02, 0x8f, 0xd6, 0xa0, 0xfa, 0xb4, 0xd5, 0x51, 0x95, 0xd6, 0xa0, 0xad, 0x08, 0x4b, 0x14, 0xb6, + 0x77, 0x20, 0x77, 0xd4, 0x3d, 0xa1, 0x2c, 0x3f, 0x38, 0xfd, 0xa3, 0x51, 0x38, 0x8d, 0x1b, 0xdc, + 0x59, 0xdc, 0xe0, 0x5e, 0xc6, 0x0d, 0xee, 0xf7, 0xb8, 0xc1, 0x7d, 0x77, 0xde, 0x28, 0x9c, 0x9d, + 0x37, 0x0a, 0x2f, 0xcf, 0x1b, 0x85, 0xe7, 0x3c, 0x2d, 0x6c, 0x58, 0x66, 0x1f, 0x9e, 0x1f, 0xfe, + 0x13, 0x00, 0x00, 0xff, 0xff, 0xad, 0xaa, 0xa4, 0x6b, 0x45, 0x0d, 0x00, 0x00, +} diff --git a/pkg/sql/schemachanger/scpb/scpb.proto b/pkg/sql/schemachanger/scpb/scpb.proto new file mode 100644 index 000000000000..a0dc68528a58 --- /dev/null +++ b/pkg/sql/schemachanger/scpb/scpb.proto @@ -0,0 +1,103 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +syntax = "proto3"; +package cockroach.sql.schemachanger.scpb; +option go_package = "scpb"; + +import "sql/catalog/descpb/structured.proto"; +import "gogoproto/gogo.proto"; + +message ElementProto { + option (gogoproto.onlyone) = true; + Column column = 1; + PrimaryIndex primary_index = 2; + SecondaryIndex secondary_index = 3; + SequenceDependency sequence_dependency = 4; + UniqueConstraint unique_constraint = 5; + CheckConstraint check_constraint = 6; +} + +message Target { + enum Direction { + UNKNOWN = 0; + ADD = 1; + DROP = 2; + } + + ElementProto element_proto = 1 [(gogoproto.embed) = true, (gogoproto.nullable) = false]; + Direction direction = 2; +} + +enum State { + UNKNOWN = 0; + ABSENT = 1; + DELETE_ONLY = 2; + DELETE_AND_WRITE_ONLY = 3; + BACKFILLED = 4; + VALIDATED = 5; + PUBLIC = 6; +} + +message Column { + option (gogoproto.equal) = true; + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; + uint32 family_id = 2 [(gogoproto.customname) = "FamilyID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.FamilyID"]; + string family_name = 3; + cockroach.sql.sqlbase.ColumnDescriptor column = 4 [(gogoproto.nullable) = false]; +} + +message PrimaryIndex { + option (gogoproto.equal) = true; + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; + cockroach.sql.sqlbase.IndexDescriptor index = 2 [(gogoproto.nullable) = false]; + uint32 other_primary_index_id = 3 [(gogoproto.customname) = "OtherPrimaryIndexID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; + repeated uint32 store_column_ids = 4 [(gogoproto.customname) = "StoreColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; + repeated string store_column_names = 5 [(gogoproto.customname) = "StoreColumnNames"]; +} + +message SecondaryIndex { + option (gogoproto.equal) = true; + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; + cockroach.sql.sqlbase.IndexDescriptor index = 2 [(gogoproto.nullable) = false]; + uint32 primary_index = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; + } + +message SequenceDependency { + option (gogoproto.equal) = true; + + enum Type { + UNKNOWN = 0; + USES = 1; + OWNS = 2; + } + + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; + uint32 column_id = 2 [(gogoproto.customname) = "ColumnID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; + uint32 sequence_id = 3 [(gogoproto.customname) = "SequenceID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; + Type type = 4; +} + +message UniqueConstraint { + option (gogoproto.equal) = true; + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; + uint32 index_id = 2 [(gogoproto.customname) = "IndexID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; + repeated uint32 column_ids = 3 [(gogoproto.customname) = "ColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; +} + +message CheckConstraint { + option (gogoproto.equal) = true; + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; + string name = 2; + string expr = 3; + repeated uint32 column_ids = 4 [(gogoproto.customname) = "ColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; + bool validated = 5; +} + From 957b1c8be9239f6b5e59cb462e13c532f80d9f89 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:02 -0500 Subject: [PATCH 03/15] schemachanger/scop: introduce schema changer operations `scop` defines `Op`s, which are specifications of low-level operations in schema change execution. Ops (grouped into stages) are the output of schema change planning. This package also generates interface definitions for operation visitors. This facilitates handling every `Op` implementation without missing cases. Release note: None --- pkg/sql/schemachanger/scop/BUILD.bazel | 32 ++++ pkg/sql/schemachanger/scop/backfill.go | 27 ++++ .../scop/backfill_visitor_generated.go | 31 ++++ .../schemachanger/scop/generate_visitor.go | 130 +++++++++++++++ pkg/sql/schemachanger/scop/mutation.go | 151 ++++++++++++++++++ .../scop/mutation_visitor_generated.go | 115 +++++++++++++ pkg/sql/schemachanger/scop/ops.go | 93 +++++++++++ pkg/sql/schemachanger/scop/type_string.go | 26 +++ pkg/sql/schemachanger/scop/validation.go | 34 ++++ .../scop/validation_visitor_generated.go | 37 +++++ 10 files changed, 676 insertions(+) create mode 100644 pkg/sql/schemachanger/scop/BUILD.bazel create mode 100644 pkg/sql/schemachanger/scop/backfill.go create mode 100644 pkg/sql/schemachanger/scop/backfill_visitor_generated.go create mode 100644 pkg/sql/schemachanger/scop/generate_visitor.go create mode 100644 pkg/sql/schemachanger/scop/mutation.go create mode 100644 pkg/sql/schemachanger/scop/mutation_visitor_generated.go create mode 100644 pkg/sql/schemachanger/scop/ops.go create mode 100644 pkg/sql/schemachanger/scop/type_string.go create mode 100644 pkg/sql/schemachanger/scop/validation.go create mode 100644 pkg/sql/schemachanger/scop/validation_visitor_generated.go diff --git a/pkg/sql/schemachanger/scop/BUILD.bazel b/pkg/sql/schemachanger/scop/BUILD.bazel new file mode 100644 index 000000000000..8e071f3a2a2d --- /dev/null +++ b/pkg/sql/schemachanger/scop/BUILD.bazel @@ -0,0 +1,32 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "ops", + srcs = ["ops.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/ops", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/catalog/descpb", + "//pkg/sql/schemachanger/targets", + ], +) + +go_library( + name = "scop", + srcs = [ + "backfill.go", + "backfill_visitor_generated.go", + "mutation.go", + "mutation_visitor_generated.go", + "ops.go", + "type_string.go", + "validation.go", + "validation_visitor_generated.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/catalog/descpb", + "@com_github_cockroachdb_errors//:errors", + ], +) diff --git a/pkg/sql/schemachanger/scop/backfill.go b/pkg/sql/schemachanger/scop/backfill.go new file mode 100644 index 000000000000..43d85713f7b5 --- /dev/null +++ b/pkg/sql/schemachanger/scop/backfill.go @@ -0,0 +1,27 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scop + +import "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + +//go:generate go run ./generate_visitor.go scop Backfill backfill.go backfill_visitor_generated.go + +type backfillOp struct{ baseOp } + +// Type implements the Op interface. +func (backfillOp) Type() Type { return BackfillType } + +// BackfillIndex specifies an index backfill operation. +type BackfillIndex struct { + backfillOp + TableID descpb.ID + IndexID descpb.IndexID +} diff --git a/pkg/sql/schemachanger/scop/backfill_visitor_generated.go b/pkg/sql/schemachanger/scop/backfill_visitor_generated.go new file mode 100644 index 000000000000..4df48d061104 --- /dev/null +++ b/pkg/sql/schemachanger/scop/backfill_visitor_generated.go @@ -0,0 +1,31 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// Code generated by generate_visitor.go. DO NOT EDIT. + +package scop + +import "context" + +// BackfillOp is an operation which can be visited by BackfillVisitor. +type BackfillOp interface { + Op + Visit(context.Context, BackfillVisitor) error +} + +// BackfillVisitor is a visitor for BackfillOp operations. +type BackfillVisitor interface { + BackfillIndex(context.Context, BackfillIndex) error +} + +// Visit is part of the BackfillOp interface. +func (op BackfillIndex) Visit(ctx context.Context, v BackfillVisitor) error { + return v.BackfillIndex(ctx, op) +} diff --git a/pkg/sql/schemachanger/scop/generate_visitor.go b/pkg/sql/schemachanger/scop/generate_visitor.go new file mode 100644 index 000000000000..aa654c04d268 --- /dev/null +++ b/pkg/sql/schemachanger/scop/generate_visitor.go @@ -0,0 +1,130 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// +build ignore + +package main + +import ( + "bytes" + "fmt" + "html/template" + "io/ioutil" + "os" + "regexp" + "strings" + + "github.com/cockroachdb/cockroach/pkg/cli/exit" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/gostdlib/go/format" +) + +func main() { + if err := run(); err != nil { + fmt.Fprintln(os.Stderr, "ERROR:", err) + exit.WithCode(exit.UnspecifiedError()) + } +} + +func run() error { + if len(os.Args) < 5 { + return errors.Newf("usage: %s \n", os.Args[0]) + } + pkg, opType, in, out := os.Args[1], os.Args[2], os.Args[3], os.Args[4] + + source, err := ioutil.ReadFile(in) + if err != nil { + return err + } + opPattern := regexp.MustCompile(`type (\w+) struct {`) + var ops []string + for _, line := range strings.Split(string(source), "\n") { + line = strings.TrimSpace(line) + if matches := opPattern.FindStringSubmatch(line); len(matches) > 0 { + ops = append(ops, matches[1]) + } + } + + tmpl, err := template.New("visitor").Parse(visitorTemplate) + if err != nil { + return err + } + + // Render the template. + var gen bytes.Buffer + if err := tmpl.Execute(&gen, info{ + Pkg: pkg, + Type: opType, + Ops: ops, + }); err != nil { + return err + } + + // Run gofmt on the generated source. + formatted, err := format.Source(gen.Bytes()) + if err != nil { + return errors.Wrap(err, "gofmt") + } + + // Write the output file. + f, err := os.OpenFile(out, os.O_CREATE|os.O_TRUNC|os.O_WRONLY, 0666) + if err != nil { + return err + } + defer func() { _ = f.Close() }() + if _, err := f.Write(formatted); err != nil { + return err + } + + return nil +} + +type info struct { + Pkg string + Type string + Ops []string +} + +const visitorTemplate = `// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// Code generated by generate_visitor.go. DO NOT EDIT. + +package {{.Pkg}} + +import "context" + +{{$type := .Type}} +// {{$type}}Op is an operation which can be visited by {{$type}}Visitor. +type {{$type}}Op interface { + Op + Visit(context.Context, {{$type}}Visitor) error +} + +// {{$type}}Visitor is a visitor for {{$type}}Op operations. +type {{$type}}Visitor interface { +{{range .Ops -}} + {{.}}(context.Context, {{.}}) error +{{end}} +} + +{{range .Ops}} +// Visit is part of the {{$type}}Op interface. +func (op {{.}}) Visit(ctx context.Context, v {{$type}}Visitor) error { + return v.{{.}}(ctx, op) +} +{{end}}` diff --git a/pkg/sql/schemachanger/scop/mutation.go b/pkg/sql/schemachanger/scop/mutation.go new file mode 100644 index 000000000000..3e07a33713f1 --- /dev/null +++ b/pkg/sql/schemachanger/scop/mutation.go @@ -0,0 +1,151 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scop + +import "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + +//go:generate go run ./generate_visitor.go scop Mutation mutation.go mutation_visitor_generated.go + +type mutationOp struct{ baseOp } + +func (mutationOp) Type() Type { return MutationType } + +// MakeAddedIndexDeleteOnly adds a non-existent primary index to the +// table. +type MakeAddedIndexDeleteOnly struct { + mutationOp + TableID descpb.ID + + // Index represents the index as it should appear in the mutation. + Index descpb.IndexDescriptor +} + +// MakeAddedIndexDeleteAndWriteOnly transitions an index addition mutation from +// DELETE_ONLY to DELETE_AND_WRITE_ONLY. +type MakeAddedIndexDeleteAndWriteOnly struct { + mutationOp + TableID descpb.ID + IndexID descpb.IndexID +} + +// MakeAddedPrimaryIndexPublic moves a new primary index from its mutation to +// public. +type MakeAddedPrimaryIndexPublic struct { + mutationOp + TableID descpb.ID + Index descpb.IndexDescriptor +} + +// MakeDroppedPrimaryIndexDeleteAndWriteOnly moves a dropped primary index from +// public to DELETE_AND_WRITE_ONLY. +type MakeDroppedPrimaryIndexDeleteAndWriteOnly struct { + mutationOp + TableID descpb.ID + + // Index is the descriptor as it should be added as part of the mutation. The + // primary index of a table has a slightly different encoding than that of + // a secondary index. The value here sets it as it should be when adding + // the mutation, including the stored columns. + Index descpb.IndexDescriptor +} + +// MakeAddedColumnDeleteAndWriteOnly transitions a column addition mutation from +// DELETE_ONLY to DELETE_AND_WRITE_ONLY. +type MakeAddedColumnDeleteAndWriteOnly struct { + mutationOp + TableID descpb.ID + ColumnID descpb.ColumnID +} + +// MakeDroppedNonPrimaryIndexDeleteAndWriteOnly moves a dropped secondary index +// from public to DELETE_AND_WRITE_ONLY. +type MakeDroppedNonPrimaryIndexDeleteAndWriteOnly struct { + mutationOp + TableID descpb.ID + IndexID descpb.IndexID +} + +// MakeDroppedIndexDeleteOnly transitions an index drop mutation from +// DELETE_AND_WRITE_ONLY to DELETE_ONLY. +type MakeDroppedIndexDeleteOnly struct { + mutationOp + TableID descpb.ID + IndexID descpb.IndexID +} + +// MakeIndexAbsent removes a dropped index mutation in DELETE_ONLY from the +// table. +type MakeIndexAbsent struct { + mutationOp + TableID descpb.ID + IndexID descpb.IndexID +} + +// MakeAddedColumnDeleteOnly adds a new column in the DELETE_ONLY state. +type MakeAddedColumnDeleteOnly struct { + mutationOp + TableID descpb.ID + FamilyID descpb.FamilyID + FamilyName string + Column descpb.ColumnDescriptor +} + +// MakeColumnPublic moves a new column from its mutation to public. +type MakeColumnPublic struct { + mutationOp + TableID descpb.ID + ColumnID descpb.ColumnID +} + +// MakeDroppedColumnDeleteAndWriteOnly moves a dropped column from public to +// DELETE_AND_WRITE_ONLY. +type MakeDroppedColumnDeleteAndWriteOnly struct { + mutationOp + TableID descpb.ID + ColumnID descpb.ColumnID +} + +// MakeDroppedColumnDeleteOnly transitions a column drop mutation from +// DELETE_AND_WRITE_ONLY to DELETE_ONLY. +type MakeDroppedColumnDeleteOnly struct { + mutationOp + TableID descpb.ID + ColumnID descpb.ColumnID +} + +// MakeColumnAbsent removes a dropped column mutation in DELETE_ONLY from the +// table. +type MakeColumnAbsent struct { + mutationOp + TableID descpb.ID + ColumnID descpb.ColumnID +} + +// AddCheckConstraint adds a check constraint in the unvalidated state. +type AddCheckConstraint struct { + mutationOp + TableID descpb.ID + Name string + Expr string + ColumnIDs descpb.ColumnIDs + Unvalidated bool + Hidden bool +} + +// AddColumnFamily adds a column family with the provided descriptor. +// +// TODO(ajwerner): Decide whether this should happen explicitly or should be a +// side-effect of adding a column. My hunch is the latter. +type AddColumnFamily struct { + mutationOp + TableID descpb.ID + Family descpb.ColumnFamilyDescriptor +} diff --git a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go new file mode 100644 index 000000000000..ee58cc9d19a8 --- /dev/null +++ b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go @@ -0,0 +1,115 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// Code generated by generate_visitor.go. DO NOT EDIT. + +package scop + +import "context" + +// MutationOp is an operation which can be visited by MutationVisitor. +type MutationOp interface { + Op + Visit(context.Context, MutationVisitor) error +} + +// MutationVisitor is a visitor for MutationOp operations. +type MutationVisitor interface { + MakeAddedIndexDeleteOnly(context.Context, MakeAddedIndexDeleteOnly) error + MakeAddedIndexDeleteAndWriteOnly(context.Context, MakeAddedIndexDeleteAndWriteOnly) error + MakeAddedPrimaryIndexPublic(context.Context, MakeAddedPrimaryIndexPublic) error + MakeDroppedPrimaryIndexDeleteAndWriteOnly(context.Context, MakeDroppedPrimaryIndexDeleteAndWriteOnly) error + MakeAddedColumnDeleteAndWriteOnly(context.Context, MakeAddedColumnDeleteAndWriteOnly) error + MakeDroppedNonPrimaryIndexDeleteAndWriteOnly(context.Context, MakeDroppedNonPrimaryIndexDeleteAndWriteOnly) error + MakeDroppedIndexDeleteOnly(context.Context, MakeDroppedIndexDeleteOnly) error + MakeIndexAbsent(context.Context, MakeIndexAbsent) error + MakeAddedColumnDeleteOnly(context.Context, MakeAddedColumnDeleteOnly) error + MakeColumnPublic(context.Context, MakeColumnPublic) error + MakeDroppedColumnDeleteAndWriteOnly(context.Context, MakeDroppedColumnDeleteAndWriteOnly) error + MakeDroppedColumnDeleteOnly(context.Context, MakeDroppedColumnDeleteOnly) error + MakeColumnAbsent(context.Context, MakeColumnAbsent) error + AddCheckConstraint(context.Context, AddCheckConstraint) error + AddColumnFamily(context.Context, AddColumnFamily) error +} + +// Visit is part of the MutationOp interface. +func (op MakeAddedIndexDeleteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeAddedIndexDeleteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeAddedIndexDeleteAndWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeAddedIndexDeleteAndWriteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeAddedPrimaryIndexPublic) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeAddedPrimaryIndexPublic(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeDroppedPrimaryIndexDeleteAndWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeDroppedPrimaryIndexDeleteAndWriteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeAddedColumnDeleteAndWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeAddedColumnDeleteAndWriteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeDroppedNonPrimaryIndexDeleteAndWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeDroppedNonPrimaryIndexDeleteAndWriteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeDroppedIndexDeleteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeDroppedIndexDeleteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeIndexAbsent) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeIndexAbsent(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeAddedColumnDeleteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeAddedColumnDeleteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeColumnPublic) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeColumnPublic(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeDroppedColumnDeleteAndWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeDroppedColumnDeleteAndWriteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeDroppedColumnDeleteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeDroppedColumnDeleteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeColumnAbsent) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeColumnAbsent(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op AddCheckConstraint) Visit(ctx context.Context, v MutationVisitor) error { + return v.AddCheckConstraint(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op AddColumnFamily) Visit(ctx context.Context, v MutationVisitor) error { + return v.AddColumnFamily(ctx, op) +} diff --git a/pkg/sql/schemachanger/scop/ops.go b/pkg/sql/schemachanger/scop/ops.go new file mode 100644 index 000000000000..a13665a7f80b --- /dev/null +++ b/pkg/sql/schemachanger/scop/ops.go @@ -0,0 +1,93 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scop + +import "github.com/cockroachdb/errors" + +// Op represents an action to be taken on a single descriptor. +type Op interface { + op() + Type() Type +} + +// Ops represents a slice of operations where all operations have the +// same type. +type Ops interface { + Type() Type + Slice() []Op +} + +// MakeOps takes a slice of ops, ensures they are all of one kind, and then +// returns an implementation of Ops corresponding to that type. The set of ops +// must all be the same type, otherwise MakeOps will panic. +func MakeOps(ops ...Op) Ops { + var typ Type + for i, op := range ops { + if i == 0 { + typ = op.Type() + continue + } + if op.Type() != typ { + panic(errors.Errorf( + "slice contains ops of type %s and %s", op.Type().String(), op)) + } + } + switch typ { + case MutationType: + return mutationOps(ops) + case BackfillType: + return backfillOps(ops) + case ValidationType: + return validationOps(ops) + default: + panic(errors.Errorf("unknown op type %s", typ.String())) + } +} + +// Type represents the type of operation for an Op. Ops can be grouped into the +// the same Stage only if they share a type. +type Type int + +//go:generate stringer -type=Type + +const ( + _ Type = iota + // MutationType represents descriptor changes. + MutationType + // BackfillType represents index backfills. + BackfillType + // ValidationType represents constraint and unique index validations + // performed using internal queries. + ValidationType +) + +type mutationOps []Op + +func (m mutationOps) Type() Type { return MutationType } +func (m mutationOps) Slice() []Op { return m } + +type backfillOps []Op + +func (b backfillOps) Type() Type { return BackfillType } +func (b backfillOps) Slice() []Op { return b } + +type validationOps []Op + +func (v validationOps) Type() Type { return ValidationType } +func (v validationOps) Slice() []Op { return v } + +var _ Ops = (mutationOps)(nil) +var _ Ops = (backfillOps)(nil) +var _ Ops = (validationOps)(nil) + +type baseOp struct{} + +func (baseOp) op() {} diff --git a/pkg/sql/schemachanger/scop/type_string.go b/pkg/sql/schemachanger/scop/type_string.go new file mode 100644 index 000000000000..a59b1bfbc30e --- /dev/null +++ b/pkg/sql/schemachanger/scop/type_string.go @@ -0,0 +1,26 @@ +// Code generated by "stringer -type=Type"; DO NOT EDIT. + +package scop + +import "strconv" + +func _() { + // An "invalid array index" compiler error signifies that the constant values have changed. + // Re-run the stringer command to generate them again. + var x [1]struct{} + _ = x[MutationType-1] + _ = x[BackfillType-2] + _ = x[ValidationType-3] +} + +const _Type_name = "MutationTypeBackfillTypeValidationType" + +var _Type_index = [...]uint8{0, 12, 24, 38} + +func (i Type) String() string { + i -= 1 + if i < 0 || i >= Type(len(_Type_index)-1) { + return "Type(" + strconv.FormatInt(int64(i+1), 10) + ")" + } + return _Type_name[_Type_index[i]:_Type_index[i+1]] +} diff --git a/pkg/sql/schemachanger/scop/validation.go b/pkg/sql/schemachanger/scop/validation.go new file mode 100644 index 000000000000..5e8c24b52b5e --- /dev/null +++ b/pkg/sql/schemachanger/scop/validation.go @@ -0,0 +1,34 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scop + +import "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + +//go:generate go run ./generate_visitor.go scop Validation validation.go validation_visitor_generated.go + +type validationOp struct{ baseOp } + +func (validationOp) Type() Type { return ValidationType } + +// ValidateUniqueIndex validates uniqueness of entries for a unique index. +type ValidateUniqueIndex struct { + validationOp + TableID descpb.ID + PrimaryIndexID descpb.IndexID + IndexID descpb.IndexID +} + +// ValidateCheckConstraint validates a check constraint on a table's columns. +type ValidateCheckConstraint struct { + validationOp + TableID descpb.ID + Name string +} diff --git a/pkg/sql/schemachanger/scop/validation_visitor_generated.go b/pkg/sql/schemachanger/scop/validation_visitor_generated.go new file mode 100644 index 000000000000..eae21486426e --- /dev/null +++ b/pkg/sql/schemachanger/scop/validation_visitor_generated.go @@ -0,0 +1,37 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// Code generated by generate_visitor.go. DO NOT EDIT. + +package scop + +import "context" + +// ValidationOp is an operation which can be visited by ValidationVisitor. +type ValidationOp interface { + Op + Visit(context.Context, ValidationVisitor) error +} + +// ValidationVisitor is a visitor for ValidationOp operations. +type ValidationVisitor interface { + ValidateUniqueIndex(context.Context, ValidateUniqueIndex) error + ValidateCheckConstraint(context.Context, ValidateCheckConstraint) error +} + +// Visit is part of the ValidationOp interface. +func (op ValidateUniqueIndex) Visit(ctx context.Context, v ValidationVisitor) error { + return v.ValidateUniqueIndex(ctx, op) +} + +// Visit is part of the ValidationOp interface. +func (op ValidateCheckConstraint) Visit(ctx context.Context, v ValidationVisitor) error { + return v.ValidateCheckConstraint(ctx, op) +} From 697d6093c1be4e4660586b120056e17e97af683d Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:02 -0500 Subject: [PATCH 04/15] schemachanger/scgraph: add library for schema change plan graphs `scgraph` defines the `Graph`, which is constructed in the process of planning schema changes and encodes all dependencies needed to eventually construct ops and stages. Release note: None --- pkg/sql/schemachanger/scgraph/BUILD.bazel | 17 ++ pkg/sql/schemachanger/scgraph/graph.go | 176 +++++++++++++++++++++ pkg/sql/schemachanger/scgraph/iteration.go | 89 +++++++++++ 3 files changed, 282 insertions(+) create mode 100644 pkg/sql/schemachanger/scgraph/BUILD.bazel create mode 100644 pkg/sql/schemachanger/scgraph/graph.go create mode 100644 pkg/sql/schemachanger/scgraph/iteration.go diff --git a/pkg/sql/schemachanger/scgraph/BUILD.bazel b/pkg/sql/schemachanger/scgraph/BUILD.bazel new file mode 100644 index 000000000000..7cecd02d1c59 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/BUILD.bazel @@ -0,0 +1,17 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "scgraph", + srcs = [ + "graph.go", + "iteration.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/schemachanger/scop", + "//pkg/sql/schemachanger/scpb", + "//pkg/util/iterutil", + "@com_github_cockroachdb_errors//:errors", + ], +) diff --git a/pkg/sql/schemachanger/scgraph/graph.go b/pkg/sql/schemachanger/scgraph/graph.go new file mode 100644 index 000000000000..013fb6bcf4d9 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/graph.go @@ -0,0 +1,176 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scgraph + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/errors" +) + +// Graph is a graph whose nodes are *scpb.Nodes. Graphs are constructed during +// schema change planning. Edges in the graph represent dependencies between +// nodes, either due to the sequencing of states for a single target or due to +// inter-target dependencies between states. +type Graph struct { + + // Targets is an interned slice of targets. + targets []*scpb.Target + + // Interns the Node so that pointer equality can be used. + targetNodes []map[scpb.State]*scpb.Node + + // Maps a target to its index in targetNodes. + targetIdxMap map[*scpb.Target]int + + // nodeOpEdges maps a Node to an opEdge that proceeds + // from it. A Node may have at most one opEdge from it. + nodeOpEdges map[*scpb.Node]*OpEdge + + // nodeDepEdges maps a Node to its dependencies. + // A Node dependency is another target state which must be + // reached before or concurrently with this targetState. + nodeDepEdges map[*scpb.Node][]*DepEdge + + edges []Edge +} + +// New constructs a new Graph. All initial nodes ought to correspond to distinct +// targets. If they do not, an error will be returned. +func New(initialNodes []*scpb.Node) (*Graph, error) { + g := Graph{ + targetIdxMap: map[*scpb.Target]int{}, + nodeOpEdges: map[*scpb.Node]*OpEdge{}, + nodeDepEdges: map[*scpb.Node][]*DepEdge{}, + } + for _, n := range initialNodes { + if existing, ok := g.targetIdxMap[n.Target]; ok { + return nil, errors.Errorf("invalid initial states contains duplicate target: %v and %v", n, initialNodes[existing]) + } + idx := len(g.targets) + g.targetIdxMap[n.Target] = idx + g.targets = append(g.targets, n.Target) + g.targetNodes = append(g.targetNodes, map[scpb.State]*scpb.Node{ + n.State: n, + }) + } + return &g, nil +} + +func (g *Graph) getNode(t *scpb.Target, s scpb.State) (*scpb.Node, bool) { + targetStates := g.getTargetStatesMap(t) + ts, ok := targetStates[s] + return ts, ok +} + +// Suppress the linter. +var _ = (*Graph)(nil).getNode + +func (g *Graph) getOrCreateNode(t *scpb.Target, s scpb.State) *scpb.Node { + targetStates := g.getTargetStatesMap(t) + if ts, ok := targetStates[s]; ok { + return ts + } + ts := &scpb.Node{ + Target: t, + State: s, + } + targetStates[s] = ts + return ts +} + +func (g *Graph) getTargetStatesMap(target *scpb.Target) map[scpb.State]*scpb.Node { + idx, ok := g.targetIdxMap[target] + if !ok { + panic(errors.Errorf("target %v does not exist", target)) + } + return g.targetNodes[idx] +} + +func (g *Graph) containsTarget(target *scpb.Target) bool { + _, exists := g.targetIdxMap[target] + return exists +} + +// Suppress the linter. +var _ = (*Graph)(nil).containsTarget + +// GetOpEdgeFrom returns the unique outgoing op edge from the specified node, +// if one exists. +func (g *Graph) GetOpEdgeFrom(n *scpb.Node) (*OpEdge, bool) { + oe, ok := g.nodeOpEdges[n] + return oe, ok +} + +// AddOpEdge adds an op edge connecting the nodes for two states of a target. +func (g *Graph) AddOpEdge(t *scpb.Target, from, to scpb.State, op scop.Op) { + oe := &OpEdge{ + from: g.getOrCreateNode(t, from), + to: g.getOrCreateNode(t, to), + op: op, + } + if existing, exists := g.nodeOpEdges[oe.from]; exists { + panic(errors.Errorf("duplicate outbound op edge %v and %v", + oe, existing)) + } + g.edges = append(g.edges, oe) + g.nodeOpEdges[oe.from] = oe +} + +// AddDepEdge adds a dep edge connecting two nodes (specified by their targets +// and states). +func (g *Graph) AddDepEdge( + fromTarget *scpb.Target, fromState scpb.State, toTarget *scpb.Target, toState scpb.State, +) { + de := &DepEdge{ + from: g.getOrCreateNode(fromTarget, fromState), + to: g.getOrCreateNode(toTarget, toState), + } + g.edges = append(g.edges, de) + g.nodeDepEdges[de.from] = append(g.nodeDepEdges[de.from], de) +} + +// Edge represents a relationship between two TargetStates. +// +// TODO(ajwerner): Consider hiding Node pointers behind an interface to clarify +// mutability. +type Edge interface { + From() *scpb.Node + To() *scpb.Node +} + +// OpEdge represents an edge changing the state of a target with an op. +type OpEdge struct { + from, to *scpb.Node + op scop.Op +} + +// From implements the Edge interface. +func (oe *OpEdge) From() *scpb.Node { return oe.from } + +// To implements the Edge interface. +func (oe *OpEdge) To() *scpb.Node { return oe.to } + +// Op returns the scop.Op for execution that is associated with the op edge. +func (oe *OpEdge) Op() scop.Op { return oe.op } + +// DepEdge represents a dependency between two target states. A dependency +// implies that the To() state cannot be reached before the From() state. It +// can be reached concurrently. +type DepEdge struct { + from, to *scpb.Node +} + +// From implements the Edge interface. +func (de *DepEdge) From() *scpb.Node { return de.from } + +// To implements the Edge interface. +func (de *DepEdge) To() *scpb.Node { return de.to } diff --git a/pkg/sql/schemachanger/scgraph/iteration.go b/pkg/sql/schemachanger/scgraph/iteration.go new file mode 100644 index 000000000000..7f90a36de070 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/iteration.go @@ -0,0 +1,89 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scgraph + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/util/iterutil" +) + +// TargetIterator is used to iterate targets. Return iterutil.StopIteration to +// return early with no error. +type TargetIterator func(t *scpb.Target) error + +// ForEachTarget iterates the targets in the graph. +func (g *Graph) ForEachTarget(it TargetIterator) error { + for _, t := range g.targets { + if err := it(t); err != nil { + if iterutil.Done(err) { + err = nil + } + return err + } + } + return nil +} + +// NodeIterator is used to iterate nodes. Return iterutil.StopIteration to +// return early with no error. +type NodeIterator func(n *scpb.Node) error + +// ForEachNode iterates the nodes in the graph. +func (g *Graph) ForEachNode(it NodeIterator) error { + for _, m := range g.targetNodes { + for i := 0; i < scpb.NumStates; i++ { + if ts, ok := m[scpb.State(i)]; ok { + if err := it(ts); err != nil { + if iterutil.Done(err) { + err = nil + } + return err + } + } + } + } + return nil +} + +// EdgeIterator is used to iterate edges. Return iterutil.StopIteration to +// return early with no error. +type EdgeIterator func(e Edge) error + +// ForEachEdge iterates the edges in the graph. +func (g *Graph) ForEachEdge(it EdgeIterator) error { + for _, e := range g.edges { + if err := it(e); err != nil { + if iterutil.Done(err) { + err = nil + } + return err + } + } + return nil +} + +// DepEdgeIterator is used to iterate dep edges. Return iterutil.StopIteration +// to return early with no error. +type DepEdgeIterator func(de *DepEdge) error + +// ForEachDepEdgeFrom iterates the dep edges in the graph. +func (g *Graph) ForEachDepEdgeFrom(n *scpb.Node, it DepEdgeIterator) error { + edges := g.nodeDepEdges[n] + for _, e := range edges { + if err := it(e); err != nil { + if iterutil.Done(err) { + err = nil + } + return err + } + } + return nil +} From 7a5993f10b0f708c98a18826ad1e5c458ce68843 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:02 -0500 Subject: [PATCH 05/15] schemachanger/scbuild: add library to convert AST nodes to schema elements `scbuild` contains the `Builder`, which takes AST nodes of DDL statements and converts them to specifications of schema elements to be added, modified, or removed. The set of nodes built up from one or more invocations of the builder becomes the input to subsequent planning steps: building a graph, and generating ops and stages. This package reimplements much of the logic in sql/alter_table.go and its friends, but adapted for generating declarative specifications of schema elements instead of directly mutating table descriptors. This commit also introduces data-driven tests for builder output. Release note: None --- pkg/sql/catalog/descriptor.go | 1 + pkg/sql/schemachanger/scbuild/BUILD.bazel | 116 +++ pkg/sql/schemachanger/scbuild/builder.go | 668 ++++++++++++++++++ pkg/sql/schemachanger/scbuild/builder_test.go | 155 ++++ pkg/sql/schemachanger/scbuild/main_test.go | 31 + .../scbuild/testdata/alter_table | 444 ++++++++++++ .../testdata/alter_table_unimplemented | 84 +++ pkg/testutils/lint/lint_test.go | 1 + 8 files changed, 1500 insertions(+) create mode 100644 pkg/sql/schemachanger/scbuild/BUILD.bazel create mode 100644 pkg/sql/schemachanger/scbuild/builder.go create mode 100644 pkg/sql/schemachanger/scbuild/builder_test.go create mode 100644 pkg/sql/schemachanger/scbuild/main_test.go create mode 100644 pkg/sql/schemachanger/scbuild/testdata/alter_table create mode 100644 pkg/sql/schemachanger/scbuild/testdata/alter_table_unimplemented diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 80366f5db3d9..dbd7e1d22b7e 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -233,6 +233,7 @@ type TableDescriptor interface { NumFamilies() int FindFamilyByID(id descpb.FamilyID) (*descpb.ColumnFamilyDescriptor, error) ForeachFamily(f func(family *descpb.ColumnFamilyDescriptor) error) error + GetNextFamilyID() descpb.FamilyID IsTable() bool IsView() bool diff --git a/pkg/sql/schemachanger/scbuild/BUILD.bazel b/pkg/sql/schemachanger/scbuild/BUILD.bazel new file mode 100644 index 000000000000..b4460c79298a --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/BUILD.bazel @@ -0,0 +1,116 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "builder", + srcs = ["builder.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/builder", + visibility = ["//visibility:public"], + deps = [ + "//pkg/clusterversion", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/resolver", + "//pkg/sql/catalog/schemaexpr", + "//pkg/sql/catalog/tabledesc", + "//pkg/sql/parser", + "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", + "//pkg/sql/schemachanger/targets", + "//pkg/sql/sem/tree", + "//pkg/sql/sqlerrors", + "//pkg/sql/types", + "//pkg/util/errorutil/unimplemented", + "//pkg/util/protoutil", + "//pkg/util/sequence", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "builder_test", + srcs = [ + "builder_test.go", + "main_test.go", + ], + deps = [ + ":builder", + "//pkg/base", + "//pkg/kv", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/sql", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/resolver", + "//pkg/sql/parser", + "//pkg/sql/schemachanger/targets", + "//pkg/sql/sem/tree", + "//pkg/sql/sessiondatapb", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/randutil", + "@com_github_stretchr_testify//require", + ], +) + +go_library( + name = "scbuild", + srcs = ["builder.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild", + visibility = ["//visibility:public"], + deps = [ + "//pkg/clusterversion", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/resolver", + "//pkg/sql/catalog/schemaexpr", + "//pkg/sql/catalog/tabledesc", + "//pkg/sql/parser", + "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", + "//pkg/sql/schemachanger/scpb", + "//pkg/sql/sem/tree", + "//pkg/sql/sqlerrors", + "//pkg/sql/types", + "//pkg/util/errorutil/unimplemented", + "//pkg/util/protoutil", + "//pkg/util/sequence", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "scbuild_test", + srcs = [ + "builder_test.go", + "main_test.go", + ], + data = glob(["testdata/**"]), + deps = [ + ":scbuild", + "//pkg/base", + "//pkg/kv", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/sql", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/resolver", + "//pkg/sql/parser", + "//pkg/sql/schemachanger/scpb", + "//pkg/sql/sem/tree", + "//pkg/sql/sessiondatapb", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/randutil", + "@com_github_cockroachdb_datadriven//:datadriven", + "@com_github_gogo_protobuf//jsonpb", + "@com_github_stretchr_testify//require", + "@in_gopkg_yaml_v2//:yaml_v2", + ], +) diff --git a/pkg/sql/schemachanger/scbuild/builder.go b/pkg/sql/schemachanger/scbuild/builder.go new file mode 100644 index 000000000000..c1f57c0f7f9d --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/builder.go @@ -0,0 +1,668 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scbuild + +import ( + "context" + "fmt" + "strings" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/sequence" + "github.com/cockroachdb/errors" +) + +// TODO(ajwerner): Eliminate all panics or add principled recovery. +// TODO(ajwerner): Add privilege checking. + +// The Builder is the entry point for planning schema changes. From AST nodes +// for DDL statements, it constructs targets which represent schema changes to +// be performed. +// +// The builder itself is essentially stateless aside from the dependencies it +// needs to resolve (immutable) descriptors, evaluate expressions, etc. The +// methods in its API take schema change graph nodes (i.e., targets and their +// current states) and DDL statement AST nodes, and output new schema change +// graph nodes that incorporate targets that were added or changed. +type Builder struct { + // TODO(ajwerner): Inject a better interface than this. + res resolver.SchemaResolver + semaCtx *tree.SemaContext + evalCtx *tree.EvalContext + + // nodes contains the internal state when building targets for an individual + // statement. + nodes []*scpb.Node +} + +type notImplementedError struct { + n tree.NodeFormatter + detail string +} + +// TODO(ajwerner): Deal with redaction. + +var _ error = (*notImplementedError)(nil) + +// HasNotImplemented returns true if the error indicates that the builder does +// not support the provided statement. +func HasNotImplemented(err error) bool { + return errors.HasType(err, (*notImplementedError)(nil)) +} + +func (e *notImplementedError) Error() string { + var buf strings.Builder + fmt.Fprintf(&buf, "%T not implemented in the new schema changer", e.n) + if e.detail != "" { + fmt.Fprintf(&buf, ": %s", e.detail) + } + return buf.String() +} + +// NewBuilder creates a new Builder. +func NewBuilder( + res resolver.SchemaResolver, semaCtx *tree.SemaContext, evalCtx *tree.EvalContext, +) *Builder { + return &Builder{ + res: res, + semaCtx: semaCtx, + evalCtx: evalCtx, + } +} + +// Build builds targets and transforms the provided schema change nodes +// accordingly, given a statement. +// +// TODO(ajwerner): Clarify whether the nodes will be mutated. Potentially just +// clone them defensively here. Similarly, close the statement as some schema +// changes mutate the AST. It's best if this method had a clear contract that +// it did not mutate its arguments. +func (b *Builder) Build( + ctx context.Context, nodes []*scpb.Node, n tree.Statement, +) ([]*scpb.Node, error) { + switch n := n.(type) { + case *tree.AlterTable: + return b.AlterTable(ctx, nodes, n) + default: + return nil, ¬ImplementedError{n: n} + } +} + +// AlterTable builds targets and transforms the provided schema change nodes +// accordingly, given an ALTER TABLE statement. +func (b *Builder) AlterTable( + ctx context.Context, nodes []*scpb.Node, n *tree.AlterTable, +) ([]*scpb.Node, error) { + // TODO (lucy): Clean this up. + b.nodes = nodes + defer func() { + b.nodes = nil + }() + + // Hoist the constraints to separate clauses because other code assumes that + // that is how the commands will look. + // + // TODO(ajwerner): Clone the AST here because this mutates it in place and + // that is bad. + n.HoistAddColumnConstraints() + + // Resolve the table. + tn := n.Table.ToTableName() + table, err := resolver.ResolveExistingTableObject(ctx, b.res, &tn, + tree.ObjectLookupFlagsWithRequired()) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) && n.IfExists { + return nodes, nil + } + return nil, err + } + for _, cmd := range n.Cmds { + if err := b.alterTableCmd(ctx, table, cmd, &tn); err != nil { + return nil, err + } + } + + result := make([]*scpb.Node, len(b.nodes)) + for i := range b.nodes { + result[i] = b.nodes[i] + } + return result, nil +} + +func (b *Builder) alterTableCmd( + ctx context.Context, table catalog.TableDescriptor, cmd tree.AlterTableCmd, tn *tree.TableName, +) error { + switch t := cmd.(type) { + case *tree.AlterTableAddColumn: + return b.alterTableAddColumn(ctx, table, t, tn) + default: + return ¬ImplementedError{n: cmd} + } +} + +func (b *Builder) alterTableAddColumn( + ctx context.Context, + table catalog.TableDescriptor, + t *tree.AlterTableAddColumn, + tn *tree.TableName, +) error { + d := t.ColumnDef + + version := b.evalCtx.Settings.Version.ActiveVersionOrEmpty(ctx) + toType, err := tree.ResolveType(ctx, d.Type, b.semaCtx.GetTypeResolver()) + if err != nil { + return err + } + if supported, err := isTypeSupportedInVersion(version, toType); err != nil { + return err + } else if !supported { + return pgerror.Newf( + pgcode.FeatureNotSupported, + "type %s is not supported until version upgrade is finalized", + toType.SQLString(), + ) + } + + if d.IsSerial { + return ¬ImplementedError{n: t.ColumnDef, detail: "contains serial data type"} + } + // Some of the building for the index exists below but end-to-end support is + // not complete so we return an error. + if d.Unique.IsUnique { + return ¬ImplementedError{n: t.ColumnDef, detail: "contains unique constraint"} + } + col, idx, defaultExpr, err := tabledesc.MakeColumnDefDescs(ctx, d, b.semaCtx, b.evalCtx) + if err != nil { + return err + } + colID := b.nextColumnID(table) + col.ID = colID + + // If the new column has a DEFAULT expression that uses a sequence, add + // references between its descriptor and this column descriptor. + if d.HasDefaultExpr() { + if err := b.maybeAddSequenceReferenceDependencies( + ctx, table.GetID(), col, defaultExpr, + ); err != nil { + return err + } + } + + if err := b.validateColumnName(table, d, col, t.IfNotExists); err != nil { + return err + } + + familyID := descpb.FamilyID(0) + familyName := string(d.Family.Name) + // TODO(ajwerner,lucy-zhang): Figure out how to compute the default column ID + // for the family. + if d.HasColumnFamily() { + if familyID, err = b.findOrAddColumnFamily( + table, familyName, d.Family.Create, d.Family.IfNotExists, + ); err != nil { + return err + } + } else { + // TODO(ajwerner,lucy-zhang): Deal with adding the first column to the + // table. + fam := table.GetFamilies()[0] + familyID = fam.ID + familyName = fam.Name + } + + if d.IsComputed() { + if d.IsVirtual() { + return unimplemented.NewWithIssue(57608, "virtual computed columns") + } + + // TODO (lucy): This is not going to work when the computed column + // references columns created in the same transaction. + computedColValidator := schemaexpr.MakeComputedColumnValidator( + ctx, + table, + b.semaCtx, + tn, + ) + serializedExpr, err := computedColValidator.Validate(d) + if err != nil { + return err + } + col.ComputeExpr = &serializedExpr + } + + b.addNode(scpb.Target_ADD, &scpb.Column{ + TableID: table.GetID(), + Column: *col, + FamilyID: familyID, + FamilyName: familyName, + }) + newPrimaryIdxID := b.addOrUpdatePrimaryIndexTargetsForAddColumn(table, colID, col.Name) + + if idx != nil { + idxID := b.nextIndexID(table) + idx.ID = idxID + b.addNode(scpb.Target_ADD, &scpb.SecondaryIndex{ + TableID: table.GetID(), + Index: *idx, + PrimaryIndex: newPrimaryIdxID, + }) + } + return nil +} + +func (b *Builder) validateColumnName( + table catalog.TableDescriptor, + d *tree.ColumnTableDef, + col *descpb.ColumnDescriptor, + ifNotExists bool, +) error { + _, err := table.FindActiveColumnByName(string(d.Name)) + if err == nil { + if ifNotExists { + return nil + } + return sqlerrors.NewColumnAlreadyExistsError(string(d.Name), table.GetName()) + } + for _, n := range b.nodes { + switch t := n.Element().(type) { + case *scpb.Column: + if t.TableID != table.GetID() || t.Column.Name != string(d.Name) { + continue + } + switch dir := n.Target.Direction; dir { + case scpb.Target_ADD: + return pgerror.Newf(pgcode.DuplicateColumn, + "duplicate: column %q in the middle of being added, not yet public", + col.Name) + case scpb.Target_DROP: + return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, + "column %q being dropped, try again later", col.Name) + default: + return errors.AssertionFailedf("unknown direction %v in %v", dir, n.Target) + } + } + } + return nil +} + +func (b *Builder) findOrAddColumnFamily( + table catalog.TableDescriptor, family string, create bool, ifNotExists bool, +) (descpb.FamilyID, error) { + if len(family) > 0 { + for i := range table.GetFamilies() { + f := &table.GetFamilies()[i] + if f.Name == family { + if create && !ifNotExists { + return 0, errors.Errorf("family %q already exists", family) + } + return f.ID, nil + } + } + } + // See if we're in the process of adding a column or dropping a column in this + // family. + // + // TODO(ajwerner): Decide what to do if the only column in a family of this + // name is being dropped and then if there is or isn't a create directive. + nextFamilyID := table.GetNextFamilyID() + for _, n := range b.nodes { + switch col := n.Element().(type) { + case *scpb.Column: + if col.TableID != table.GetID() { + continue + } + if col.FamilyName == family { + if create && !ifNotExists { + return 0, errors.Errorf("family %q already exists", family) + } + return col.FamilyID, nil + } + if col.FamilyID >= nextFamilyID { + nextFamilyID = col.FamilyID + 1 + } + } + } + if !create { + return 0, errors.Errorf("unknown family %q", family) + } + return nextFamilyID, nil +} + +func (b *Builder) alterTableDropColumn( + ctx context.Context, table catalog.TableDescriptor, t *tree.AlterTableDropColumn, +) error { + if b.evalCtx.SessionData.SafeUpdates { + return pgerror.DangerousStatementf("ALTER TABLE DROP COLUMN will " + + "remove all data in that column") + } + + // TODO(ajwerner): Deal with drop column for columns which are being added + // currently. + colToDrop, _, err := table.FindColumnByName(t.Column) + if err != nil { + if t.IfExists { + // Noop. + return nil + } + return err + } + // Check whether the column is being dropped. + for _, n := range b.nodes { + switch col := n.Element().(type) { + case *scpb.Column: + if col.TableID != table.GetID() || + n.Target.Direction != scpb.Target_DROP || + col.Column.ColName() != t.Column { + continue + } + // Column drops are, while the column is in the process of being dropped, + // for whatever reason, idempotent. Return silently here. + return nil + } + } + + // TODO: + // remove sequence dependencies + // drop sequences owned by column (if not referenced by other columns) + // drop view (if cascade specified) + // check that no computed columns reference this column + // check that column is not in the PK + // drop secondary indexes + // drop all indexes that index/store the column or use it as a partial index predicate + // drop check constraints + // remove comments + // drop foreign keys + + // TODO(ajwerner): Add family information to the column. + b.addNode(scpb.Target_DROP, &scpb.Column{ + TableID: table.GetID(), + Column: *colToDrop, + }) + + b.addOrUpdatePrimaryIndexTargetsForDropColumn(table, colToDrop.ID) + return nil +} + +// Suppress the linter. We're not ready to fully implement this schema change +// yet. +var _ = (*Builder)(nil).alterTableDropColumn + +func (b *Builder) maybeAddSequenceReferenceDependencies( + ctx context.Context, tableID descpb.ID, col *descpb.ColumnDescriptor, defaultExpr tree.TypedExpr, +) error { + seqNames, err := sequence.GetUsedSequenceNames(defaultExpr) + if err != nil { + return err + } + for _, seqName := range seqNames { + parsedSeqName, err := parser.ParseTableName(seqName) + if err != nil { + return err + } + tn := parsedSeqName.ToTableName() + seqDesc, err := resolver.ResolveExistingTableObject(ctx, b.res, &tn, + tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return err + } + + col.UsesSequenceIds = append(col.UsesSequenceIds, seqDesc.GetID()) + b.addNode(scpb.Target_ADD, &scpb.SequenceDependency{ + SequenceID: seqDesc.GetID(), + TableID: tableID, + ColumnID: col.ID, + }) + } + return nil +} + +func (b *Builder) addOrUpdatePrimaryIndexTargetsForAddColumn( + table catalog.TableDescriptor, colID descpb.ColumnID, colName string, +) (idxID descpb.IndexID) { + // Check whether a target to add a PK already exists. If so, update its + // storing columns. + for i, n := range b.nodes { + if t, ok := n.Element().(*scpb.PrimaryIndex); ok && + b.nodes[i].Target.Direction == scpb.Target_ADD && + t.TableID == table.GetID() { + t.StoreColumnIDs = append(t.StoreColumnIDs, colID) + t.StoreColumnNames = append(t.StoreColumnNames, colName) + return t.Index.ID + } + } + + // Create a new primary index, identical to the existing one except for its + // ID and name. + idxID = b.nextIndexID(table) + newIdx := protoutil.Clone(table.GetPrimaryIndex().IndexDesc()).(*descpb.IndexDescriptor) + newIdx.Name = tabledesc.GenerateUniqueConstraintName( + "new_primary_key", + func(name string) bool { + // TODO (lucy): Also check the new indexes specified in the targets. + _, err := table.FindIndexWithName(name) + return err == nil + }, + ) + newIdx.ID = idxID + + var storeColIDs []descpb.ColumnID + var storeColNames []string + for _, col := range table.GetPublicColumns() { + containsCol := false + for _, id := range newIdx.ColumnIDs { + if id == col.ID { + containsCol = true + break + } + } + if !containsCol { + storeColIDs = append(storeColIDs, col.ID) + storeColNames = append(storeColNames, col.Name) + } + } + + b.addNode(scpb.Target_ADD, &scpb.PrimaryIndex{ + TableID: table.GetID(), + Index: *newIdx, + OtherPrimaryIndexID: table.GetPrimaryIndexID(), + StoreColumnIDs: append(storeColIDs, colID), + StoreColumnNames: append(storeColNames, colName), + }) + + // Drop the existing primary index. + b.addNode(scpb.Target_DROP, &scpb.PrimaryIndex{ + TableID: table.GetID(), + Index: *(protoutil.Clone(table.GetPrimaryIndex().IndexDesc()).(*descpb.IndexDescriptor)), + OtherPrimaryIndexID: newIdx.ID, + StoreColumnIDs: storeColIDs, + StoreColumnNames: storeColNames, + }) + + return idxID +} + +// TODO (lucy): refactor this to share with the add column case. +func (b *Builder) addOrUpdatePrimaryIndexTargetsForDropColumn( + table catalog.TableDescriptor, colID descpb.ColumnID, +) (idxID descpb.IndexID) { + // Check whether a target to add a PK already exists. If so, update its + // storing columns. + for _, n := range b.nodes { + if t, ok := n.Element().(*scpb.PrimaryIndex); ok && + n.Target.Direction == scpb.Target_ADD && + t.TableID == table.GetID() { + for j := range t.StoreColumnIDs { + if t.StoreColumnIDs[j] == colID { + t.StoreColumnIDs = append(t.StoreColumnIDs[:j], t.StoreColumnIDs[j+1:]...) + t.StoreColumnNames = append(t.StoreColumnNames[:j], t.StoreColumnNames[j+1:]...) + return t.Index.ID + } + + panic("index not found") + } + } + } + + // Create a new primary index, identical to the existing one except for its + // ID and name. + idxID = b.nextIndexID(table) + newIdx := protoutil.Clone(table.GetPrimaryIndex().IndexDesc()).(*descpb.IndexDescriptor) + newIdx.Name = tabledesc.GenerateUniqueConstraintName( + "new_primary_key", + func(name string) bool { + // TODO (lucy): Also check the new indexes specified in the targets. + _, err := table.FindIndexWithName(name) + return err == nil + }, + ) + newIdx.ID = idxID + + var addStoreColIDs []descpb.ColumnID + var addStoreColNames []string + var dropStoreColIDs []descpb.ColumnID + var dropStoreColNames []string + for _, col := range table.GetPublicColumns() { + containsCol := false + for _, id := range newIdx.ColumnIDs { + if id == col.ID { + containsCol = true + break + } + } + if !containsCol { + if colID != col.ID { + addStoreColIDs = append(addStoreColIDs, col.ID) + addStoreColNames = append(addStoreColNames, col.Name) + } + dropStoreColIDs = append(dropStoreColIDs, col.ID) + dropStoreColNames = append(dropStoreColNames, col.Name) + } + } + + b.addNode(scpb.Target_ADD, &scpb.PrimaryIndex{ + TableID: table.GetID(), + Index: *newIdx, + OtherPrimaryIndexID: table.GetPrimaryIndexID(), + StoreColumnIDs: addStoreColIDs, + StoreColumnNames: addStoreColNames, + }) + + // Drop the existing primary index. + b.addNode(scpb.Target_DROP, &scpb.PrimaryIndex{ + TableID: table.GetID(), + Index: *(protoutil.Clone(table.GetPrimaryIndex().IndexDesc()).(*descpb.IndexDescriptor)), + OtherPrimaryIndexID: idxID, + StoreColumnIDs: dropStoreColIDs, + StoreColumnNames: dropStoreColNames, + }) + return idxID +} + +// Suppress the linter. We're not ready to fully implement this schema change +// yet. +var _ = (*Builder)(nil).addOrUpdatePrimaryIndexTargetsForDropColumn + +func (b *Builder) nextColumnID(table catalog.TableDescriptor) descpb.ColumnID { + nextColID := table.GetNextColumnID() + var maxColID descpb.ColumnID + + for _, n := range b.nodes { + if n.Target.Direction != scpb.Target_ADD || n.Element().DescriptorID() != table.GetID() { + continue + } + if ac, ok := n.Element().(*scpb.Column); ok { + if ac.Column.ID > maxColID { + maxColID = ac.Column.ID + } + } + } + if maxColID != 0 { + nextColID = maxColID + 1 + } + return nextColID +} + +func (b *Builder) nextIndexID(table catalog.TableDescriptor) descpb.IndexID { + nextMaxID := table.GetNextIndexID() + var maxIdxID descpb.IndexID + for _, n := range b.nodes { + if n.Target.Direction != scpb.Target_ADD || n.Element().DescriptorID() != table.GetID() { + continue + } + if ai, ok := n.Element().(*scpb.SecondaryIndex); ok { + if ai.Index.ID > maxIdxID { + maxIdxID = ai.Index.ID + } + } else if ai, ok := n.Element().(*scpb.PrimaryIndex); ok { + if ai.Index.ID > maxIdxID { + maxIdxID = ai.Index.ID + } + } + } + if maxIdxID != 0 { + nextMaxID = maxIdxID + 1 + } + return nextMaxID +} + +func (b *Builder) addNode(dir scpb.Target_Direction, elem scpb.Element) { + var s scpb.State + switch dir { + case scpb.Target_ADD: + s = scpb.State_ABSENT + case scpb.Target_DROP: + s = scpb.State_PUBLIC + default: + panic(errors.Errorf("unknown direction %s", dir)) + } + b.nodes = append(b.nodes, &scpb.Node{ + Target: scpb.NewTarget(dir, elem), + State: s, + }) +} + +// minimumTypeUsageVersions defines the minimum version needed for a new +// data type. +var minimumTypeUsageVersions = map[types.Family]clusterversion.Key{ + types.GeographyFamily: clusterversion.GeospatialType, + types.GeometryFamily: clusterversion.GeospatialType, + types.Box2DFamily: clusterversion.Box2DType, +} + +// isTypeSupportedInVersion returns whether a given type is supported in the given version. +// This is copied straight from the sql package. +func isTypeSupportedInVersion(v clusterversion.ClusterVersion, t *types.T) (bool, error) { + // For these checks, if we have an array, we only want to find whether + // we support the array contents. + if t.Family() == types.ArrayFamily { + t = t.ArrayContents() + } + + minVersion, ok := minimumTypeUsageVersions[t.Family()] + if !ok { + return true, nil + } + return v.IsActive(minVersion), nil +} diff --git a/pkg/sql/schemachanger/scbuild/builder_test.go b/pkg/sql/schemachanger/scbuild/builder_test.go new file mode 100644 index 000000000000..974766bc9d54 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/builder_test.go @@ -0,0 +1,155 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scbuild_test + +import ( + "bytes" + "context" + gojson "encoding/json" + "fmt" + "path/filepath" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/datadriven" + "github.com/gogo/protobuf/jsonpb" + "github.com/stretchr/testify/require" + "gopkg.in/yaml.v2" +) + +func TestBuilderAlterTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + datadriven.Walk(t, filepath.Join("testdata"), func(t *testing.T, path string) { + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + tdb := sqlutils.MakeSQLRunner(sqlDB) + run := func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "create-table": + stmts, err := parser.Parse(d.Input) + require.NoError(t, err) + require.Len(t, stmts, 1) + create, ok := stmts[0].AST.(*tree.CreateTable) + if !ok { + t.Fatal("not a CREATE TABLE statement") + } + + tdb.Exec(t, d.Input) + + tableName := create.Table.String() + var tableID descpb.ID + tdb.QueryRow(t, `SELECT $1::regclass::int`, tableName).Scan(&tableID) + if tableID == 0 { + t.Fatalf("failed to read ID of new table %s", tableName) + } + t.Logf("created table with id %d", tableID) + + return "" + case "build": + b, cleanup := newTestingBuilder(s) + defer cleanup() + + stmts, err := parser.Parse(d.Input) + require.NoError(t, err) + + var ts []*scpb.Node + for i := range stmts { + next, err := b.Build(ctx, ts, stmts[i].AST) + require.NoError(t, err) + ts = next + } + + return marshalNodes(t, ts) + case "unimplemented": + b, cleanup := newTestingBuilder(s) + defer cleanup() + + stmts, err := parser.Parse(d.Input) + require.NoError(t, err) + require.Len(t, stmts, 1) + + stmt := stmts[0] + alter, ok := stmt.AST.(*tree.AlterTable) + require.Truef(t, ok, "not an ALTER TABLE statement: %s", stmt.SQL) + _, err = b.AlterTable(ctx, nil, alter) + require.Truef(t, scbuild.HasNotImplemented(err), "expected unimplemented, got %v", err) + return "" + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + } + datadriven.RunTest(t, path, run) + }) +} + +// marshalNodes marshals a []*scpb.Node to YAML. +func marshalNodes(t *testing.T, nodes []*scpb.Node) string { + type mapNode struct { + Target map[string]interface{} + State string + } + mapNodes := make([]mapNode, 0, len(nodes)) + for _, node := range nodes { + var buf bytes.Buffer + require.NoError(t, (&jsonpb.Marshaler{}).Marshal(&buf, node.Target)) + + target := make(map[string]interface{}) + require.NoError(t, gojson.Unmarshal(buf.Bytes(), &target)) + + mapNodes = append(mapNodes, mapNode{ + Target: target, + State: node.State.String(), + }) + } + + out, err := yaml.Marshal(mapNodes) + require.NoError(t, err) + return string(out) +} + +func newTestingBuilder(s serverutils.TestServerInterface) (*scbuild.Builder, func()) { + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + ip, cleanup := sql.NewInternalPlanner( + "test", + kv.NewTxn(context.Background(), s.DB(), s.NodeID()), + security.RootUserName(), + &sql.MemoryMetrics{}, + &execCfg, + // Setting the database on the session data to "defaultdb" in the obvious + // way doesn't seem to do what we want. + sessiondatapb.SessionData{}, + ) + planner := ip.(interface { + resolver.SchemaResolver + SemaCtx() *tree.SemaContext + EvalContext() *tree.EvalContext + }) + return scbuild.NewBuilder(planner, planner.SemaCtx(), planner.EvalContext()), cleanup +} diff --git a/pkg/sql/schemachanger/scbuild/main_test.go b/pkg/sql/schemachanger/scbuild/main_test.go new file mode 100644 index 000000000000..d6f73f452c55 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/main_test.go @@ -0,0 +1,31 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scbuild_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table b/pkg/sql/schemachanger/scbuild/testdata/alter_table new file mode 100644 index 000000000000..26bae009ed42 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table @@ -0,0 +1,444 @@ +create-table +CREATE TABLE defaultdb.foo (i INT PRIMARY KEY) +---- + +build +ALTER TABLE defaultdb.foo ADD COLUMN j INT +---- +- target: + direction: ADD + elementProto: + column: + column: + id: 2 + name: j + nullable: true + type: + family: IntFamily + oid: 20 + width: 64 + familyName: primary + tableId: 52 + state: ABSENT +- target: + direction: ADD + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 1 + columnNames: + - i + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + name: new_primary_key + partitioning: {} + sharded: {} + unique: true + version: 2 + otherPrimaryIndexId: 1 + storeColumnIds: + - 2 + storeColumnNames: + - j + tableId: 52 + state: ABSENT +- target: + direction: DROP + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 1 + columnNames: + - i + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + name: primary + partitioning: {} + sharded: {} + unique: true + version: 2 + otherPrimaryIndexId: 2 + tableId: 52 + state: PUBLIC + +build +ALTER TABLE defaultdb.foo ADD COLUMN j INT DEFAULT 123 +---- +- target: + direction: ADD + elementProto: + column: + column: + defaultExpr: 123:::INT8 + id: 2 + name: j + nullable: true + type: + family: IntFamily + oid: 20 + width: 64 + familyName: primary + tableId: 52 + state: ABSENT +- target: + direction: ADD + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 1 + columnNames: + - i + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + name: new_primary_key + partitioning: {} + sharded: {} + unique: true + version: 2 + otherPrimaryIndexId: 1 + storeColumnIds: + - 2 + storeColumnNames: + - j + tableId: 52 + state: ABSENT +- target: + direction: DROP + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 1 + columnNames: + - i + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + name: primary + partitioning: {} + sharded: {} + unique: true + version: 2 + otherPrimaryIndexId: 2 + tableId: 52 + state: PUBLIC + +build +ALTER TABLE defaultdb.foo ADD COLUMN j INT DEFAULT 123; +ALTER TABLE defaultdb.foo ADD COLUMN k INT DEFAULT 456; +---- +- target: + direction: ADD + elementProto: + column: + column: + defaultExpr: 123:::INT8 + id: 2 + name: j + nullable: true + type: + family: IntFamily + oid: 20 + width: 64 + familyName: primary + tableId: 52 + state: ABSENT +- target: + direction: ADD + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 1 + columnNames: + - i + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + name: new_primary_key + partitioning: {} + sharded: {} + unique: true + version: 2 + otherPrimaryIndexId: 1 + storeColumnIds: + - 2 + - 3 + storeColumnNames: + - j + - k + tableId: 52 + state: ABSENT +- target: + direction: DROP + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 1 + columnNames: + - i + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + name: primary + partitioning: {} + sharded: {} + unique: true + version: 2 + otherPrimaryIndexId: 2 + tableId: 52 + state: PUBLIC +- target: + direction: ADD + elementProto: + column: + column: + defaultExpr: 456:::INT8 + id: 3 + name: k + nullable: true + type: + family: IntFamily + oid: 20 + width: 64 + familyName: primary + tableId: 52 + state: ABSENT + +build +ALTER TABLE defaultdb.foo ADD COLUMN a INT AS (i+1) STORED +---- +- target: + direction: ADD + elementProto: + column: + column: + computeExpr: i + 1:::INT8 + id: 2 + name: a + nullable: true + type: + family: IntFamily + oid: 20 + width: 64 + familyName: primary + tableId: 52 + state: ABSENT +- target: + direction: ADD + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 1 + columnNames: + - i + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + name: new_primary_key + partitioning: {} + sharded: {} + unique: true + version: 2 + otherPrimaryIndexId: 1 + storeColumnIds: + - 2 + storeColumnNames: + - a + tableId: 52 + state: ABSENT +- target: + direction: DROP + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 1 + columnNames: + - i + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + name: primary + partitioning: {} + sharded: {} + unique: true + version: 2 + otherPrimaryIndexId: 2 + tableId: 52 + state: PUBLIC + +create-table +CREATE TABLE defaultdb.bar (j INT); +---- + +build +ALTER TABLE defaultdb.foo ADD COLUMN a INT; +ALTER TABLE defaultdb.bar ADD COLUMN b INT; +---- +- target: + direction: ADD + elementProto: + column: + column: + id: 2 + name: a + nullable: true + type: + family: IntFamily + oid: 20 + width: 64 + familyName: primary + tableId: 52 + state: ABSENT +- target: + direction: ADD + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 1 + columnNames: + - i + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + name: new_primary_key + partitioning: {} + sharded: {} + unique: true + version: 2 + otherPrimaryIndexId: 1 + storeColumnIds: + - 2 + storeColumnNames: + - a + tableId: 52 + state: ABSENT +- target: + direction: DROP + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 1 + columnNames: + - i + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + name: primary + partitioning: {} + sharded: {} + unique: true + version: 2 + otherPrimaryIndexId: 2 + tableId: 52 + state: PUBLIC +- target: + direction: ADD + elementProto: + column: + column: + id: 3 + name: b + nullable: true + type: + family: IntFamily + oid: 20 + width: 64 + familyName: primary + tableId: 53 + state: ABSENT +- target: + direction: ADD + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 2 + columnNames: + - rowid + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + name: new_primary_key + partitioning: {} + sharded: {} + unique: true + otherPrimaryIndexId: 1 + storeColumnIds: + - 1 + - 3 + storeColumnNames: + - j + - b + tableId: 53 + state: ABSENT +- target: + direction: DROP + elementProto: + primaryIndex: + index: + columnDirections: + - ASC + columnIds: + - 2 + columnNames: + - rowid + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + name: primary + partitioning: {} + sharded: {} + unique: true + otherPrimaryIndexId: 2 + storeColumnIds: + - 1 + storeColumnNames: + - j + tableId: 53 + state: PUBLIC diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_unimplemented b/pkg/sql/schemachanger/scbuild/testdata/alter_table_unimplemented new file mode 100644 index 000000000000..c46c4a4109d9 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_unimplemented @@ -0,0 +1,84 @@ +create-table +CREATE TABLE defaultdb.foo (i INT PRIMARY KEY) +---- + +unimplemented +ALTER TABLE defaultdb.foo ADD COLUMN j INT UNIQUE +---- + +unimplemented +ALTER TABLE defaultdb.foo ADD COLUMN j INT CHECK (j > 0) +---- + +unimplemented +ALTER TABLE defaultdb.foo ADD COLUMN j INT REFERENCES defaultdb.foo(i) +---- + +unimplemented +ALTER TABLE defaultdb.foo ADD COLUMN j SERIAL +---- + +unimplemented +ALTER TABLE defaultdb.foo ADD CONSTRAINT j CHECK (i > 0) +---- + +unimplemented +ALTER TABLE defaultdb.foo ALTER COLUMN i SET DATA TYPE STRING +---- + +unimplemented +ALTER TABLE defaultdb.foo ALTER PRIMARY KEY USING COLUMNS (i) +---- + +unimplemented +ALTER TABLE defaultdb.foo DROP COLUMN i +---- + +unimplemented +ALTER TABLE defaultdb.foo DROP CONSTRAINT foobar +---- + +unimplemented +ALTER TABLE defaultdb.foo ALTER COLUMN i DROP NOT NULL +---- + +unimplemented +ALTER TABLE defaultdb.foo ALTER COLUMN i DROP STORED +---- + +unimplemented +ALTER TABLE defaultdb.foo ALTER COLUMN i SET NOT NULL +---- + +unimplemented +ALTER TABLE defaultdb.foo RENAME COLUMN i TO j +---- + +unimplemented +ALTER TABLE defaultdb.foo RENAME CONSTRAINT foobar TO baz +---- + +unimplemented +ALTER TABLE defaultdb.foo EXPERIMENTAL_AUDIT SET READ WRITE +---- + +unimplemented +ALTER TABLE defaultdb.foo ALTER COLUMN i SET DEFAULT 42 +---- + +unimplemented +ALTER TABLE defaultdb.foo ALTER COLUMN i DROP DEFAULT +---- + +unimplemented +ALTER TABLE defaultdb.foo VALIDATE CONSTRAINT foobar +---- + +unimplemented +ALTER TABLE defaultdb.foo PARTITION BY NOTHING +---- + +unimplemented +ALTER TABLE defaultdb.foo INJECT STATISTICS '[]' +---- + diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 7de7525896ed..3694d3fc05a0 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -982,6 +982,7 @@ func TestLint(t *testing.T) { ":!util/protoutil/marshaler.go", ":!settings/settings_test.go", ":!sql/types/types_jsonpb.go", + ":!sql/schemachanger/scbuild/builder_test.go", ) if err != nil { t.Fatal(err) From 0d1b8fed2ebb3308ec0f101555c55e77003541f9 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:02 -0500 Subject: [PATCH 06/15] schemachanger/scplan: add library to plan schema changes `scplan` generates schema change plans from a set of initial states for targets. It does this by constructing a graph, computing all edges in the graph (either op edges or dep edges) representing ordering constraints, and then generating a sequence of stages that respects the edges' constraints. This package contains rules for generating both op edges and dep edges, specified declaratively in terms of predicates on pairs of nodes. Release note: None --- pkg/sql/schemachanger/scplan/BUILD.bazel | 57 ++++ pkg/sql/schemachanger/scplan/declarative.go | 215 ++++++++++++ pkg/sql/schemachanger/scplan/helpers.go | 28 ++ pkg/sql/schemachanger/scplan/plan.go | 235 +++++++++++++ pkg/sql/schemachanger/scplan/rules.go | 346 ++++++++++++++++++++ 5 files changed, 881 insertions(+) create mode 100644 pkg/sql/schemachanger/scplan/BUILD.bazel create mode 100644 pkg/sql/schemachanger/scplan/declarative.go create mode 100644 pkg/sql/schemachanger/scplan/helpers.go create mode 100644 pkg/sql/schemachanger/scplan/plan.go create mode 100644 pkg/sql/schemachanger/scplan/rules.go diff --git a/pkg/sql/schemachanger/scplan/BUILD.bazel b/pkg/sql/schemachanger/scplan/BUILD.bazel new file mode 100644 index 000000000000..e06038704863 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/BUILD.bazel @@ -0,0 +1,57 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "compiler", + srcs = [ + "compiler.go", + "gen_dep_edges.go", + "gen_op_edges.go", + "graphviz.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/compiler", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/catalog/descpb", + "//pkg/sql/schemachanger/ops", + "//pkg/sql/schemachanger/targets", + "//pkg/util", + "@com_github_cockroachdb_errors//:errors", + "@com_github_emicklei_dot//:dot", + ], +) + +go_test( + name = "compiler_test", + srcs = ["compiler_test.go"], + embed = [":compiler"], + deps = [ + "//pkg/sql/catalog/descpb", + "//pkg/sql/schemachanger/ops", + "//pkg/sql/schemachanger/targets", + "//pkg/sql/types", + "//pkg/util/leaktest", + "@com_github_emicklei_dot//:dot", + "@com_github_stretchr_testify//require", + ], +) + +go_library( + name = "scplan", + srcs = [ + "declarative.go", + "helpers.go", + "plan.go", + "rules.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/schemachanger/scgraph", + "//pkg/sql/schemachanger/scop", + "//pkg/sql/schemachanger/scpb", + "//pkg/util/iterutil", + "@com_github_cockroachdb_errors//:errors", + ], +) diff --git a/pkg/sql/schemachanger/scplan/declarative.go b/pkg/sql/schemachanger/scplan/declarative.go new file mode 100644 index 000000000000..9191fcfaba30 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/declarative.go @@ -0,0 +1,215 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scplan + +import ( + "reflect" + + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/errors" +) + +type depMatcher struct { + dirPredicate func(thisDir, thatDir scpb.Target_Direction) bool + thatState scpb.State + predicate interface{} +} + +type decOpEdge struct { + nextState scpb.State + predicate interface{} + op interface{} +} + +type targetRules struct { + deps targetDepRules + forward, backwards targetOpRules +} + +type targetDepRules map[scpb.State][]depMatcher + +type targetOpRules map[scpb.State][]decOpEdge + +var p = buildSchemaChangePlanner(rules) + +type opGenFunc func(builder *scgraph.Graph, t *scpb.Target, s scpb.State, flags Params) +type depGenFunc func(g *scgraph.Graph, t *scpb.Target, s scpb.State) + +type schemaChangeTargetPlanner struct { + ops opGenFunc + deps depGenFunc +} + +type schemaChangePlanner map[reflect.Type]schemaChangeTargetPlanner + +func buildSchemaChangePlanner(m map[scpb.Element]targetRules) schemaChangePlanner { + tp := make(map[reflect.Type]schemaChangeTargetPlanner) + for e, r := range m { + tp[reflect.TypeOf(e)] = schemaChangeTargetPlanner{ + ops: buildSchemaChangeOpGenFunc(e, r.forward, r.backwards), + deps: buildSchemaChangeDepGenFunc(e, r.deps), + } + } + return tp +} + +func buildSchemaChangeDepGenFunc(e scpb.Element, deps targetDepRules) depGenFunc { + // We want to walk all of the edges and ensure that they have the proper + // signature. + tTyp := reflect.TypeOf(e) + type matcher struct { + dirPred func(thisDir, thatDir scpb.Target_Direction) bool + pred func(this, that scpb.Element) bool + thatState scpb.State + } + matchers := map[scpb.State]map[reflect.Type][]matcher{} + for s, rules := range deps { + for i, rule := range rules { + mt := reflect.TypeOf(rule.predicate) + if mt.NumIn() != 2 { + panic(errors.Errorf("expected two args, got %d for (%T,%s)[%d]", mt.NumIn(), e, s, i)) + } + if got := mt.In(0); got != tTyp { + panic(errors.Errorf("expected %T, got %v for (%T,%s)[%d]", e, got, e, s, i)) + } + other := mt.In(1) + if !other.Implements(elementInterfaceType) { + panic(errors.Errorf("expected %T to implement %v for (%T,%s)[%d]", other, elementInterfaceType, e, s, i)) + } + if mt.NumOut() != 1 { + panic(errors.Errorf("expected one return value, got %d for (%T,%s)[%d]", mt.NumOut(), e, s, i)) + } + if mt.Out(0) != boolType { + panic(errors.Errorf("expected bool return value, got %v for (%T,%s)[%d]", mt.Out(0), e, s, i)) + } + if rule.dirPredicate == nil { + panic(errors.Errorf("invalid missing direction predicate for (%T,%s)[%d]", e, s, i)) + } + if matchers[s] == nil { + matchers[s] = map[reflect.Type][]matcher{} + } + predV := reflect.ValueOf(rule.predicate) + f := func(a, b scpb.Element) bool { + out := predV.Call([]reflect.Value{ + reflect.ValueOf(a), + reflect.ValueOf(b), + }) + return out[0].Bool() + } + matchers[s][other] = append(matchers[s][other], matcher{ + dirPred: rule.dirPredicate, + pred: f, + thatState: rule.thatState, + }) + } + } + return func(g *scgraph.Graph, this *scpb.Target, thisState scpb.State) { + for t, matchers := range matchers[thisState] { + if err := g.ForEachTarget(func(that *scpb.Target) error { + if reflect.TypeOf(that.Element()) != t { + return nil + } + for _, m := range matchers { + if m.dirPred(this.Direction, that.Direction) && + m.pred(this.Element(), that.Element()) { + g.AddDepEdge(this, thisState, that, m.thatState) + } + } + return nil + }); err != nil { + panic(err) + } + } + } +} + +var ( + compileFlagsTyp = reflect.TypeOf((*Params)(nil)).Elem() + opsType = reflect.TypeOf((*scop.Op)(nil)).Elem() + boolType = reflect.TypeOf((*bool)(nil)).Elem() + elementInterfaceType = reflect.TypeOf((*scpb.Element)(nil)).Elem() +) + +func buildSchemaChangeOpGenFunc(e scpb.Element, forward, backwards targetOpRules) opGenFunc { + // We want to walk all of the edges and ensure that they have the proper + // signature. + tTyp := reflect.TypeOf(e) + predicateTyp := reflect.FuncOf( + []reflect.Type{tTyp, compileFlagsTyp}, + []reflect.Type{boolType}, + false, /* variadic */ + ) + opType := reflect.FuncOf( + []reflect.Type{tTyp}, + []reflect.Type{opsType}, + false, /* variadic */ + ) + for s, rules := range forward { + for i, rule := range rules { + if rule.nextState == s { + panic(errors.Errorf("detected rule into same state: %s for %T[%d]", s, e, i)) + } + if rule.predicate != nil { + if pt := reflect.TypeOf(rule.predicate); pt != predicateTyp { + panic(errors.Errorf("invalid predicate with signature %v != %v for %T[%d]", pt, predicateTyp, e, i)) + } + } + if rule.nextState == scpb.State_UNKNOWN { + if rule.op != nil { + panic(errors.Errorf("invalid stopping rule with non-nil op func for %T[%d]", e, i)) + } + continue + } + if rule.nextState != scpb.State_UNKNOWN && rule.op == nil { + panic(errors.Errorf("invalid nil op with next state %s for %T[%d]", rule.nextState, e, i)) + } + if ot := reflect.TypeOf(rule.op); ot != opType { + panic(errors.Errorf("invalid ops with signature %v != %v %p %p for (%T, %s)[%d]", ot, opType, ot, opsType, e, s, i)) + } + } + } + + return func(builder *scgraph.Graph, t *scpb.Target, s scpb.State, flags Params) { + cur := s + tv := reflect.ValueOf(t.Element()) + flagsV := reflect.ValueOf(flags) + predicateArgs := []reflect.Value{tv, flagsV} + opsArgs := []reflect.Value{tv} + var stateRules targetOpRules + if t.Direction == scpb.Target_ADD { + stateRules = forward + } else { + stateRules = backwards + } + + outer: + for { + rules := stateRules[cur] + for _, rule := range rules { + if rule.predicate != nil { + if out := reflect.ValueOf(rule.predicate).Call(predicateArgs); !out[0].Bool() { + continue + } + } + if rule.nextState == scpb.State_UNKNOWN { + return + } + out := reflect.ValueOf(rule.op).Call(opsArgs) + builder.AddOpEdge(t, cur, rule.nextState, out[0].Interface().(scop.Op)) + cur = rule.nextState + continue outer + } + break + } + } +} diff --git a/pkg/sql/schemachanger/scplan/helpers.go b/pkg/sql/schemachanger/scplan/helpers.go new file mode 100644 index 000000000000..9d27c4420a9c --- /dev/null +++ b/pkg/sql/schemachanger/scplan/helpers.go @@ -0,0 +1,28 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scplan + +import "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + +func indexContainsColumn(idx *descpb.IndexDescriptor, colID descpb.ColumnID) bool { + return columnsContainsID(idx.ColumnIDs, colID) || + columnsContainsID(idx.StoreColumnIDs, colID) || + columnsContainsID(idx.ExtraColumnIDs, colID) +} + +func columnsContainsID(haystack []descpb.ColumnID, needle descpb.ColumnID) bool { + for _, id := range haystack { + if id == needle { + return true + } + } + return false +} diff --git a/pkg/sql/schemachanger/scplan/plan.go b/pkg/sql/schemachanger/scplan/plan.go new file mode 100644 index 000000000000..eae3b84ac59a --- /dev/null +++ b/pkg/sql/schemachanger/scplan/plan.go @@ -0,0 +1,235 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scplan + +import ( + "reflect" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/util/iterutil" + "github.com/cockroachdb/errors" +) + +// A Phase represents the context in which an op is executed within a schema +// change. Different phases require different dependencies for the execution of +// the ops to be plumbed in. +// +// Today, we support the phases corresponding to async schema changes initiated +// and partially executed in the user transaction. This will change as we +// transition to transactional schema changes. +type Phase int + +const ( + // PostStatementPhase refers to execution of ops occurring after statement + // execution during the user transaction. + // TODO(ajwerner,lucy-zhang): Move the execution for these ops to actual + // statement execution for the planNodes and update the name. + PostStatementPhase Phase = iota + // PreCommitPhase refers to execution of ops occurring during the user + // transaction immediately before commit. + PreCommitPhase + // PostCommitPhase refers to execution of ops occurring after the user + // transaction has committed (i.e., in the async schema change job). + PostCommitPhase +) + +// Params holds the arguments for planning. +type Params struct { + // ExecutionPhase indicates the phase that the plan should be constructed for. + ExecutionPhase Phase + // CreatedDescriptorIDs contains IDs for new descriptors created by the same + // schema changer (i.e., earlier in the same transaction). New descriptors + // can have most of their schema changes fully executed in the same + // transaction. + // + // This doesn't do anything right now. + CreatedDescriptorIDs catalog.DescriptorIDSet +} + +// A Plan is a schema change plan, primarily containing ops to be executed that +// are partitioned into stages. +type Plan struct { + Params Params + InitialNodes []*scpb.Node + Graph *scgraph.Graph + Stages []Stage +} + +// A Stage is a sequence of ops to be executed "together" as part of a schema +// change. +// +// Stages also contain their corresponding targets and states before and after +// the execution of the ops in the stage, reflecting the fact that any set of +// ops can be thought of as a transition from a set of target states to another. +type Stage struct { + Before, After []*scpb.Node + Ops scop.Ops +} + +// MakePlan generates a Plan for a particular phase of a schema change, given +// the initial states for a set of targets. +func MakePlan(initialStates []*scpb.Node, params Params) (_ Plan, err error) { + defer func() { + if r := recover(); r != nil { + rAsErr, ok := r.(error) + if !ok { + rAsErr = errors.Errorf("panic during MakePlan: %v", r) + } + err = errors.CombineErrors(err, rAsErr) + } + }() + + g, err := scgraph.New(initialStates) + if err != nil { + return Plan{}, err + } + // TODO(ajwerner): Generate the stages for all of the phases as it will make + // debugging easier. + for _, ts := range initialStates { + p[reflect.TypeOf(ts.Element())].ops(g, ts.Target, ts.State, params) + } + if err := g.ForEachNode(func(n *scpb.Node) error { + d, ok := p[reflect.TypeOf(n.Element())] + if !ok { + return errors.Errorf("not implemented for %T", n.Target) + } + d.deps(g, n.Target, n.State) + return nil + }); err != nil { + return Plan{}, err + } + stages := buildStages(initialStates, g) + return Plan{ + Params: params, + InitialNodes: initialStates, + Graph: g, + Stages: stages, + }, nil +} + +func buildStages(init []*scpb.Node, g *scgraph.Graph) []Stage { + // TODO(ajwerner): deal with the case where the target state was + // fulfilled by something that preceded the initial state. + cur := init + fulfilled := map[*scpb.Node]struct{}{} + filterUnsatisfiedEdgesStep := func(edges []*scgraph.OpEdge) ([]*scgraph.OpEdge, bool) { + candidates := make(map[*scpb.Node]struct{}) + for _, e := range edges { + candidates[e.To()] = struct{}{} + } + // Check to see if the current set of edges will have their dependencies met + // if they are all run. Any which will not must be pruned. This greedy + // algorithm works, but a justification is in order. + failed := map[*scgraph.OpEdge]struct{}{} + for _, e := range edges { + _ = g.ForEachDepEdgeFrom(e.To(), func(de *scgraph.DepEdge) error { + _, isFulfilled := fulfilled[de.To()] + _, isCandidate := candidates[de.To()] + if isFulfilled || isCandidate { + return nil + } + failed[e] = struct{}{} + return iterutil.StopIteration() + }) + } + if len(failed) == 0 { + return edges, true + } + truncated := edges[:0] + for _, e := range edges { + if _, found := failed[e]; !found { + truncated = append(truncated, e) + } + } + return truncated, false + } + filterUnsatisfiedEdges := func(edges []*scgraph.OpEdge) ([]*scgraph.OpEdge, bool) { + for len(edges) > 0 { + if filtered, done := filterUnsatisfiedEdgesStep(edges); !done { + edges = filtered + } else { + return filtered, true + } + } + return edges, false + } + buildStageType := func(edges []*scgraph.OpEdge) (Stage, bool) { + edges, ok := filterUnsatisfiedEdges(edges) + if !ok { + return Stage{}, false + } + next := append(cur[:0:0], cur...) + var ops []scop.Op + for i, ts := range cur { + for _, e := range edges { + if e.From() == ts { + next[i] = e.To() + ops = append(ops, e.Op()) + break + } + } + } + return Stage{ + Before: cur, + After: next, + Ops: scop.MakeOps(ops...), + }, true + } + + var stages []Stage + for { + // Note that the current nodes are fulfilled for the sake of dependency + // checking. + for _, ts := range cur { + fulfilled[ts] = struct{}{} + } + + // Extract the set of op edges for the current stage. + var opEdges []*scgraph.OpEdge + for _, t := range cur { + // TODO(ajwerner): improve the efficiency of this lookup. + // Look for an opEdge from this node. Then, for the other side + // of the opEdge, look for dependencies. + if oe, ok := g.GetOpEdgeFrom(t); ok { + opEdges = append(opEdges, oe) + } + } + + // Group the op edges a per-type basis. + opTypes := make(map[scop.Type][]*scgraph.OpEdge) + for _, oe := range opEdges { + opTypes[oe.Op().Type()] = append(opTypes[oe.Op().Type()], oe) + } + + // Greedily attempt to find a stage which can be executed. This is sane + // because once a dependency is met, it never becomes unmet. + var didSomething bool + var s Stage + for _, typ := range []scop.Type{ + scop.MutationType, + scop.BackfillType, + scop.ValidationType, + } { + if s, didSomething = buildStageType(opTypes[typ]); didSomething { + break + } + } + if !didSomething { + break + } + stages = append(stages, s) + cur = s.After + } + return stages +} diff --git a/pkg/sql/schemachanger/scplan/rules.go b/pkg/sql/schemachanger/scplan/rules.go new file mode 100644 index 000000000000..7665f2f75d62 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/rules.go @@ -0,0 +1,346 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scplan + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" +) + +func columnInSecondaryIndex(this *scpb.Column, that *scpb.SecondaryIndex) bool { + return this.TableID == that.TableID && + indexContainsColumn(&that.Index, this.Column.ID) +} + +func columnInPrimaryIndex(this *scpb.Column, that *scpb.PrimaryIndex) bool { + return this.TableID == that.TableID && + indexContainsColumn(&that.Index, this.Column.ID) || + columnsContainsID(that.StoreColumnIDs, this.Column.ID) +} + +func primaryIndexContainsColumn(this *scpb.PrimaryIndex, that *scpb.Column) bool { + return columnInPrimaryIndex(that, this) +} + +func primaryIndexesReferenceEachOther(this, that *scpb.PrimaryIndex) bool { + return this.TableID == that.TableID && + this.OtherPrimaryIndexID == that.Index.ID +} + +func sameDirection(a, b scpb.Target_Direction) bool { + return a == b +} + +func oppositeDirection(a, b scpb.Target_Direction) bool { + return a != b +} + +// Suppress the linter. +var _ = oppositeDirection + +func bothDirectionsEqual(dir scpb.Target_Direction) func(a, b scpb.Target_Direction) bool { + return directionsMatch(dir, dir) +} + +func directionsMatch(thisDir, thatDir scpb.Target_Direction) func(a, b scpb.Target_Direction) bool { + return func(a, b scpb.Target_Direction) bool { + return a == thisDir && b == thatDir + } +} + +var rules = map[scpb.Element]targetRules{ + (*scpb.Column)(nil): { + deps: targetDepRules{ + scpb.State_DELETE_AND_WRITE_ONLY: { + { + dirPredicate: sameDirection, + thatState: scpb.State_DELETE_AND_WRITE_ONLY, + predicate: columnInSecondaryIndex, + }, + { + dirPredicate: sameDirection, + thatState: scpb.State_DELETE_AND_WRITE_ONLY, + predicate: columnInPrimaryIndex, + }, + }, + scpb.State_PUBLIC: { + { + dirPredicate: bothDirectionsEqual(scpb.Target_ADD), + thatState: scpb.State_PUBLIC, + predicate: columnInSecondaryIndex, + }, + { + dirPredicate: bothDirectionsEqual(scpb.Target_ADD), + thatState: scpb.State_PUBLIC, + predicate: columnInPrimaryIndex, + }, + }, + }, + forward: targetOpRules{ + scpb.State_ABSENT: { + { + predicate: func(this *scpb.Column, flags Params) bool { + return flags.ExecutionPhase == PostStatementPhase && + !flags.CreatedDescriptorIDs.Contains(this.TableID) + }, + }, + { + nextState: scpb.State_DELETE_ONLY, + op: func(this *scpb.Column) scop.Op { + return scop.MakeAddedColumnDeleteOnly{ + TableID: this.TableID, + FamilyID: this.FamilyID, + FamilyName: this.FamilyName, + Column: this.Column, + } + }, + }, + }, + scpb.State_DELETE_ONLY: { + { + predicate: func(this *scpb.Column, flags Params) bool { + return flags.ExecutionPhase == PreCommitPhase && + !flags.CreatedDescriptorIDs.Contains(this.TableID) + }, + }, + { + nextState: scpb.State_DELETE_AND_WRITE_ONLY, + op: func(this *scpb.Column) scop.Op { + return scop.MakeAddedColumnDeleteAndWriteOnly{ + TableID: this.TableID, + ColumnID: this.Column.ID, + } + }, + }, + }, + scpb.State_DELETE_AND_WRITE_ONLY: { + { + nextState: scpb.State_PUBLIC, + op: func(this *scpb.Column) scop.Op { + return scop.MakeColumnPublic{ + TableID: this.TableID, + ColumnID: this.Column.ID, + } + }, + }, + }, + }, + backwards: targetOpRules{ + scpb.State_PUBLIC: { + { + nextState: scpb.State_DELETE_AND_WRITE_ONLY, + op: func(this *scpb.Column) scop.Op { + return scop.MakeDroppedColumnDeleteAndWriteOnly{ + TableID: this.TableID, + ColumnID: this.Column.ID, + } + }, + }, + }, + scpb.State_DELETE_AND_WRITE_ONLY: { + { + predicate: func(this *scpb.Column, flags Params) bool { + return !flags.CreatedDescriptorIDs.Contains(this.TableID) && + (flags.ExecutionPhase == PostStatementPhase || + flags.ExecutionPhase == PreCommitPhase) + }, + }, + { + nextState: scpb.State_DELETE_ONLY, + op: func(this *scpb.Column) scop.Op { + return scop.MakeDroppedColumnDeleteOnly{ + TableID: this.TableID, + ColumnID: this.Column.ID, + } + }, + }, + }, + scpb.State_DELETE_ONLY: { + { + nextState: scpb.State_ABSENT, + op: func(this *scpb.Column) scop.Op { + return scop.MakeColumnAbsent{ + TableID: this.TableID, + ColumnID: this.Column.ID, + } + }, + }, + }, + }, + }, + (*scpb.PrimaryIndex)(nil): { + deps: targetDepRules{ + scpb.State_PUBLIC: { + { + dirPredicate: directionsMatch(scpb.Target_ADD, scpb.Target_DROP), + thatState: scpb.State_DELETE_AND_WRITE_ONLY, + predicate: primaryIndexesReferenceEachOther, + }, + }, + scpb.State_DELETE_AND_WRITE_ONLY: { + { + dirPredicate: directionsMatch(scpb.Target_DROP, scpb.Target_ADD), + thatState: scpb.State_PUBLIC, + predicate: primaryIndexesReferenceEachOther, + }, + { + dirPredicate: bothDirectionsEqual(scpb.Target_DROP), + thatState: scpb.State_DELETE_AND_WRITE_ONLY, + predicate: primaryIndexContainsColumn, + }, + }, + }, + forward: targetOpRules{ + scpb.State_ABSENT: { + { + predicate: func(this *scpb.PrimaryIndex, flags Params) bool { + return flags.ExecutionPhase == PostStatementPhase && + !flags.CreatedDescriptorIDs.Contains(this.TableID) + }, + }, + { + nextState: scpb.State_DELETE_ONLY, + op: func(this *scpb.PrimaryIndex) scop.Op { + idx := this.Index + idx.StoreColumnNames = this.StoreColumnNames + idx.StoreColumnIDs = this.StoreColumnIDs + idx.EncodingType = descpb.PrimaryIndexEncoding + return scop.MakeAddedIndexDeleteOnly{ + TableID: this.TableID, + Index: idx, + } + }, + }, + }, + scpb.State_DELETE_ONLY: { + { + predicate: func(this *scpb.PrimaryIndex, flags Params) bool { + return flags.ExecutionPhase == PreCommitPhase && + !flags.CreatedDescriptorIDs.Contains(this.TableID) + }, + }, + { + nextState: scpb.State_DELETE_AND_WRITE_ONLY, + op: func(this *scpb.PrimaryIndex) scop.Op { + return scop.MakeAddedIndexDeleteAndWriteOnly{ + TableID: this.TableID, + IndexID: this.Index.ID, + } + }, + }, + }, + scpb.State_DELETE_AND_WRITE_ONLY: { + { + // If this index is unique (which primary indexes should be) and + // there's not already a covering primary index, then we'll need to + // validate that this index indeed is unique. + // + // TODO(ajwerner): Rationalize this and hook up the optimization. + predicate: func(this *scpb.PrimaryIndex, flags Params) bool { + return this.Index.Unique + }, + nextState: scpb.State_BACKFILLED, + op: func(this *scpb.PrimaryIndex) scop.Op { + return scop.BackfillIndex{ + TableID: this.TableID, + IndexID: this.Index.ID, + } + }, + }, + { + nextState: scpb.State_VALIDATED, + op: func(this *scpb.PrimaryIndex) scop.Op { + return scop.BackfillIndex{ + TableID: this.TableID, + IndexID: this.Index.ID, + } + }, + }, + }, + scpb.State_BACKFILLED: { + { + nextState: scpb.State_VALIDATED, + op: func(this *scpb.PrimaryIndex) scop.Op { + return scop.ValidateUniqueIndex{ + TableID: this.TableID, + PrimaryIndexID: this.OtherPrimaryIndexID, + IndexID: this.Index.ID, + } + }, + }, + }, + scpb.State_VALIDATED: { + { + nextState: scpb.State_PUBLIC, + op: func(this *scpb.PrimaryIndex) scop.Op { + return scop.MakeAddedPrimaryIndexPublic{ + TableID: this.TableID, + Index: this.Index, + } + }, + }, + }, + }, + backwards: targetOpRules{ + scpb.State_PUBLIC: { + { + predicate: func(this *scpb.PrimaryIndex, flags Params) bool { + return flags.ExecutionPhase == PostStatementPhase && + !flags.CreatedDescriptorIDs.Contains(this.TableID) + }, + }, + { + nextState: scpb.State_DELETE_AND_WRITE_ONLY, + op: func(this *scpb.PrimaryIndex) scop.Op { + // Most of this logic is taken from MakeMutationComplete(). + idx := this.Index + idx.StoreColumnIDs = this.StoreColumnIDs + idx.StoreColumnNames = this.StoreColumnNames + idx.EncodingType = descpb.PrimaryIndexEncoding + return scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly{ + TableID: this.TableID, + Index: idx, + } + }, + }, + }, + scpb.State_DELETE_AND_WRITE_ONLY: { + { + predicate: func(this *scpb.PrimaryIndex, flags Params) bool { + return flags.ExecutionPhase == PreCommitPhase && + !flags.CreatedDescriptorIDs.Contains(this.TableID) + }, + }, + { + nextState: scpb.State_DELETE_ONLY, + op: func(this *scpb.PrimaryIndex) scop.Op { + return scop.MakeDroppedIndexDeleteOnly{ + TableID: this.TableID, + IndexID: this.Index.ID, + } + }, + }, + }, + scpb.State_DELETE_ONLY: { + { + nextState: scpb.State_ABSENT, + op: func(this *scpb.PrimaryIndex) scop.Op { + return scop.MakeIndexAbsent{ + TableID: this.TableID, + IndexID: this.Index.ID, + } + }, + }, + }, + }, + }, +} From 40f58fc0190738d100fe463f321aaa816a79a603 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:02 -0500 Subject: [PATCH 07/15] schemachange/scexec: add library to execute schema change operations `scexec` contains a framework, the `Executor, `for executing schema change operations which are specified in terms of low-level `Op`s. `scmutationexec` contains implementations of `Op`s which mutate (specific elements on) descriptors. These correspond to moving indexes, columns, etc., through the familiar non-public states. Release note: None --- pkg/sql/schemachanger/scexec/BUILD.bazel | 126 +++++ .../scexec/descriptorutils/BUILD.bazel | 13 + .../scexec/descriptorutils/helpers.go | 49 ++ pkg/sql/schemachanger/scexec/executor.go | 201 ++++++++ .../scexec/executor_external_test.go | 439 ++++++++++++++++++ pkg/sql/schemachanger/scexec/main_test.go | 32 ++ .../scexec/mutation_desc_getter.go | 42 ++ .../scexec/scmutationexec/BUILD.bazel | 20 + .../scexec/scmutationexec/helpers.go | 120 +++++ .../scexec/scmutationexec/scmutationexec.go | 328 +++++++++++++ 10 files changed, 1370 insertions(+) create mode 100644 pkg/sql/schemachanger/scexec/BUILD.bazel create mode 100644 pkg/sql/schemachanger/scexec/descriptorutils/BUILD.bazel create mode 100644 pkg/sql/schemachanger/scexec/descriptorutils/helpers.go create mode 100644 pkg/sql/schemachanger/scexec/executor.go create mode 100644 pkg/sql/schemachanger/scexec/executor_external_test.go create mode 100644 pkg/sql/schemachanger/scexec/main_test.go create mode 100644 pkg/sql/schemachanger/scexec/mutation_desc_getter.go create mode 100644 pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel create mode 100644 pkg/sql/schemachanger/scexec/scmutationexec/helpers.go create mode 100644 pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go diff --git a/pkg/sql/schemachanger/scexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/BUILD.bazel new file mode 100644 index 000000000000..9a8caafa287f --- /dev/null +++ b/pkg/sql/schemachanger/scexec/BUILD.bazel @@ -0,0 +1,126 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "executor", + srcs = ["executor.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/executor", + visibility = ["//visibility:public"], + deps = [ + "//pkg/keys", + "//pkg/kv", + "//pkg/roachpb", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", + "//pkg/sql/schemachanger/ops", + "//pkg/sql/schemachanger/targets", + "//pkg/sql/sem/tree", + "//pkg/util/log", + "//pkg/util/protoutil", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "executor_test", + srcs = [ + "executor_external_test.go", + "main_test.go", + ], + deps = [ + ":executor", + "//pkg/base", + "//pkg/kv", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/settings/cluster", + "//pkg/sql", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/lease", + "//pkg/sql/catalog/resolver", + "//pkg/sql/catalog/tabledesc", + "//pkg/sql/parser", + "//pkg/sql/schemachanger/builder", + "//pkg/sql/schemachanger/compiler", + "//pkg/sql/schemachanger/ops", + "//pkg/sql/schemachanger/targets", + "//pkg/sql/sem/tree", + "//pkg/sql/sessiondatapb", + "//pkg/sql/sqlutil", + "//pkg/sql/types", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/randutil", + "//pkg/util/retry", + "@com_github_stretchr_testify//require", + ], +) + +go_library( + name = "scexec", + srcs = [ + "executor.go", + "mutation_desc_getter.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec", + visibility = ["//visibility:public"], + deps = [ + "//pkg/keys", + "//pkg/kv", + "//pkg/roachpb", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/tabledesc", + "//pkg/sql/schemachanger/scexec/descriptorutils", + "//pkg/sql/schemachanger/scexec/scmutationexec", + "//pkg/sql/schemachanger/scop", + "//pkg/sql/sem/tree", + "//pkg/util/log", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "scexec_test", + srcs = [ + "executor_external_test.go", + "main_test.go", + ], + deps = [ + ":scexec", + "//pkg/base", + "//pkg/kv", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/settings/cluster", + "//pkg/sql", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/lease", + "//pkg/sql/catalog/resolver", + "//pkg/sql/catalog/tabledesc", + "//pkg/sql/parser", + "//pkg/sql/schemachanger/scbuild", + "//pkg/sql/schemachanger/scop", + "//pkg/sql/schemachanger/scpb", + "//pkg/sql/schemachanger/scplan", + "//pkg/sql/sem/tree", + "//pkg/sql/sessiondatapb", + "//pkg/sql/sqlutil", + "//pkg/sql/types", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/randutil", + "//pkg/util/retry", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/sql/schemachanger/scexec/descriptorutils/BUILD.bazel b/pkg/sql/schemachanger/scexec/descriptorutils/BUILD.bazel new file mode 100644 index 000000000000..6d6af7db2b2b --- /dev/null +++ b/pkg/sql/schemachanger/scexec/descriptorutils/BUILD.bazel @@ -0,0 +1,13 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "descriptorutils", + srcs = ["helpers.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/descriptorutils", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "@com_github_cockroachdb_errors//:errors", + ], +) diff --git a/pkg/sql/schemachanger/scexec/descriptorutils/helpers.go b/pkg/sql/schemachanger/scexec/descriptorutils/helpers.go new file mode 100644 index 000000000000..dad331f8b2e3 --- /dev/null +++ b/pkg/sql/schemachanger/scexec/descriptorutils/helpers.go @@ -0,0 +1,49 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package descriptorutils + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/errors" +) + +// GetIndexMutation returns a reference to a specified index add/drop mutation +// on a table. +func GetIndexMutation( + table catalog.TableDescriptor, idxID descpb.IndexID, +) (mut *descpb.DescriptorMutation, sliceIdx int, err error) { + mutations := table.TableDesc().Mutations + for i := range mutations { + mut := &mutations[i] + idx := mut.GetIndex() + if idx != nil && idx.ID == idxID { + return mut, i, nil + } + } + return nil, 0, errors.AssertionFailedf("mutation not found") +} + +// GetColumnMutation returns a reference to a specified column add/drop mutation +// on a table. +func GetColumnMutation( + table catalog.TableDescriptor, colID descpb.ColumnID, +) (mut *descpb.DescriptorMutation, sliceIdx int, err error) { + mutations := table.TableDesc().Mutations + for i := range mutations { + mut := &mutations[i] + col := mut.GetColumn() + if col != nil && col.ID == colID { + return mut, i, nil + } + } + return nil, 0, errors.AssertionFailedf("mutation not found") +} diff --git a/pkg/sql/schemachanger/scexec/executor.go b/pkg/sql/schemachanger/scexec/executor.go new file mode 100644 index 000000000000..845492702d7f --- /dev/null +++ b/pkg/sql/schemachanger/scexec/executor.go @@ -0,0 +1,201 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scexec + +import ( + "context" + "time" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/descriptorutils" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" +) + +// An Executor executes ops generated during planning. It mostly holds +// dependencies for execution and has little additional logic of its own. +type Executor struct { + txn *kv.Txn + descsCollection *descs.Collection + codec keys.SQLCodec + indexBackfiller IndexBackfiller + jobTracker JobProgressTracker +} + +// NewExecutor creates a new Executor. +func NewExecutor( + txn *kv.Txn, + descsCollection *descs.Collection, + codec keys.SQLCodec, + backfiller IndexBackfiller, + tracker JobProgressTracker, +) *Executor { + return &Executor{ + txn: txn, + descsCollection: descsCollection, + codec: codec, + indexBackfiller: backfiller, + jobTracker: tracker, + } +} + +// ExecuteOps executes the provided ops. The ops must all be of the same type. +func (ex *Executor) ExecuteOps(ctx context.Context, toExecute scop.Ops) error { + switch typ := toExecute.Type(); typ { + case scop.MutationType: + return ex.executeDescriptorMutationOps(ctx, toExecute.Slice()) + case scop.BackfillType: + return ex.executeBackfillOps(ctx, toExecute.Slice()) + case scop.ValidationType: + return ex.executeValidationOps(ctx, toExecute.Slice()) + default: + return errors.AssertionFailedf("unknown ops type %d", typ) + } +} + +func (ex *Executor) executeValidationOps(ctx context.Context, execute []scop.Op) error { + log.Errorf(ctx, "not implemented") + return nil +} + +func (ex *Executor) executeBackfillOps(ctx context.Context, execute []scop.Op) error { + // TODO(ajwerner): Run backfills in parallel. Will require some plumbing for + // checkpointing at the very least. + + for _, op := range execute { + var err error + switch op := op.(type) { + case scop.BackfillIndex: + err = ex.executeIndexBackfillOp(ctx, op) + default: + panic("unimplemented") + } + if err != nil { + return err + } + } + return nil +} + +func (ex *Executor) executeIndexBackfillOp(ctx context.Context, op scop.BackfillIndex) error { + // Note that the leasing here is subtle. We'll avoid the cache and ensure that + // the descriptor is read from the store. That means it will not be leased. + // This relies on changed to the descriptor not messing with this index + // backfill. + table, err := ex.descsCollection.GetImmutableTableByID(ctx, ex.txn, op.TableID, tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, + RequireMutable: false, + AvoidCached: true, + }, + }) + if err != nil { + return err + } + mut, _, err := descriptorutils.GetIndexMutation(table, op.IndexID) + if err != nil { + return err + } + + // Must be the right index given the above call. + idxToBackfill := mut.GetIndex() + + // Split off the index span prior to backfilling. + if err := ex.maybeSplitIndexSpans(ctx, table.IndexSpan(ex.codec, idxToBackfill.ID)); err != nil { + return err + } + return ex.indexBackfiller.BackfillIndex(ctx, ex.jobTracker, table, table.GetPrimaryIndexID(), idxToBackfill.ID) +} + +// IndexBackfiller is an abstract index backfiller that performs index backfills +// when provided with a specification of tables and indexes and a way to track +// job progress. +type IndexBackfiller interface { + BackfillIndex( + ctx context.Context, + _ JobProgressTracker, + _ catalog.TableDescriptor, + source descpb.IndexID, + destinations ...descpb.IndexID, + ) error +} + +// JobProgressTracker abstracts the infrastructure to read and write backfill +// progress to job state. +type JobProgressTracker interface { + + // This interface is implicitly implying that there is only one stage of + // index backfills for a given table in a schema change. It implies that + // because it assumes that it's safe and reasonable to just store one set of + // resume spans per table on the job. + // + // Potentially something close to interface could still work if there were + // multiple stages of backfills for a table if we tracked which stage this + // were somehow. Maybe we could do something like increment a stage counter + // per table after finishing the backfills. + // + // It definitely is possible that there are multiple index backfills on a + // table in the context of a single schema change that changes the set of + // columns (primary index) and adds secondary indexes. + // + // Really this complexity arises in the computation of the fraction completed. + // We'll want to know whether there are more index backfills to come. + // + // One idea is to index secondarily on the source index. + + GetResumeSpans(ctx context.Context, tableID descpb.ID, indexID descpb.IndexID) ([]roachpb.Span, error) + SetResumeSpans(ctx context.Context, tableID descpb.ID, indexID descpb.IndexID, total, done []roachpb.Span) error +} + +func (ex *Executor) maybeSplitIndexSpans(ctx context.Context, span roachpb.Span) error { + // Only perform splits on the system tenant. + if !ex.codec.ForSystemTenant() { + return nil + } + const backfillSplitExpiration = time.Hour + expirationTime := ex.txn.DB().Clock().Now().Add(backfillSplitExpiration.Nanoseconds(), 0) + return ex.txn.DB().AdminSplit(ctx, span.Key, expirationTime) +} + +func (ex *Executor) executeDescriptorMutationOps(ctx context.Context, ops []scop.Op) error { + dg := &mutationDescGetter{ + descs: ex.descsCollection, + txn: ex.txn, + } + v := scmutationexec.NewMutationVisitor(dg) + for _, op := range ops { + if err := op.(scop.MutationOp).Visit(ctx, v); err != nil { + return err + } + } + ba := ex.txn.NewBatch() + for _, id := range dg.retrieved.Ordered() { + desc, err := ex.descsCollection.GetMutableDescriptorByID(ctx, id, ex.txn) + if err != nil { + return errors.NewAssertionErrorWithWrappedErrf(err, "failed to retrieve modified descriptor") + } + if err := ex.descsCollection.WriteDescToBatch(ctx, false, desc, ba); err != nil { + return err + } + } + if err := ex.txn.Run(ctx, ba); err != nil { + return errors.Wrap(err, "writing descriptors") + } + return nil +} diff --git a/pkg/sql/schemachanger/scexec/executor_external_test.go b/pkg/sql/schemachanger/scexec/executor_external_test.go new file mode 100644 index 000000000000..108b1b4fa116 --- /dev/null +++ b/pkg/sql/schemachanger/scexec/executor_external_test.go @@ -0,0 +1,439 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scexec_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/retry" + "github.com/stretchr/testify/require" +) + +type testInfra struct { + tc *testcluster.TestCluster + settings *cluster.Settings + ie sqlutil.InternalExecutor + db *kv.DB + lm *lease.Manager + tsql *sqlutils.SQLRunner +} + +func setupTestInfra(t testing.TB) *testInfra { + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + return &testInfra{ + tc: tc, + settings: tc.Server(0).ClusterSettings(), + ie: tc.Server(0).InternalExecutor().(sqlutil.InternalExecutor), + db: tc.Server(0).DB(), + lm: tc.Server(0).LeaseManager().(*lease.Manager), + tsql: sqlutils.MakeSQLRunner(tc.ServerConn(0)), + } +} + +func (ti *testInfra) txn( + ctx context.Context, + f func(ctx context.Context, txn *kv.Txn, descriptors *descs.Collection) error, +) error { + return descs.Txn(ctx, ti.settings, ti.lm, ti.ie, ti.db, f) +} + +func TestExecutorDescriptorMutationOps(t *testing.T) { + defer leaktest.AfterTest(t)() + + type testCase struct { + name string + orig, exp func() catalog.TableDescriptor + ops func() scop.Ops + } + var table *tabledesc.Mutable + makeTable := func(f func(mutable *tabledesc.Mutable)) func() catalog.TableDescriptor { + return func() catalog.TableDescriptor { + cpy := tabledesc.NewExistingMutable( + *table.ImmutableCopy().(catalog.TableDescriptor).TableDesc()) + if f != nil { + f(cpy) + } + return cpy.ImmutableCopy().(catalog.TableDescriptor) + } + } + mutFlags := tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, + RequireMutable: true, + AvoidCached: true, + }, + } + immFlags := tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, + AvoidCached: true, + }, + } + run := func(t *testing.T, c testCase) { + ctx := context.Background() + ti := setupTestInfra(t) + defer ti.tc.Stopper().Stop(ctx) + + ti.tsql.Exec(t, `CREATE DATABASE db`) + ti.tsql.Exec(t, ` +CREATE TABLE db.t ( + i INT PRIMARY KEY +)`) + + tn := tree.MakeTableName("db", "t") + require.NoError(t, ti.txn(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) (err error) { + if _, table, err = descriptors.GetMutableTableByName( + ctx, txn, &tn, mutFlags, + ); err != nil { + return err + } + return nil + })) + + require.NoError(t, ti.txn(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) error { + ex := scexec.NewExecutor(txn, descriptors, ti.lm.Codec(), nil, nil) + _, orig, err := descriptors.GetImmutableTableByName(ctx, txn, &tn, immFlags) + require.NoError(t, err) + require.Equal(t, c.orig(), orig) + require.NoError(t, ex.ExecuteOps(ctx, c.ops())) + _, after, err := descriptors.GetImmutableTableByName(ctx, txn, &tn, immFlags) + require.NoError(t, err) + require.Equal(t, c.exp(), after) + return nil + })) + } + + indexToAdd := descpb.IndexDescriptor{ + ID: 2, + Name: "foo", + ColumnIDs: []descpb.ColumnID{1}, + ColumnNames: []string{"i"}, + ColumnDirections: []descpb.IndexDescriptor_Direction{ + descpb.IndexDescriptor_ASC, + }, + } + for _, tc := range []testCase{ + { + name: "add index", + orig: makeTable(nil), + exp: makeTable(func(mutable *tabledesc.Mutable) { + mutable.MaybeIncrementVersion() + mutable.NextIndexID++ + mutable.Mutations = append(mutable.Mutations, descpb.DescriptorMutation{ + Descriptor_: &descpb.DescriptorMutation_Index{ + Index: &indexToAdd, + }, + State: descpb.DescriptorMutation_DELETE_ONLY, + Direction: descpb.DescriptorMutation_ADD, + MutationID: mutable.NextMutationID, + }) + mutable.NextMutationID++ + }), + ops: func() scop.Ops { + return scop.MakeOps( + scop.MakeAddedIndexDeleteOnly{ + TableID: table.ID, + Index: indexToAdd, + }, + ) + }, + }, + { + name: "add check constraint", + orig: makeTable(nil), + exp: makeTable(func(mutable *tabledesc.Mutable) { + mutable.MaybeIncrementVersion() + mutable.Checks = append(mutable.Checks, &descpb.TableDescriptor_CheckConstraint{ + Expr: "i > 1", + Name: "check_foo", + Validity: descpb.ConstraintValidity_Validating, + ColumnIDs: []descpb.ColumnID{1}, + IsNonNullConstraint: false, + Hidden: false, + }) + }), + ops: func() scop.Ops { + return scop.MakeOps( + scop.AddCheckConstraint{ + TableID: table.GetID(), + Name: "check_foo", + Expr: "i > 1", + ColumnIDs: []descpb.ColumnID{1}, + Unvalidated: false, + Hidden: false, + }, + ) + }, + }, + } { + t.Run(tc.name, func(t *testing.T) { + run(t, tc) + }) + } +} + +// TODO(ajwerner): Move this out into the schemachanger_test package once that +// is fixed up. +func TestSchemaChanger(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + t.Run("add column", func(t *testing.T) { + ti := setupTestInfra(t) + defer ti.tc.Stopper().Stop(ctx) + ti.tsql.Exec(t, `CREATE DATABASE db`) + ti.tsql.Exec(t, `CREATE TABLE db.foo (i INT PRIMARY KEY)`) + + var id descpb.ID + var ts []*scpb.Node + var targetSlice []*scpb.Target + require.NoError(t, ti.txn(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) (err error) { + tn := tree.MakeTableName("db", "foo") + _, fooTable, err := descriptors.GetImmutableTableByName(ctx, txn, &tn, tree.ObjectLookupFlagsWithRequired()) + require.NoError(t, err) + id = fooTable.GetID() + + // Corresponds to: + // + // ALTER TABLE foo ADD COLUMN j INT; + // + targetSlice = []*scpb.Target{ + scpb.NewTarget(scpb.Target_ADD, &scpb.PrimaryIndex{ + TableID: fooTable.GetID(), + Index: descpb.IndexDescriptor{ + Name: "new_primary_key", + ID: 2, + ColumnIDs: []descpb.ColumnID{1}, + ColumnNames: []string{"i"}, + ColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC}, + Unique: true, + Type: descpb.IndexDescriptor_FORWARD, + }, + OtherPrimaryIndexID: fooTable.GetPrimaryIndexID(), + StoreColumnIDs: []descpb.ColumnID{2}, + StoreColumnNames: []string{"j"}, + }), + scpb.NewTarget(scpb.Target_ADD, &scpb.Column{ + TableID: fooTable.GetID(), + FamilyID: descpb.FamilyID(0), + FamilyName: "primary", + Column: descpb.ColumnDescriptor{ + Name: "j", + ID: 2, + Type: types.Int, + Nullable: true, + PGAttributeNum: 2, + }, + }), + scpb.NewTarget(scpb.Target_DROP, &scpb.PrimaryIndex{ + TableID: fooTable.GetID(), + Index: descpb.IndexDescriptor{ + Name: "primary", + ID: 1, + ColumnIDs: []descpb.ColumnID{1}, + ColumnNames: []string{"i"}, + ColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC}, + Unique: true, + Type: descpb.IndexDescriptor_FORWARD, + }, + OtherPrimaryIndexID: 2, + StoreColumnIDs: []descpb.ColumnID{}, + StoreColumnNames: []string{}, + }), + } + + targetStates := []*scpb.Node{ + { + Target: targetSlice[0], + State: scpb.State_ABSENT, + }, + { + Target: targetSlice[1], + State: scpb.State_ABSENT, + }, + { + Target: targetSlice[2], + State: scpb.State_PUBLIC, + }, + } + + for _, phase := range []scplan.Phase{ + scplan.PostStatementPhase, + scplan.PreCommitPhase, + } { + sc, err := scplan.MakePlan(targetStates, scplan.Params{ + ExecutionPhase: phase, + }) + require.NoError(t, err) + stages := sc.Stages + for _, s := range stages { + exec := scexec.NewExecutor( + txn, + descriptors, + ti.lm.Codec(), + noopBackfiller{}, + nil, + ) + require.NoError(t, exec.ExecuteOps(ctx, s.Ops)) + ts = s.After + } + } + return nil + })) + var after []*scpb.Node + require.NoError(t, ti.txn(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) error { + sc, err := scplan.MakePlan(ts, scplan.Params{ + ExecutionPhase: scplan.PostCommitPhase, + }) + require.NoError(t, err) + for _, s := range sc.Stages { + exec := scexec.NewExecutor(txn, descriptors, ti.lm.Codec(), noopBackfiller{}, nil) + require.NoError(t, exec.ExecuteOps(ctx, s.Ops)) + after = s.After + } + return nil + })) + require.Equal(t, []*scpb.Node{ + { + Target: targetSlice[0], + State: scpb.State_PUBLIC, + }, + { + Target: targetSlice[1], + State: scpb.State_PUBLIC, + }, + { + Target: targetSlice[2], + State: scpb.State_ABSENT, + }, + }, after) + _, err := ti.lm.WaitForOneVersion(ctx, id, retry.Options{}) + require.NoError(t, err) + ti.tsql.Exec(t, "INSERT INTO db.foo VALUES (1, 1)") + }) + t.Run("with builder", func(t *testing.T) { + ti := setupTestInfra(t) + defer ti.tc.Stopper().Stop(ctx) + ti.tsql.Exec(t, `CREATE DATABASE db`) + ti.tsql.Exec(t, `CREATE TABLE db.foo (i INT PRIMARY KEY)`) + + var id descpb.ID + var ts []*scpb.Node + require.NoError(t, ti.txn(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) (err error) { + tn := tree.MakeTableName("db", "foo") + _, fooTable, err := descriptors.GetImmutableTableByName(ctx, txn, &tn, tree.ObjectLookupFlagsWithRequired()) + require.NoError(t, err) + id = fooTable.GetID() + + execCfg := ti.tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) + ip, cleanup := sql.NewInternalPlanner( + "foo", + kv.NewTxn(context.Background(), ti.db, ti.tc.Server(0).NodeID()), + security.RootUserName(), + &sql.MemoryMetrics{}, + &execCfg, + sessiondatapb.SessionData{}, + ) + planner := ip.(interface { + resolver.SchemaResolver + SemaCtx() *tree.SemaContext + EvalContext() *tree.EvalContext + }) + defer cleanup() + b := scbuild.NewBuilder(planner, planner.SemaCtx(), planner.EvalContext()) + parsed, err := parser.Parse("ALTER TABLE db.foo ADD COLUMN j INT") + require.NoError(t, err) + require.Len(t, parsed, 1) + targetStates, err := b.AlterTable(ctx, nil, parsed[0].AST.(*tree.AlterTable)) + require.NoError(t, err) + + for _, phase := range []scplan.Phase{ + scplan.PostStatementPhase, + scplan.PreCommitPhase, + } { + sc, err := scplan.MakePlan(targetStates, scplan.Params{ + ExecutionPhase: phase, + }) + require.NoError(t, err) + for _, s := range sc.Stages { + require.NoError(t, scexec.NewExecutor(txn, descriptors, ti.lm.Codec(), noopBackfiller{}, nil).ExecuteOps(ctx, s.Ops)) + ts = s.After + } + } + return nil + })) + require.NoError(t, ti.txn(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) error { + sc, err := scplan.MakePlan(ts, scplan.Params{ + ExecutionPhase: scplan.PostCommitPhase, + }) + require.NoError(t, err) + for _, s := range sc.Stages { + exec := scexec.NewExecutor(txn, descriptors, ti.lm.Codec(), noopBackfiller{}, nil) + require.NoError(t, exec.ExecuteOps(ctx, s.Ops)) + } + return nil + })) + _, err := ti.lm.WaitForOneVersion(ctx, id, retry.Options{}) + require.NoError(t, err) + ti.tsql.Exec(t, "INSERT INTO db.foo VALUES (1, 1)") + }) +} + +type noopBackfiller struct{} + +func (n noopBackfiller) BackfillIndex( + ctx context.Context, + _ scexec.JobProgressTracker, + _ catalog.TableDescriptor, + source descpb.IndexID, + destinations ...descpb.IndexID, +) error { + return nil +} + +var _ scexec.IndexBackfiller = noopBackfiller{} diff --git a/pkg/sql/schemachanger/scexec/main_test.go b/pkg/sql/schemachanger/scexec/main_test.go new file mode 100644 index 000000000000..bfef565818b7 --- /dev/null +++ b/pkg/sql/schemachanger/scexec/main_test.go @@ -0,0 +1,32 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. +package scexec_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} + +//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go diff --git a/pkg/sql/schemachanger/scexec/mutation_desc_getter.go b/pkg/sql/schemachanger/scexec/mutation_desc_getter.go new file mode 100644 index 000000000000..03af22a8028f --- /dev/null +++ b/pkg/sql/schemachanger/scexec/mutation_desc_getter.go @@ -0,0 +1,42 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scexec + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" +) + +type mutationDescGetter struct { + descs *descs.Collection + txn *kv.Txn + retrieved catalog.DescriptorIDSet +} + +func (m *mutationDescGetter) GetMutableTableByID( + ctx context.Context, id descpb.ID, +) (*tabledesc.Mutable, error) { + table, err := m.descs.GetMutableTableVersionByID(ctx, id, m.txn) + if err != nil { + return nil, err + } + table.MaybeIncrementVersion() + m.retrieved.Add(table.GetID()) + return table, nil +} + +var _ scmutationexec.MutableDescGetter = (*mutationDescGetter)(nil) diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel new file mode 100644 index 000000000000..efeb6b51c6eb --- /dev/null +++ b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel @@ -0,0 +1,20 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "scmutationexec", + srcs = [ + "helpers.go", + "scmutationexec.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/tabledesc", + "//pkg/sql/schemachanger/scexec/descriptorutils", + "//pkg/sql/schemachanger/scop", + "//pkg/util/protoutil", + "@com_github_cockroachdb_errors//:errors", + ], +) diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go new file mode 100644 index 000000000000..6c5cad492c79 --- /dev/null +++ b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go @@ -0,0 +1,120 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scmutationexec + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/descriptorutils" + "github.com/cockroachdb/errors" +) + +type mutationSelector func(descriptor catalog.TableDescriptor) (mut *descpb.DescriptorMutation, sliceIdx int, err error) + +func mutationStateChange( + ctx context.Context, + table *tabledesc.Mutable, + f mutationSelector, + exp, next descpb.DescriptorMutation_State, +) error { + mut, _, err := f(table) + if err != nil { + return err + } + if mut.State != exp { + return errors.AssertionFailedf("update mutation for %d from %v to %v: unexpected state: %v", + table.GetID(), exp, mut.State, table) + } + mut.State = next + return nil +} + +func removeMutation( + ctx context.Context, + table *tabledesc.Mutable, + f mutationSelector, + exp descpb.DescriptorMutation_State, +) (descpb.DescriptorMutation, error) { + mut, foundIdx, err := f(table) + if err != nil { + return descpb.DescriptorMutation{}, err + } + cpy := *mut + if mut.State != exp { + return descpb.DescriptorMutation{}, errors.AssertionFailedf( + "remove mutation from %d: unexpected state: got %v, expected %v: %v", + table.GetID(), mut.State, exp, table, + ) + } + table.Mutations = append(table.Mutations[:foundIdx], table.Mutations[foundIdx+1:]...) + return cpy, nil +} + +func getIndexMutation( + idxID descpb.IndexID, +) func(table catalog.TableDescriptor) (mut *descpb.DescriptorMutation, sliceIdx int, err error) { + return func(table catalog.TableDescriptor) (mut *descpb.DescriptorMutation, sliceIdx int, err error) { + return descriptorutils.GetIndexMutation(table, idxID) + } +} + +func getColumnMutation( + colID descpb.ColumnID, +) func(table catalog.TableDescriptor) (mut *descpb.DescriptorMutation, sliceIdx int, err error) { + return func(table catalog.TableDescriptor) (mut *descpb.DescriptorMutation, sliceIdx int, err error) { + return descriptorutils.GetColumnMutation(table, colID) + } +} + +// findFamilyOrdinalForColumnID finds a family which contains the needle column +// id and returns its index in the families slice. +func findFamilyOrdinalForColumnID( + table catalog.TableDescriptor, needle descpb.ColumnID, +) (int, error) { + families := table.GetFamilies() + for i := range families { + for _, colID := range families[i].ColumnIDs { + if colID == needle { + return i, nil + } + } + } + return -1, errors.Errorf("failed to find column family for column %d in table %d: %v", + needle, table.GetID(), table) +} + +// Suppress the linter. +var _ = findFamilyOrdinalForColumnID + +func removeColumnFromFamily(table *tabledesc.Mutable, colID descpb.ColumnID) error { + famIdx, err := findFamilyOrdinalForColumnID(table, colID) + if err != nil { + return errors.WithAssertionFailure(err) + } + f := &table.Families[famIdx] + for i, id := range f.ColumnIDs { + if id == colID { + f.ColumnIDs = append(f.ColumnIDs[:i], f.ColumnIDs[i+1:]...) + f.ColumnNames = append(f.ColumnNames[:i], f.ColumnNames[i+1:]...) + break + } + } + if len(f.ColumnIDs) == 0 { + table.Families = append(table.Families[:famIdx], table.Families[famIdx+1:]...) + } + return nil +} + +// Suppress the linter. +var _ = removeColumnFromFamily diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go b/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go new file mode 100644 index 000000000000..741d9eecb57d --- /dev/null +++ b/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go @@ -0,0 +1,328 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scmutationexec + +import ( + "context" + "sort" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +// MutableDescGetter encapsulates the logic to retrieve descriptors. +// All retrieved descriptors are modified. +type MutableDescGetter interface { + GetMutableTableByID(ctx context.Context, id descpb.ID) (*tabledesc.Mutable, error) +} + +// NewMutationVisitor creates a new scop.MutationVisitor. +func NewMutationVisitor(descs MutableDescGetter) scop.MutationVisitor { + return &visitor{descs: descs} +} + +type visitor struct { + descs MutableDescGetter +} + +func (m *visitor) MakeAddedColumnDeleteAndWriteOnly( + ctx context.Context, op scop.MakeAddedColumnDeleteAndWriteOnly, +) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + return mutationStateChange( + ctx, + table, + getColumnMutation(op.ColumnID), + descpb.DescriptorMutation_DELETE_ONLY, + descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY, + ) +} + +func (m *visitor) MakeColumnPublic(ctx context.Context, op scop.MakeColumnPublic) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + mut, err := removeMutation( + ctx, + table, + getColumnMutation(op.ColumnID), + descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY, + ) + if err != nil { + return err + } + // TODO(ajwerner): Should the op just have the column descriptor? What's the + // type hydration status here? Cloning is going to blow away hydration. Is + // that okay? + table.Columns = append(table.Columns, + *(protoutil.Clone(mut.GetColumn())).(*descpb.ColumnDescriptor)) + return nil +} + +func (m *visitor) MakeDroppedNonPrimaryIndexDeleteAndWriteOnly( + ctx context.Context, op scop.MakeDroppedNonPrimaryIndexDeleteAndWriteOnly, +) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + var idx descpb.IndexDescriptor + for i := range table.Indexes { + if table.Indexes[i].ID != op.IndexID { + continue + } + idx = table.Indexes[i] + table.Indexes = append(table.Indexes[:i], table.Indexes[i+1:]...) + break + } + if idx.ID == 0 { + return errors.AssertionFailedf("failed to find index %d in descriptor %v", + op.IndexID, table) + } + return table.AddIndexMutation(&idx, descpb.DescriptorMutation_DROP) +} + +func (m *visitor) MakeDroppedColumnDeleteAndWriteOnly( + ctx context.Context, op scop.MakeDroppedColumnDeleteAndWriteOnly, +) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + var col descpb.ColumnDescriptor + for i := range table.Columns { + if table.Columns[i].ID != op.ColumnID { + continue + } + col = table.Columns[i] + table.Columns = append(table.Columns[:i], table.Columns[i+1:]...) + break + } + if col.ID == 0 { + return errors.AssertionFailedf("failed to find column %d in %v", col.ID, table) + } + table.AddColumnMutation(&col, descpb.DescriptorMutation_DROP) + return nil +} + +func (m *visitor) MakeDroppedColumnDeleteOnly( + ctx context.Context, op scop.MakeDroppedColumnDeleteOnly, +) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + return mutationStateChange( + ctx, + table, + getColumnMutation(op.ColumnID), + descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY, + descpb.DescriptorMutation_DELETE_ONLY, + ) +} + +func (m *visitor) MakeColumnAbsent(ctx context.Context, op scop.MakeColumnAbsent) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + mut, err := removeMutation( + ctx, + table, + getColumnMutation(op.ColumnID), + descpb.DescriptorMutation_DELETE_ONLY, + ) + if err != nil { + return err + } + col := mut.GetColumn() + table.RemoveColumnFromFamily(col.ID) + return nil +} + +func (m *visitor) MakeAddedIndexDeleteAndWriteOnly( + ctx context.Context, op scop.MakeAddedIndexDeleteAndWriteOnly, +) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + return mutationStateChange( + ctx, + table, + getIndexMutation(op.IndexID), + descpb.DescriptorMutation_DELETE_ONLY, + descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY, + ) +} + +func (m *visitor) MakeAddedColumnDeleteOnly( + ctx context.Context, op scop.MakeAddedColumnDeleteOnly, +) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + + // TODO(ajwerner): deal with ordering the indexes or sanity checking this + // or what-not. + if op.Column.ID >= table.NextColumnID { + table.NextColumnID = op.Column.ID + 1 + } + var foundFamily bool + for i := range table.Families { + fam := &table.Families[i] + if foundFamily = fam.ID == op.FamilyID; foundFamily { + fam.ColumnIDs = append(fam.ColumnIDs, op.Column.ID) + fam.ColumnNames = append(fam.ColumnNames, op.Column.Name) + break + } + } + if !foundFamily { + table.Families = append(table.Families, descpb.ColumnFamilyDescriptor{ + Name: op.FamilyName, + ID: op.FamilyID, + ColumnNames: []string{op.Column.Name}, + ColumnIDs: []descpb.ColumnID{op.Column.ID}, + }) + sort.Slice(table.Families, func(i, j int) bool { + return table.Families[i].ID < table.Families[j].ID + }) + if table.NextFamilyID <= op.FamilyID { + table.NextFamilyID = op.FamilyID + 1 + } + } + table.AddColumnMutation(&op.Column, descpb.DescriptorMutation_ADD) + return nil +} + +func (m *visitor) MakeDroppedIndexDeleteOnly( + ctx context.Context, op scop.MakeDroppedIndexDeleteOnly, +) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + return mutationStateChange( + ctx, + table, + getIndexMutation(op.IndexID), + descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY, + descpb.DescriptorMutation_DELETE_ONLY, + ) +} + +func (m *visitor) MakeDroppedPrimaryIndexDeleteAndWriteOnly( + ctx context.Context, op scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly, +) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + + // NOTE: There is no ordering guarantee between operations which might + // touch the primary index. Remove it if it has not already been overwritten. + if table.PrimaryIndex.ID == op.Index.ID { + table.PrimaryIndex = descpb.IndexDescriptor{} + } + + idx := protoutil.Clone(&op.Index).(*descpb.IndexDescriptor) + return table.AddIndexMutation(idx, descpb.DescriptorMutation_DROP) +} + +func (m *visitor) MakeAddedIndexDeleteOnly( + ctx context.Context, op scop.MakeAddedIndexDeleteOnly, +) error { + + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + + // TODO(ajwerner): deal with ordering the indexes or sanity checking this + // or what-not. + if op.Index.ID >= table.NextIndexID { + table.NextIndexID = op.Index.ID + 1 + } + // Make some adjustments to the index descriptor so that it behaves correctly + // as a secondary index while being added. + idx := protoutil.Clone(&op.Index).(*descpb.IndexDescriptor) + return table.AddIndexMutation(idx, descpb.DescriptorMutation_ADD) +} + +func (m *visitor) AddCheckConstraint(ctx context.Context, op scop.AddCheckConstraint) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + ck := &descpb.TableDescriptor_CheckConstraint{ + Expr: op.Expr, + Name: op.Name, + ColumnIDs: op.ColumnIDs, + Hidden: op.Hidden, + } + if op.Unvalidated { + ck.Validity = descpb.ConstraintValidity_Unvalidated + } else { + ck.Validity = descpb.ConstraintValidity_Validating + } + table.Checks = append(table.Checks, ck) + return nil +} + +func (m *visitor) MakeAddedPrimaryIndexPublic( + ctx context.Context, op scop.MakeAddedPrimaryIndexPublic, +) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + if _, err := removeMutation( + ctx, + table, + getIndexMutation(op.Index.ID), + descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY, + ); err != nil { + return err + } + table.PrimaryIndex = *(protoutil.Clone(&op.Index)).(*descpb.IndexDescriptor) + return nil +} + +func (m *visitor) MakeIndexAbsent(ctx context.Context, op scop.MakeIndexAbsent) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + _, err = removeMutation(ctx, table, getIndexMutation(op.IndexID), descpb.DescriptorMutation_DELETE_ONLY) + return err +} + +func (m *visitor) AddColumnFamily(ctx context.Context, op scop.AddColumnFamily) error { + table, err := m.descs.GetMutableTableByID(ctx, op.TableID) + if err != nil { + return err + } + table.AddFamily(op.Family) + if op.Family.ID >= table.NextFamilyID { + table.NextFamilyID = op.Family.ID + 1 + } + return nil +} + +var _ scop.MutationVisitor = (*visitor)(nil) From cd76842fe8e37e17df6be22c83ecadb1952227bd Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:02 -0500 Subject: [PATCH 08/15] sql: add an adapter to the index backfiller for use in scexec This commit introduces `sql.IndexBackfillPlanner`, which is a wrapper that holds dependencies for the index backfiller so that `scexec` can call it via an interface. Release note: None --- pkg/server/server_sql.go | 1 + pkg/sql/BUILD.bazel | 2 + pkg/sql/backfill.go | 15 +- pkg/sql/distsql_plan_backfill.go | 14 +- pkg/sql/exec_util.go | 4 + pkg/sql/execinfrapb/processors_bulk_io.pb.go | 360 ++++++++++++------- pkg/sql/execinfrapb/processors_bulk_io.proto | 5 + pkg/sql/index_backfiller.go | 160 +++++++++ pkg/sql/indexbackfiller_test.go | 2 +- pkg/sql/schema_changer_helpers_test.go | 4 +- 10 files changed, 422 insertions(+), 145 deletions(-) create mode 100644 pkg/sql/index_backfiller.go diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 281d5420862e..56a712ac8d0b 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -600,6 +600,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { } distSQLServer.ServerConfig.SessionBoundInternalExecutorFactory = ieFactory jobRegistry.SetSessionBoundInternalExecutorFactory(ieFactory) + execCfg.IndexBackfiller = sql.NewIndexBackfiller(execCfg, ieFactory) distSQLServer.ServerConfig.ProtectedTimestampProvider = execCfg.ProtectedTimestampProvider diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index ceafad59b2f5..f2eabe46c22c 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -98,6 +98,7 @@ go_library( "grant_revoke.go", "grant_role.go", "group.go", + "index_backfiller.go", "index_join.go", "information_schema.go", "insert.go", @@ -315,6 +316,7 @@ go_library( "//pkg/sql/rowenc", "//pkg/sql/rowexec", "//pkg/sql/schemachange", + "//pkg/sql/schemachanger/scexec", "//pkg/sql/scrub", "//pkg/sql/sem/builtins", "//pkg/sql/sem/transform", diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 81cedf6e2899..7cb84cae21a1 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -169,6 +169,7 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { // mutations. Collect the elements that are part of the mutation. var droppedIndexDescs []descpb.IndexDescriptor var addedIndexSpans []roachpb.Span + var addedIndexes []descpb.IndexID var constraintsToDrop []descpb.ConstraintToUpdate var constraintsToAddBeforeValidation []descpb.ConstraintToUpdate @@ -220,6 +221,7 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { } case *descpb.DescriptorMutation_Index: addedIndexSpans = append(addedIndexSpans, tableDesc.IndexSpan(sc.execCfg.Codec, t.Index.ID)) + addedIndexes = append(addedIndexes, t.Index.ID) case *descpb.DescriptorMutation_Constraint: switch t.Constraint.ConstraintType { case descpb.ConstraintToUpdate_CHECK: @@ -310,7 +312,7 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { // Add new indexes. if len(addedIndexSpans) > 0 { // Check if bulk-adding is enabled and supported by indexes (ie non-unique). - if err := sc.backfillIndexes(ctx, version, addedIndexSpans); err != nil { + if err := sc.backfillIndexes(ctx, version, addedIndexSpans, addedIndexes); err != nil { return err } } @@ -945,6 +947,7 @@ func (sc *SchemaChanger) distIndexBackfill( ctx context.Context, version descpb.DescriptorVersion, targetSpans []roachpb.Span, + addedIndexes []descpb.IndexID, filter backfill.MutationFilter, indexBackfillBatchSize int64, ) error { @@ -1023,7 +1026,7 @@ func (sc *SchemaChanger) distIndexBackfill( planCtx = sc.distSQLPlanner.NewPlanningCtx(ctx, &evalCtx, nil /* planner */, txn, true /* distribute */) chunkSize := sc.getChunkSize(indexBackfillBatchSize) - spec, err := initIndexBackfillerSpec(*tableDesc.TableDesc(), readAsOf, chunkSize) + spec, err := initIndexBackfillerSpec(*tableDesc.TableDesc(), readAsOf, chunkSize, addedIndexes) if err != nil { return err } @@ -1733,7 +1736,10 @@ func (sc *SchemaChanger) validateForwardIndexes( // This operates over multiple goroutines concurrently and is thus not // able to reuse the original kv.Txn safely. func (sc *SchemaChanger) backfillIndexes( - ctx context.Context, version descpb.DescriptorVersion, addingSpans []roachpb.Span, + ctx context.Context, + version descpb.DescriptorVersion, + addingSpans []roachpb.Span, + addedIndexes []descpb.IndexID, ) error { log.Infof(ctx, "backfilling %d indexes", len(addingSpans)) @@ -1754,7 +1760,8 @@ func (sc *SchemaChanger) backfillIndexes( } if err := sc.distIndexBackfill( - ctx, version, addingSpans, backfill.IndexMutationFilter, indexBackfillBatchSize); err != nil { + ctx, version, addingSpans, addedIndexes, backfill.IndexMutationFilter, indexBackfillBatchSize, + ); err != nil { return err } diff --git a/pkg/sql/distsql_plan_backfill.go b/pkg/sql/distsql_plan_backfill.go index d446445ea306..21712572a497 100644 --- a/pkg/sql/distsql_plan_backfill.go +++ b/pkg/sql/distsql_plan_backfill.go @@ -34,13 +34,17 @@ func initColumnBackfillerSpec( } func initIndexBackfillerSpec( - desc descpb.TableDescriptor, readAsOf hlc.Timestamp, chunkSize int64, + desc descpb.TableDescriptor, + readAsOf hlc.Timestamp, + chunkSize int64, + indexesToBackfill []descpb.IndexID, ) (execinfrapb.BackfillerSpec, error) { return execinfrapb.BackfillerSpec{ - Table: desc, - ReadAsOf: readAsOf, - Type: execinfrapb.BackfillerSpec_Index, - ChunkSize: chunkSize, + Table: desc, + ReadAsOf: readAsOf, + Type: execinfrapb.BackfillerSpec_Index, + ChunkSize: chunkSize, + IndexesToBackfill: indexesToBackfill, }, nil } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index bc3d05bed842..b6ff99bc70ce 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -782,6 +782,10 @@ type ExecutorConfig struct { // version` but before executing it. It can carry out arbitrary migrations // that allow us to eventually remove legacy code. VersionUpgradeHook func(ctx context.Context, from, to clusterversion.ClusterVersion) error + + // IndexBackfiller is used to backfill indexes. It is another rather circular + // object which mostly just holds on to an ExecConfig. + IndexBackfiller *IndexBackfillPlanner } // Organization returns the value of cluster.organization. diff --git a/pkg/sql/execinfrapb/processors_bulk_io.pb.go b/pkg/sql/execinfrapb/processors_bulk_io.pb.go index 0ab475caabbb..bbf197a9d798 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.pb.go +++ b/pkg/sql/execinfrapb/processors_bulk_io.pb.go @@ -17,6 +17,7 @@ import descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import time "time" +import github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" import github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security" import github_com_cockroachdb_cockroach_pkg_ccl_streamingccl "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl" @@ -72,7 +73,7 @@ func (x *FileCompression) UnmarshalJSON(data []byte) error { return nil } func (FileCompression) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{0} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{0} } type BackfillerSpec_Type int32 @@ -111,7 +112,7 @@ func (x *BackfillerSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{0, 0} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{0, 0} } // BackfillerSpec is the specification for a "schema change backfiller". @@ -137,13 +138,17 @@ type BackfillerSpec struct { ChunkSize int64 `protobuf:"varint,5,opt,name=chunk_size,json=chunkSize" json:"chunk_size"` // The timestamp to perform index backfill historical scans at. ReadAsOf hlc.Timestamp `protobuf:"bytes,7,opt,name=readAsOf" json:"readAsOf"` + // IndexesToBackfill is the set of indexes to backfill. This is populated only + // starting in 21.1, prior to that the implied index set are those containing + // the mutation ID of the first mutation on the table descriptor. + IndexesToBackfill []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID `protobuf:"varint,8,rep,name=indexes_to_backfill,json=indexesToBackfill,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID" json:"indexes_to_backfill,omitempty"` } func (m *BackfillerSpec) Reset() { *m = BackfillerSpec{} } func (m *BackfillerSpec) String() string { return proto.CompactTextString(m) } func (*BackfillerSpec) ProtoMessage() {} func (*BackfillerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{0} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{0} } func (m *BackfillerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -183,7 +188,7 @@ func (m *JobProgress) Reset() { *m = JobProgress{} } func (m *JobProgress) String() string { return proto.CompactTextString(m) } func (*JobProgress) ProtoMessage() {} func (*JobProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{1} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{1} } func (m *JobProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -246,7 +251,7 @@ func (m *ReadImportDataSpec) Reset() { *m = ReadImportDataSpec{} } func (m *ReadImportDataSpec) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec) ProtoMessage() {} func (*ReadImportDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{2} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{2} } func (m *ReadImportDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -284,7 +289,7 @@ func (m *ReadImportDataSpec_ImportTable) Reset() { *m = ReadImportDataSp func (m *ReadImportDataSpec_ImportTable) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec_ImportTable) ProtoMessage() {} func (*ReadImportDataSpec_ImportTable) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{2, 0} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{2, 0} } func (m *ReadImportDataSpec_ImportTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -323,7 +328,7 @@ func (m *StreamIngestionDataSpec) Reset() { *m = StreamIngestionDataSpec func (m *StreamIngestionDataSpec) String() string { return proto.CompactTextString(m) } func (*StreamIngestionDataSpec) ProtoMessage() {} func (*StreamIngestionDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{3} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{3} } func (m *StreamIngestionDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -366,7 +371,7 @@ func (m *StreamIngestionFrontierSpec) Reset() { *m = StreamIngestionFron func (m *StreamIngestionFrontierSpec) String() string { return proto.CompactTextString(m) } func (*StreamIngestionFrontierSpec) ProtoMessage() {} func (*StreamIngestionFrontierSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{4} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{4} } func (m *StreamIngestionFrontierSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -412,7 +417,7 @@ func (m *BackupDataSpec) Reset() { *m = BackupDataSpec{} } func (m *BackupDataSpec) String() string { return proto.CompactTextString(m) } func (*BackupDataSpec) ProtoMessage() {} func (*BackupDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{5} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{5} } func (m *BackupDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -451,7 +456,7 @@ func (m *RestoreSpanEntry) Reset() { *m = RestoreSpanEntry{} } func (m *RestoreSpanEntry) String() string { return proto.CompactTextString(m) } func (*RestoreSpanEntry) ProtoMessage() {} func (*RestoreSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{6} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{6} } func (m *RestoreSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -489,7 +494,7 @@ func (m *RestoreDataSpec) Reset() { *m = RestoreDataSpec{} } func (m *RestoreDataSpec) String() string { return proto.CompactTextString(m) } func (*RestoreDataSpec) ProtoMessage() {} func (*RestoreDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{7} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{7} } func (m *RestoreDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -523,7 +528,7 @@ func (m *SplitAndScatterSpec) Reset() { *m = SplitAndScatterSpec{} } func (m *SplitAndScatterSpec) String() string { return proto.CompactTextString(m) } func (*SplitAndScatterSpec) ProtoMessage() {} func (*SplitAndScatterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{8} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{8} } func (m *SplitAndScatterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -556,7 +561,7 @@ func (m *SplitAndScatterSpec_RestoreEntryChunk) Reset() { *m = SplitAndS func (m *SplitAndScatterSpec_RestoreEntryChunk) String() string { return proto.CompactTextString(m) } func (*SplitAndScatterSpec_RestoreEntryChunk) ProtoMessage() {} func (*SplitAndScatterSpec_RestoreEntryChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{8, 0} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{8, 0} } func (m *SplitAndScatterSpec_RestoreEntryChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -603,7 +608,7 @@ func (m *CSVWriterSpec) Reset() { *m = CSVWriterSpec{} } func (m *CSVWriterSpec) String() string { return proto.CompactTextString(m) } func (*CSVWriterSpec) ProtoMessage() {} func (*CSVWriterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{9} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{9} } func (m *CSVWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -638,7 +643,7 @@ func (m *BulkRowWriterSpec) Reset() { *m = BulkRowWriterSpec{} } func (m *BulkRowWriterSpec) String() string { return proto.CompactTextString(m) } func (*BulkRowWriterSpec) ProtoMessage() {} func (*BulkRowWriterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{10} + return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{10} } func (m *BulkRowWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -738,6 +743,13 @@ func (m *BackfillerSpec) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n2 + if len(m.IndexesToBackfill) > 0 { + for _, num := range m.IndexesToBackfill { + dAtA[i] = 0x40 + i++ + i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(num)) + } + } return i, nil } @@ -1434,6 +1446,11 @@ func (m *BackfillerSpec) Size() (n int) { n += 1 + sovProcessorsBulkIo(uint64(m.ChunkSize)) l = m.ReadAsOf.Size() n += 1 + l + sovProcessorsBulkIo(uint64(l)) + if len(m.IndexesToBackfill) > 0 { + for _, e := range m.IndexesToBackfill { + n += 1 + sovProcessorsBulkIo(uint64(e)) + } + } return n } @@ -1905,6 +1922,79 @@ func (m *BackfillerSpec) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 8: + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsBulkIo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.IndexesToBackfill = append(m.IndexesToBackfill, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsBulkIo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthProcessorsBulkIo + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.IndexesToBackfill) == 0 { + m.IndexesToBackfill = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID, 0, elementCount) + } + for iNdEx < postIndex { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsBulkIo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.IndexesToBackfill = append(m.IndexesToBackfill, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field IndexesToBackfill", wireType) + } default: iNdEx = preIndex skippy, err := skipProcessorsBulkIo(dAtA[iNdEx:]) @@ -4382,125 +4472,127 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_4b564fcb5cc1c063) + proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_ac83a229416c6ca4) } -var fileDescriptor_processors_bulk_io_4b564fcb5cc1c063 = []byte{ - // 1842 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0xcf, 0x52, 0x1b, 0xc9, - 0x19, 0x67, 0xf4, 0x0f, 0xe9, 0x93, 0x01, 0xd1, 0xf6, 0xee, 0x4e, 0x48, 0x05, 0x28, 0xed, 0xe2, - 0x28, 0x4e, 0x59, 0xaa, 0xb5, 0x93, 0x94, 0x2b, 0xc9, 0xae, 0x83, 0x84, 0xf1, 0x0a, 0x76, 0x6d, - 0x32, 0x32, 0xb8, 0x6a, 0x2b, 0xa9, 0xa9, 0xd6, 0x4c, 0x23, 0xda, 0x1a, 0x4d, 0x0f, 0xdd, 0x3d, - 0x60, 0xf9, 0x92, 0x54, 0xe5, 0x94, 0x5b, 0x1e, 0x21, 0x6f, 0x90, 0x5c, 0xf2, 0x04, 0xb9, 0xf8, - 0xb8, 0x95, 0xd3, 0x56, 0x0e, 0x54, 0x82, 0xdf, 0x22, 0xa7, 0x54, 0xf7, 0xf4, 0x88, 0x01, 0x03, - 0x86, 0x75, 0xed, 0xc5, 0x1e, 0xba, 0xfb, 0xf7, 0xeb, 0xef, 0xff, 0xf7, 0xb5, 0xa0, 0x21, 0xf6, - 0x83, 0x16, 0x79, 0x49, 0x3c, 0x1a, 0xee, 0x72, 0x1c, 0xf5, 0x5b, 0x11, 0x67, 0x1e, 0x11, 0x82, - 0x71, 0xe1, 0xf6, 0xe3, 0x60, 0xe8, 0x52, 0xd6, 0x8c, 0x38, 0x93, 0x0c, 0xd9, 0x1e, 0xf3, 0x86, - 0x9c, 0x61, 0x6f, 0xaf, 0x29, 0xf6, 0x83, 0xa6, 0x4f, 0x85, 0x14, 0xfb, 0x01, 0x8f, 0xc3, 0x85, - 0x0f, 0x5f, 0xb0, 0xbe, 0x68, 0xa9, 0x7f, 0xa2, 0xbe, 0xfe, 0x2f, 0x41, 0x2c, 0xd8, 0xfa, 0x74, - 0xd4, 0x6f, 0x51, 0x76, 0x77, 0x97, 0xf1, 0x11, 0x96, 0xe9, 0xce, 0xc7, 0xea, 0x56, 0x0f, 0x4b, - 0x1c, 0xb0, 0x41, 0xcb, 0x27, 0xc2, 0x8b, 0xfa, 0x2d, 0x21, 0x79, 0xec, 0xc9, 0x98, 0x13, 0xdf, - 0x1c, 0x5a, 0xb9, 0x4c, 0x34, 0x2c, 0x48, 0x7a, 0x4b, 0x2c, 0x69, 0xd0, 0xda, 0x0b, 0xbc, 0x96, - 0xa4, 0x23, 0x22, 0x24, 0x1e, 0x45, 0x66, 0xe7, 0xd6, 0x80, 0x0d, 0x98, 0xfe, 0x6c, 0xa9, 0x2f, - 0xb3, 0x8a, 0x52, 0xa9, 0x7c, 0x2c, 0xb1, 0x59, 0x9b, 0x4f, 0xd7, 0x70, 0x44, 0x93, 0xa5, 0xfa, - 0xdf, 0xf2, 0x30, 0xdb, 0xc6, 0xde, 0x70, 0x97, 0x06, 0x01, 0xe1, 0xbd, 0x88, 0x78, 0xe8, 0x31, - 0x14, 0xe4, 0x38, 0x22, 0xb6, 0xb5, 0x6c, 0x35, 0x66, 0xef, 0xdd, 0x6d, 0x5e, 0x64, 0x90, 0xe6, - 0x69, 0x5c, 0xf3, 0xd9, 0x38, 0x22, 0xed, 0xc2, 0xeb, 0xa3, 0xa5, 0x29, 0x47, 0x13, 0xa0, 0x36, - 0x14, 0x25, 0xee, 0x07, 0xc4, 0xce, 0x2d, 0x5b, 0x8d, 0xea, 0xbd, 0xdb, 0x67, 0x98, 0xc4, 0x7e, - 0xa0, 0xf5, 0x7b, 0xa6, 0xce, 0xac, 0x11, 0xe1, 0x71, 0x1a, 0x49, 0xc6, 0x0d, 0x45, 0x02, 0x45, - 0x8f, 0xa0, 0x28, 0x22, 0x1c, 0x0a, 0x3b, 0xbf, 0x9c, 0x6f, 0x54, 0xef, 0xfd, 0xe4, 0x62, 0x69, - 0x34, 0x8d, 0x43, 0xb0, 0xaf, 0xc4, 0xc1, 0x61, 0x4a, 0xa3, 0xd1, 0xe8, 0x53, 0x28, 0xfb, 0x31, - 0xc7, 0x92, 0xb2, 0xd0, 0x2e, 0x2c, 0x5b, 0x8d, 0x7c, 0xfb, 0x03, 0xb5, 0xfd, 0xbf, 0xa3, 0xa5, - 0x19, 0x65, 0xce, 0xe6, 0x9a, 0xd9, 0x74, 0x26, 0xc7, 0xd0, 0xc7, 0x00, 0xde, 0x5e, 0x1c, 0x0e, - 0x5d, 0x41, 0x5f, 0x11, 0xbb, 0xa8, 0x41, 0x09, 0x67, 0x45, 0xaf, 0xf7, 0xe8, 0x2b, 0x82, 0x1e, - 0x42, 0x99, 0x13, 0xec, 0xaf, 0x8a, 0xa7, 0xbb, 0xf6, 0xb4, 0xd6, 0xf2, 0x47, 0x19, 0x09, 0x95, - 0xcb, 0x9a, 0x7b, 0x81, 0xd7, 0x7c, 0x96, 0xba, 0xcc, 0x30, 0x4c, 0x40, 0xf5, 0x3b, 0x50, 0x50, - 0x76, 0x43, 0x55, 0x98, 0xee, 0x86, 0x07, 0x38, 0xa0, 0x7e, 0x6d, 0x0a, 0x01, 0x94, 0x3a, 0x2c, - 0x88, 0x47, 0x61, 0xcd, 0x42, 0x15, 0x28, 0x76, 0x43, 0x9f, 0xbc, 0xac, 0xe5, 0x36, 0x0a, 0xe5, - 0x52, 0x6d, 0xba, 0x7e, 0x08, 0xd5, 0x0d, 0xd6, 0xdf, 0xe2, 0x6c, 0xc0, 0x89, 0x10, 0xe8, 0x13, - 0x28, 0xbd, 0x60, 0x7d, 0x97, 0xfa, 0xda, 0x5f, 0xf9, 0xf6, 0x8c, 0xba, 0xe0, 0xf8, 0x68, 0xa9, - 0xb8, 0xc1, 0xfa, 0xdd, 0x35, 0xa7, 0xf8, 0x82, 0xf5, 0xbb, 0x3e, 0x6a, 0xc0, 0x0d, 0x8f, 0x85, - 0x92, 0xd3, 0x7e, 0xac, 0x6d, 0xa0, 0x3c, 0x92, 0x33, 0xc2, 0x9c, 0xda, 0x41, 0x36, 0x14, 0x44, - 0xc0, 0xa4, 0x9d, 0x5f, 0xb6, 0x1a, 0xc5, 0xd4, 0x9d, 0x6a, 0xa5, 0xfe, 0xba, 0x0c, 0x48, 0xd9, - 0xb7, 0x3b, 0x8a, 0x18, 0x97, 0x6b, 0x58, 0x62, 0x1d, 0x2e, 0x2b, 0x50, 0x15, 0x78, 0x14, 0x05, - 0x24, 0x31, 0x54, 0x2e, 0x83, 0x83, 0x64, 0x43, 0x5b, 0xea, 0x31, 0x94, 0x23, 0x23, 0xb3, 0x5d, - 0xd2, 0x96, 0x5a, 0xb9, 0xd8, 0x97, 0x19, 0x05, 0x53, 0x8b, 0xa5, 0x60, 0xf4, 0x18, 0xf2, 0x31, - 0xa7, 0xf6, 0xb4, 0x8e, 0x87, 0x9f, 0x5f, 0xcc, 0xf1, 0xb6, 0xa8, 0xcd, 0x6d, 0x4e, 0x1f, 0x85, - 0x92, 0x8f, 0x1d, 0xc5, 0x80, 0x3e, 0x83, 0x52, 0x92, 0xae, 0x76, 0x59, 0xcb, 0xb3, 0x94, 0xe1, - 0x32, 0x89, 0xd2, 0xec, 0x3e, 0x5d, 0xa7, 0x01, 0x59, 0xd7, 0xc7, 0x8c, 0x24, 0x06, 0x84, 0x76, - 0xa0, 0xa4, 0x43, 0x54, 0xd8, 0x15, 0x2d, 0xca, 0x83, 0x6b, 0x89, 0xa2, 0xa3, 0x55, 0x68, 0x69, - 0x34, 0xaf, 0xe5, 0x18, 0x36, 0xf4, 0x10, 0x7e, 0x20, 0x86, 0x34, 0x72, 0x47, 0x54, 0x08, 0x1a, - 0x0e, 0xdc, 0x5d, 0xc6, 0x09, 0x1d, 0x84, 0xee, 0x90, 0x8c, 0x85, 0x0d, 0xcb, 0x56, 0xa3, 0x6c, - 0x04, 0xf9, 0x50, 0x1d, 0xfb, 0x2a, 0x39, 0xb5, 0x9e, 0x1c, 0xda, 0x24, 0x63, 0x81, 0xee, 0xc0, - 0xcc, 0x21, 0x0e, 0x02, 0x15, 0xd7, 0x4f, 0x70, 0xc8, 0x84, 0x5d, 0xcd, 0xc4, 0xee, 0xe9, 0x2d, - 0x74, 0x0f, 0xe6, 0xb9, 0x4e, 0x99, 0x2d, 0xcc, 0x71, 0x10, 0x90, 0x80, 0x8a, 0x91, 0x3d, 0x93, - 0x71, 0xe1, 0xdb, 0xdb, 0xe8, 0x6b, 0x00, 0x4e, 0x44, 0x3c, 0x22, 0x6e, 0xc4, 0x84, 0x3d, 0xab, - 0x95, 0xff, 0xd5, 0xb5, 0x94, 0x77, 0x34, 0x7c, 0x8b, 0x25, 0xfa, 0x3b, 0x15, 0x9e, 0xfe, 0x8d, - 0x08, 0x40, 0x2c, 0x08, 0x77, 0x75, 0x71, 0xb2, 0xe7, 0x96, 0xad, 0x46, 0xa5, 0xbd, 0x6e, 0x32, - 0xf5, 0xf3, 0x01, 0x95, 0x7b, 0x71, 0xbf, 0xe9, 0xb1, 0x51, 0x6b, 0x72, 0x9b, 0xdf, 0x3f, 0xf9, - 0x6e, 0x45, 0xc3, 0x41, 0x4b, 0x10, 0x2f, 0xe6, 0x54, 0x8e, 0x9b, 0xbd, 0xdf, 0x7e, 0xb9, 0x2d, - 0x08, 0x0f, 0xf1, 0x88, 0x6c, 0x29, 0x36, 0xa7, 0xa2, 0x98, 0xf5, 0xe7, 0x42, 0x0c, 0xd5, 0x44, - 0x24, 0xed, 0x06, 0xf4, 0x1b, 0x28, 0xa8, 0xea, 0xac, 0x33, 0xe8, 0x7a, 0x75, 0xca, 0x72, 0x34, - 0x12, 0x7d, 0x02, 0x20, 0x31, 0x1f, 0x10, 0xd9, 0x61, 0x81, 0xb0, 0x73, 0xcb, 0xf9, 0x46, 0xc5, - 0xec, 0x67, 0xd6, 0x17, 0x04, 0x54, 0x33, 0x7e, 0x47, 0x35, 0xc8, 0x0f, 0xc9, 0x58, 0xdf, 0x5a, - 0x71, 0xd4, 0x27, 0x7a, 0x02, 0xc5, 0x03, 0x1c, 0xc4, 0x69, 0xc5, 0xbc, 0x5e, 0x48, 0x65, 0x34, - 0x72, 0x12, 0x9a, 0x5f, 0xe6, 0x1e, 0x58, 0x0b, 0xbf, 0x80, 0x72, 0x1a, 0xf7, 0xd9, 0x1b, 0x8b, - 0xc9, 0x8d, 0xb7, 0xb2, 0x37, 0x56, 0xb2, 0xb8, 0x5f, 0xc3, 0xec, 0x69, 0x3f, 0xbd, 0x0b, 0x9d, - 0xcf, 0xa0, 0x37, 0x0a, 0x65, 0x4b, 0x57, 0xac, 0x7c, 0xad, 0xb0, 0x51, 0x28, 0x17, 0x6a, 0xc5, - 0x8d, 0x42, 0xb9, 0x58, 0x2b, 0x6d, 0x14, 0xca, 0x37, 0x6a, 0x33, 0xf5, 0x7f, 0xe5, 0xe0, 0xa3, - 0x9e, 0xe4, 0x04, 0x8f, 0xba, 0xe1, 0x80, 0x08, 0x55, 0x78, 0x26, 0xf5, 0xe4, 0x0f, 0x70, 0x33, - 0xc2, 0x5c, 0x52, 0xb5, 0xe8, 0x62, 0xdf, 0x57, 0x49, 0x4f, 0x84, 0x6d, 0x69, 0x9b, 0x3e, 0x51, - 0xb1, 0xf0, 0xef, 0xa3, 0xa5, 0xf5, 0x2b, 0xc5, 0x82, 0xe7, 0x05, 0xaa, 0xdf, 0x12, 0x3c, 0xa2, - 0xe1, 0xc0, 0xf3, 0x82, 0xe6, 0x56, 0x4a, 0xbc, 0x9a, 0xf0, 0x3a, 0x28, 0x3a, 0xb3, 0x42, 0x04, - 0x6a, 0x03, 0x08, 0x89, 0xb9, 0x74, 0x55, 0x9a, 0x18, 0x4f, 0x5c, 0xa9, 0xaa, 0x57, 0x34, 0x4c, - 0xad, 0x22, 0x0e, 0xb3, 0xc9, 0xc5, 0xa9, 0x06, 0xba, 0x9e, 0x56, 0xda, 0x9b, 0x46, 0xfe, 0xce, - 0x77, 0x93, 0x3f, 0xb1, 0x59, 0x2a, 0xfc, 0x8c, 0xc8, 0xfe, 0x59, 0xff, 0x87, 0x05, 0x3f, 0x3c, - 0x63, 0xd4, 0x75, 0xce, 0x42, 0x49, 0x4d, 0x5f, 0x77, 0xe0, 0xe6, 0x1e, 0x1d, 0xec, 0xb9, 0x87, - 0x58, 0x12, 0xee, 0x62, 0xe9, 0x6a, 0x71, 0x4d, 0xd0, 0x5f, 0x49, 0xc1, 0x9a, 0xc2, 0x3f, 0x57, - 0xf0, 0x55, 0xd9, 0x53, 0x60, 0xd4, 0x86, 0x19, 0xc9, 0xb1, 0x37, 0x24, 0xbe, 0x9b, 0xb4, 0xe9, - 0x9c, 0x2e, 0x07, 0x1f, 0x9d, 0x53, 0x4a, 0x33, 0x4d, 0xf9, 0x86, 0xc1, 0xa8, 0x25, 0x51, 0xff, - 0xe7, 0x74, 0x32, 0x82, 0xc4, 0xd1, 0x24, 0x06, 0xee, 0xa7, 0x5d, 0xdf, 0xba, 0x0a, 0x9d, 0xe9, - 0xf1, 0x5f, 0x40, 0x8d, 0x86, 0x92, 0x33, 0x3f, 0xf6, 0xae, 0x27, 0xce, 0xdc, 0x09, 0x4c, 0x4b, - 0x84, 0xee, 0x43, 0xd5, 0x27, 0xbb, 0x38, 0x0e, 0xa4, 0xab, 0x5a, 0x4d, 0xe2, 0x3a, 0x64, 0x1a, - 0x2b, 0xac, 0x25, 0x5b, 0xdb, 0x4e, 0xd7, 0x01, 0x73, 0x6c, 0x9b, 0x53, 0xf4, 0x27, 0x0b, 0x6e, - 0xc6, 0x9c, 0x0a, 0xb7, 0x3f, 0x76, 0x03, 0xe6, 0xe1, 0x80, 0xca, 0xb1, 0x3b, 0x3c, 0xb0, 0x0b, - 0x5a, 0x84, 0xcf, 0x2f, 0x1f, 0xa3, 0x4e, 0x74, 0x57, 0x4d, 0x4a, 0xb4, 0xc7, 0x5f, 0x1a, 0x86, - 0xcd, 0x83, 0xa4, 0x47, 0xdc, 0x3a, 0x3e, 0x5a, 0xaa, 0x6d, 0x3b, 0xdd, 0xec, 0xd6, 0x8e, 0x53, - 0x8b, 0xcf, 0x1c, 0x46, 0x0e, 0x54, 0x47, 0x07, 0x9e, 0xe7, 0xee, 0xd2, 0x40, 0x12, 0xae, 0xc7, - 0x96, 0xd9, 0x53, 0xce, 0x4d, 0xf5, 0xff, 0x6a, 0xa7, 0xd3, 0x59, 0xd7, 0x87, 0x4e, 0x34, 0x3b, - 0x59, 0x73, 0x40, 0xb1, 0x24, 0xdf, 0xe8, 0x0b, 0x00, 0x12, 0x7a, 0x7c, 0x1c, 0xe9, 0xd1, 0x21, - 0x69, 0xde, 0x8d, 0x73, 0x28, 0x55, 0xab, 0x7c, 0x34, 0x39, 0xf8, 0x54, 0xff, 0x2b, 0x9c, 0x0c, - 0x16, 0x3d, 0x85, 0xf9, 0xbe, 0xd6, 0xd6, 0xcd, 0x64, 0xd8, 0x35, 0xe6, 0xa6, 0xb9, 0x04, 0xdd, - 0x9b, 0xe4, 0xd9, 0x26, 0x98, 0x25, 0x97, 0x84, 0x7e, 0x42, 0x57, 0xbe, 0x3a, 0xdd, 0x4c, 0x82, - 0x7d, 0x14, 0xfa, 0x9a, 0x6c, 0x1b, 0x4a, 0xd1, 0xd0, 0xa5, 0x7e, 0xda, 0xd1, 0xef, 0x5f, 0xd9, - 0x67, 0x5b, 0xc3, 0xae, 0x6f, 0x9a, 0x79, 0x45, 0xcd, 0x5e, 0x5b, 0x9b, 0xdd, 0x35, 0xe1, 0x14, - 0x23, 0xb5, 0x7c, 0xa6, 0xa7, 0xc1, 0xf7, 0xd5, 0xd3, 0x3a, 0xf0, 0xc1, 0xb9, 0xa1, 0x73, 0x4e, - 0x9b, 0xb9, 0xb8, 0xe8, 0x3f, 0x00, 0x38, 0xd1, 0x25, 0x8b, 0x2c, 0x9c, 0x83, 0x2c, 0x67, 0x90, - 0xf5, 0xbf, 0x5b, 0x50, 0x73, 0x88, 0x90, 0x8c, 0x13, 0x95, 0x44, 0x09, 0xc1, 0xa7, 0x50, 0x50, - 0x79, 0x68, 0x6a, 0xcc, 0x3b, 0xd2, 0x50, 0x1f, 0x45, 0xab, 0x50, 0xdc, 0xa5, 0x6a, 0xaa, 0x4a, - 0x52, 0x77, 0xe5, 0xbc, 0xa1, 0x4c, 0x37, 0x3a, 0x87, 0xec, 0xc7, 0x44, 0x48, 0x1d, 0x75, 0x69, - 0x21, 0xd0, 0x48, 0x74, 0x1b, 0xaa, 0xe9, 0xb4, 0xd8, 0xf5, 0x5f, 0xea, 0xf4, 0x4d, 0xc7, 0x9f, - 0xec, 0x46, 0xfd, 0x8f, 0x79, 0x98, 0x33, 0x22, 0x4f, 0x2a, 0xcf, 0x3a, 0xdc, 0xe0, 0xc9, 0x52, - 0x12, 0x4d, 0xd7, 0xa8, 0x8e, 0x55, 0x03, 0xd4, 0xb1, 0x74, 0x3a, 0x67, 0x72, 0xef, 0x91, 0x33, - 0x5d, 0x28, 0x71, 0xa2, 0x87, 0xbf, 0xe4, 0x09, 0xf4, 0xd3, 0x77, 0x5a, 0xc4, 0xbc, 0x84, 0x86, - 0x64, 0x9c, 0x8e, 0xac, 0x09, 0x81, 0x1a, 0x59, 0x4d, 0x80, 0x27, 0x45, 0xe9, 0x67, 0x97, 0xcd, - 0x17, 0xa7, 0xec, 0x72, 0x69, 0x84, 0xbf, 0x47, 0xd4, 0xfc, 0x35, 0x07, 0x37, 0x7b, 0x51, 0x40, - 0xe5, 0x6a, 0xe8, 0xf7, 0x3c, 0x2c, 0xa5, 0xe9, 0x55, 0xbf, 0x87, 0x92, 0x7e, 0x64, 0xa5, 0x1d, - 0xe0, 0xe1, 0xc5, 0x92, 0x9e, 0x03, 0x4f, 0xa5, 0xd7, 0xf2, 0x74, 0x14, 0x4f, 0x6a, 0x88, 0x84, - 0x34, 0x63, 0xd3, 0xdc, 0x7b, 0xda, 0x74, 0xc1, 0x85, 0xf9, 0xb7, 0x6e, 0x43, 0x1b, 0x30, 0x4d, - 0xd4, 0x9b, 0x8a, 0xa4, 0xf2, 0xdf, 0x79, 0xa7, 0xa5, 0x27, 0x49, 0x63, 0xf8, 0x53, 0x82, 0xfa, - 0x9f, 0xf3, 0x30, 0xd3, 0xe9, 0xed, 0x3c, 0xe7, 0x34, 0x35, 0xce, 0x6d, 0xd5, 0x9e, 0x84, 0xa4, - 0x61, 0xf2, 0x9e, 0xd5, 0x89, 0x9d, 0xc6, 0x60, 0x66, 0x03, 0xfd, 0x18, 0x6e, 0xa8, 0x4a, 0xe1, - 0x46, 0xda, 0x30, 0x49, 0x14, 0x4e, 0x0e, 0xea, 0x1a, 0x92, 0x6c, 0xa0, 0xcf, 0x60, 0x9a, 0x25, - 0x91, 0xa7, 0x93, 0xa5, 0x7a, 0x6e, 0xc3, 0xe8, 0xf4, 0x76, 0x4c, 0x78, 0xa6, 0x12, 0x1a, 0xcc, - 0xc9, 0x4b, 0x99, 0xb3, 0x43, 0x61, 0x9e, 0xd7, 0xd9, 0x97, 0xb2, 0xc3, 0x0e, 0x05, 0xfa, 0x1d, - 0xcc, 0x7b, 0x6c, 0x14, 0xa9, 0xdc, 0x53, 0x83, 0x9d, 0xc7, 0x7c, 0xe2, 0x99, 0xf6, 0x74, 0xc9, - 0xa3, 0x5e, 0xa5, 0x47, 0xe7, 0x04, 0x96, 0xce, 0x21, 0x19, 0xa6, 0x8e, 0x22, 0x3a, 0x53, 0x63, - 0x4b, 0xdf, 0x53, 0x8d, 0xad, 0x3f, 0x87, 0xf9, 0x76, 0x1c, 0x28, 0x85, 0x32, 0xee, 0x98, 0xfc, - 0xcc, 0x61, 0x7d, 0xe7, 0x9f, 0x39, 0xee, 0xac, 0xc0, 0xdc, 0x19, 0x55, 0x51, 0x19, 0x0a, 0x4f, - 0x58, 0x48, 0x6a, 0x53, 0xea, 0xeb, 0xf1, 0x2b, 0x1a, 0xd5, 0xac, 0xf6, 0xdd, 0xd7, 0xff, 0x5d, - 0x9c, 0x7a, 0x7d, 0xbc, 0x68, 0x7d, 0x73, 0xbc, 0x68, 0x7d, 0x7b, 0xbc, 0x68, 0xfd, 0xe7, 0x78, - 0xd1, 0xfa, 0xcb, 0x9b, 0xc5, 0xa9, 0x6f, 0xde, 0x2c, 0x4e, 0x7d, 0xfb, 0x66, 0x71, 0xea, 0xeb, - 0x6a, 0xe6, 0x97, 0xa4, 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0x19, 0x65, 0xa0, 0x8f, 0xf6, 0x12, - 0x00, 0x00, +var fileDescriptor_processors_bulk_io_ac83a229416c6ca4 = []byte{ + // 1886 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4d, 0x6f, 0x1b, 0xc7, + 0xf9, 0xd7, 0xf2, 0x4d, 0xe4, 0x43, 0x53, 0xa6, 0xc6, 0x4e, 0xb2, 0x7f, 0xfd, 0x51, 0x49, 0x60, + 0x62, 0x97, 0x75, 0x61, 0x12, 0xb1, 0xdb, 0xc2, 0x68, 0x9b, 0xb8, 0x22, 0x65, 0x39, 0x94, 0x12, + 0x5b, 0x5d, 0x5a, 0x32, 0x10, 0xb4, 0x58, 0x0c, 0x77, 0x47, 0xd4, 0x98, 0xcb, 0x9d, 0xd5, 0xcc, + 0xac, 0x64, 0xfa, 0xd2, 0x02, 0x3d, 0xf5, 0xd6, 0x8f, 0xd0, 0x43, 0x3f, 0x40, 0x2f, 0xfd, 0x04, + 0xbd, 0xf8, 0x18, 0xf4, 0x14, 0xf4, 0x20, 0xb4, 0xf2, 0xb7, 0xc8, 0xa9, 0x98, 0xd9, 0x59, 0x6a, + 0x25, 0x4b, 0xb2, 0x14, 0x23, 0x17, 0x69, 0x39, 0x33, 0xbf, 0xdf, 0x3e, 0xef, 0xcf, 0x33, 0x0b, + 0x4d, 0xb1, 0x17, 0xb4, 0xc9, 0x4b, 0xe2, 0xd1, 0x70, 0x87, 0xe3, 0x68, 0xd0, 0x8e, 0x38, 0xf3, + 0x88, 0x10, 0x8c, 0x0b, 0x77, 0x10, 0x07, 0x23, 0x97, 0xb2, 0x56, 0xc4, 0x99, 0x64, 0xc8, 0xf6, + 0x98, 0x37, 0xe2, 0x0c, 0x7b, 0xbb, 0x2d, 0xb1, 0x17, 0xb4, 0x7c, 0x2a, 0xa4, 0xd8, 0x0b, 0x78, + 0x1c, 0x2e, 0x7c, 0xf8, 0x82, 0x0d, 0x44, 0x5b, 0xfd, 0x89, 0x06, 0xfa, 0x5f, 0x82, 0x58, 0xb0, + 0xf5, 0xe9, 0x68, 0xd0, 0xa6, 0xec, 0xee, 0x0e, 0xe3, 0x63, 0x2c, 0xd3, 0x9d, 0x8f, 0xd5, 0x5b, + 0x3d, 0x2c, 0x71, 0xc0, 0x86, 0x6d, 0x9f, 0x08, 0x2f, 0x1a, 0xb4, 0x85, 0xe4, 0xb1, 0x27, 0x63, + 0x4e, 0x7c, 0x73, 0xe8, 0xd6, 0x45, 0xa2, 0x61, 0x41, 0xd2, 0xb7, 0xc4, 0x92, 0x06, 0xed, 0xdd, + 0xc0, 0x6b, 0x4b, 0x3a, 0x26, 0x42, 0xe2, 0x71, 0x64, 0x76, 0x6e, 0x0e, 0xd9, 0x90, 0xe9, 0xc7, + 0xb6, 0x7a, 0x32, 0xab, 0x28, 0x95, 0xca, 0xc7, 0x12, 0x9b, 0xb5, 0xf9, 0x74, 0x0d, 0x47, 0x34, + 0x59, 0x6a, 0xfc, 0xad, 0x00, 0x73, 0x1d, 0xec, 0x8d, 0x76, 0x68, 0x10, 0x10, 0xde, 0x8f, 0x88, + 0x87, 0x1e, 0x43, 0x41, 0x4e, 0x22, 0x62, 0x5b, 0xcb, 0x56, 0x73, 0xee, 0xde, 0xdd, 0xd6, 0x79, + 0x06, 0x69, 0x9d, 0xc4, 0xb5, 0x9e, 0x4d, 0x22, 0xd2, 0x29, 0xbc, 0x3e, 0x5c, 0x9a, 0x71, 0x34, + 0x01, 0xea, 0x40, 0x51, 0xe2, 0x41, 0x40, 0xec, 0xdc, 0xb2, 0xd5, 0xac, 0xde, 0xbb, 0x7d, 0x8a, + 0x49, 0xec, 0x05, 0x5a, 0xbf, 0x67, 0xea, 0xcc, 0x2a, 0x11, 0x1e, 0xa7, 0x91, 0x64, 0xdc, 0x50, + 0x24, 0x50, 0xf4, 0x08, 0x8a, 0x22, 0xc2, 0xa1, 0xb0, 0xf3, 0xcb, 0xf9, 0x66, 0xf5, 0xde, 0x4f, + 0xce, 0x97, 0x46, 0xd3, 0x38, 0x04, 0xfb, 0x4a, 0x1c, 0x1c, 0xa6, 0x34, 0x1a, 0x8d, 0x3e, 0x85, + 0xb2, 0x1f, 0x73, 0x2c, 0x29, 0x0b, 0xed, 0xc2, 0xb2, 0xd5, 0xcc, 0x77, 0x3e, 0x50, 0xdb, 0xdf, + 0x1d, 0x2e, 0xd5, 0x94, 0x39, 0x5b, 0xab, 0x66, 0xd3, 0x99, 0x1e, 0x43, 0x1f, 0x03, 0x78, 0xbb, + 0x71, 0x38, 0x72, 0x05, 0x7d, 0x45, 0xec, 0xa2, 0x06, 0x25, 0x9c, 0x15, 0xbd, 0xde, 0xa7, 0xaf, + 0x08, 0x7a, 0x08, 0x65, 0x4e, 0xb0, 0xbf, 0x22, 0x9e, 0xee, 0xd8, 0xb3, 0x5a, 0xcb, 0x1f, 0x65, + 0x24, 0x54, 0x2e, 0x6b, 0xed, 0x06, 0x5e, 0xeb, 0x59, 0xea, 0x32, 0xc3, 0x30, 0x05, 0x21, 0x01, + 0x37, 0x68, 0xe8, 0x93, 0x97, 0x44, 0xb8, 0x92, 0xb9, 0x03, 0x63, 0x51, 0xbb, 0xbc, 0x9c, 0x6f, + 0xd6, 0x3a, 0xdd, 0xef, 0x0e, 0x97, 0x1e, 0x0e, 0xa9, 0xdc, 0x8d, 0x07, 0x2d, 0x8f, 0x8d, 0xdb, + 0x53, 0x66, 0x7f, 0x70, 0xfc, 0xdc, 0x8e, 0x46, 0xc3, 0xf6, 0xdb, 0x81, 0xd6, 0xea, 0x29, 0xda, + 0xde, 0xaa, 0x33, 0x6f, 0xf8, 0x9f, 0xb1, 0xd4, 0x5f, 0x8d, 0x3b, 0x50, 0x50, 0xce, 0x42, 0x55, + 0x98, 0xed, 0x85, 0xfb, 0x38, 0xa0, 0x7e, 0x7d, 0x06, 0x01, 0x94, 0xba, 0x2c, 0x88, 0xc7, 0x61, + 0xdd, 0x42, 0x15, 0x28, 0x6a, 0x78, 0x3d, 0xb7, 0x5e, 0x28, 0x97, 0xea, 0xb3, 0x8d, 0x03, 0xa8, + 0xae, 0xb3, 0xc1, 0x26, 0x67, 0x43, 0x4e, 0x84, 0x40, 0x9f, 0x40, 0xe9, 0x05, 0x1b, 0xb8, 0xd4, + 0xd7, 0x41, 0x92, 0xef, 0xd4, 0x94, 0x56, 0x47, 0x87, 0x4b, 0xc5, 0x75, 0x36, 0xe8, 0xad, 0x3a, + 0xc5, 0x17, 0x6c, 0xd0, 0xf3, 0x51, 0x13, 0xae, 0x79, 0x2c, 0x94, 0x9c, 0x0e, 0x62, 0x6d, 0x78, + 0x15, 0x06, 0x39, 0x63, 0x81, 0x13, 0x3b, 0xc8, 0x86, 0x82, 0x08, 0x98, 0xb4, 0xf3, 0xcb, 0x56, + 0xb3, 0x98, 0xc6, 0x90, 0x5a, 0x69, 0xbc, 0x2e, 0x03, 0x52, 0x4e, 0xed, 0x8d, 0x23, 0xc6, 0xe5, + 0x2a, 0x96, 0x58, 0xc7, 0xe8, 0x2d, 0xa8, 0x0a, 0x3c, 0x8e, 0x02, 0x92, 0x78, 0x27, 0x97, 0xc1, + 0x41, 0xb2, 0xa1, 0xdd, 0xf3, 0x18, 0xca, 0x91, 0x91, 0xd9, 0x2e, 0x69, 0xf7, 0xdc, 0x3a, 0x3f, + 0x80, 0x32, 0x0a, 0xa6, 0x6e, 0x4a, 0xc1, 0xe8, 0x31, 0xe4, 0x63, 0x4e, 0xed, 0x59, 0x1d, 0x84, + 0x3f, 0x3f, 0x9f, 0xe3, 0x6d, 0x51, 0x5b, 0x5b, 0x9c, 0x3e, 0x0a, 0x25, 0x9f, 0x38, 0x8a, 0x01, + 0x7d, 0x06, 0xa5, 0xa4, 0x46, 0xd8, 0x65, 0x2d, 0xcf, 0x52, 0x86, 0xcb, 0x64, 0x67, 0xab, 0xf7, + 0x74, 0x8d, 0x06, 0x64, 0x4d, 0x1f, 0x33, 0x92, 0x18, 0x10, 0xda, 0x86, 0x92, 0xce, 0x0b, 0x61, + 0x57, 0xb4, 0x28, 0x0f, 0xae, 0x24, 0x8a, 0x4e, 0x11, 0xa1, 0xa5, 0xd1, 0xbc, 0x96, 0x63, 0xd8, + 0xd0, 0x43, 0xf8, 0x3f, 0x31, 0xa2, 0x91, 0x3b, 0xa6, 0x42, 0xd0, 0x70, 0xe8, 0xee, 0x30, 0x4e, + 0xe8, 0x30, 0x74, 0x47, 0x64, 0x22, 0x6c, 0x58, 0xb6, 0x9a, 0x65, 0x23, 0xc8, 0x87, 0xea, 0xd8, + 0x57, 0xc9, 0xa9, 0xb5, 0xe4, 0xd0, 0x06, 0x99, 0x08, 0x74, 0x07, 0x6a, 0x07, 0x38, 0x08, 0x54, + 0x32, 0x3d, 0xc1, 0x21, 0x13, 0x76, 0x35, 0x93, 0x30, 0x27, 0xb7, 0xd0, 0x3d, 0x98, 0xe7, 0x3a, + 0x4f, 0x37, 0x31, 0xc7, 0x41, 0x40, 0x02, 0x2a, 0xc6, 0x76, 0x2d, 0xe3, 0xc2, 0xb7, 0xb7, 0xd1, + 0xd7, 0x00, 0x9c, 0x88, 0x78, 0x4c, 0xdc, 0x88, 0x09, 0x7b, 0x4e, 0x2b, 0xff, 0xab, 0x2b, 0x29, + 0xef, 0x68, 0xf8, 0x26, 0x4b, 0xf4, 0x77, 0x2a, 0x3c, 0xfd, 0x8d, 0x08, 0x40, 0x2c, 0x08, 0x77, + 0x75, 0x45, 0xb4, 0xaf, 0x2f, 0x5b, 0xcd, 0x4a, 0x67, 0xcd, 0x94, 0x87, 0xcf, 0x2f, 0x97, 0x7e, + 0xc4, 0x8b, 0x39, 0x95, 0x93, 0x56, 0xff, 0xb7, 0x5f, 0x6e, 0x09, 0xc2, 0x43, 0x3c, 0x26, 0x9b, + 0x8a, 0xcd, 0xa9, 0x28, 0x66, 0xfd, 0xb8, 0x10, 0x43, 0x35, 0x11, 0x49, 0xbb, 0x01, 0xfd, 0x06, + 0x0a, 0x2a, 0x53, 0x75, 0x06, 0x5d, 0xad, 0x38, 0x5a, 0x8e, 0x46, 0xa2, 0x4f, 0x00, 0x24, 0xe6, + 0x43, 0x22, 0xbb, 0x2c, 0x10, 0x76, 0x6e, 0x39, 0xdf, 0xac, 0x98, 0xfd, 0xcc, 0xfa, 0x82, 0x80, + 0x6a, 0xc6, 0xef, 0xa8, 0x0e, 0xf9, 0x11, 0x99, 0xe8, 0xb7, 0x56, 0x1c, 0xf5, 0x88, 0x9e, 0x40, + 0x71, 0x1f, 0x07, 0x71, 0x5a, 0xa6, 0xaf, 0x16, 0x52, 0x19, 0x8d, 0x9c, 0x84, 0xe6, 0x97, 0xb9, + 0x07, 0xd6, 0xc2, 0x2f, 0xa0, 0x9c, 0xc6, 0x7d, 0xf6, 0x8d, 0xc5, 0xe4, 0x8d, 0x37, 0xb3, 0x6f, + 0xac, 0x64, 0x71, 0xbf, 0x86, 0xb9, 0x93, 0x7e, 0x7a, 0x17, 0x3a, 0x9f, 0x41, 0xaf, 0x17, 0xca, + 0x96, 0xae, 0x58, 0xf9, 0x7a, 0x61, 0xbd, 0x50, 0x2e, 0xd4, 0x8b, 0xeb, 0x85, 0x72, 0xb1, 0x5e, + 0x5a, 0x2f, 0x94, 0xaf, 0xd5, 0x6b, 0x8d, 0x7f, 0xe5, 0xe0, 0xa3, 0xbe, 0xe4, 0x04, 0x8f, 0x7b, + 0xe1, 0x90, 0x08, 0x55, 0x78, 0xa6, 0xf5, 0xe4, 0x0f, 0x70, 0x23, 0xc2, 0x5c, 0x52, 0xb5, 0xe8, + 0x62, 0xdf, 0x57, 0x49, 0x4f, 0x84, 0x6d, 0x69, 0x9b, 0x3e, 0x51, 0xb1, 0xf0, 0xef, 0xc3, 0xa5, + 0xb5, 0x4b, 0xc5, 0x82, 0xe7, 0x05, 0xaa, 0xc9, 0x13, 0x3c, 0xa6, 0xe1, 0xd0, 0xf3, 0x82, 0xd6, + 0x66, 0x4a, 0xbc, 0x92, 0xf0, 0x3a, 0x28, 0x3a, 0xb5, 0x42, 0x04, 0xea, 0x00, 0x08, 0x89, 0xb9, + 0x74, 0x55, 0x9a, 0x18, 0x4f, 0x5c, 0xaa, 0x95, 0x54, 0x34, 0x4c, 0xad, 0x22, 0x0e, 0x73, 0xc9, + 0x8b, 0x53, 0x0d, 0x74, 0x3d, 0xad, 0x74, 0x36, 0x8c, 0xfc, 0xdd, 0xef, 0x27, 0x7f, 0x62, 0xb3, + 0x54, 0xf8, 0x9a, 0xc8, 0xfe, 0x6c, 0xfc, 0xc3, 0x82, 0xff, 0x3f, 0x65, 0xd4, 0x35, 0xce, 0x42, + 0x49, 0xcd, 0x30, 0xe1, 0xc0, 0x8d, 0x5d, 0x3a, 0xdc, 0x75, 0x0f, 0xb0, 0x24, 0xdc, 0xc5, 0xd2, + 0xd5, 0xe2, 0x9a, 0xa0, 0xbf, 0x94, 0x82, 0x75, 0x85, 0x7f, 0xae, 0xe0, 0x2b, 0xb2, 0xaf, 0xc0, + 0xa8, 0x03, 0x35, 0xc9, 0xb1, 0x37, 0x22, 0xbe, 0x9b, 0xcc, 0x06, 0x39, 0x5d, 0x0e, 0x3e, 0x3a, + 0xa3, 0x94, 0x66, 0x26, 0x81, 0x6b, 0x06, 0xa3, 0x96, 0x44, 0xe3, 0x9f, 0xb3, 0xc9, 0xdc, 0x13, + 0x47, 0xd3, 0x18, 0xb8, 0x9f, 0x8e, 0x1a, 0xd6, 0x65, 0xe8, 0xcc, 0x60, 0xf1, 0x05, 0xd4, 0x69, + 0x28, 0x39, 0xf3, 0x63, 0xef, 0x6a, 0xe2, 0x5c, 0x3f, 0x86, 0x69, 0x89, 0xd0, 0x7d, 0xa8, 0xfa, + 0x64, 0x07, 0xc7, 0x81, 0x74, 0x55, 0xab, 0x49, 0x5c, 0x87, 0x4c, 0x63, 0x85, 0xd5, 0x64, 0x6b, + 0xcb, 0xe9, 0x39, 0x60, 0x8e, 0x6d, 0x71, 0x8a, 0xfe, 0x64, 0xc1, 0x8d, 0x98, 0x53, 0xe1, 0x0e, + 0x26, 0x6e, 0xc0, 0x3c, 0x1c, 0x50, 0x39, 0x71, 0x47, 0xfb, 0x76, 0x41, 0x8b, 0xf0, 0xf9, 0xc5, + 0xb3, 0xdb, 0xb1, 0xee, 0xaa, 0x49, 0x89, 0xce, 0xe4, 0x4b, 0xc3, 0xb0, 0xb1, 0x9f, 0xf4, 0x88, + 0x9b, 0x47, 0x87, 0x4b, 0xf5, 0x2d, 0xa7, 0x97, 0xdd, 0xda, 0x76, 0xea, 0xf1, 0xa9, 0xc3, 0xc8, + 0x81, 0xea, 0x78, 0xdf, 0xf3, 0xdc, 0x1d, 0x1a, 0x48, 0xc2, 0xf5, 0xac, 0x34, 0x77, 0xc2, 0xb9, + 0xa9, 0xfe, 0x5f, 0x6d, 0x77, 0xbb, 0x6b, 0xfa, 0xd0, 0xb1, 0x66, 0xc7, 0x6b, 0x0e, 0x28, 0x96, + 0xe4, 0x19, 0x7d, 0x01, 0x40, 0x42, 0x8f, 0x4f, 0x22, 0x3d, 0x3a, 0x24, 0xcd, 0xbb, 0x79, 0x06, + 0xa5, 0x6a, 0x95, 0x8f, 0xa6, 0x07, 0x9f, 0xea, 0xbf, 0xc2, 0xc9, 0x60, 0xd1, 0x53, 0x98, 0x1f, + 0x68, 0x6d, 0xdd, 0x4c, 0x86, 0x5d, 0x61, 0x58, 0xbb, 0x9e, 0xa0, 0xfb, 0xd3, 0x3c, 0xdb, 0x00, + 0xb3, 0xe4, 0x92, 0xd0, 0x4f, 0xe8, 0xca, 0x97, 0xa7, 0xab, 0x25, 0xd8, 0x47, 0xa1, 0xaf, 0xc9, + 0xb6, 0xa0, 0x14, 0x8d, 0x5c, 0xea, 0xa7, 0x1d, 0xfd, 0xfe, 0xa5, 0x7d, 0xb6, 0x39, 0xea, 0xf9, + 0xa6, 0x99, 0x57, 0xd4, 0xec, 0xb5, 0xb9, 0xd1, 0x5b, 0x15, 0x4e, 0x31, 0x52, 0xcb, 0xa7, 0x7a, + 0x1a, 0xfc, 0x50, 0x3d, 0xad, 0x0b, 0x1f, 0x9c, 0x19, 0x3a, 0x67, 0xb4, 0x99, 0xf3, 0x8b, 0xfe, + 0x03, 0x80, 0x63, 0x5d, 0xb2, 0xc8, 0xc2, 0x19, 0xc8, 0x72, 0x06, 0xd9, 0xf8, 0xbb, 0x05, 0x75, + 0x87, 0x08, 0xc9, 0x38, 0x51, 0x49, 0x94, 0x10, 0x7c, 0x0a, 0x05, 0x95, 0x87, 0xa6, 0xc6, 0xbc, + 0x23, 0x0d, 0xf5, 0x51, 0xb4, 0x02, 0xc5, 0x1d, 0xaa, 0xa6, 0xaa, 0x24, 0x75, 0x6f, 0x9d, 0x35, + 0x94, 0xe9, 0x46, 0xe7, 0x90, 0xbd, 0x98, 0x08, 0xa9, 0xa3, 0x2e, 0x2d, 0x04, 0x1a, 0x89, 0x6e, + 0x43, 0x35, 0x9d, 0x16, 0x7b, 0xfe, 0x4b, 0x9d, 0xbe, 0xe9, 0xf8, 0x93, 0xdd, 0x68, 0xfc, 0x31, + 0x0f, 0xd7, 0x8d, 0xc8, 0xd3, 0xca, 0xb3, 0x06, 0xd7, 0x78, 0xb2, 0x94, 0x44, 0xd3, 0x15, 0xaa, + 0x63, 0xd5, 0x00, 0x75, 0x2c, 0x9d, 0xcc, 0x99, 0xdc, 0x7b, 0xe4, 0x4c, 0x0f, 0x4a, 0x9c, 0xe8, + 0xe1, 0x2f, 0xb9, 0x77, 0xfd, 0xf4, 0x9d, 0x16, 0x31, 0xd7, 0xaf, 0x11, 0x99, 0xa4, 0x23, 0x6b, + 0x42, 0xa0, 0x46, 0x56, 0x13, 0xe0, 0x49, 0x51, 0xfa, 0xd9, 0x45, 0xf3, 0xc5, 0x09, 0xbb, 0x5c, + 0x18, 0xe1, 0xef, 0x11, 0x35, 0x7f, 0xcd, 0xc1, 0x8d, 0x7e, 0x14, 0x50, 0xb9, 0x12, 0xfa, 0x7d, + 0x0f, 0x4b, 0x69, 0x7a, 0xd5, 0xef, 0xa1, 0xa4, 0x6f, 0x76, 0x69, 0x07, 0x78, 0x78, 0xbe, 0xa4, + 0x67, 0xc0, 0x53, 0xe9, 0xb5, 0x3c, 0x5d, 0xc5, 0x93, 0x1a, 0x22, 0x21, 0xcd, 0xd8, 0x34, 0xf7, + 0x9e, 0x36, 0x5d, 0x70, 0x61, 0xfe, 0xad, 0xb7, 0xa1, 0x75, 0x98, 0x25, 0xea, 0x4e, 0x45, 0x52, + 0xf9, 0xef, 0xbc, 0xd3, 0xd2, 0xd3, 0xa4, 0x31, 0xfc, 0x29, 0x41, 0xe3, 0xcf, 0x79, 0xa8, 0x75, + 0xfb, 0xdb, 0xcf, 0x39, 0x4d, 0x8d, 0x73, 0x5b, 0xb5, 0x27, 0x21, 0x69, 0x98, 0x5c, 0xa2, 0x75, + 0x62, 0xa7, 0x31, 0x98, 0xd9, 0x40, 0x3f, 0x86, 0x6b, 0xaa, 0x52, 0xb8, 0x91, 0x36, 0x4c, 0x12, + 0x85, 0xd3, 0x83, 0xba, 0x86, 0x24, 0x1b, 0xe8, 0x33, 0x98, 0x65, 0x49, 0xe4, 0xe9, 0x64, 0xa9, + 0x9e, 0xd9, 0x30, 0xba, 0xfd, 0x6d, 0x13, 0x9e, 0xa9, 0x84, 0x06, 0x73, 0x7c, 0x3d, 0xe7, 0xec, + 0x40, 0x98, 0x3b, 0x7d, 0xf6, 0x7a, 0xee, 0xb0, 0x03, 0x81, 0x7e, 0x07, 0xf3, 0x1e, 0x1b, 0x47, + 0x2a, 0xf7, 0xd4, 0x60, 0xe7, 0x31, 0x9f, 0x78, 0xa6, 0x3d, 0x5d, 0xf0, 0x25, 0x41, 0xa5, 0x47, + 0xf7, 0x18, 0x96, 0xce, 0x21, 0x19, 0xa6, 0xae, 0x22, 0x3a, 0x55, 0x63, 0x4b, 0x3f, 0x50, 0x8d, + 0x6d, 0x3c, 0x87, 0xf9, 0x4e, 0x1c, 0x28, 0x85, 0x32, 0xee, 0x98, 0x7e, 0x5b, 0xb1, 0xbe, 0xf7, + 0xb7, 0x95, 0x3b, 0xb7, 0xe0, 0xfa, 0x29, 0x55, 0x51, 0x19, 0x0a, 0x4f, 0x58, 0x48, 0xea, 0x33, + 0xea, 0xe9, 0xf1, 0x2b, 0x1a, 0xd5, 0xad, 0xce, 0xdd, 0xd7, 0xff, 0x5d, 0x9c, 0x79, 0x7d, 0xb4, + 0x68, 0x7d, 0x73, 0xb4, 0x68, 0x7d, 0x7b, 0xb4, 0x68, 0xfd, 0xe7, 0x68, 0xd1, 0xfa, 0xcb, 0x9b, + 0xc5, 0x99, 0x6f, 0xde, 0x2c, 0xce, 0x7c, 0xfb, 0x66, 0x71, 0xe6, 0xeb, 0x6a, 0xe6, 0xf3, 0xd5, + 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0x47, 0x83, 0x27, 0x38, 0x6b, 0x13, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 365153f4213e..471ed54cb36d 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -61,6 +61,11 @@ message BackfillerSpec { // The timestamp to perform index backfill historical scans at. optional util.hlc.Timestamp readAsOf = 7 [(gogoproto.nullable) = false]; + // IndexesToBackfill is the set of indexes to backfill. This is populated only + // starting in 21.1, prior to that the implied index set are those containing + // the mutation ID of the first mutation on the table descriptor. + repeated uint32 indexes_to_backfill = 8 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; + reserved 6; } diff --git a/pkg/sql/index_backfiller.go b/pkg/sql/index_backfiller.go new file mode 100644 index 000000000000..ce6b92934d5e --- /dev/null +++ b/pkg/sql/index_backfiller.go @@ -0,0 +1,160 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// IndexBackfillPlanner holds dependencies for an index backfiller. +type IndexBackfillPlanner struct { + execCfg *ExecutorConfig + ieFactory sqlutil.SessionBoundInternalExecutorFactory +} + +// NewIndexBackfiller creates a new IndexBackfillPlanner. +func NewIndexBackfiller( + execCfg *ExecutorConfig, ieFactory sqlutil.SessionBoundInternalExecutorFactory, +) *IndexBackfillPlanner { + return &IndexBackfillPlanner{execCfg: execCfg, ieFactory: ieFactory} +} + +// BackfillIndex will backfill the specified index on the passed table. +// +// TODO(ajwerner): allow backfilling multiple indexes. +func (ib *IndexBackfillPlanner) BackfillIndex( + ctx context.Context, + tracker scexec.JobProgressTracker, + descriptor catalog.TableDescriptor, + source descpb.IndexID, + toBackfill ...descpb.IndexID, +) error { + + // Pick an arbitrary read timestamp for the reads of the backfill. + // It's safe to use any timestamp to read even if we've partially backfilled + // at an earlier timestamp because other writing transactions have been + // writing at the appropriate timestamps in-between. + backfillReadTimestamp := ib.execCfg.DB.Clock().Now() + targetSpans := make([]roachpb.Span, len(toBackfill)) + for i, idxID := range toBackfill { + targetSpans[i] = descriptor.IndexSpan(ib.execCfg.Codec, idxID) + } + if err := ib.scanTargetSpansToPushTimestampCache( + ctx, backfillReadTimestamp, targetSpans, + ); err != nil { + return err + } + + resumeSpans, err := tracker.GetResumeSpans(ctx, descriptor.GetID(), source) + if err != nil { + return err + } + run, err := ib.plan(ctx, descriptor, backfillReadTimestamp, backfillReadTimestamp, resumeSpans, toBackfill, func( + ctx context.Context, meta *execinfrapb.ProducerMetadata, + ) error { + // TODO(ajwerner): Hook up the jobs tracking stuff. + log.Infof(ctx, "got update: %v", meta) + return nil + }) + if err != nil { + return err + } + return run(ctx) +} + +// Index backfilling ingests SSTs that don't play nicely with running txns +// since they just add their keys blindly. Running a Scan of the target +// spans at the time the SSTs' keys will be written will calcify history up +// to then since the scan will resolve intents and populate tscache to keep +// anything else from sneaking under us. Since these are new indexes, these +// spans should be essentially empty, so this should be a pretty quick and +// cheap scan. +func (ib *IndexBackfillPlanner) scanTargetSpansToPushTimestampCache( + ctx context.Context, backfillTimestamp hlc.Timestamp, targetSpans []roachpb.Span, +) error { + const pageSize = 10000 + return ib.execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.SetFixedTimestamp(ctx, backfillTimestamp) + for _, span := range targetSpans { + // TODO(dt): a Count() request would be nice here if the target isn't + // empty, since we don't need to drag all the results back just to + // then ignore them -- we just need the iteration on the far end. + if err := txn.Iterate(ctx, span.Key, span.EndKey, pageSize, iterateNoop); err != nil { + return err + } + } + return nil + }) +} + +func iterateNoop(_ []kv.KeyValue) error { return nil } + +var _ scexec.IndexBackfiller = (*IndexBackfillPlanner)(nil) + +func (ib *IndexBackfillPlanner) plan( + ctx context.Context, + tableDesc catalog.TableDescriptor, + nowTimestamp, readAsOf hlc.Timestamp, + sourceSpans []roachpb.Span, + indexesToBackfill []descpb.IndexID, + callback func(_ context.Context, meta *execinfrapb.ProducerMetadata) error, +) (runFunc func(context.Context) error, _ error) { + + var p *PhysicalPlan + var evalCtx extendedEvalContext + var planCtx *PlanningCtx + td := tabledesc.NewExistingMutable(*tableDesc.TableDesc()) + if err := ib.execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + evalCtx = createSchemaChangeEvalCtx(ctx, ib.execCfg, nowTimestamp, ib.ieFactory) + planCtx = ib.execCfg.DistSQLPlanner.NewPlanningCtx(ctx, &evalCtx, nil /* planner */, txn, + true /* distribute */) + // TODO(ajwerner): Adopt util.ConstantWithMetamorphicTestRange for the + // batch size. Also plumb in a testing knob. + spec, err := initIndexBackfillerSpec( + *td.TableDesc(), readAsOf, indexBackfillBatchSize, indexesToBackfill) + if err != nil { + return err + } + p, err = ib.execCfg.DistSQLPlanner.createBackfillerPhysicalPlan(planCtx, spec, sourceSpans) + return err + }); err != nil { + return nil, err + } + + return func(ctx context.Context) error { + cbw := MetadataCallbackWriter{rowResultWriter: &errOnlyResultWriter{}, fn: callback} + recv := MakeDistSQLReceiver( + ctx, + &cbw, + tree.Rows, /* stmtType - doesn't matter here since no result are produced */ + ib.execCfg.RangeDescriptorCache, + nil, /* txn - the flow does not run wholly in a txn */ + ib.execCfg.Clock, + evalCtx.Tracing, + ) + defer recv.Release() + evalCtxCopy := evalCtx + ib.execCfg.DistSQLPlanner.Run(planCtx, nil, p, recv, &evalCtxCopy, nil)() + return cbw.Err() + }, nil +} diff --git a/pkg/sql/indexbackfiller_test.go b/pkg/sql/indexbackfiller_test.go index 091a8932de97..9724265d95f0 100644 --- a/pkg/sql/indexbackfiller_test.go +++ b/pkg/sql/indexbackfiller_test.go @@ -524,7 +524,7 @@ INSERT INTO foo VALUES (1), (10), (100); changer.SetJob(j) spans := []roachpb.Span{table.IndexSpan(keys.SystemSQLCodec, test.indexToBackfill)} require.NoError(t, changer.TestingDistIndexBackfill( - ctx, table.GetVersion(), spans, backfill.IndexMutationFilter, 10, + ctx, table.GetVersion(), spans, []descpb.IndexID{test.indexToBackfill}, backfill.IndexMutationFilter, 10, )) // Make the mutation complete, then read the index and validate that it diff --git a/pkg/sql/schema_changer_helpers_test.go b/pkg/sql/schema_changer_helpers_test.go index deea56cc7486..7dde63c130f3 100644 --- a/pkg/sql/schema_changer_helpers_test.go +++ b/pkg/sql/schema_changer_helpers_test.go @@ -25,10 +25,12 @@ func (sc *SchemaChanger) TestingDistIndexBackfill( ctx context.Context, version descpb.DescriptorVersion, targetSpans []roachpb.Span, + addedIndexes []descpb.IndexID, filter backfill.MutationFilter, indexBackfillBatchSize int64, ) error { - return sc.distIndexBackfill(ctx, version, targetSpans, filter, indexBackfillBatchSize) + return sc.distIndexBackfill( + ctx, version, targetSpans, addedIndexes, filter, indexBackfillBatchSize) } // SetJob sets the job. From 0f003e11e2373a8ae6a49e63f9e1a115ebf73f57 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:02 -0500 Subject: [PATCH 09/15] schemachanger/scjob,jobs: add a job to execute new schema changes This commit introduces a `NewSchemaChange` job to run the async portion of schema changes. The job reads the serialized schema change targets where the in-transaction phases left off, plans the post-commit phase of the schema change, and executes the ops. There is an incomplete implementation of backfill progress tracking in the `badJobTracker`. Release note: None --- pkg/jobs/jobspb/BUILD.bazel | 2 + pkg/jobs/jobspb/jobs.pb.go | 1439 ++++++++++++++++------- pkg/jobs/jobspb/jobs.proto | 14 + pkg/jobs/jobspb/wrap.go | 14 +- pkg/server/BUILD.bazel | 1 + pkg/server/server.go | 1 + pkg/sql/schema_changer.go | 13 + pkg/sql/schemachanger/scjob/BUILD.bazel | 51 + pkg/sql/schemachanger/scjob/job.go | 168 +++ pkg/ts/catalog/chart_catalog.go | 13 + 10 files changed, 1274 insertions(+), 442 deletions(-) create mode 100644 pkg/sql/schemachanger/scjob/BUILD.bazel create mode 100644 pkg/sql/schemachanger/scjob/job.go diff --git a/pkg/jobs/jobspb/BUILD.bazel b/pkg/jobs/jobspb/BUILD.bazel index 1a583087a505..99e4d9544f56 100644 --- a/pkg/jobs/jobspb/BUILD.bazel +++ b/pkg/jobs/jobspb/BUILD.bazel @@ -26,6 +26,7 @@ proto_library( deps = [ "//pkg/roachpb:roachpb_proto", "//pkg/sql/catalog/descpb:descpb_proto", + "//pkg/sql/schemachanger/scpb:scpb_proto", "//pkg/util/hlc:hlc_proto", "@com_github_cockroachdb_errors//errorspb:errorspb_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", @@ -44,6 +45,7 @@ go_proto_library( "//pkg/roachpb", "//pkg/security", # keep "//pkg/sql/catalog/descpb", + "//pkg/sql/schemachanger/scpb", "//pkg/sql/sem/tree", # keep "//pkg/util/hlc", "@com_github_cockroachdb_errors//errorspb", diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 2a0509e51fc7..9c33c7f27662 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -9,6 +9,7 @@ import math "math" import errorspb "github.com/cockroachdb/errors/errorspb" import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" +import scpb "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -56,7 +57,7 @@ func (x EncryptionMode) String() string { return proto.EnumName(EncryptionMode_name, int32(x)) } func (EncryptionMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{0} } type Status int32 @@ -85,7 +86,7 @@ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } func (Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{1} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{1} } type Type int32 @@ -104,6 +105,7 @@ const ( // names for this enum, which cause a conflict with the SCHEMA_CHANGE entry. TypeTypeSchemaChange Type = 9 TypeStreamIngestion Type = 10 + TypeNewSchemaChange Type = 11 ) var Type_name = map[int32]string{ @@ -118,6 +120,7 @@ var Type_name = map[int32]string{ 8: "SCHEMA_CHANGE_GC", 9: "TYPEDESC_SCHEMA_CHANGE", 10: "STREAM_INGESTION", + 11: "NEW_SCHEMA_CHANGE", } var Type_value = map[string]int32{ "UNSPECIFIED": 0, @@ -131,10 +134,11 @@ var Type_value = map[string]int32{ "SCHEMA_CHANGE_GC": 8, "TYPEDESC_SCHEMA_CHANGE": 9, "STREAM_INGESTION": 10, + "NEW_SCHEMA_CHANGE": 11, } func (Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{2} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{2} } type EncryptionInfo_Scheme int32 @@ -154,7 +158,7 @@ func (x EncryptionInfo_Scheme) String() string { return proto.EnumName(EncryptionInfo_Scheme_name, int32(x)) } func (EncryptionInfo_Scheme) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{2, 0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{2, 0} } type SchemaChangeGCProgress_Status int32 @@ -184,7 +188,7 @@ func (x SchemaChangeGCProgress_Status) String() string { return proto.EnumName(SchemaChangeGCProgress_Status_name, int32(x)) } func (SchemaChangeGCProgress_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{20, 0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{22, 0} } type Lease struct { @@ -198,7 +202,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_a7a6d19cca6b70ee, []int{0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{0} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -239,7 +243,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_a7a6d19cca6b70ee, []int{1} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{1} } func (m *BackupEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -273,7 +277,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_a7a6d19cca6b70ee, []int{1, 0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{1, 0} } func (m *BackupEncryptionOptions_KMSInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -313,7 +317,7 @@ func (m *EncryptionInfo) Reset() { *m = EncryptionInfo{} } func (m *EncryptionInfo) String() string { return proto.CompactTextString(m) } func (*EncryptionInfo) ProtoMessage() {} func (*EncryptionInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{2} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{2} } func (m *EncryptionInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -356,7 +360,7 @@ func (m *StreamIngestionDetails) Reset() { *m = StreamIngestionDetails{} func (m *StreamIngestionDetails) String() string { return proto.CompactTextString(m) } func (*StreamIngestionDetails) ProtoMessage() {} func (*StreamIngestionDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{3} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{3} } func (m *StreamIngestionDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -388,7 +392,7 @@ func (m *StreamIngestionProgress) Reset() { *m = StreamIngestionProgress func (m *StreamIngestionProgress) String() string { return proto.CompactTextString(m) } func (*StreamIngestionProgress) ProtoMessage() {} func (*StreamIngestionProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{4} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{4} } func (m *StreamIngestionProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -443,7 +447,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_a7a6d19cca6b70ee, []int{5} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{5} } func (m *BackupDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -475,7 +479,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_a7a6d19cca6b70ee, []int{6} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{6} } func (m *BackupProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -537,7 +541,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_a7a6d19cca6b70ee, []int{7} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{7} } func (m *RestoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -574,7 +578,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_a7a6d19cca6b70ee, []int{7, 0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{7, 0} } func (m *RestoreDetails_DescriptorRewrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -607,7 +611,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_a7a6d19cca6b70ee, []int{7, 1} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{7, 1} } func (m *RestoreDetails_BackupLocalityInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -640,7 +644,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_a7a6d19cca6b70ee, []int{8} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{8} } func (m *RestoreProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -701,7 +705,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_a7a6d19cca6b70ee, []int{9} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{9} } func (m *ImportDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -739,7 +743,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_a7a6d19cca6b70ee, []int{9, 0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{9, 0} } func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -781,7 +785,7 @@ func (m *SequenceValChunk) Reset() { *m = SequenceValChunk{} } func (m *SequenceValChunk) String() string { return proto.CompactTextString(m) } func (*SequenceValChunk) ProtoMessage() {} func (*SequenceValChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{10} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{10} } func (m *SequenceValChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -817,7 +821,7 @@ func (m *SequenceDetails) Reset() { *m = SequenceDetails{} } func (m *SequenceDetails) String() string { return proto.CompactTextString(m) } func (*SequenceDetails) ProtoMessage() {} func (*SequenceDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{11} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{11} } func (m *SequenceDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -852,7 +856,7 @@ func (m *SequenceDetails_SequenceChunks) Reset() { *m = SequenceDetails_ func (m *SequenceDetails_SequenceChunks) String() string { return proto.CompactTextString(m) } func (*SequenceDetails_SequenceChunks) ProtoMessage() {} func (*SequenceDetails_SequenceChunks) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{11, 0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{11, 0} } func (m *SequenceDetails_SequenceChunks) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -898,7 +902,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_a7a6d19cca6b70ee, []int{12} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{12} } func (m *ImportProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -932,7 +936,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_a7a6d19cca6b70ee, []int{13} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{13} } func (m *TypeSchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -965,7 +969,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_a7a6d19cca6b70ee, []int{14} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{14} } func (m *TypeSchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -990,6 +994,74 @@ func (m *TypeSchemaChangeProgress) XXX_DiscardUnknown() { var xxx_messageInfo_TypeSchemaChangeProgress proto.InternalMessageInfo +// TypeSchemaChangeDetails is the job detail information for the new schema change job. +type NewSchemaChangeDetails struct { + Targets []*scpb.Target `protobuf:"bytes,1,rep,name=targets,proto3" json:"targets,omitempty"` +} + +func (m *NewSchemaChangeDetails) Reset() { *m = NewSchemaChangeDetails{} } +func (m *NewSchemaChangeDetails) String() string { return proto.CompactTextString(m) } +func (*NewSchemaChangeDetails) ProtoMessage() {} +func (*NewSchemaChangeDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{15} +} +func (m *NewSchemaChangeDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NewSchemaChangeDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NewSchemaChangeDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_NewSchemaChangeDetails.Merge(dst, src) +} +func (m *NewSchemaChangeDetails) XXX_Size() int { + return m.Size() +} +func (m *NewSchemaChangeDetails) XXX_DiscardUnknown() { + xxx_messageInfo_NewSchemaChangeDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_NewSchemaChangeDetails proto.InternalMessageInfo + +// NewSchemaChangeProgress is the persisted progress for the new schema change job. +type NewSchemaChangeProgress struct { + States []scpb.State `protobuf:"varint,1,rep,packed,name=states,proto3,enum=cockroach.sql.schemachanger.scpb.State" json:"states,omitempty"` +} + +func (m *NewSchemaChangeProgress) Reset() { *m = NewSchemaChangeProgress{} } +func (m *NewSchemaChangeProgress) String() string { return proto.CompactTextString(m) } +func (*NewSchemaChangeProgress) ProtoMessage() {} +func (*NewSchemaChangeProgress) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{16} +} +func (m *NewSchemaChangeProgress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NewSchemaChangeProgress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NewSchemaChangeProgress) XXX_Merge(src proto.Message) { + xxx_messageInfo_NewSchemaChangeProgress.Merge(dst, src) +} +func (m *NewSchemaChangeProgress) XXX_Size() int { + return m.Size() +} +func (m *NewSchemaChangeProgress) XXX_DiscardUnknown() { + xxx_messageInfo_NewSchemaChangeProgress.DiscardUnknown(m) +} + +var xxx_messageInfo_NewSchemaChangeProgress proto.InternalMessageInfo + type ResumeSpanList struct { ResumeSpans []roachpb.Span `protobuf:"bytes,1,rep,name=resume_spans,json=resumeSpans,proto3" json:"resume_spans"` } @@ -998,7 +1070,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_a7a6d19cca6b70ee, []int{15} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{17} } func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1033,7 +1105,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_a7a6d19cca6b70ee, []int{16} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{18} } func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1096,7 +1168,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_a7a6d19cca6b70ee, []int{17} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{19} } func (m *SchemaChangeGCDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1130,7 +1202,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_a7a6d19cca6b70ee, []int{17, 0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{19, 0} } func (m *SchemaChangeGCDetails_DroppedIndex) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1164,7 +1236,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_a7a6d19cca6b70ee, []int{17, 1} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{19, 1} } func (m *SchemaChangeGCDetails_DroppedID) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1198,7 +1270,7 @@ func (m *SchemaChangeGCDetails_DroppedTenant) Reset() { *m = SchemaChang func (m *SchemaChangeGCDetails_DroppedTenant) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedTenant) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedTenant) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{17, 2} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{19, 2} } func (m *SchemaChangeGCDetails_DroppedTenant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1262,7 +1334,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_a7a6d19cca6b70ee, []int{18} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{20} } func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1294,7 +1366,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_a7a6d19cca6b70ee, []int{19} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{21} } func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1332,7 +1404,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_a7a6d19cca6b70ee, []int{20} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{22} } func (m *SchemaChangeGCProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1366,7 +1438,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_a7a6d19cca6b70ee, []int{20, 0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{22, 0} } func (m *SchemaChangeGCProgress_IndexProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1400,7 +1472,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_a7a6d19cca6b70ee, []int{20, 1} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{22, 1} } func (m *SchemaChangeGCProgress_TableProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1433,7 +1505,7 @@ func (m *SchemaChangeGCProgress_TenantProgress) Reset() { *m = SchemaCha func (m *SchemaChangeGCProgress_TenantProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_TenantProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_TenantProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_a7a6d19cca6b70ee, []int{20, 2} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{22, 2} } func (m *SchemaChangeGCProgress_TenantProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1466,7 +1538,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_a7a6d19cca6b70ee, []int{21} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{23} } func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1519,7 +1591,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_a7a6d19cca6b70ee, []int{22} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{24} } func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1554,7 +1626,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_a7a6d19cca6b70ee, []int{23} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{25} } func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1597,7 +1669,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_a7a6d19cca6b70ee, []int{24} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{26} } func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1641,7 +1713,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_a7a6d19cca6b70ee, []int{25} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{27} } func (m *CreateStatsDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1682,7 +1754,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_a7a6d19cca6b70ee, []int{25, 0} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{27, 0} } func (m *CreateStatsDetails_ColStat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1714,7 +1786,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_a7a6d19cca6b70ee, []int{26} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{28} } func (m *CreateStatsProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1775,6 +1847,7 @@ type Payload struct { // *Payload_SchemaChangeGC // *Payload_TypeSchemaChange // *Payload_StreamIngestion + // *Payload_NewSchemaChange Details isPayload_Details `protobuf_oneof:"details"` } @@ -1782,7 +1855,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_a7a6d19cca6b70ee, []int{27} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{29} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1840,6 +1913,9 @@ type Payload_TypeSchemaChange struct { type Payload_StreamIngestion struct { StreamIngestion *StreamIngestionDetails `protobuf:"bytes,23,opt,name=streamIngestion,proto3,oneof"` } +type Payload_NewSchemaChange struct { + NewSchemaChange *NewSchemaChangeDetails `protobuf:"bytes,24,opt,name=newSchemaChange,proto3,oneof"` +} func (*Payload_Backup) isPayload_Details() {} func (*Payload_Restore) isPayload_Details() {} @@ -1850,6 +1926,7 @@ func (*Payload_CreateStats) isPayload_Details() {} func (*Payload_SchemaChangeGC) isPayload_Details() {} func (*Payload_TypeSchemaChange) isPayload_Details() {} func (*Payload_StreamIngestion) isPayload_Details() {} +func (*Payload_NewSchemaChange) isPayload_Details() {} func (m *Payload) GetDetails() isPayload_Details { if m != nil { @@ -1921,6 +1998,13 @@ func (m *Payload) GetStreamIngestion() *StreamIngestionDetails { return nil } +func (m *Payload) GetNewSchemaChange() *NewSchemaChangeDetails { + if x, ok := m.GetDetails().(*Payload_NewSchemaChange); ok { + return x.NewSchemaChange + } + return nil +} + // XXX_OneofFuncs is for the internal use of the proto package. func (*Payload) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { return _Payload_OneofMarshaler, _Payload_OneofUnmarshaler, _Payload_OneofSizer, []interface{}{ @@ -1933,6 +2017,7 @@ func (*Payload) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error (*Payload_SchemaChangeGC)(nil), (*Payload_TypeSchemaChange)(nil), (*Payload_StreamIngestion)(nil), + (*Payload_NewSchemaChange)(nil), } } @@ -1985,6 +2070,11 @@ func _Payload_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.StreamIngestion); err != nil { return err } + case *Payload_NewSchemaChange: + _ = b.EncodeVarint(24<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.NewSchemaChange); err != nil { + return err + } case nil: default: return fmt.Errorf("Payload.Details has unexpected type %T", x) @@ -2067,6 +2157,14 @@ func _Payload_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer err := b.DecodeMessage(msg) m.Details = &Payload_StreamIngestion{msg} return true, err + case 24: // details.newSchemaChange + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(NewSchemaChangeDetails) + err := b.DecodeMessage(msg) + m.Details = &Payload_NewSchemaChange{msg} + return true, err default: return false, nil } @@ -2121,6 +2219,11 @@ func _Payload_OneofSizer(msg proto.Message) (n int) { n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *Payload_NewSchemaChange: + s := proto.Size(x.NewSchemaChange) + n += 2 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s case nil: default: panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) @@ -2145,6 +2248,7 @@ type Progress struct { // *Progress_SchemaChangeGC // *Progress_TypeSchemaChange // *Progress_StreamIngest + // *Progress_NewSchemaChange Details isProgress_Details `protobuf_oneof:"details"` } @@ -2152,7 +2256,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_a7a6d19cca6b70ee, []int{28} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{30} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2221,6 +2325,9 @@ type Progress_TypeSchemaChange struct { type Progress_StreamIngest struct { StreamIngest *StreamIngestionProgress `protobuf:"bytes,18,opt,name=streamIngest,proto3,oneof"` } +type Progress_NewSchemaChange struct { + NewSchemaChange *NewSchemaChangeProgress `protobuf:"bytes,19,opt,name=newSchemaChange,proto3,oneof"` +} func (*Progress_FractionCompleted) isProgress_Progress() {} func (*Progress_HighWater) isProgress_Progress() {} @@ -2233,6 +2340,7 @@ func (*Progress_CreateStats) isProgress_Details() {} func (*Progress_SchemaChangeGC) isProgress_Details() {} func (*Progress_TypeSchemaChange) isProgress_Details() {} func (*Progress_StreamIngest) isProgress_Details() {} +func (*Progress_NewSchemaChange) isProgress_Details() {} func (m *Progress) GetProgress() isProgress_Progress { if m != nil { @@ -2324,6 +2432,13 @@ func (m *Progress) GetStreamIngest() *StreamIngestionProgress { return nil } +func (m *Progress) GetNewSchemaChange() *NewSchemaChangeProgress { + if x, ok := m.GetDetails().(*Progress_NewSchemaChange); ok { + return x.NewSchemaChange + } + return nil +} + // XXX_OneofFuncs is for the internal use of the proto package. func (*Progress) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { return _Progress_OneofMarshaler, _Progress_OneofUnmarshaler, _Progress_OneofSizer, []interface{}{ @@ -2338,6 +2453,7 @@ func (*Progress) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) erro (*Progress_SchemaChangeGC)(nil), (*Progress_TypeSchemaChange)(nil), (*Progress_StreamIngest)(nil), + (*Progress_NewSchemaChange)(nil), } } @@ -2404,6 +2520,11 @@ func _Progress_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.StreamIngest); err != nil { return err } + case *Progress_NewSchemaChange: + _ = b.EncodeVarint(19<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.NewSchemaChange); err != nil { + return err + } case nil: default: return fmt.Errorf("Progress.Details has unexpected type %T", x) @@ -2501,6 +2622,14 @@ func _Progress_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffe err := b.DecodeMessage(msg) m.Details = &Progress_StreamIngest{msg} return true, err + case 19: // details.newSchemaChange + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(NewSchemaChangeProgress) + err := b.DecodeMessage(msg) + m.Details = &Progress_NewSchemaChange{msg} + return true, err default: return false, nil } @@ -2569,6 +2698,11 @@ func _Progress_OneofSizer(msg proto.Message) (n int) { n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *Progress_NewSchemaChange: + s := proto.Size(x.NewSchemaChange) + n += 2 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s case nil: default: panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) @@ -2588,7 +2722,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_a7a6d19cca6b70ee, []int{29} + return fileDescriptor_jobs_8f6d1b2aa13f3feb, []int{31} } func (m *Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2639,6 +2773,8 @@ func init() { proto.RegisterType((*ImportProgress)(nil), "cockroach.sql.jobs.jobspb.ImportProgress") proto.RegisterType((*TypeSchemaChangeDetails)(nil), "cockroach.sql.jobs.jobspb.TypeSchemaChangeDetails") proto.RegisterType((*TypeSchemaChangeProgress)(nil), "cockroach.sql.jobs.jobspb.TypeSchemaChangeProgress") + proto.RegisterType((*NewSchemaChangeDetails)(nil), "cockroach.sql.jobs.jobspb.NewSchemaChangeDetails") + proto.RegisterType((*NewSchemaChangeProgress)(nil), "cockroach.sql.jobs.jobspb.NewSchemaChangeProgress") proto.RegisterType((*ResumeSpanList)(nil), "cockroach.sql.jobs.jobspb.ResumeSpanList") proto.RegisterType((*DroppedTableDetails)(nil), "cockroach.sql.jobs.jobspb.DroppedTableDetails") proto.RegisterType((*SchemaChangeGCDetails)(nil), "cockroach.sql.jobs.jobspb.SchemaChangeGCDetails") @@ -3904,6 +4040,71 @@ func (m *TypeSchemaChangeProgress) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *NewSchemaChangeDetails) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *NewSchemaChangeDetails) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Targets) > 0 { + for _, msg := range m.Targets { + dAtA[i] = 0xa + i++ + i = encodeVarintJobs(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *NewSchemaChangeProgress) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *NewSchemaChangeProgress) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.States) > 0 { + dAtA22 := make([]byte, len(m.States)*10) + var j21 int + for _, num := range m.States { + for num >= 1<<7 { + dAtA22[j21] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j21++ + } + dAtA22[j21] = uint8(num) + j21++ + } + dAtA[i] = 0xa + i++ + i = encodeVarintJobs(dAtA, i, uint64(j21)) + i += copy(dAtA[i:], dAtA22[:j21]) + } + return i, nil +} + func (m *ResumeSpanList) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4016,11 +4217,11 @@ func (m *SchemaChangeGCDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintJobs(dAtA, i, uint64(m.InterleavedTable.Size())) - n21, err := m.InterleavedTable.MarshalTo(dAtA[i:]) + n23, err := m.InterleavedTable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n21 + i += n23 } if len(m.InterleavedIndexes) > 0 { for _, msg := range m.InterleavedIndexes { @@ -4038,11 +4239,11 @@ func (m *SchemaChangeGCDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Tenant.Size())) - n22, err := m.Tenant.MarshalTo(dAtA[i:]) + n24, err := m.Tenant.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n22 + i += n24 } return i, nil } @@ -4191,38 +4392,38 @@ func (m *SchemaChangeDetails) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(m.FormatVersion)) } if len(m.DroppedTypes) > 0 { - dAtA24 := make([]byte, len(m.DroppedTypes)*10) - var j23 int + dAtA26 := make([]byte, len(m.DroppedTypes)*10) + var j25 int for _, num := range m.DroppedTypes { for num >= 1<<7 { - dAtA24[j23] = uint8(uint64(num)&0x7f | 0x80) + dAtA26[j25] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j23++ + j25++ } - dAtA24[j23] = uint8(num) - j23++ + dAtA26[j25] = uint8(num) + j25++ } dAtA[i] = 0x42 i++ - i = encodeVarintJobs(dAtA, i, uint64(j23)) - i += copy(dAtA[i:], dAtA24[:j23]) + i = encodeVarintJobs(dAtA, i, uint64(j25)) + i += copy(dAtA[i:], dAtA26[:j25]) } if len(m.DroppedSchemas) > 0 { - dAtA26 := make([]byte, len(m.DroppedSchemas)*10) - var j25 int + dAtA28 := make([]byte, len(m.DroppedSchemas)*10) + var j27 int for _, num := range m.DroppedSchemas { for num >= 1<<7 { - dAtA26[j25] = uint8(uint64(num)&0x7f | 0x80) + dAtA28[j27] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j25++ + j27++ } - dAtA26[j25] = uint8(num) - j25++ + dAtA28[j27] = uint8(num) + j27++ } dAtA[i] = 0x4a i++ - i = encodeVarintJobs(dAtA, i, uint64(j25)) - i += copy(dAtA[i:], dAtA26[:j25]) + i = encodeVarintJobs(dAtA, i, uint64(j27)) + i += copy(dAtA[i:], dAtA28[:j27]) } return i, nil } @@ -4288,11 +4489,11 @@ func (m *SchemaChangeGCProgress) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Tenant.Size())) - n27, err := m.Tenant.MarshalTo(dAtA[i:]) + n29, err := m.Tenant.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n29 } return i, nil } @@ -4466,21 +4667,21 @@ func (m *ChangefeedDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64((&v).Size())) - n28, err := (&v).MarshalTo(dAtA[i:]) + n30, err := (&v).MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n28 + i += n30 } } dAtA[i] = 0x3a i++ i = encodeVarintJobs(dAtA, i, uint64(m.StatementTime.Size())) - n29, err := m.StatementTime.MarshalTo(dAtA[i:]) + n31, err := m.StatementTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n31 return i, nil } @@ -4502,19 +4703,19 @@ func (m *ResolvedSpan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintJobs(dAtA, i, uint64(m.Span.Size())) - n30, err := m.Span.MarshalTo(dAtA[i:]) + n32, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n30 + i += n32 dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Timestamp.Size())) - n31, err := m.Timestamp.MarshalTo(dAtA[i:]) + n33, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n31 + i += n33 if m.BoundaryReached { dAtA[i] = 0x18 i++ @@ -4558,11 +4759,11 @@ func (m *ChangefeedProgress) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.ProtectedTimestampRecord.Size())) - n32, err := m.ProtectedTimestampRecord.MarshalTo(dAtA[i:]) + n34, err := m.ProtectedTimestampRecord.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n32 + i += n34 return i, nil } @@ -4590,11 +4791,11 @@ func (m *CreateStatsDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Table.Size())) - n33, err := m.Table.MarshalTo(dAtA[i:]) + n35, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n33 + i += n35 if len(m.ColumnStats) > 0 { for _, msg := range m.ColumnStats { dAtA[i] = 0x1a @@ -4617,11 +4818,11 @@ func (m *CreateStatsDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintJobs(dAtA, i, uint64(m.AsOf.Size())) - n34, err := m.AsOf.MarshalTo(dAtA[i:]) + n36, err := m.AsOf.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n34 + i += n36 } if len(m.FQTableName) > 0 { dAtA[i] = 0x32 @@ -4654,21 +4855,21 @@ func (m *CreateStatsDetails_ColStat) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.ColumnIDs) > 0 { - dAtA36 := make([]byte, len(m.ColumnIDs)*10) - var j35 int + dAtA38 := make([]byte, len(m.ColumnIDs)*10) + var j37 int for _, num := range m.ColumnIDs { for num >= 1<<7 { - dAtA36[j35] = uint8(uint64(num)&0x7f | 0x80) + dAtA38[j37] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j35++ + j37++ } - dAtA36[j35] = uint8(num) - j35++ + dAtA38[j37] = uint8(num) + j37++ } dAtA[i] = 0xa i++ - i = encodeVarintJobs(dAtA, i, uint64(j35)) - i += copy(dAtA[i:], dAtA36[:j35]) + i = encodeVarintJobs(dAtA, i, uint64(j37)) + i += copy(dAtA[i:], dAtA38[:j37]) } if m.HasHistogram { dAtA[i] = 0x10 @@ -4754,21 +4955,21 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(m.FinishedMicros)) } if len(m.DescriptorIDs) > 0 { - dAtA38 := make([]byte, len(m.DescriptorIDs)*10) - var j37 int + dAtA40 := make([]byte, len(m.DescriptorIDs)*10) + var j39 int for _, num := range m.DescriptorIDs { for num >= 1<<7 { - dAtA38[j37] = uint8(uint64(num)&0x7f | 0x80) + dAtA40[j39] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j37++ + j39++ } - dAtA38[j37] = uint8(num) - j37++ + dAtA40[j39] = uint8(num) + j39++ } dAtA[i] = 0x32 i++ - i = encodeVarintJobs(dAtA, i, uint64(j37)) - i += copy(dAtA[i:], dAtA38[:j37]) + i = encodeVarintJobs(dAtA, i, uint64(j39)) + i += copy(dAtA[i:], dAtA40[:j39]) } if len(m.Error) > 0 { dAtA[i] = 0x42 @@ -4780,18 +4981,18 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Lease.Size())) - n39, err := m.Lease.MarshalTo(dAtA[i:]) + n41, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n39 + i += n41 } if m.Details != nil { - nn40, err := m.Details.MarshalTo(dAtA[i:]) + nn42, err := m.Details.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn40 + i += nn42 } if len(m.Statement) > 0 { dAtA[i] = 0x82 @@ -4835,11 +5036,11 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.FinalResumeError.Size())) - n41, err := m.FinalResumeError.MarshalTo(dAtA[i:]) + n43, err := m.FinalResumeError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n43 } if m.Noncancelable { dAtA[i] = 0xa0 @@ -4862,11 +5063,11 @@ func (m *Payload_Backup) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Backup.Size())) - n42, err := m.Backup.MarshalTo(dAtA[i:]) + n44, err := m.Backup.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n44 } return i, nil } @@ -4876,11 +5077,11 @@ func (m *Payload_Restore) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Restore.Size())) - n43, err := m.Restore.MarshalTo(dAtA[i:]) + n45, err := m.Restore.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n43 + i += n45 } return i, nil } @@ -4890,11 +5091,11 @@ func (m *Payload_SchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChange.Size())) - n44, err := m.SchemaChange.MarshalTo(dAtA[i:]) + n46, err := m.SchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n44 + i += n46 } return i, nil } @@ -4904,11 +5105,11 @@ func (m *Payload_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Import.Size())) - n45, err := m.Import.MarshalTo(dAtA[i:]) + n47, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n45 + i += n47 } return i, nil } @@ -4918,11 +5119,11 @@ func (m *Payload_Changefeed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Changefeed.Size())) - n46, err := m.Changefeed.MarshalTo(dAtA[i:]) + n48, err := m.Changefeed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n46 + i += n48 } return i, nil } @@ -4932,11 +5133,11 @@ func (m *Payload_CreateStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x7a i++ i = encodeVarintJobs(dAtA, i, uint64(m.CreateStats.Size())) - n47, err := m.CreateStats.MarshalTo(dAtA[i:]) + n49, err := m.CreateStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n47 + i += n49 } return i, nil } @@ -4948,11 +5149,11 @@ func (m *Payload_SchemaChangeGC) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChangeGC.Size())) - n48, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) + n50, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n48 + i += n50 } return i, nil } @@ -4964,11 +5165,11 @@ func (m *Payload_TypeSchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.TypeSchemaChange.Size())) - n49, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) + n51, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n51 } return i, nil } @@ -4980,11 +5181,27 @@ func (m *Payload_StreamIngestion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.StreamIngestion.Size())) - n50, err := m.StreamIngestion.MarshalTo(dAtA[i:]) + n52, err := m.StreamIngestion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n52 + } + return i, nil +} +func (m *Payload_NewSchemaChange) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.NewSchemaChange != nil { + dAtA[i] = 0xc2 + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintJobs(dAtA, i, uint64(m.NewSchemaChange.Size())) + n53, err := m.NewSchemaChange.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n53 } return i, nil } @@ -5004,11 +5221,11 @@ func (m *Progress) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Progress != nil { - nn51, err := m.Progress.MarshalTo(dAtA[i:]) + nn54, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn51 + i += nn54 } if m.ModifiedMicros != 0 { dAtA[i] = 0x10 @@ -5022,11 +5239,11 @@ func (m *Progress) MarshalTo(dAtA []byte) (int, error) { i += copy(dAtA[i:], m.RunningStatus) } if m.Details != nil { - nn52, err := m.Details.MarshalTo(dAtA[i:]) + nn55, err := m.Details.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn52 + i += nn55 } return i, nil } @@ -5045,11 +5262,11 @@ func (m *Progress_HighWater) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.HighWater.Size())) - n53, err := m.HighWater.MarshalTo(dAtA[i:]) + n56, err := m.HighWater.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n56 } return i, nil } @@ -5059,11 +5276,11 @@ func (m *Progress_Backup) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Backup.Size())) - n54, err := m.Backup.MarshalTo(dAtA[i:]) + n57, err := m.Backup.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n57 } return i, nil } @@ -5073,11 +5290,11 @@ func (m *Progress_Restore) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Restore.Size())) - n55, err := m.Restore.MarshalTo(dAtA[i:]) + n58, err := m.Restore.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n58 } return i, nil } @@ -5087,11 +5304,11 @@ func (m *Progress_SchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChange.Size())) - n56, err := m.SchemaChange.MarshalTo(dAtA[i:]) + n59, err := m.SchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n59 } return i, nil } @@ -5101,11 +5318,11 @@ func (m *Progress_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Import.Size())) - n57, err := m.Import.MarshalTo(dAtA[i:]) + n60, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n60 } return i, nil } @@ -5115,11 +5332,11 @@ func (m *Progress_Changefeed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Changefeed.Size())) - n58, err := m.Changefeed.MarshalTo(dAtA[i:]) + n61, err := m.Changefeed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n58 + i += n61 } return i, nil } @@ -5129,11 +5346,11 @@ func (m *Progress_CreateStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x7a i++ i = encodeVarintJobs(dAtA, i, uint64(m.CreateStats.Size())) - n59, err := m.CreateStats.MarshalTo(dAtA[i:]) + n62, err := m.CreateStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n59 + i += n62 } return i, nil } @@ -5145,11 +5362,11 @@ func (m *Progress_SchemaChangeGC) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChangeGC.Size())) - n60, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) + n63, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n60 + i += n63 } return i, nil } @@ -5161,11 +5378,11 @@ func (m *Progress_TypeSchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.TypeSchemaChange.Size())) - n61, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) + n64, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n61 + i += n64 } return i, nil } @@ -5177,11 +5394,27 @@ func (m *Progress_StreamIngest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.StreamIngest.Size())) - n62, err := m.StreamIngest.MarshalTo(dAtA[i:]) + n65, err := m.StreamIngest.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n62 + i += n65 + } + return i, nil +} +func (m *Progress_NewSchemaChange) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.NewSchemaChange != nil { + dAtA[i] = 0x9a + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintJobs(dAtA, i, uint64(m.NewSchemaChange.Size())) + n66, err := m.NewSchemaChange.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n66 } return i, nil } @@ -5209,21 +5442,21 @@ func (m *Job) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Progress.Size())) - n63, err := m.Progress.MarshalTo(dAtA[i:]) + n67, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n63 + i += n67 } if m.Payload != nil { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Payload.Size())) - n64, err := m.Payload.MarshalTo(dAtA[i:]) + n68, err := m.Payload.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n64 + i += n68 } return i, nil } @@ -5743,6 +5976,37 @@ func (m *TypeSchemaChangeProgress) Size() (n int) { return n } +func (m *NewSchemaChangeDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Targets) > 0 { + for _, e := range m.Targets { + l = e.Size() + n += 1 + l + sovJobs(uint64(l)) + } + } + return n +} + +func (m *NewSchemaChangeProgress) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.States) > 0 { + l = 0 + for _, e := range m.States { + l += sovJobs(uint64(e)) + } + n += 1 + sovJobs(uint64(l)) + l + } + return n +} + func (m *ResumeSpanList) Size() (n int) { if m == nil { return 0 @@ -6303,6 +6567,18 @@ func (m *Payload_StreamIngestion) Size() (n int) { } return n } +func (m *Payload_NewSchemaChange) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NewSchemaChange != nil { + l = m.NewSchemaChange.Size() + n += 2 + l + sovJobs(uint64(l)) + } + return n +} func (m *Progress) Size() (n int) { if m == nil { return 0 @@ -6454,6 +6730,18 @@ func (m *Progress_StreamIngest) Size() (n int) { } return n } +func (m *Progress_NewSchemaChange) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NewSchemaChange != nil { + l = m.NewSchemaChange.Size() + n += 2 + l + sovJobs(uint64(l)) + } + return n +} func (m *Job) Size() (n int) { if m == nil { return 0 @@ -10028,6 +10316,203 @@ func (m *TypeSchemaChangeProgress) Unmarshal(dAtA []byte) error { } return nil } +func (m *NewSchemaChangeDetails) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: NewSchemaChangeDetails: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: NewSchemaChangeDetails: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Targets", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Targets = append(m.Targets, &scpb.Target{}) + if err := m.Targets[len(m.Targets)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipJobs(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthJobs + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *NewSchemaChangeProgress) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: NewSchemaChangeProgress: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: NewSchemaChangeProgress: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v scpb.State + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (scpb.State(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.States = append(m.States, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.States) == 0 { + m.States = make([]scpb.State, 0, elementCount) + } + for iNdEx < postIndex { + var v scpb.State + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (scpb.State(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.States = append(m.States, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field States", wireType) + } + default: + iNdEx = preIndex + skippy, err := skipJobs(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthJobs + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ResumeSpanList) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -13317,6 +13802,38 @@ func (m *Payload) Unmarshal(dAtA []byte) error { } m.Details = &Payload_StreamIngestion{v} iNdEx = postIndex + case 24: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NewSchemaChange", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &NewSchemaChangeDetails{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Details = &Payload_NewSchemaChange{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipJobs(dAtA[iNdEx:]) @@ -13746,6 +14263,38 @@ func (m *Progress) Unmarshal(dAtA []byte) error { } m.Details = &Progress_StreamIngest{v} iNdEx = postIndex + case 19: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NewSchemaChange", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &NewSchemaChangeProgress{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Details = &Progress_NewSchemaChange{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipJobs(dAtA[iNdEx:]) @@ -14007,294 +14556,302 @@ var ( ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_a7a6d19cca6b70ee) } - -var fileDescriptor_jobs_a7a6d19cca6b70ee = []byte{ - // 4571 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5b, 0x4b, 0x6c, 0x23, 0x47, - 0x7a, 0x56, 0x93, 0x14, 0xd9, 0xfc, 0x29, 0x52, 0xad, 0x92, 0x66, 0x86, 0x66, 0x6c, 0x51, 0xa1, - 0x5f, 0x33, 0x63, 0x9b, 0xf2, 0xca, 0x59, 0xaf, 0x3d, 0xb1, 0xc7, 0xe6, 0x6b, 0x24, 0x52, 0xa3, - 0x87, 0x9b, 0xd2, 0xf8, 0xb1, 0x70, 0x3a, 0xcd, 0xee, 0x12, 0xd5, 0x11, 0xd9, 0xcd, 0xe9, 0x6a, - 0xce, 0x8c, 0x36, 0x41, 0x12, 0x6c, 0x10, 0x60, 0x31, 0xa7, 0x04, 0x48, 0x72, 0x49, 0x06, 0x08, - 0x90, 0x5d, 0x20, 0x87, 0x00, 0x01, 0x8c, 0x20, 0xc9, 0x21, 0xb7, 0xbd, 0x18, 0xc1, 0x06, 0xd8, - 0x4b, 0x00, 0x23, 0x07, 0x6e, 0x22, 0x5f, 0x72, 0x0c, 0x92, 0xdb, 0x5c, 0x12, 0xd4, 0xa3, 0x9b, - 0x4d, 0xea, 0x45, 0x8d, 0xec, 0xcd, 0x45, 0xc3, 0xfe, 0xeb, 0xaf, 0xaf, 0xaa, 0xfe, 0xfa, 0x9f, - 0x55, 0x35, 0x70, 0xf5, 0xb7, 0x9c, 0x16, 0x59, 0xa6, 0x7f, 0x7a, 0x2d, 0xf6, 0x4f, 0xb1, 0xe7, - 0x3a, 0x9e, 0x83, 0x9e, 0x33, 0x1c, 0xe3, 0xc0, 0x75, 0x74, 0x63, 0xbf, 0x48, 0xee, 0x77, 0x8a, - 0xac, 0x85, 0x73, 0xe5, 0xae, 0x60, 0xd7, 0x75, 0x5c, 0xca, 0xcf, 0x7f, 0xf0, 0x1e, 0xb9, 0x85, - 0xb6, 0xd3, 0x76, 0xd8, 0xcf, 0x65, 0xfa, 0x4b, 0x50, 0x11, 0xc3, 0xe8, 0xb5, 0x96, 0x4d, 0xdd, - 0xd3, 0x05, 0x2d, 0xeb, 0xd3, 0x2c, 0xe7, 0x8d, 0x3d, 0xc7, 0xed, 0xea, 0x9e, 0x8f, 0xf1, 0x22, - 0xb9, 0xdf, 0x59, 0x36, 0x74, 0x4f, 0xef, 0x38, 0xed, 0x65, 0x13, 0x13, 0xa3, 0xd7, 0x5a, 0x26, - 0x9e, 0xdb, 0x37, 0xbc, 0xbe, 0x8b, 0x4d, 0xc1, 0x94, 0x3f, 0x81, 0xc9, 0xc3, 0xb6, 0x6e, 0x7b, - 0x3e, 0x7e, 0xdf, 0xb3, 0x3a, 0xcb, 0xfb, 0x1d, 0x63, 0xd9, 0xb3, 0xba, 0x98, 0x78, 0x7a, 0xb7, - 0xc7, 0x5b, 0x0a, 0xbf, 0x07, 0xd3, 0x77, 0xb1, 0x4e, 0x30, 0xfa, 0x0c, 0x12, 0xb6, 0x63, 0x62, - 0xcd, 0x32, 0xb3, 0xd2, 0x92, 0x74, 0x3d, 0x5d, 0x2e, 0x1d, 0x0d, 0xf2, 0xf1, 0x4d, 0xc7, 0xc4, - 0xf5, 0xea, 0xd3, 0x41, 0xfe, 0xad, 0xb6, 0xe5, 0xed, 0xf7, 0x5b, 0x45, 0xc3, 0xe9, 0x2e, 0x07, - 0x82, 0x30, 0x5b, 0xc3, 0xdf, 0xcb, 0xbd, 0x83, 0xf6, 0xb2, 0x58, 0x46, 0x91, 0x77, 0x53, 0xe3, - 0x14, 0xb1, 0x6e, 0xa2, 0x05, 0x98, 0xc6, 0x3d, 0xc7, 0xd8, 0xcf, 0x46, 0x96, 0xa4, 0xeb, 0x51, - 0x95, 0x7f, 0xdc, 0x8a, 0xfd, 0xe7, 0x5f, 0xe6, 0xa5, 0xc2, 0x4f, 0x22, 0x70, 0xad, 0xac, 0x1b, - 0x07, 0xfd, 0x5e, 0xcd, 0x36, 0xdc, 0xc3, 0x9e, 0x67, 0x39, 0xf6, 0x16, 0xfb, 0x4b, 0x90, 0x02, - 0xd1, 0x03, 0x7c, 0xc8, 0xe6, 0x33, 0xa3, 0xd2, 0x9f, 0xe8, 0x7d, 0x88, 0x75, 0x1d, 0x13, 0x33, - 0xa0, 0xcc, 0xca, 0x8d, 0xe2, 0xa9, 0x7b, 0x52, 0x1c, 0xa2, 0x6d, 0x38, 0x26, 0x56, 0x59, 0x37, - 0xd4, 0x02, 0xf9, 0xa0, 0x4b, 0x34, 0xcb, 0xde, 0x73, 0xb2, 0xd1, 0x25, 0xe9, 0x7a, 0x6a, 0xe5, - 0xd6, 0x19, 0x10, 0xa7, 0x4c, 0xab, 0xb8, 0xbe, 0xd1, 0xac, 0xdb, 0x7b, 0x4e, 0x39, 0x75, 0x34, - 0xc8, 0x27, 0xc4, 0x87, 0x9a, 0x38, 0xe8, 0x12, 0xfa, 0x23, 0xb7, 0x05, 0x3e, 0x8d, 0xce, 0xbf, - 0xef, 0x5a, 0x6c, 0xfe, 0x49, 0x95, 0xfe, 0x44, 0xaf, 0x03, 0xc2, 0x1c, 0x0f, 0x9b, 0x1a, 0x55, - 0x00, 0x8d, 0x2e, 0x30, 0xc2, 0x16, 0xa8, 0x04, 0x2d, 0x55, 0xdd, 0xd3, 0xd7, 0xf1, 0x21, 0x97, - 0x90, 0x90, 0xd3, 0xef, 0x47, 0x21, 0x33, 0x9c, 0x0a, 0x83, 0x5f, 0x83, 0x38, 0x31, 0xf6, 0x71, - 0x17, 0xb3, 0x11, 0x32, 0x2b, 0x6f, 0x4e, 0x24, 0x0e, 0xda, 0xb5, 0xd8, 0x64, 0xfd, 0x54, 0xd1, - 0x1f, 0x21, 0x88, 0x11, 0xbd, 0xe3, 0x89, 0x89, 0xb0, 0xdf, 0xe8, 0xcf, 0x25, 0x58, 0x1a, 0x9f, - 0x51, 0xf9, 0x70, 0x7d, 0xa3, 0xb9, 0xa1, 0x13, 0x0f, 0xbb, 0xeb, 0xf8, 0xb0, 0x5e, 0xcd, 0x46, - 0x97, 0xa2, 0xd7, 0x53, 0x2b, 0x5b, 0x93, 0x0f, 0x5c, 0x3b, 0x07, 0xb1, 0x66, 0x7b, 0xee, 0xa1, - 0x7a, 0xee, 0xc0, 0xb9, 0x26, 0xbc, 0x3c, 0x11, 0x54, 0x58, 0x87, 0x92, 0x5c, 0x87, 0x16, 0x60, - 0xfa, 0x81, 0xde, 0xe9, 0x63, 0xb1, 0x5a, 0xfe, 0x71, 0x2b, 0xf2, 0x8e, 0x54, 0xb8, 0x06, 0x71, - 0x2e, 0x18, 0x94, 0x86, 0x64, 0xa9, 0xd6, 0x5c, 0xf9, 0xee, 0xdb, 0xab, 0x95, 0x0d, 0x65, 0x4a, - 0x6c, 0xc1, 0xff, 0x4a, 0x70, 0xb5, 0xe9, 0xb9, 0x58, 0xef, 0xd6, 0xed, 0x36, 0x26, 0x74, 0x4d, - 0x55, 0xec, 0xe9, 0x56, 0x87, 0x20, 0x1b, 0x32, 0x84, 0xb5, 0x68, 0xba, 0x69, 0xba, 0x98, 0x10, - 0x3e, 0x60, 0x79, 0xf5, 0xe9, 0x20, 0x5f, 0x99, 0xc8, 0x74, 0x0c, 0xa3, 0xb3, 0xcc, 0x21, 0x2c, - 0xbb, 0x6d, 0x18, 0x9d, 0x22, 0x1f, 0xa9, 0xc4, 0xe1, 0xd4, 0x34, 0x09, 0x7f, 0xa2, 0xef, 0x40, - 0x8c, 0xf4, 0x74, 0x9b, 0x2d, 0x21, 0xb5, 0x72, 0x2d, 0x24, 0x7f, 0xdf, 0x04, 0x9b, 0x3d, 0xdd, - 0x2e, 0xc7, 0xbe, 0x1c, 0xe4, 0xa7, 0x54, 0xc6, 0x8a, 0xca, 0x00, 0xc4, 0xd3, 0x5d, 0x4f, 0xa3, - 0x2e, 0x40, 0x68, 0xff, 0x0b, 0xa1, 0x8e, 0xd4, 0x45, 0x14, 0xf7, 0x3b, 0x46, 0x71, 0xc7, 0x77, - 0x11, 0xa2, 0x7b, 0x92, 0x75, 0xa3, 0xd4, 0xc2, 0x73, 0x70, 0x6d, 0x4c, 0x00, 0xdb, 0xae, 0xd3, - 0xa6, 0x33, 0x2a, 0xfc, 0x53, 0x1c, 0xd2, 0xdc, 0x60, 0x7c, 0x99, 0x8c, 0x0e, 0x28, 0x3d, 0xcb, - 0x80, 0xe8, 0x36, 0xc8, 0xd8, 0x36, 0x39, 0x42, 0x64, 0x72, 0x84, 0x04, 0xb6, 0x4d, 0xd6, 0xff, - 0x39, 0x6e, 0x81, 0x51, 0xb6, 0x19, 0x89, 0xa3, 0x41, 0x3e, 0xba, 0xab, 0xd6, 0xb9, 0x29, 0xbe, - 0x07, 0x39, 0x13, 0xf7, 0x5c, 0x6c, 0xe8, 0xd4, 0x16, 0x5b, 0x6c, 0xea, 0x5a, 0x57, 0xb7, 0xad, - 0x3d, 0x4c, 0xbc, 0x6c, 0x8c, 0xe9, 0x46, 0x76, 0xc8, 0xc1, 0xd7, 0xb6, 0x21, 0xda, 0xd1, 0x1f, - 0x48, 0x30, 0xdf, 0x77, 0x2d, 0xa2, 0xb5, 0x0e, 0xb5, 0x8e, 0x63, 0xe8, 0x1d, 0xcb, 0x3b, 0xd4, - 0x0e, 0x1e, 0x64, 0xa7, 0x99, 0x41, 0xdc, 0x3e, 0xd7, 0xab, 0x08, 0x21, 0x15, 0x77, 0x5d, 0x8b, - 0x94, 0x0f, 0xef, 0x0a, 0x84, 0xf5, 0x07, 0x4c, 0x69, 0xcb, 0x0b, 0x47, 0x83, 0xbc, 0xb2, 0xab, - 0xd6, 0xc3, 0x4d, 0xf7, 0x54, 0xa5, 0x3f, 0xc6, 0x8c, 0xf4, 0xc0, 0x9d, 0x58, 0x8e, 0xad, 0x39, - 0xdc, 0x3f, 0x65, 0xe3, 0x4c, 0x50, 0x2b, 0x17, 0xf7, 0x6c, 0xea, 0x1c, 0x3e, 0xe6, 0x83, 0xff, - 0x58, 0x82, 0x1c, 0x0d, 0x15, 0xd8, 0xa0, 0x62, 0x0a, 0xc2, 0x87, 0xe6, 0x62, 0xc3, 0x71, 0xcd, - 0x6c, 0x82, 0xca, 0xa9, 0xdc, 0xfc, 0xb7, 0x49, 0x23, 0x04, 0x0b, 0x44, 0xfd, 0xbe, 0x65, 0x16, - 0x77, 0x77, 0xeb, 0xd5, 0xa3, 0x41, 0x3e, 0xbb, 0xed, 0x83, 0x07, 0x9b, 0xa8, 0x32, 0x68, 0x35, - 0xdb, 0x3b, 0xa5, 0x05, 0xbd, 0x03, 0x19, 0xc3, 0xe9, 0x74, 0xb0, 0xc1, 0x96, 0xbd, 0xab, 0xd6, - 0xb3, 0x32, 0xdb, 0xe0, 0xb9, 0xa3, 0x41, 0x3e, 0x5d, 0x09, 0x5a, 0xe8, 0x56, 0xa7, 0x8d, 0xf0, - 0x27, 0x52, 0x61, 0x36, 0x24, 0x30, 0x16, 0x07, 0x92, 0x4c, 0x5a, 0x37, 0x26, 0x76, 0x61, 0x6a, - 0x06, 0x8f, 0x7c, 0xe7, 0x2a, 0x70, 0xe5, 0xc4, 0x5d, 0x3c, 0xcf, 0xf5, 0x24, 0xc3, 0xae, 0x47, - 0x81, 0x0c, 0xdf, 0x94, 0xc0, 0xa0, 0xbe, 0xc8, 0x40, 0x46, 0xc5, 0xc4, 0x73, 0x5c, 0xec, 0x5b, - 0xd4, 0x17, 0x12, 0xcc, 0xd3, 0xf0, 0xee, 0x5a, 0x3d, 0xcf, 0x71, 0x35, 0x17, 0x3f, 0x74, 0x2d, - 0x0f, 0x93, 0x6c, 0x84, 0x29, 0x5d, 0xe9, 0x8c, 0x25, 0x8c, 0x02, 0x15, 0xab, 0x01, 0x88, 0x2a, - 0x30, 0xb8, 0xde, 0xdd, 0xfe, 0xe1, 0x2f, 0xf2, 0xb7, 0x26, 0xda, 0xc7, 0xe3, 0x19, 0x47, 0xb1, - 0x5e, 0x55, 0x91, 0x79, 0x0c, 0x18, 0x3d, 0x0f, 0x31, 0xaa, 0xb7, 0x2c, 0x54, 0x24, 0xcb, 0xf2, - 0xd1, 0x20, 0x1f, 0xa3, 0x9a, 0xad, 0x32, 0xea, 0x88, 0x81, 0xc7, 0x9e, 0xc1, 0xc0, 0x57, 0x21, - 0xe5, 0xe9, 0xad, 0x0e, 0xd6, 0xe8, 0xc8, 0x44, 0x98, 0xdf, 0x2b, 0x63, 0x92, 0x20, 0xf7, 0x3b, - 0x2d, 0x9d, 0xe0, 0xe2, 0x0e, 0xe5, 0x0c, 0xad, 0x1d, 0x3c, 0x9f, 0x40, 0xd0, 0x32, 0xa4, 0x9c, - 0x07, 0xd8, 0x75, 0x2d, 0x13, 0x6b, 0x66, 0x8b, 0xd9, 0x50, 0xb2, 0x9c, 0x39, 0x1a, 0xe4, 0x61, - 0x4b, 0x90, 0xab, 0x65, 0x15, 0x7c, 0x96, 0x6a, 0x0b, 0x79, 0xb0, 0x20, 0x9c, 0x46, 0x60, 0xff, - 0x4c, 0x9f, 0x12, 0x6c, 0x0a, 0xef, 0x4d, 0xbe, 0x19, 0x7c, 0xdf, 0x7d, 0xe5, 0x61, 0x99, 0x05, - 0x5f, 0x24, 0x6a, 0x1d, 0x6b, 0x41, 0xaf, 0xc1, 0x5c, 0xcf, 0xc5, 0x3d, 0xdd, 0xc5, 0x9a, 0xe1, - 0x74, 0x7b, 0x1d, 0xec, 0x61, 0x93, 0x69, 0xbf, 0xac, 0x2a, 0xa2, 0xa1, 0xe2, 0xd3, 0xd1, 0xcb, - 0x34, 0x2a, 0xe9, 0x1e, 0x4d, 0x78, 0x08, 0x76, 0x29, 0x67, 0x92, 0x71, 0xa6, 0x19, 0xb5, 0x2e, - 0x88, 0xe8, 0x2d, 0xb8, 0x32, 0xdc, 0x37, 0xa2, 0xf5, 0xfa, 0xad, 0x8e, 0x45, 0xf6, 0xb1, 0x99, - 0x05, 0xc6, 0xbd, 0x10, 0x6a, 0xdc, 0xf6, 0xdb, 0xd0, 0xe1, 0x88, 0x2a, 0x1a, 0x54, 0x30, 0x7a, - 0x1b, 0x67, 0x53, 0x4b, 0xd2, 0xf5, 0xe9, 0xf2, 0xda, 0xd3, 0x41, 0xbe, 0x3a, 0xb1, 0x1e, 0x11, - 0xdc, 0x5d, 0xf6, 0x5c, 0x8c, 0x43, 0x6a, 0x59, 0x11, 0x78, 0x61, 0x8d, 0xf2, 0x69, 0x48, 0x05, - 0x18, 0x9a, 0x60, 0x76, 0xe6, 0x99, 0xbd, 0x5d, 0x08, 0x05, 0x95, 0x20, 0xc1, 0x33, 0x66, 0x92, - 0x4d, 0xb3, 0x0d, 0xfc, 0xd5, 0xd3, 0x74, 0x88, 0x71, 0x85, 0x76, 0xc9, 0xef, 0x87, 0xaa, 0x00, - 0xde, 0x61, 0xcf, 0xd7, 0xc4, 0x0c, 0x43, 0x79, 0xf9, 0x34, 0x94, 0xc3, 0x5e, 0x58, 0x11, 0x93, - 0x9e, 0xf8, 0x26, 0xa8, 0x01, 0x33, 0x2c, 0x29, 0xd3, 0x05, 0xce, 0x2c, 0xc3, 0x79, 0xf5, 0x14, - 0x1c, 0x96, 0xae, 0xe8, 0x21, 0xa4, 0x14, 0x09, 0x28, 0x04, 0x6d, 0x43, 0x86, 0xe6, 0x98, 0x94, - 0x53, 0xa0, 0x29, 0x0c, 0xed, 0xc6, 0x29, 0x68, 0x55, 0xc1, 0x1c, 0xc2, 0x4b, 0x9b, 0x21, 0x1a, - 0xc9, 0xfd, 0x8f, 0x04, 0x73, 0xc7, 0x9c, 0x07, 0xda, 0x81, 0x48, 0x50, 0x36, 0x50, 0x9f, 0x1e, - 0x61, 0x25, 0xc3, 0x65, 0x1c, 0x49, 0xc4, 0x32, 0x51, 0x1b, 0x92, 0x54, 0x9d, 0x6d, 0x8f, 0xd6, - 0x24, 0x11, 0x06, 0xde, 0x38, 0x1a, 0xe4, 0xe5, 0x6d, 0x46, 0xbc, 0xf4, 0x10, 0x32, 0x07, 0xaf, - 0x9b, 0x28, 0x0f, 0x29, 0xcf, 0xd1, 0xf0, 0x23, 0x8b, 0x78, 0x96, 0xdd, 0x66, 0xc9, 0x82, 0xac, - 0x82, 0xe7, 0xd4, 0x04, 0x25, 0xf7, 0x17, 0x11, 0x40, 0xc7, 0xad, 0x14, 0xfd, 0xa3, 0x04, 0xcf, - 0xfb, 0x39, 0x80, 0xe3, 0x5a, 0x6d, 0xcb, 0xd6, 0x3b, 0x23, 0xc9, 0x80, 0xc4, 0xa4, 0xfd, 0xd9, - 0x65, 0x5c, 0x81, 0x48, 0x10, 0xb6, 0x04, 0xfc, 0x78, 0xa2, 0xf0, 0x3c, 0x8d, 0xa0, 0x3c, 0x51, - 0x38, 0xc6, 0x72, 0x4f, 0xcd, 0xf6, 0x4f, 0xe9, 0x9c, 0x5b, 0x87, 0x17, 0xce, 0x04, 0xbe, 0x48, - 0xec, 0xca, 0xfd, 0x50, 0x82, 0x6b, 0xa7, 0x44, 0x94, 0x30, 0x4e, 0x9a, 0xe3, 0x7c, 0x14, 0xc6, - 0x49, 0xad, 0xfc, 0xfa, 0x25, 0xa2, 0x56, 0x68, 0x12, 0x8d, 0x98, 0x2c, 0x29, 0x91, 0xc2, 0x9b, - 0x30, 0x2b, 0x3a, 0xf9, 0x71, 0x14, 0xbd, 0x00, 0xb0, 0x6f, 0xb5, 0xf7, 0xb5, 0x87, 0xba, 0x87, - 0x5d, 0x51, 0x4b, 0x26, 0x29, 0xe5, 0x63, 0x4a, 0x28, 0xfc, 0xab, 0x0c, 0xe9, 0x7a, 0xb7, 0xe7, - 0xb8, 0x9e, 0x1f, 0x65, 0xef, 0x42, 0x9c, 0xc5, 0x05, 0x22, 0xf6, 0xaf, 0x78, 0xc6, 0x0c, 0x47, - 0x7a, 0xf2, 0xf8, 0x22, 0xdc, 0x82, 0xc0, 0x08, 0xc2, 0x5f, 0xe4, 0xc4, 0xf0, 0xf7, 0x3e, 0xc4, - 0x79, 0xbd, 0x2f, 0x12, 0xf2, 0xfc, 0x09, 0x99, 0x7c, 0x7d, 0xeb, 0x8e, 0xd5, 0xc1, 0x77, 0x18, - 0x9b, 0x0f, 0xce, 0x3b, 0xa1, 0x57, 0x40, 0x26, 0xc4, 0xd3, 0x88, 0xf5, 0x03, 0x1e, 0x3d, 0xa3, - 0xbc, 0x26, 0x6d, 0x36, 0x77, 0x9a, 0xd6, 0x0f, 0xb0, 0x9a, 0x20, 0xc4, 0xa3, 0x3f, 0x50, 0x0e, - 0xe4, 0x87, 0x7a, 0xa7, 0xc3, 0xa2, 0xec, 0x34, 0xab, 0xc1, 0x83, 0xef, 0x51, 0x33, 0x8b, 0x7f, - 0xbb, 0x66, 0x26, 0x02, 0x66, 0x4f, 0xf7, 0xf6, 0x59, 0xe6, 0x98, 0x54, 0x81, 0x93, 0xb6, 0x75, - 0x6f, 0x1f, 0x65, 0x21, 0x41, 0x74, 0x1a, 0xbb, 0x48, 0x56, 0x5e, 0x8a, 0x5e, 0x9f, 0x51, 0xfd, - 0x4f, 0xb4, 0x08, 0x2c, 0xf2, 0xf2, 0x4f, 0x16, 0xc4, 0xa2, 0x6a, 0x88, 0xc2, 0xe4, 0x70, 0x60, - 0xf5, 0xb4, 0xbd, 0x03, 0xc2, 0x83, 0x96, 0x90, 0xc3, 0x81, 0xd5, 0xbb, 0xb3, 0x4e, 0xd4, 0x04, - 0x6d, 0xbc, 0x73, 0x40, 0xd0, 0xab, 0x30, 0x6b, 0xb1, 0xca, 0x45, 0x33, 0x2d, 0x17, 0x1b, 0x5e, - 0xe7, 0x90, 0x05, 0x2c, 0x59, 0xcd, 0x70, 0x72, 0x55, 0x50, 0xd1, 0x0d, 0x50, 0xc6, 0xc3, 0x2c, - 0x0b, 0x34, 0xb2, 0x3a, 0x3b, 0x16, 0x65, 0x29, 0x2b, 0xdf, 0xea, 0x50, 0xe0, 0x4c, 0x73, 0x56, - 0x4e, 0x1f, 0xc6, 0xcc, 0x22, 0xcc, 0xf7, 0x74, 0x97, 0x60, 0xad, 0xd5, 0xb7, 0xcd, 0x0e, 0xd6, - 0xb8, 0xaf, 0xce, 0x66, 0x18, 0xf7, 0x1c, 0x6b, 0x2a, 0xb3, 0x16, 0xee, 0xd6, 0xcf, 0xcb, 0xbd, - 0xaf, 0xfe, 0x3f, 0xe4, 0xde, 0xb9, 0x9f, 0x44, 0x60, 0x9a, 0xe9, 0x39, 0xba, 0x05, 0x31, 0xba, - 0xcd, 0xa2, 0xb2, 0x9b, 0x34, 0xe7, 0x62, 0x7d, 0x10, 0x82, 0x98, 0xad, 0x77, 0x71, 0x16, 0x31, - 0x25, 0x60, 0xbf, 0xd1, 0x35, 0x48, 0x10, 0x7c, 0x5f, 0x7b, 0xa0, 0x77, 0xb2, 0xf3, 0x6c, 0x87, - 0xe3, 0x04, 0xdf, 0xbf, 0xa7, 0x77, 0xd0, 0x15, 0x88, 0x5b, 0x44, 0xb3, 0xf1, 0xc3, 0xec, 0x02, - 0x93, 0xd4, 0xb4, 0x45, 0x36, 0xf1, 0x43, 0xe6, 0xb6, 0x75, 0xb7, 0x8d, 0x3d, 0xcd, 0x70, 0x3a, - 0x24, 0x7b, 0x85, 0x1a, 0x18, 0x4d, 0xe9, 0x28, 0xa9, 0xe2, 0x74, 0x08, 0xfa, 0x15, 0x48, 0x3e, - 0xd4, 0x89, 0x86, 0xbb, 0x3d, 0xef, 0x90, 0x09, 0x4b, 0xa6, 0x6a, 0x4f, 0x6a, 0xf4, 0xbb, 0x11, - 0x93, 0x23, 0x4a, 0xb4, 0x11, 0x93, 0xa3, 0x4a, 0xac, 0x11, 0x93, 0x63, 0xca, 0x74, 0x23, 0x26, - 0x4f, 0x2b, 0xf1, 0x46, 0x4c, 0x8e, 0x2b, 0x89, 0x46, 0x4c, 0x4e, 0x28, 0x72, 0x23, 0x26, 0xcb, - 0x4a, 0xb2, 0x11, 0x93, 0x93, 0x0a, 0x34, 0x62, 0x32, 0x28, 0xa9, 0x46, 0x4c, 0x4e, 0x29, 0x33, - 0x8d, 0x98, 0x3c, 0xa3, 0xa4, 0x1b, 0x31, 0x39, 0xad, 0x64, 0x1a, 0x31, 0x39, 0xa3, 0xcc, 0x36, - 0x62, 0xf2, 0xac, 0xa2, 0x34, 0x62, 0xb2, 0xa2, 0xcc, 0x35, 0x62, 0xf2, 0x9c, 0x82, 0x0a, 0x5f, - 0x48, 0xa0, 0x34, 0xf1, 0xfd, 0x3e, 0xb6, 0x0d, 0x7c, 0x4f, 0xef, 0x54, 0xf6, 0xfb, 0xf6, 0x01, - 0x7a, 0x05, 0x66, 0x0d, 0xfa, 0x43, 0xe3, 0x85, 0x31, 0x5d, 0xaa, 0xc4, 0x96, 0x9a, 0x66, 0xe4, - 0x26, 0xa5, 0xd2, 0x15, 0xbf, 0x00, 0x20, 0xf8, 0xa8, 0x65, 0xf3, 0x53, 0xb3, 0x24, 0x67, 0xa1, - 0xe6, 0x3c, 0x06, 0xe3, 0x3a, 0x0f, 0x99, 0xfb, 0x18, 0x81, 0x51, 0x9d, 0x87, 0x68, 0x19, 0x16, - 0x6c, 0xfc, 0xc8, 0xd3, 0xc6, 0x99, 0x99, 0xab, 0x50, 0xe7, 0x68, 0x5b, 0x25, 0xdc, 0xa1, 0xf0, - 0x2f, 0x11, 0x98, 0xf5, 0x27, 0xed, 0xbb, 0xc3, 0x3d, 0x50, 0xe8, 0xb6, 0x58, 0xa6, 0xe6, 0x39, - 0x1c, 0xc9, 0x77, 0x8c, 0xef, 0x9f, 0xe1, 0x18, 0xc7, 0x50, 0xe8, 0x77, 0xdd, 0xdc, 0x71, 0xd8, - 0x70, 0x3c, 0x34, 0xa8, 0x69, 0x12, 0xa6, 0xe5, 0x76, 0x21, 0xe3, 0x77, 0xe2, 0x14, 0x54, 0x81, - 0xf8, 0xc8, 0x78, 0xaf, 0x4d, 0x30, 0x9e, 0x2f, 0x6a, 0x55, 0x74, 0xcd, 0xfd, 0x36, 0xa0, 0xe3, - 0x63, 0x87, 0xc3, 0xd2, 0x34, 0x0f, 0x4b, 0x5b, 0xa3, 0x61, 0xe9, 0xdd, 0x8b, 0xad, 0x2d, 0x34, - 0xed, 0x70, 0x55, 0xf7, 0xd3, 0x08, 0x64, 0x78, 0x88, 0x08, 0xc2, 0xd1, 0x6b, 0x30, 0xc7, 0x9c, - 0x96, 0x65, 0xb7, 0xb5, 0x9e, 0x20, 0xb2, 0xf5, 0x45, 0x54, 0xc5, 0x6f, 0x08, 0x98, 0x5f, 0x84, - 0xb4, 0x8b, 0x75, 0x73, 0xc8, 0x18, 0x61, 0x8c, 0x33, 0x94, 0x18, 0x30, 0xbd, 0x0c, 0x19, 0x16, - 0x0d, 0x87, 0x5c, 0x51, 0xc6, 0x95, 0x66, 0xd4, 0x80, 0xad, 0x0c, 0x69, 0xd2, 0xd3, 0xed, 0x21, - 0x57, 0x8c, 0x09, 0xf5, 0x9c, 0xb3, 0xa3, 0x19, 0xda, 0x27, 0x1c, 0x4b, 0x5d, 0x4c, 0xfa, 0x5d, - 0xac, 0xf5, 0x1c, 0x5e, 0x6c, 0x45, 0xd5, 0x24, 0xa7, 0x6c, 0x3b, 0x04, 0xed, 0x32, 0x55, 0x61, - 0xb2, 0xd0, 0x4c, 0x2e, 0x9c, 0x6c, 0x9c, 0x8d, 0x72, 0x73, 0x72, 0x71, 0xaa, 0xb3, 0x64, 0x94, - 0x50, 0xf8, 0x1d, 0xb8, 0x46, 0xf3, 0x65, 0xee, 0x14, 0x2b, 0xfb, 0xba, 0xdd, 0x0e, 0x94, 0x53, - 0x87, 0x04, 0xcb, 0xb9, 0x83, 0xf4, 0x73, 0xed, 0x68, 0x90, 0x8f, 0x53, 0xee, 0x4b, 0x07, 0xae, - 0x38, 0x05, 0xae, 0x9b, 0x85, 0x1c, 0x64, 0xc7, 0x47, 0x0f, 0x6a, 0x74, 0x95, 0x95, 0xe8, 0xfd, - 0x2e, 0xa6, 0x12, 0xbb, 0x6b, 0x11, 0x0f, 0x7d, 0x08, 0x33, 0x42, 0x42, 0x54, 0x70, 0xbe, 0xe6, - 0x9e, 0x23, 0xe4, 0x94, 0x1b, 0x80, 0x90, 0xc2, 0xdf, 0x49, 0x30, 0x5f, 0x75, 0x9d, 0x5e, 0x0f, - 0x9b, 0xc2, 0x77, 0xf2, 0xa5, 0xfa, 0x2e, 0x53, 0x0a, 0xb9, 0xcc, 0x4d, 0x88, 0xd4, 0xab, 0x22, - 0x37, 0xbe, 0x7d, 0xd9, 0x94, 0xbb, 0x5e, 0x45, 0xef, 0x42, 0x9c, 0x96, 0x86, 0x7d, 0xc2, 0xfc, - 0x49, 0xe6, 0x58, 0x11, 0x34, 0xb2, 0x6d, 0x8c, 0x51, 0x15, 0x1d, 0x0a, 0x3f, 0x4e, 0xc0, 0x95, - 0xb0, 0x8c, 0x56, 0x2b, 0xfe, 0xc4, 0x3f, 0x87, 0x84, 0x65, 0x9b, 0xf8, 0x11, 0x9e, 0xc8, 0x6f, - 0x9c, 0x04, 0x51, 0x14, 0xf2, 0xa8, 0x53, 0x18, 0xbf, 0xec, 0x12, 0x98, 0xe8, 0x93, 0x20, 0x5d, - 0xe3, 0xc7, 0x20, 0xb7, 0x9e, 0x19, 0xbd, 0x3a, 0x96, 0xba, 0x8d, 0x64, 0x46, 0xcc, 0xc1, 0x7e, - 0x4b, 0x99, 0x51, 0x13, 0xe6, 0x2c, 0xdb, 0xc3, 0x6e, 0x07, 0xeb, 0x0f, 0x68, 0xa0, 0xa7, 0xc3, - 0x8b, 0xd3, 0x90, 0x49, 0xc3, 0xaa, 0x12, 0x02, 0xe0, 0xe1, 0xf9, 0x73, 0x98, 0x0f, 0x83, 0xfa, - 0x5b, 0x70, 0xf6, 0x09, 0x09, 0x93, 0xf0, 0x10, 0xd6, 0x3f, 0x88, 0x08, 0x01, 0xd5, 0x85, 0xd8, - 0xef, 0x41, 0x9c, 0x17, 0xbe, 0xe2, 0xb8, 0xf1, 0xf6, 0xb3, 0x8a, 0x9d, 0x17, 0xd4, 0xaa, 0x40, - 0xcb, 0xfd, 0x99, 0x04, 0x33, 0xe1, 0xed, 0x46, 0x16, 0xc8, 0x6c, 0xee, 0xbe, 0x8d, 0x47, 0xcb, - 0x9b, 0x34, 0xb7, 0x63, 0x8d, 0x6c, 0x0f, 0x3e, 0x78, 0xe6, 0x3d, 0xe0, 0x10, 0x42, 0x95, 0xea, - 0x26, 0x4d, 0x18, 0x4c, 0xd7, 0xe9, 0x0d, 0x8f, 0x9b, 0xa3, 0xaa, 0x4c, 0x09, 0x34, 0x07, 0xca, - 0xfd, 0x2e, 0x24, 0x03, 0x45, 0x09, 0x55, 0xbc, 0xd1, 0x6f, 0xb0, 0xe2, 0x3d, 0x73, 0xfc, 0x2a, - 0xa4, 0x47, 0x24, 0x86, 0xae, 0x06, 0x73, 0x88, 0x95, 0xe3, 0x7c, 0x0e, 0xe7, 0xa2, 0x14, 0x7e, - 0x11, 0x87, 0xf9, 0x93, 0x1c, 0xe9, 0xa7, 0xa0, 0x84, 0xfc, 0x96, 0xd6, 0xb1, 0x88, 0x27, 0xec, - 0xe9, 0xc6, 0xd9, 0x05, 0x5a, 0xc8, 0xf9, 0x09, 0x6d, 0xc9, 0xb8, 0xa3, 0x2e, 0xf1, 0xfb, 0x90, - 0x31, 0xf9, 0xc4, 0x35, 0x61, 0xa8, 0xd1, 0x73, 0xeb, 0xaa, 0x13, 0x1c, 0xa0, 0x40, 0x4f, 0x9b, - 0xa1, 0x26, 0xc2, 0xce, 0xe1, 0x7d, 0xf4, 0xe0, 0xac, 0xc3, 0x32, 0x99, 0xf5, 0xa4, 0xcb, 0xcd, - 0xa3, 0x41, 0x7e, 0x4e, 0x60, 0xf9, 0x87, 0x1b, 0x97, 0xde, 0xa9, 0x39, 0x73, 0x0c, 0xd0, 0xa4, - 0x61, 0x88, 0xb6, 0xd3, 0x81, 0xa7, 0x87, 0x61, 0x88, 0xda, 0xd1, 0xe5, 0xc3, 0x10, 0xfd, 0x59, - 0x37, 0xd1, 0x1f, 0x4a, 0x30, 0xc7, 0x4f, 0x3a, 0xbb, 0x7d, 0x4f, 0xe7, 0xc7, 0xd7, 0x7e, 0xbd, - 0xf6, 0xe9, 0xd1, 0x20, 0x3f, 0xcb, 0x04, 0xb2, 0x21, 0xda, 0xd8, 0xb0, 0xe5, 0x67, 0x1d, 0x76, - 0x88, 0x22, 0x6a, 0x98, 0x80, 0x60, 0xa2, 0x75, 0xc8, 0xf0, 0xe2, 0x53, 0xa3, 0xe5, 0x97, 0xe5, - 0xd8, 0xac, 0x90, 0x4b, 0x97, 0x5f, 0x7a, 0x3a, 0xc8, 0x2f, 0x9d, 0xa0, 0x59, 0xbc, 0x6e, 0xbd, - 0xc7, 0x79, 0xd5, 0xf4, 0x5e, 0xf8, 0x13, 0x19, 0x90, 0x0e, 0x54, 0xe3, 0xb0, 0x27, 0xea, 0xbe, - 0xcb, 0x87, 0xb2, 0x19, 0x5f, 0x47, 0x28, 0x26, 0x6a, 0xc3, 0xac, 0x3f, 0x08, 0x2f, 0xb8, 0x48, - 0x36, 0xf9, 0x8d, 0x0c, 0xe3, 0xab, 0x35, 0x5f, 0x35, 0x11, 0x47, 0x10, 0x57, 0x61, 0xe1, 0xc4, - 0x5c, 0xe1, 0xab, 0x38, 0x5c, 0x1d, 0x75, 0x84, 0x41, 0x5a, 0xa5, 0x8d, 0x47, 0xc8, 0x0f, 0x26, - 0x76, 0xa6, 0x3e, 0x06, 0x77, 0x66, 0xfe, 0xd7, 0x78, 0x8c, 0xfc, 0x7c, 0x2c, 0x46, 0x3e, 0x03, - 0x3e, 0x53, 0xaf, 0x31, 0x7c, 0x3f, 0x50, 0x7e, 0x12, 0xc4, 0x02, 0x7e, 0x8a, 0xf1, 0xe1, 0x33, - 0xc0, 0xb3, 0xfe, 0xfe, 0x67, 0x10, 0x0d, 0x7e, 0x26, 0x41, 0x7a, 0x64, 0x65, 0xbf, 0xcc, 0x70, - 0xb0, 0x1d, 0x64, 0x43, 0xfc, 0xb9, 0xc1, 0x3b, 0x17, 0x5f, 0xd6, 0x68, 0x92, 0x94, 0xfb, 0x07, - 0x09, 0xd2, 0x23, 0x82, 0xfc, 0x96, 0x02, 0xc9, 0x37, 0x3f, 0xf3, 0x16, 0x64, 0x46, 0xb7, 0x28, - 0x34, 0x86, 0xf4, 0xcd, 0x8c, 0x51, 0xf8, 0x1e, 0xc4, 0x39, 0x05, 0x21, 0xc8, 0x7c, 0x5c, 0xaa, - 0xef, 0xd4, 0x37, 0x57, 0xb5, 0x3b, 0x5b, 0xaa, 0xb6, 0x5a, 0x51, 0xa6, 0xd0, 0x0c, 0xc8, 0xd5, - 0xda, 0xdd, 0x1a, 0x25, 0x2a, 0x12, 0x4a, 0x41, 0x82, 0x7d, 0xd5, 0xaa, 0x4a, 0xa4, 0x50, 0x06, - 0x85, 0x63, 0xef, 0x61, 0x1a, 0x18, 0xdc, 0x36, 0xf6, 0x50, 0x11, 0xe6, 0x29, 0x2c, 0xee, 0xd2, - 0xfc, 0x8d, 0x86, 0x42, 0x2d, 0x94, 0x3d, 0xcf, 0x05, 0x4d, 0x34, 0x28, 0x6e, 0xea, 0x5d, 0x5c, - 0xf8, 0xfb, 0x18, 0xcc, 0x0d, 0x41, 0xfc, 0xb0, 0xf8, 0x0a, 0xc8, 0xc4, 0xb2, 0x0f, 0xb4, 0xe1, - 0x25, 0x32, 0x3f, 0x58, 0xb2, 0xec, 0x83, 0x5d, 0xb5, 0xae, 0x26, 0x68, 0xe3, 0xae, 0x6b, 0xa1, - 0x06, 0xc4, 0x9c, 0x9e, 0xe7, 0xd7, 0x54, 0x6f, 0x9f, 0x21, 0x8a, 0x63, 0x63, 0x14, 0xb7, 0x7a, - 0x9e, 0xa8, 0x88, 0x19, 0x06, 0xfa, 0x1b, 0x09, 0x12, 0xfc, 0x14, 0xc3, 0xaf, 0x9e, 0xde, 0xbd, - 0x10, 0x1e, 0x17, 0x80, 0xb8, 0xd1, 0xfb, 0x98, 0x1a, 0xea, 0xd3, 0x41, 0x7e, 0x6e, 0x5c, 0x40, - 0xe4, 0x92, 0x57, 0x7d, 0xfe, 0x14, 0x51, 0x83, 0x5f, 0x32, 0x0d, 0x05, 0xcd, 0x02, 0xc2, 0x84, - 0xf7, 0x78, 0xe9, 0x91, 0x8d, 0xc8, 0xb5, 0x61, 0x26, 0x3c, 0xfb, 0x13, 0x4e, 0x8f, 0x4b, 0xa3, - 0x65, 0xfa, 0x6b, 0x13, 0x49, 0x86, 0x63, 0x86, 0x8f, 0xac, 0xbf, 0x07, 0xc9, 0x40, 0xec, 0x17, - 0x39, 0xeb, 0xe6, 0x3e, 0x3e, 0x38, 0x3c, 0x9a, 0x56, 0xe2, 0x85, 0xbf, 0x95, 0x60, 0x46, 0xc5, - 0xc4, 0xe9, 0x3c, 0xc0, 0x26, 0xcd, 0x79, 0x82, 0xb7, 0x19, 0xd2, 0xe4, 0x6f, 0x33, 0x4a, 0x90, - 0x0c, 0x4e, 0xf7, 0x2e, 0xf2, 0xce, 0x61, 0xd8, 0x0b, 0xdd, 0x00, 0xa5, 0xe5, 0xf4, 0x6d, 0x53, - 0x77, 0x0f, 0x35, 0x17, 0xeb, 0xc6, 0x3e, 0x36, 0xc5, 0x4d, 0xc6, 0xac, 0x4f, 0x57, 0x39, 0xb9, - 0xf0, 0xa3, 0x08, 0xa0, 0xa1, 0x70, 0x42, 0xae, 0x88, 0x66, 0x6e, 0x6c, 0x1d, 0xa2, 0x78, 0x8d, - 0x9c, 0x78, 0xf7, 0x34, 0x96, 0x00, 0x06, 0x0b, 0xf7, 0xb7, 0xd4, 0x0d, 0xd1, 0x08, 0xfa, 0xd3, - 0xb3, 0xcf, 0x30, 0xa3, 0xec, 0x0c, 0x93, 0x69, 0xe9, 0x2f, 0xf5, 0x1c, 0x53, 0x04, 0xeb, 0xff, - 0x8e, 0x01, 0xaa, 0xb8, 0x58, 0xf7, 0x30, 0xf5, 0x3c, 0xe4, 0xac, 0x5a, 0xbb, 0x0c, 0xd3, 0xbc, - 0x30, 0x8b, 0x5c, 0xa4, 0x30, 0x13, 0x42, 0xe1, 0x5d, 0xd1, 0x6f, 0xc0, 0x8c, 0xe1, 0x74, 0xfa, - 0x5d, 0x5b, 0x63, 0x37, 0xb0, 0x22, 0x11, 0xfe, 0xee, 0x59, 0x4a, 0x7c, 0x6c, 0x72, 0xc5, 0x8a, - 0xd3, 0xa1, 0xdf, 0xfe, 0xd9, 0x01, 0x07, 0x64, 0x1c, 0xe8, 0x79, 0x48, 0x06, 0x06, 0xc5, 0x52, - 0xe0, 0xa4, 0x3a, 0x24, 0xa0, 0x15, 0x98, 0xd6, 0x89, 0xe6, 0xec, 0xb1, 0x1c, 0xf5, 0x3c, 0x0d, - 0x53, 0x63, 0x3a, 0xd9, 0xda, 0x43, 0x6f, 0x41, 0x7a, 0xef, 0x3e, 0xcf, 0xdb, 0xb9, 0x03, 0xe5, - 0x17, 0xe3, 0xb3, 0x47, 0x83, 0x7c, 0xea, 0xce, 0x47, 0x6c, 0xb1, 0xd4, 0x7d, 0xaa, 0xa9, 0xbd, - 0xfb, 0xc1, 0x07, 0xba, 0x09, 0x73, 0x5d, 0xfd, 0x91, 0xb6, 0xe7, 0xea, 0x86, 0x48, 0x54, 0x3b, - 0xdc, 0x2b, 0x48, 0xea, 0x6c, 0x57, 0x7f, 0x74, 0x47, 0xd0, 0xeb, 0x66, 0x07, 0xe7, 0xfe, 0x4b, - 0x82, 0x84, 0x58, 0x11, 0xea, 0x01, 0x08, 0xf1, 0x58, 0x26, 0x4f, 0x85, 0xd2, 0xe5, 0x8f, 0x8e, - 0x06, 0xf9, 0x64, 0x85, 0x51, 0xeb, 0x55, 0xf2, 0x74, 0x90, 0xff, 0xf0, 0x59, 0x9d, 0x96, 0x0f, - 0xa2, 0x26, 0xf9, 0x20, 0x75, 0x93, 0x1d, 0xb0, 0xed, 0xeb, 0x44, 0xdb, 0xb7, 0x88, 0xe7, 0xb4, - 0x5d, 0xbd, 0xcb, 0x36, 0x57, 0x56, 0x67, 0xf6, 0x75, 0xb2, 0xe6, 0xd3, 0x50, 0x8e, 0xa6, 0x1c, - 0x0f, 0xf8, 0x05, 0x3a, 0x37, 0xa9, 0xe0, 0x1b, 0xad, 0xc0, 0x95, 0xa0, 0xb3, 0x46, 0x17, 0xdd, - 0xea, 0x1b, 0x07, 0x98, 0x45, 0x02, 0xea, 0xb3, 0xe6, 0x83, 0xc6, 0x0d, 0xfd, 0x51, 0x99, 0x37, - 0x15, 0xae, 0xc0, 0x7c, 0x68, 0x5b, 0x83, 0x04, 0xf1, 0x67, 0x29, 0x48, 0x6c, 0xeb, 0x87, 0x1d, - 0x47, 0x37, 0xd1, 0x12, 0xa4, 0xfc, 0x8b, 0x6f, 0x9a, 0x62, 0x73, 0x3d, 0x0c, 0x93, 0x90, 0x05, - 0x99, 0x3e, 0xc1, 0x2e, 0xdd, 0x13, 0x8d, 0x3d, 0xe5, 0xe4, 0xbe, 0xaa, 0x5c, 0x7e, 0x3a, 0xc8, - 0xdf, 0x9e, 0x4c, 0x44, 0xd8, 0xe8, 0xbb, 0x96, 0x77, 0x58, 0x6c, 0x7e, 0x74, 0x77, 0x57, 0x40, - 0x51, 0x43, 0x72, 0xd4, 0x74, 0x3f, 0xfc, 0x29, 0x9e, 0x11, 0xd0, 0xe5, 0x6a, 0x5d, 0xcb, 0x70, - 0x1d, 0xe2, 0x9f, 0x36, 0x0b, 0xea, 0x06, 0x23, 0xa2, 0x57, 0x61, 0x76, 0xcf, 0xb2, 0xd9, 0x4d, - 0x87, 0xcf, 0xc7, 0x0f, 0x9a, 0x33, 0x3e, 0x59, 0x30, 0x3e, 0x80, 0x4c, 0xe8, 0xe9, 0x00, 0xdd, - 0xea, 0x38, 0xdb, 0xea, 0xad, 0xa3, 0x41, 0x3e, 0x3d, 0x34, 0x1d, 0xbe, 0xdd, 0x97, 0x89, 0x51, - 0xe9, 0xe1, 0x30, 0x74, 0xb3, 0x17, 0x60, 0x9a, 0xbd, 0xcf, 0xe5, 0xaf, 0x85, 0x54, 0xfe, 0x81, - 0xde, 0x86, 0xe9, 0x0e, 0xd6, 0x09, 0x16, 0x0f, 0x81, 0x96, 0xce, 0x30, 0x46, 0xf6, 0x52, 0x56, - 0xe5, 0xec, 0xa8, 0x0c, 0x71, 0x7e, 0x77, 0xc5, 0x6e, 0x9c, 0x52, 0x2b, 0xd7, 0x27, 0x7d, 0xf3, - 0xb5, 0x36, 0xa5, 0x8a, 0x9e, 0xa8, 0x06, 0x09, 0x97, 0x5f, 0x57, 0xb2, 0x7b, 0xa8, 0x73, 0x8b, - 0xed, 0xd0, 0x6d, 0xe8, 0xda, 0x94, 0xea, 0xf7, 0x45, 0x3b, 0xfe, 0x9b, 0x01, 0xee, 0xd5, 0xc5, - 0x93, 0x88, 0xe2, 0x84, 0x09, 0xd9, 0x10, 0x70, 0x04, 0x85, 0x2e, 0xd0, 0x62, 0x67, 0xd7, 0xec, - 0x3a, 0xeb, 0xec, 0x05, 0x8e, 0xdc, 0x83, 0xd2, 0x05, 0xf2, 0x9e, 0x68, 0x13, 0xc0, 0x08, 0x22, - 0x0d, 0xbb, 0xe8, 0x4a, 0xad, 0xbc, 0x7e, 0x91, 0x6c, 0x66, 0x6d, 0x4a, 0x0d, 0x21, 0xa0, 0x8f, - 0x20, 0x65, 0x0c, 0x4d, 0x27, 0x3b, 0xcb, 0x00, 0xdf, 0xb8, 0x90, 0xff, 0x5c, 0xa3, 0x3e, 0x73, - 0x48, 0x1d, 0xf5, 0x99, 0xca, 0xb8, 0xcf, 0xac, 0x41, 0x5a, 0x9c, 0x8b, 0xf0, 0xa7, 0xdd, 0xd9, - 0x39, 0xe6, 0xb2, 0xc3, 0x5a, 0xe2, 0x3f, 0xfe, 0x2e, 0xd6, 0x6c, 0xc3, 0x31, 0xb1, 0x59, 0xa3, - 0xdf, 0xaa, 0x38, 0x06, 0x66, 0x1f, 0x04, 0xad, 0x42, 0xc6, 0xe8, 0x60, 0xdd, 0xee, 0xf7, 0x7c, - 0x1c, 0x34, 0x21, 0x4e, 0x5a, 0xf4, 0x13, 0x40, 0x9b, 0x80, 0xf6, 0xd8, 0x3b, 0x83, 0xf0, 0xac, - 0xd8, 0x7d, 0xd9, 0x24, 0x60, 0x0a, 0xeb, 0xab, 0x0e, 0x67, 0x86, 0x5e, 0x82, 0xb4, 0xed, 0xd8, - 0x86, 0x6e, 0x1b, 0xb8, 0xc3, 0xa2, 0x1b, 0xbf, 0x62, 0x1b, 0x25, 0xa2, 0xcf, 0x20, 0x43, 0x46, - 0x52, 0xf8, 0xec, 0x15, 0x36, 0xe2, 0x9b, 0x17, 0x3d, 0xf4, 0x5b, 0x9b, 0x52, 0xc7, 0x90, 0xd0, - 0x6f, 0x82, 0xe2, 0x8d, 0x9d, 0xaf, 0xb3, 0xcb, 0xba, 0xb3, 0xdf, 0xf4, 0x9c, 0x72, 0x21, 0xb0, - 0x36, 0xa5, 0x1e, 0x43, 0x43, 0x9f, 0xc3, 0x2c, 0x19, 0x7d, 0xb5, 0x9a, 0xbd, 0xc6, 0x06, 0xf8, - 0xce, 0x99, 0xc7, 0xdb, 0x27, 0x3d, 0xf4, 0x5d, 0x9b, 0x52, 0xc7, 0xb1, 0xca, 0x49, 0x48, 0x88, - 0xcb, 0x8e, 0xe0, 0x0a, 0x31, 0xa1, 0xc8, 0x85, 0x7f, 0x4e, 0x80, 0x1c, 0xe4, 0x56, 0xcb, 0x80, - 0x82, 0x68, 0x38, 0x7c, 0xb7, 0x45, 0xdd, 0x7a, 0x64, 0x6d, 0x4a, 0x9d, 0xf3, 0xdb, 0x86, 0x4f, - 0xb7, 0x5e, 0x85, 0xd9, 0xae, 0x63, 0x5a, 0x7b, 0xd6, 0xd0, 0x99, 0xf2, 0xa3, 0xbc, 0x8c, 0x4f, - 0x16, 0xce, 0xf4, 0xf6, 0xc8, 0xf3, 0x86, 0x49, 0x9e, 0xf5, 0xae, 0x4d, 0x85, 0xde, 0x3f, 0x50, - 0xe7, 0xee, 0xf6, 0x6d, 0xdb, 0xb2, 0xdb, 0x9a, 0x28, 0xe7, 0x78, 0xde, 0x90, 0x16, 0x54, 0x51, - 0x91, 0x55, 0xc6, 0xbc, 0xdd, 0x8d, 0x73, 0xbd, 0x9d, 0xbf, 0xf6, 0x35, 0x29, 0x70, 0x77, 0x77, - 0xc6, 0xdd, 0xdd, 0xcd, 0xf3, 0xdd, 0x5d, 0x08, 0x26, 0xf0, 0x77, 0xbb, 0x27, 0xfa, 0xbb, 0xe5, - 0x09, 0x95, 0x31, 0x84, 0x38, 0xea, 0xf0, 0x2a, 0x63, 0x0e, 0xef, 0xc6, 0xb9, 0x0e, 0x2f, 0xbc, - 0x46, 0xe1, 0xf1, 0xb6, 0x4e, 0xf0, 0x78, 0x6f, 0x4c, 0xe4, 0xf1, 0x42, 0x60, 0x61, 0x97, 0xa7, - 0x9e, 0xe4, 0xf2, 0x8a, 0x93, 0xb9, 0xbc, 0x10, 0xe4, 0x88, 0xcf, 0xfb, 0xfe, 0x31, 0x7b, 0x56, - 0xce, 0x37, 0x88, 0x13, 0x6b, 0xf8, 0x35, 0xe9, 0x98, 0x41, 0xeb, 0x27, 0x18, 0xf4, 0x1c, 0x83, - 0x7f, 0xeb, 0x02, 0x06, 0x1d, 0x1a, 0xe0, 0xb8, 0x45, 0x7f, 0x02, 0x33, 0x61, 0x2b, 0x64, 0xcf, - 0x08, 0xce, 0xf6, 0x17, 0xa7, 0x3c, 0x5b, 0x67, 0x3a, 0x10, 0x6a, 0x2a, 0x03, 0xc8, 0xfe, 0x05, - 0x69, 0xc8, 0xb0, 0x0b, 0x7f, 0x22, 0x41, 0xb4, 0xe1, 0xb4, 0x50, 0x66, 0x78, 0x5c, 0xc3, 0x0e, - 0x5a, 0x3e, 0x18, 0xb2, 0x8b, 0xba, 0xe0, 0xc5, 0x33, 0x26, 0x11, 0x1c, 0x6f, 0x05, 0x9d, 0xd0, - 0x7b, 0x90, 0xe8, 0xf1, 0x9c, 0x4f, 0xd8, 0x6e, 0xe1, 0xac, 0xfe, 0x9c, 0x53, 0xf5, 0xbb, 0xdc, - 0xbc, 0x11, 0xfe, 0x3f, 0x21, 0x1b, 0x8e, 0x89, 0x51, 0x06, 0x60, 0x5b, 0x27, 0xa4, 0xb7, 0xef, - 0xea, 0x04, 0x2b, 0x53, 0x28, 0x01, 0xd1, 0xf5, 0x8d, 0xa6, 0x22, 0xdd, 0xfc, 0x24, 0x7c, 0xb8, - 0x52, 0x55, 0x4b, 0xf5, 0xcd, 0xfa, 0xe6, 0xaa, 0xb6, 0x59, 0xda, 0xa8, 0x35, 0x95, 0x29, 0x94, - 0x85, 0x85, 0x8f, 0x4b, 0xf5, 0x1d, 0x71, 0xda, 0xa2, 0xd5, 0x37, 0x77, 0x6a, 0xea, 0xbd, 0xd2, - 0x5d, 0x45, 0x42, 0x57, 0x01, 0xa9, 0x5b, 0x95, 0xf5, 0x66, 0xb5, 0xac, 0x55, 0xb6, 0x36, 0xb6, - 0x4b, 0x95, 0x9d, 0xfa, 0xd6, 0xa6, 0x12, 0x41, 0x32, 0xc4, 0xaa, 0x5b, 0x9b, 0x35, 0x05, 0x6e, - 0xfe, 0x34, 0x0a, 0x31, 0xba, 0x7b, 0xe8, 0x25, 0x48, 0xed, 0x6e, 0x36, 0xb7, 0x6b, 0x95, 0xfa, - 0x9d, 0x7a, 0xad, 0xaa, 0x4c, 0xe5, 0xe6, 0x1f, 0x3f, 0x59, 0x9a, 0xa5, 0x4d, 0xbb, 0x36, 0xe9, - 0x61, 0x83, 0xb9, 0x2d, 0x94, 0x83, 0x78, 0xb9, 0x54, 0x59, 0xdf, 0xdd, 0x56, 0xa4, 0x5c, 0xe6, - 0xf1, 0x93, 0x25, 0xa0, 0x0c, 0xdc, 0x65, 0xa0, 0xe7, 0x21, 0xa1, 0xd6, 0x9a, 0x3b, 0x5b, 0x6a, - 0x4d, 0x89, 0xe4, 0x66, 0x1f, 0x3f, 0x59, 0x4a, 0xd1, 0x46, 0xe1, 0x09, 0xd0, 0xab, 0x90, 0x6e, - 0x56, 0xd6, 0x6a, 0x1b, 0x25, 0xad, 0xb2, 0x56, 0xda, 0x5c, 0xad, 0x29, 0xd1, 0xdc, 0xc2, 0xe3, - 0x27, 0x4b, 0xca, 0xb8, 0xea, 0xd0, 0x21, 0xea, 0x1b, 0xdb, 0x5b, 0xea, 0x8e, 0x12, 0x1b, 0x0e, - 0xc1, 0x2d, 0x16, 0x15, 0x00, 0x78, 0xef, 0x3b, 0xb5, 0x5a, 0x55, 0x99, 0xce, 0xa1, 0xc7, 0x4f, - 0x96, 0x32, 0xb4, 0x7d, 0x68, 0x88, 0xe8, 0x65, 0x98, 0xa9, 0xa8, 0xb5, 0xd2, 0x4e, 0x4d, 0x6b, - 0xee, 0x94, 0x76, 0x9a, 0x4a, 0x7c, 0xb8, 0x92, 0x90, 0x71, 0xa1, 0x22, 0xcc, 0x95, 0x76, 0x77, - 0xb6, 0xb4, 0x11, 0xde, 0x44, 0xee, 0xda, 0xe3, 0x27, 0x4b, 0xf3, 0x94, 0xb7, 0xd4, 0xf7, 0x9c, - 0x30, 0xff, 0xeb, 0xa0, 0x8c, 0xcc, 0x5f, 0x5b, 0xad, 0x28, 0x72, 0xee, 0xea, 0xe3, 0x27, 0x4b, - 0x68, 0x7c, 0x09, 0xab, 0x15, 0xf4, 0x6b, 0x70, 0x75, 0xe7, 0xd3, 0xed, 0x5a, 0xb5, 0xd6, 0xac, - 0x68, 0xa3, 0xcb, 0x4e, 0xe6, 0xb2, 0x8f, 0x9f, 0x2c, 0x2d, 0xd0, 0x3e, 0xc7, 0x96, 0xfe, 0x06, - 0x28, 0xcd, 0x1d, 0xb5, 0x56, 0xda, 0xd0, 0xea, 0x9b, 0xab, 0xb5, 0x26, 0xdb, 0x2c, 0x18, 0x4e, - 0x69, 0xcc, 0x0c, 0x72, 0xf2, 0x8f, 0xfe, 0x6a, 0x71, 0xea, 0xaf, 0x7f, 0xbc, 0x38, 0x55, 0xbe, - 0xfe, 0xe5, 0x7f, 0x2c, 0x4e, 0x7d, 0x79, 0xb4, 0x28, 0xfd, 0xfc, 0x68, 0x51, 0xfa, 0xea, 0x68, - 0x51, 0xfa, 0xf7, 0xa3, 0x45, 0xe9, 0x8f, 0xbe, 0x5e, 0x9c, 0xfa, 0xf9, 0xd7, 0x8b, 0x53, 0x5f, - 0x7d, 0xbd, 0x38, 0xf5, 0x59, 0x9c, 0x6b, 0x61, 0x2b, 0xce, 0xca, 0x8d, 0xb7, 0xfe, 0x2f, 0x00, - 0x00, 0xff, 0xff, 0x4e, 0x91, 0x4d, 0x7c, 0x29, 0x37, 0x00, 0x00, +func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_8f6d1b2aa13f3feb) } + +var fileDescriptor_jobs_8f6d1b2aa13f3feb = []byte{ + // 4692 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x7b, 0x4b, 0x6c, 0x23, 0x47, + 0x7a, 0xbf, 0x9a, 0xa4, 0xc8, 0xe6, 0x47, 0x91, 0x6a, 0x95, 0x34, 0x33, 0x5c, 0xfe, 0x6d, 0x51, + 0x4b, 0xbf, 0x66, 0xc6, 0x36, 0xe5, 0x95, 0xff, 0xeb, 0xb5, 0x27, 0xf6, 0xd8, 0x7c, 0x8d, 0x44, + 0x6a, 0xf4, 0x70, 0x53, 0x1a, 0x3f, 0x36, 0xde, 0x4e, 0xb3, 0xbb, 0x24, 0x75, 0x44, 0x76, 0x73, + 0xba, 0x9a, 0x33, 0xa3, 0x4d, 0x90, 0x04, 0x1b, 0x04, 0x58, 0xcc, 0x29, 0x01, 0x92, 0x5c, 0x92, + 0x01, 0x02, 0x64, 0x17, 0x08, 0x92, 0x00, 0x01, 0x8c, 0x20, 0xc9, 0x21, 0xb7, 0x5c, 0x7c, 0x48, + 0x80, 0xbd, 0x04, 0x30, 0x72, 0xe0, 0x26, 0xf2, 0x25, 0xc7, 0x20, 0x7b, 0x9b, 0x4b, 0x82, 0x7a, + 0x74, 0xb3, 0x49, 0xea, 0x41, 0x8d, 0xec, 0xcd, 0x45, 0xc3, 0xfe, 0xea, 0xab, 0x5f, 0xbd, 0xbe, + 0xfa, 0x7d, 0x5f, 0x7d, 0x55, 0x03, 0x57, 0x7f, 0xdd, 0x69, 0x91, 0x65, 0xfa, 0xa7, 0xdb, 0x62, + 0xff, 0x14, 0xbb, 0xae, 0xe3, 0x39, 0xe8, 0x5b, 0x86, 0x63, 0x1c, 0xba, 0x8e, 0x6e, 0x1c, 0x14, + 0xc9, 0xfd, 0x76, 0x91, 0x95, 0x70, 0xad, 0xdc, 0x15, 0xec, 0xba, 0x8e, 0x4b, 0xf5, 0xf9, 0x0f, + 0x5e, 0x23, 0xb7, 0xb0, 0xef, 0xec, 0x3b, 0xec, 0xe7, 0x32, 0xfd, 0x25, 0xa4, 0x88, 0x61, 0x74, + 0x5b, 0xcb, 0xa6, 0xee, 0xe9, 0x42, 0x96, 0xf5, 0x65, 0x96, 0xf3, 0xfa, 0x9e, 0xe3, 0x76, 0x74, + 0xcf, 0xc7, 0x78, 0x81, 0xdc, 0x6f, 0x2f, 0x1b, 0xba, 0xa7, 0xb7, 0x9d, 0xfd, 0x65, 0x13, 0x13, + 0xa3, 0xdb, 0x5a, 0x26, 0x9e, 0xdb, 0x33, 0xbc, 0x9e, 0x8b, 0x4d, 0xa1, 0x94, 0x3f, 0x41, 0xc9, + 0xc3, 0xb6, 0x6e, 0x7b, 0x3e, 0x7e, 0xcf, 0xb3, 0xda, 0xcb, 0x07, 0x6d, 0x63, 0xd9, 0xb3, 0x3a, + 0x98, 0x78, 0x7a, 0xa7, 0x2b, 0x4a, 0xbe, 0x4d, 0xab, 0x12, 0xe3, 0x00, 0x77, 0x74, 0xe3, 0x40, + 0xb7, 0xf7, 0xb1, 0xbb, 0xcc, 0xdb, 0x30, 0xba, 0x2d, 0xae, 0x52, 0xf8, 0x6d, 0x98, 0xbe, 0x8b, + 0x75, 0x82, 0xd1, 0xa7, 0x90, 0xb0, 0x1d, 0x13, 0x6b, 0x96, 0x99, 0x95, 0x96, 0xa4, 0xeb, 0xe9, + 0x72, 0xe9, 0xb8, 0x9f, 0x8f, 0x6f, 0x3a, 0x26, 0xae, 0x57, 0x9f, 0xf6, 0xf3, 0x6f, 0xee, 0x5b, + 0xde, 0x41, 0xaf, 0x55, 0x34, 0x9c, 0xce, 0x72, 0x30, 0x57, 0x66, 0x6b, 0xf0, 0x7b, 0xb9, 0x7b, + 0xb8, 0xbf, 0x2c, 0x46, 0x5a, 0xe4, 0xd5, 0xd4, 0x38, 0x45, 0xac, 0x9b, 0x68, 0x01, 0xa6, 0x71, + 0xd7, 0x31, 0x0e, 0xb2, 0x91, 0x25, 0xe9, 0x7a, 0x54, 0xe5, 0x1f, 0xb7, 0x62, 0xff, 0xf9, 0x67, + 0x79, 0xa9, 0xf0, 0xd3, 0x08, 0x5c, 0x2b, 0xeb, 0xc6, 0x61, 0xaf, 0x5b, 0xb3, 0x0d, 0xf7, 0xa8, + 0xeb, 0x59, 0x8e, 0xbd, 0xc5, 0xfe, 0x12, 0xa4, 0x40, 0xf4, 0x10, 0x1f, 0xb1, 0xfe, 0xcc, 0xa8, + 0xf4, 0x27, 0x7a, 0x0f, 0x62, 0x1d, 0xc7, 0xc4, 0x0c, 0x28, 0xb3, 0x72, 0xa3, 0x78, 0xea, 0xb2, + 0x15, 0x07, 0x68, 0x1b, 0x8e, 0x89, 0x55, 0x56, 0x0d, 0xb5, 0x40, 0x3e, 0xec, 0x10, 0xcd, 0xb2, + 0xf7, 0x9c, 0x6c, 0x74, 0x49, 0xba, 0x9e, 0x5a, 0xb9, 0x75, 0x06, 0xc4, 0x29, 0xdd, 0x2a, 0xae, + 0x6f, 0x34, 0xeb, 0xf6, 0x9e, 0x53, 0x4e, 0x1d, 0xf7, 0xf3, 0x09, 0xf1, 0xa1, 0x26, 0x0e, 0x3b, + 0x84, 0xfe, 0xc8, 0x6d, 0x81, 0x2f, 0xa3, 0xfd, 0xef, 0xb9, 0x16, 0xeb, 0x7f, 0x52, 0xa5, 0x3f, + 0xd1, 0x6b, 0x80, 0x30, 0xc7, 0xc3, 0xa6, 0x46, 0x6d, 0x44, 0xa3, 0x03, 0x8c, 0xb0, 0x01, 0x2a, + 0x41, 0x49, 0x55, 0xf7, 0xf4, 0x75, 0x7c, 0xc4, 0x67, 0x48, 0xcc, 0xd3, 0xef, 0x44, 0x21, 0x33, + 0xe8, 0x0a, 0x83, 0x5f, 0x83, 0x38, 0x5b, 0x5c, 0xcc, 0x5a, 0xc8, 0xac, 0xbc, 0x31, 0xd1, 0x74, + 0xd0, 0xaa, 0xc5, 0x26, 0xab, 0xa7, 0x8a, 0xfa, 0x08, 0x41, 0x8c, 0xe8, 0x6d, 0x4f, 0x74, 0x84, + 0xfd, 0x46, 0x7f, 0x22, 0xc1, 0xd2, 0x68, 0x8f, 0xca, 0x47, 0xeb, 0x1b, 0xcd, 0x0d, 0x9d, 0x78, + 0xd8, 0x5d, 0xc7, 0x47, 0xf5, 0x6a, 0x36, 0xba, 0x14, 0xbd, 0x9e, 0x5a, 0xd9, 0x9a, 0xbc, 0xe1, + 0xda, 0x39, 0x88, 0x35, 0xdb, 0x73, 0x8f, 0xd4, 0x73, 0x1b, 0xce, 0x35, 0xe1, 0xa5, 0x89, 0xa0, + 0xc2, 0x36, 0x94, 0xe4, 0x36, 0xb4, 0x00, 0xd3, 0x0f, 0xf4, 0x76, 0x0f, 0x8b, 0xd1, 0xf2, 0x8f, + 0x5b, 0x91, 0xb7, 0xa5, 0xc2, 0x35, 0x88, 0xf3, 0x89, 0x41, 0x69, 0x48, 0x96, 0x6a, 0xcd, 0x95, + 0xef, 0xbe, 0xb5, 0x5a, 0xd9, 0x50, 0xa6, 0xc4, 0x12, 0xfc, 0x8f, 0x04, 0x57, 0x9b, 0x9e, 0x8b, + 0xf5, 0x4e, 0xdd, 0xde, 0xc7, 0x84, 0x8e, 0xa9, 0x8a, 0x3d, 0xdd, 0x6a, 0x13, 0x64, 0x43, 0x86, + 0xb0, 0x12, 0x4d, 0x37, 0x4d, 0x17, 0x13, 0xc2, 0x1b, 0x2c, 0xaf, 0x3e, 0xed, 0xe7, 0x2b, 0x13, + 0x6d, 0x1d, 0xc3, 0x68, 0x2f, 0x73, 0x08, 0xcb, 0xde, 0x37, 0x8c, 0x76, 0x91, 0xb7, 0x54, 0xe2, + 0x70, 0x6a, 0x9a, 0x84, 0x3f, 0xd1, 0x77, 0x20, 0x46, 0xba, 0xba, 0xcd, 0x86, 0x90, 0x5a, 0xb9, + 0x16, 0x9a, 0x7f, 0x7f, 0x0b, 0x36, 0xbb, 0xba, 0x5d, 0x8e, 0x7d, 0xd1, 0xcf, 0x4f, 0xa9, 0x4c, + 0x15, 0x95, 0x01, 0x88, 0xa7, 0xbb, 0x9e, 0x46, 0x59, 0x42, 0x58, 0xff, 0xf3, 0xa1, 0x8a, 0x94, + 0x45, 0x8a, 0x07, 0x6d, 0xa3, 0xb8, 0xe3, 0xb3, 0x88, 0xa8, 0x9e, 0x64, 0xd5, 0xa8, 0xb4, 0xf0, + 0x2d, 0xb8, 0x36, 0x32, 0x01, 0xdb, 0xae, 0xb3, 0x4f, 0x7b, 0x54, 0xf8, 0xc7, 0x38, 0xa4, 0xf9, + 0x86, 0xf1, 0xe7, 0x64, 0xb8, 0x41, 0xe9, 0x59, 0x1a, 0x44, 0xb7, 0x41, 0xc6, 0xb6, 0xc9, 0x11, + 0x22, 0x93, 0x23, 0x24, 0xb0, 0x6d, 0xb2, 0xfa, 0xdf, 0xe2, 0x3b, 0x30, 0xca, 0x16, 0x23, 0x71, + 0xdc, 0xcf, 0x47, 0x77, 0xd5, 0x3a, 0xdf, 0x8a, 0xef, 0x42, 0xce, 0xc4, 0x5d, 0x17, 0x1b, 0x3a, + 0xdd, 0x8b, 0x2d, 0xd6, 0x75, 0xad, 0xa3, 0xdb, 0xd6, 0x1e, 0x26, 0x5e, 0x36, 0xc6, 0x6c, 0x23, + 0x3b, 0xd0, 0xe0, 0x63, 0xdb, 0x10, 0xe5, 0xe8, 0x77, 0x25, 0x98, 0xef, 0xb9, 0x16, 0xd1, 0x5a, + 0x47, 0x5a, 0xdb, 0x31, 0xf4, 0xb6, 0xe5, 0x1d, 0x69, 0x87, 0x0f, 0xb2, 0xd3, 0x6c, 0x43, 0xdc, + 0x3e, 0x97, 0x55, 0xc4, 0x24, 0x15, 0x77, 0x5d, 0x8b, 0x94, 0x8f, 0xee, 0x0a, 0x84, 0xf5, 0x07, + 0xcc, 0x68, 0xcb, 0x0b, 0xc7, 0xfd, 0xbc, 0xb2, 0xab, 0xd6, 0xc3, 0x45, 0xf7, 0x54, 0xa5, 0x37, + 0xa2, 0x8c, 0xf4, 0x80, 0x4e, 0x2c, 0xc7, 0xd6, 0x1c, 0xce, 0x4f, 0xd9, 0x38, 0x9b, 0xa8, 0x95, + 0x8b, 0x33, 0x9b, 0x3a, 0x87, 0xc7, 0x38, 0xf8, 0x0f, 0x24, 0xc8, 0x51, 0x57, 0x81, 0x0d, 0x3a, + 0x4d, 0x81, 0x87, 0xd1, 0x5c, 0x6c, 0x38, 0xae, 0x99, 0x4d, 0xd0, 0x79, 0x2a, 0x37, 0xff, 0x6d, + 0x52, 0x0f, 0xc1, 0x7c, 0x55, 0xaf, 0x67, 0x99, 0xc5, 0xdd, 0xdd, 0x7a, 0xf5, 0xb8, 0x9f, 0xcf, + 0x6e, 0xfb, 0xe0, 0xc1, 0x22, 0xaa, 0x0c, 0x5a, 0xcd, 0x76, 0x4f, 0x29, 0x41, 0x6f, 0x43, 0xc6, + 0x70, 0xda, 0x6d, 0x6c, 0xb0, 0x61, 0xef, 0xaa, 0xf5, 0xac, 0xcc, 0x16, 0x78, 0xee, 0xb8, 0x9f, + 0x4f, 0x57, 0x82, 0x12, 0xba, 0xd4, 0x69, 0x23, 0xfc, 0x89, 0x54, 0x98, 0x0d, 0x4d, 0x18, 0xf3, + 0x03, 0x49, 0x36, 0x5b, 0x37, 0x26, 0xa6, 0x30, 0x35, 0x83, 0x87, 0xbe, 0x73, 0x15, 0xb8, 0x72, + 0xe2, 0x2a, 0x9e, 0x47, 0x3d, 0xc9, 0x30, 0xf5, 0x28, 0x90, 0xe1, 0x8b, 0x12, 0x6c, 0xa8, 0xcf, + 0x33, 0x90, 0x51, 0x31, 0xf1, 0x1c, 0x17, 0xfb, 0x3b, 0xea, 0x73, 0x09, 0xe6, 0x69, 0x04, 0xe0, + 0x5a, 0x5d, 0xcf, 0x71, 0x35, 0x17, 0x3f, 0x74, 0x2d, 0x0f, 0x93, 0x6c, 0x84, 0x19, 0x5d, 0xe9, + 0x8c, 0x21, 0x0c, 0x03, 0x15, 0xab, 0x01, 0x88, 0x2a, 0x30, 0xb8, 0xdd, 0xdd, 0xfe, 0xd1, 0xcf, + 0xf3, 0xb7, 0x26, 0x5a, 0xc7, 0xf1, 0xa0, 0xa4, 0x58, 0xaf, 0xaa, 0xc8, 0x1c, 0x03, 0x46, 0xcf, + 0x41, 0x8c, 0xda, 0x2d, 0x73, 0x15, 0xc9, 0xb2, 0x7c, 0xdc, 0xcf, 0xc7, 0xa8, 0x65, 0xab, 0x4c, + 0x3a, 0xb4, 0xc1, 0x63, 0xcf, 0xb0, 0xc1, 0x57, 0x21, 0xe5, 0xe9, 0xad, 0x36, 0xd6, 0x68, 0xcb, + 0x44, 0x6c, 0xbf, 0x97, 0x47, 0x66, 0x82, 0xdc, 0x6f, 0xb7, 0x74, 0x82, 0x8b, 0x3b, 0x54, 0x33, + 0x34, 0x76, 0xf0, 0x7c, 0x01, 0x41, 0xcb, 0x90, 0x72, 0x1e, 0x60, 0xd7, 0xb5, 0x4c, 0xac, 0x99, + 0x2d, 0xb6, 0x87, 0x92, 0xe5, 0xcc, 0x71, 0x3f, 0x0f, 0x5b, 0x42, 0x5c, 0x2d, 0xab, 0xe0, 0xab, + 0x54, 0x5b, 0xc8, 0x83, 0x05, 0x41, 0x1a, 0xc1, 0xfe, 0x67, 0xf6, 0x94, 0x60, 0x5d, 0x78, 0x77, + 0xf2, 0xc5, 0xe0, 0xeb, 0xee, 0x1b, 0x0f, 0x8b, 0x2c, 0xf8, 0x20, 0x51, 0x6b, 0xac, 0x04, 0xbd, + 0x0a, 0x73, 0x5d, 0x17, 0x77, 0x75, 0x17, 0x6b, 0x86, 0xd3, 0xe9, 0xb6, 0xb1, 0x87, 0x4d, 0x66, + 0xfd, 0xb2, 0xaa, 0x88, 0x82, 0x8a, 0x2f, 0x47, 0x2f, 0x51, 0xaf, 0xa4, 0x7b, 0x34, 0xe0, 0x21, + 0xd8, 0xa5, 0x9a, 0x49, 0xa6, 0x99, 0x66, 0xd2, 0xba, 0x10, 0xa2, 0x37, 0xe1, 0xca, 0x60, 0xdd, + 0x88, 0xd6, 0xed, 0xb5, 0xda, 0x16, 0x39, 0xc0, 0x66, 0x16, 0x98, 0xf6, 0x42, 0xa8, 0x70, 0xdb, + 0x2f, 0x43, 0x47, 0x43, 0xa6, 0x68, 0xd0, 0x89, 0xd1, 0xf7, 0x71, 0x36, 0xb5, 0x24, 0x5d, 0x9f, + 0x2e, 0xaf, 0x3d, 0xed, 0xe7, 0xab, 0x13, 0xdb, 0x11, 0xc1, 0x9d, 0x65, 0xcf, 0xc5, 0x38, 0x64, + 0x96, 0x15, 0x81, 0x17, 0xb6, 0x28, 0x5f, 0x86, 0x54, 0x80, 0xc1, 0x16, 0xcc, 0xce, 0x3c, 0x33, + 0xdb, 0x85, 0x50, 0x50, 0x09, 0x12, 0x3c, 0xa8, 0x26, 0xd9, 0x34, 0x5b, 0xc0, 0x6f, 0x9f, 0x66, + 0x43, 0x4c, 0x2b, 0xb4, 0x4a, 0x7e, 0x3d, 0x54, 0x05, 0xf0, 0x8e, 0xba, 0xbe, 0x25, 0x66, 0x18, + 0xca, 0x4b, 0xa7, 0xa1, 0x1c, 0x75, 0xc3, 0x86, 0x98, 0xf4, 0xc4, 0x37, 0x41, 0x0d, 0x98, 0xe1, + 0x11, 0xbb, 0xc0, 0x99, 0x65, 0x38, 0xaf, 0x9c, 0x82, 0xc3, 0xc2, 0x15, 0x3d, 0x84, 0x94, 0x22, + 0x81, 0x84, 0xa0, 0x6d, 0xc8, 0xd0, 0x18, 0x93, 0x6a, 0x0a, 0x34, 0x85, 0xa1, 0xdd, 0x38, 0x05, + 0xad, 0x2a, 0x94, 0x43, 0x78, 0x69, 0x33, 0x24, 0x23, 0xb9, 0x5f, 0x48, 0x30, 0x37, 0x46, 0x1e, + 0x68, 0x07, 0x22, 0xc1, 0xb1, 0x81, 0x72, 0x7a, 0x84, 0x1d, 0x19, 0x2e, 0x43, 0x24, 0x11, 0xcb, + 0x44, 0xfb, 0x90, 0xa4, 0xe6, 0x6c, 0x7b, 0xf4, 0x4c, 0x12, 0x61, 0xe0, 0x8d, 0xe3, 0x7e, 0x5e, + 0xde, 0x66, 0xc2, 0x4b, 0x37, 0x21, 0x73, 0xf0, 0xba, 0x89, 0xf2, 0x90, 0xf2, 0x1c, 0x0d, 0x3f, + 0xb2, 0x88, 0x67, 0xd9, 0xfb, 0x2c, 0x58, 0x90, 0x55, 0xf0, 0x9c, 0x9a, 0x90, 0xe4, 0xfe, 0x34, + 0x02, 0x68, 0x7c, 0x97, 0xa2, 0x7f, 0x90, 0xe0, 0x39, 0x3f, 0x06, 0x70, 0x5c, 0x6b, 0xdf, 0xb2, + 0xf5, 0xf6, 0x50, 0x30, 0x20, 0xb1, 0xd9, 0xfe, 0xf4, 0x32, 0x54, 0x20, 0x02, 0x84, 0x2d, 0x01, + 0x3f, 0x1a, 0x28, 0x3c, 0x47, 0x3d, 0x28, 0x0f, 0x14, 0xc6, 0x54, 0xee, 0xa9, 0xd9, 0xde, 0x29, + 0x95, 0x73, 0xeb, 0xf0, 0xfc, 0x99, 0xc0, 0x17, 0xf1, 0x5d, 0xb9, 0x1f, 0x49, 0x70, 0xed, 0x14, + 0x8f, 0x12, 0xc6, 0x49, 0x73, 0x9c, 0x0f, 0xc3, 0x38, 0xa9, 0x95, 0x5f, 0xb9, 0x84, 0xd7, 0x0a, + 0x75, 0xa2, 0x11, 0x93, 0x25, 0x25, 0x52, 0x78, 0x03, 0x66, 0x45, 0x25, 0xdf, 0x8f, 0xa2, 0xe7, + 0x01, 0x0e, 0xac, 0xfd, 0x03, 0xed, 0xa1, 0xee, 0x61, 0x57, 0x9c, 0x25, 0x93, 0x54, 0xf2, 0x11, + 0x15, 0x14, 0xfe, 0x55, 0x86, 0x74, 0xbd, 0xd3, 0x75, 0x5c, 0xcf, 0xf7, 0xb2, 0x77, 0x21, 0xce, + 0xfc, 0x02, 0x11, 0xeb, 0x57, 0x3c, 0xa3, 0x87, 0x43, 0x35, 0xb9, 0x7f, 0x11, 0xb4, 0x20, 0x30, + 0x02, 0xf7, 0x17, 0x39, 0xd1, 0xfd, 0xbd, 0x07, 0x71, 0x9e, 0x12, 0x10, 0x01, 0x79, 0xfe, 0x84, + 0x48, 0xbe, 0xbe, 0x75, 0xc7, 0x6a, 0xe3, 0x3b, 0x4c, 0xcd, 0x07, 0xe7, 0x95, 0xd0, 0xcb, 0x20, + 0x13, 0xe2, 0x69, 0xc4, 0xfa, 0x21, 0xf7, 0x9e, 0x51, 0x7e, 0x26, 0x6d, 0x36, 0x77, 0x9a, 0xd6, + 0x0f, 0xb1, 0x9a, 0x20, 0xc4, 0xa3, 0x3f, 0x50, 0x0e, 0xe4, 0x87, 0x7a, 0xbb, 0xcd, 0xbc, 0xec, + 0x34, 0x3b, 0x83, 0x07, 0xdf, 0xc3, 0xdb, 0x2c, 0xfe, 0xcd, 0x6e, 0x33, 0xe1, 0x30, 0xbb, 0xba, + 0x77, 0xc0, 0x22, 0xc7, 0xa4, 0x0a, 0x5c, 0xb4, 0xad, 0x7b, 0x07, 0x28, 0x0b, 0x09, 0xa2, 0x53, + 0xdf, 0x45, 0xb2, 0xf2, 0x52, 0xf4, 0xfa, 0x8c, 0xea, 0x7f, 0xa2, 0x45, 0x60, 0x9e, 0x97, 0x7f, + 0x32, 0x27, 0x16, 0x55, 0x43, 0x12, 0x36, 0x0f, 0x87, 0x56, 0x57, 0xdb, 0x3b, 0x24, 0xdc, 0x69, + 0x89, 0x79, 0x38, 0xb4, 0xba, 0x77, 0xd6, 0x89, 0x9a, 0xa0, 0x85, 0x77, 0x0e, 0x09, 0x7a, 0x05, + 0x66, 0x2d, 0x76, 0x72, 0xd1, 0x4c, 0xcb, 0xc5, 0x86, 0xd7, 0x3e, 0x62, 0x0e, 0x4b, 0x56, 0x33, + 0x5c, 0x5c, 0x15, 0x52, 0x74, 0x03, 0x94, 0x51, 0x37, 0xcb, 0x1c, 0x8d, 0xac, 0xce, 0x8e, 0x78, + 0x59, 0xaa, 0xca, 0x97, 0x3a, 0xe4, 0x38, 0xd3, 0x5c, 0x95, 0xcb, 0x07, 0x3e, 0xb3, 0x08, 0xf3, + 0x5d, 0xdd, 0x25, 0x58, 0x6b, 0xf5, 0x6c, 0xb3, 0x8d, 0x35, 0xce, 0xd5, 0xd9, 0x0c, 0xd3, 0x9e, + 0x63, 0x45, 0x65, 0x56, 0xc2, 0x69, 0xfd, 0xbc, 0xd8, 0xfb, 0xea, 0xff, 0x41, 0xec, 0x9d, 0xfb, + 0x69, 0x04, 0xa6, 0x99, 0x9d, 0xa3, 0x5b, 0x10, 0xa3, 0xcb, 0x2c, 0x4e, 0x76, 0x93, 0xc6, 0x5c, + 0xac, 0x0e, 0x42, 0x10, 0xb3, 0xf5, 0x0e, 0xce, 0x22, 0x66, 0x04, 0xec, 0x37, 0xba, 0x06, 0x09, + 0x82, 0xef, 0x6b, 0x0f, 0xf4, 0x76, 0x76, 0x9e, 0xad, 0x70, 0x9c, 0xe0, 0xfb, 0xf7, 0xf4, 0x36, + 0xba, 0x02, 0x71, 0x8b, 0x68, 0x36, 0x7e, 0x98, 0x5d, 0x60, 0x33, 0x35, 0x6d, 0x91, 0x4d, 0xfc, + 0x90, 0xd1, 0xb6, 0xee, 0xee, 0x63, 0x4f, 0x33, 0x9c, 0x36, 0xc9, 0x5e, 0xa1, 0x1b, 0x8c, 0x86, + 0x74, 0x54, 0x54, 0x71, 0xda, 0x04, 0xfd, 0x3f, 0x48, 0x3e, 0xd4, 0x89, 0x86, 0x3b, 0x5d, 0xef, + 0x88, 0x4d, 0x96, 0x4c, 0xcd, 0x9e, 0xd4, 0xe8, 0x77, 0x23, 0x26, 0x47, 0x94, 0x68, 0x23, 0x26, + 0x47, 0x95, 0x58, 0x23, 0x26, 0xc7, 0x94, 0xe9, 0x46, 0x4c, 0x9e, 0x56, 0xe2, 0x8d, 0x98, 0x1c, + 0x57, 0x12, 0x8d, 0x98, 0x9c, 0x50, 0xe4, 0x46, 0x4c, 0x96, 0x95, 0x64, 0x23, 0x26, 0x27, 0x15, + 0x68, 0xc4, 0x64, 0x50, 0x52, 0x8d, 0x98, 0x9c, 0x52, 0x66, 0x1a, 0x31, 0x79, 0x46, 0x49, 0x37, + 0x62, 0x72, 0x5a, 0xc9, 0x34, 0x62, 0x72, 0x46, 0x99, 0x6d, 0xc4, 0xe4, 0x59, 0x45, 0x69, 0xc4, + 0x64, 0x45, 0x99, 0x6b, 0xc4, 0xe4, 0x39, 0x05, 0x15, 0x3e, 0x97, 0x40, 0x69, 0xe2, 0xfb, 0x3d, + 0x6c, 0x1b, 0xf8, 0x9e, 0xde, 0xae, 0x1c, 0xf4, 0xec, 0x43, 0xf4, 0x32, 0xcc, 0x1a, 0xf4, 0x87, + 0xc6, 0x0f, 0xc6, 0x74, 0xa8, 0x12, 0x1b, 0x6a, 0x9a, 0x89, 0x9b, 0x54, 0x4a, 0x47, 0xfc, 0x3c, + 0x80, 0xd0, 0xa3, 0x3b, 0x9b, 0x67, 0xcd, 0x92, 0x5c, 0x85, 0x6e, 0xe7, 0x11, 0x18, 0xd7, 0x79, + 0xc8, 0xe8, 0x63, 0x08, 0x46, 0x75, 0x1e, 0xa2, 0x65, 0x58, 0xb0, 0xf1, 0x23, 0x4f, 0x1b, 0x55, + 0x66, 0x54, 0xa1, 0xce, 0xd1, 0xb2, 0x4a, 0xb8, 0x42, 0xe1, 0x5f, 0x22, 0x30, 0xeb, 0x77, 0xda, + 0xa7, 0xc3, 0x3d, 0x50, 0xe8, 0xb2, 0x58, 0xa6, 0xe6, 0x39, 0x1c, 0xc9, 0x27, 0xc6, 0xf7, 0xce, + 0x20, 0xc6, 0x11, 0x14, 0xfa, 0x5d, 0x37, 0x77, 0x1c, 0xd6, 0x1c, 0x77, 0x0d, 0x6a, 0x9a, 0x84, + 0x65, 0xb9, 0x5d, 0xc8, 0xf8, 0x95, 0xb8, 0x04, 0x55, 0x20, 0x3e, 0xd4, 0xde, 0xab, 0x13, 0xb4, + 0xe7, 0x4f, 0xb5, 0x2a, 0xaa, 0xe6, 0x7e, 0x03, 0xd0, 0x78, 0xdb, 0x61, 0xb7, 0x34, 0xcd, 0xdd, + 0xd2, 0xd6, 0xb0, 0x5b, 0x7a, 0xe7, 0x62, 0x63, 0x0b, 0x75, 0x3b, 0x7c, 0xaa, 0xfb, 0xa7, 0x08, + 0x64, 0xb8, 0x8b, 0x08, 0xdc, 0xd1, 0xab, 0x30, 0xc7, 0x48, 0xcb, 0xb2, 0xf7, 0xb5, 0xae, 0x10, + 0xb2, 0xf1, 0x45, 0x54, 0xc5, 0x2f, 0x08, 0x94, 0x5f, 0x80, 0xb4, 0x8b, 0x75, 0x73, 0xa0, 0x18, + 0x61, 0x8a, 0x33, 0x54, 0x18, 0x28, 0xbd, 0x04, 0x19, 0xe6, 0x0d, 0x07, 0x5a, 0x51, 0xa6, 0x95, + 0x66, 0xd2, 0x40, 0xad, 0x0c, 0x69, 0xd2, 0xd5, 0xed, 0x81, 0x56, 0x8c, 0x4d, 0xea, 0x39, 0xb9, + 0xa3, 0x19, 0x5a, 0x27, 0xec, 0x4b, 0x5d, 0x4c, 0x7a, 0x1d, 0xac, 0x75, 0x1d, 0x7e, 0xd8, 0x8a, + 0xaa, 0x49, 0x2e, 0xd9, 0x76, 0x08, 0xda, 0x65, 0xa6, 0xc2, 0xe6, 0x42, 0x33, 0xf9, 0xe4, 0x64, + 0xe3, 0xac, 0x95, 0x9b, 0x93, 0x4f, 0xa7, 0x3a, 0x4b, 0x86, 0x05, 0x85, 0xdf, 0x84, 0x6b, 0x34, + 0x5e, 0xe6, 0xa4, 0x58, 0x61, 0x89, 0x6c, 0xdf, 0x38, 0x75, 0x48, 0xb0, 0x98, 0x3b, 0x08, 0x3f, + 0xd7, 0x8e, 0xfb, 0xf9, 0x38, 0xd5, 0xbe, 0xb4, 0xe3, 0x8a, 0x53, 0xe0, 0xba, 0x59, 0xc8, 0x41, + 0x76, 0xb4, 0xf5, 0xe0, 0x8c, 0xfe, 0xab, 0x70, 0x75, 0x13, 0x3f, 0x3c, 0xa9, 0x63, 0x65, 0x48, + 0x70, 0x26, 0xf2, 0x8d, 0xf7, 0xfa, 0x28, 0x3f, 0x86, 0xd3, 0xf2, 0x45, 0xd6, 0xe8, 0x0e, 0xab, + 0xa0, 0xfa, 0x15, 0x0b, 0x9f, 0xc2, 0xb5, 0x11, 0xf4, 0x60, 0x21, 0xde, 0x87, 0x38, 0x3d, 0xc3, + 0x89, 0x18, 0x25, 0x33, 0x7e, 0x3e, 0x18, 0x47, 0x6f, 0x52, 0x7d, 0x55, 0x54, 0x2b, 0xa8, 0x2c, + 0xb9, 0xd0, 0xeb, 0x60, 0xba, 0xd6, 0x77, 0x2d, 0xe2, 0xa1, 0x0f, 0x60, 0x46, 0xac, 0x2d, 0x5d, + 0x72, 0xbf, 0xdb, 0xe7, 0x98, 0x47, 0xca, 0x0d, 0x40, 0x48, 0xe1, 0x6f, 0x25, 0x98, 0xaf, 0xba, + 0x4e, 0xb7, 0x8b, 0x4d, 0xc1, 0xfa, 0x7c, 0x2e, 0x7c, 0xb2, 0x97, 0x42, 0x64, 0xbf, 0x09, 0x91, + 0x7a, 0x55, 0x44, 0xf5, 0xb7, 0x2f, 0x7b, 0x58, 0xa8, 0x57, 0xd1, 0x3b, 0x7c, 0x42, 0x7a, 0x84, + 0x31, 0x61, 0x66, 0xec, 0xf8, 0x36, 0x64, 0x70, 0x4c, 0x51, 0x15, 0x15, 0x0a, 0x3f, 0x49, 0xc0, + 0x95, 0xf0, 0x24, 0xaf, 0x56, 0xfc, 0x8e, 0x7f, 0x06, 0x09, 0xcb, 0x36, 0xf1, 0x23, 0x3c, 0x11, + 0xe3, 0x9d, 0x04, 0x51, 0x14, 0xf3, 0x51, 0xa7, 0x30, 0xfe, 0x81, 0x51, 0x60, 0xa2, 0x8f, 0x83, + 0x40, 0x93, 0x27, 0x70, 0x6e, 0x3d, 0x33, 0x7a, 0x75, 0x24, 0xe8, 0x1c, 0x8a, 0xe9, 0x98, 0x6b, + 0xf8, 0x86, 0x62, 0xba, 0x26, 0xcc, 0x59, 0xb6, 0x87, 0xdd, 0x36, 0xd6, 0x1f, 0xd0, 0x10, 0x85, + 0x36, 0x2f, 0xf2, 0x38, 0x93, 0x06, 0x04, 0x4a, 0x08, 0x80, 0x07, 0x16, 0x9f, 0xc1, 0x7c, 0x18, + 0xd4, 0x5f, 0x82, 0xb3, 0x73, 0x3b, 0x6c, 0x86, 0x07, 0xb0, 0x7e, 0x0a, 0x25, 0x04, 0x54, 0x17, + 0xd3, 0x7e, 0x0f, 0xe2, 0xfc, 0xc8, 0x2e, 0x12, 0xa5, 0xb7, 0x9f, 0x75, 0xda, 0x79, 0x2a, 0x40, + 0x15, 0x68, 0xb9, 0x3f, 0x96, 0x60, 0x26, 0xbc, 0xdc, 0xc8, 0x02, 0x99, 0xf5, 0xdd, 0x67, 0xa7, + 0x68, 0x79, 0x93, 0x46, 0xa5, 0xac, 0x90, 0xad, 0xc1, 0xfb, 0xcf, 0xbc, 0x06, 0x1c, 0x42, 0x98, + 0x52, 0xdd, 0xa4, 0xa1, 0x8e, 0xe9, 0x3a, 0xdd, 0x41, 0xa2, 0x3c, 0xaa, 0xca, 0x54, 0x40, 0xa3, + 0xb7, 0xdc, 0x6f, 0x41, 0x32, 0x30, 0x94, 0xd0, 0x59, 0x3d, 0xfa, 0x35, 0x9e, 0xd5, 0xcf, 0x6c, + 0xbf, 0x0a, 0xe9, 0xa1, 0x19, 0x43, 0x57, 0x83, 0x3e, 0xc4, 0xca, 0x71, 0xde, 0x87, 0x73, 0x51, + 0x0a, 0x3f, 0x8f, 0xc3, 0xfc, 0x49, 0x4c, 0xfb, 0x09, 0x28, 0x21, 0xde, 0xd2, 0xda, 0x16, 0xf1, + 0xc4, 0x7e, 0xba, 0x71, 0xf6, 0xd1, 0x32, 0x44, 0x7e, 0xc2, 0x5a, 0x32, 0xee, 0x30, 0x25, 0x7e, + 0x1f, 0x32, 0x26, 0xef, 0xb8, 0x26, 0x36, 0x6a, 0xf4, 0xdc, 0x13, 0xe1, 0x09, 0x04, 0x28, 0xd0, + 0xd3, 0x66, 0xa8, 0x88, 0xb0, 0x1b, 0x04, 0x1f, 0x3d, 0xc8, 0xd2, 0x58, 0x26, 0xdb, 0x3d, 0xe9, + 0x72, 0xf3, 0xb8, 0x9f, 0x9f, 0x13, 0x58, 0x7e, 0x5a, 0xe6, 0xd2, 0x2b, 0x35, 0x67, 0x8e, 0x00, + 0x9a, 0xd4, 0x81, 0xd2, 0x72, 0xda, 0xf0, 0xf4, 0xc0, 0x81, 0xd2, 0x7d, 0x74, 0x79, 0x07, 0x4a, + 0x7f, 0xd6, 0x4d, 0xf4, 0x7b, 0x12, 0xcc, 0xf1, 0x1c, 0x6d, 0xa7, 0xe7, 0xe9, 0x3c, 0xf1, 0xee, + 0x9f, 0x34, 0x3f, 0x39, 0xee, 0xe7, 0x67, 0xd9, 0x84, 0x6c, 0x88, 0x32, 0xd6, 0x6c, 0xf9, 0x59, + 0x9b, 0x1d, 0xa0, 0x88, 0xd3, 0x57, 0x20, 0x30, 0xd1, 0x3a, 0x64, 0xf8, 0xb1, 0x59, 0xa3, 0x07, + 0x47, 0xcb, 0xb1, 0xd9, 0x11, 0x34, 0x5d, 0x7e, 0xf1, 0x69, 0x3f, 0xbf, 0x74, 0x82, 0x65, 0xf1, + 0x13, 0xf7, 0x3d, 0xae, 0xab, 0xa6, 0xf7, 0xc2, 0x9f, 0xc8, 0x80, 0x74, 0x60, 0x1a, 0x47, 0x5d, + 0x71, 0x62, 0xbd, 0xbc, 0x2b, 0x9b, 0xf1, 0x6d, 0x84, 0x62, 0xa2, 0x7d, 0x98, 0xf5, 0x1b, 0xe1, + 0x0e, 0x9d, 0x64, 0x93, 0x5f, 0x4b, 0x33, 0xbe, 0x59, 0xf3, 0x51, 0x13, 0x91, 0x3c, 0xb9, 0x0a, + 0x0b, 0x27, 0x46, 0x39, 0x5f, 0xc6, 0xe1, 0xea, 0x30, 0x11, 0x06, 0x71, 0x88, 0x36, 0xea, 0x21, + 0xdf, 0x9f, 0x98, 0x4c, 0x7d, 0x0c, 0x4e, 0x66, 0xfe, 0xd7, 0xa8, 0x8f, 0xfc, 0x6c, 0xc4, 0x47, + 0x3e, 0x03, 0x3e, 0x33, 0xaf, 0x11, 0x7c, 0xdf, 0x51, 0x7e, 0x1c, 0xf8, 0x02, 0x9e, 0x7f, 0xf9, + 0xe0, 0x19, 0xe0, 0x59, 0x7d, 0xff, 0x33, 0xf0, 0x06, 0xff, 0x2c, 0x41, 0x7a, 0x68, 0x64, 0xbf, + 0x4c, 0x77, 0xb0, 0x1d, 0x44, 0x43, 0xfc, 0xa1, 0xc4, 0xdb, 0x17, 0x1f, 0xd6, 0x70, 0x90, 0x94, + 0xfb, 0x7b, 0x09, 0xd2, 0x43, 0x13, 0xf9, 0x0d, 0x39, 0x92, 0xaf, 0xbf, 0xe7, 0x2d, 0xc8, 0x0c, + 0x2f, 0x51, 0xa8, 0x0d, 0xe9, 0xeb, 0x69, 0xa3, 0xf0, 0x3d, 0x88, 0x73, 0x09, 0x42, 0x90, 0xf9, + 0xa8, 0x54, 0xdf, 0xa9, 0x6f, 0xae, 0x6a, 0x77, 0xb6, 0x54, 0x6d, 0xb5, 0xa2, 0x4c, 0xa1, 0x19, + 0x90, 0xab, 0xb5, 0xbb, 0x35, 0x2a, 0x54, 0x24, 0x94, 0x82, 0x04, 0xfb, 0xaa, 0x55, 0x95, 0x48, + 0xa1, 0x0c, 0x0a, 0xc7, 0xde, 0xc3, 0xd4, 0x31, 0xd0, 0xb8, 0x1f, 0x15, 0x61, 0x9e, 0x05, 0xe9, + 0x1d, 0x1a, 0xbf, 0x51, 0x57, 0xa8, 0x85, 0xa2, 0xe7, 0xb9, 0xa0, 0x88, 0x3a, 0xc5, 0x4d, 0xbd, + 0x83, 0x0b, 0x7f, 0x17, 0x83, 0xb9, 0x01, 0x88, 0xef, 0x16, 0x5f, 0x06, 0x99, 0x58, 0xf6, 0xa1, + 0x36, 0xb8, 0xfe, 0xe6, 0x29, 0x31, 0xcb, 0x3e, 0xdc, 0x55, 0xeb, 0x6a, 0x82, 0x16, 0xee, 0xba, + 0x16, 0x6a, 0x40, 0xcc, 0xe9, 0x7a, 0xfe, 0x69, 0xf0, 0xad, 0x33, 0xa6, 0x62, 0xac, 0x8d, 0xe2, + 0x56, 0xd7, 0x13, 0x67, 0x79, 0x86, 0x81, 0xfe, 0x5a, 0x1a, 0x9c, 0x7a, 0xf8, 0xb9, 0xef, 0x9d, + 0x0b, 0xe1, 0xf1, 0x09, 0x10, 0x77, 0x91, 0x1f, 0xd1, 0x8d, 0xfa, 0xb4, 0x9f, 0x9f, 0x1b, 0x9d, + 0x20, 0x72, 0xc9, 0x4b, 0x4a, 0xbf, 0x8b, 0xa8, 0xc1, 0xaf, 0xc7, 0x06, 0x13, 0xcd, 0x1c, 0xc2, + 0x84, 0x37, 0x90, 0xe9, 0xa1, 0x85, 0xc8, 0xed, 0xc3, 0x4c, 0xb8, 0xf7, 0x27, 0xe4, 0xbd, 0x4b, + 0xc3, 0x09, 0x86, 0x57, 0x27, 0x9a, 0x19, 0x71, 0x24, 0x0c, 0x25, 0xdb, 0xbf, 0x07, 0xc9, 0x60, + 0xda, 0x2f, 0x92, 0xa5, 0xe7, 0x1c, 0x1f, 0xa4, 0xbd, 0xa6, 0x95, 0x78, 0xe1, 0x6f, 0x24, 0x98, + 0x51, 0x31, 0x71, 0xda, 0x0f, 0xb0, 0x49, 0x63, 0x9e, 0xe0, 0x55, 0x89, 0x34, 0xf9, 0xab, 0x92, + 0x12, 0x24, 0x83, 0xbc, 0xe4, 0x45, 0x5e, 0x68, 0x0c, 0x6a, 0xa1, 0x1b, 0xa0, 0xb4, 0x9c, 0x9e, + 0x6d, 0xea, 0xee, 0x91, 0xe6, 0x62, 0xdd, 0x38, 0xc0, 0xa6, 0xb8, 0x83, 0x99, 0xf5, 0xe5, 0x2a, + 0x17, 0x17, 0x7e, 0x1c, 0x01, 0x34, 0x98, 0x9c, 0x10, 0x15, 0xd1, 0xc8, 0x8d, 0x8d, 0x43, 0x1c, + 0x5e, 0x23, 0x27, 0xde, 0x9a, 0x8d, 0x04, 0x80, 0xc1, 0xc0, 0xfd, 0x25, 0x75, 0x43, 0x32, 0x82, + 0xfe, 0xe8, 0xec, 0xec, 0x6b, 0x94, 0x65, 0x5f, 0x99, 0x95, 0xfe, 0x52, 0x33, 0xb0, 0xc2, 0x59, + 0xff, 0x77, 0x0c, 0x50, 0xc5, 0xc5, 0xba, 0x87, 0x29, 0xf3, 0x90, 0xb3, 0xce, 0xda, 0x65, 0x98, + 0xe6, 0x07, 0xb3, 0xc8, 0x45, 0x0e, 0x66, 0x62, 0x52, 0x78, 0x55, 0xf4, 0x03, 0x98, 0x31, 0x9c, + 0x76, 0xaf, 0x63, 0x6b, 0xec, 0xee, 0x58, 0x04, 0xc2, 0xdf, 0x3d, 0xcb, 0x88, 0xc7, 0x3a, 0x57, + 0xac, 0x38, 0x6d, 0xfa, 0xed, 0xe7, 0x0e, 0x38, 0x20, 0xd3, 0x40, 0xcf, 0x41, 0x32, 0xd8, 0x50, + 0x2c, 0x04, 0x4e, 0xaa, 0x03, 0x01, 0x5a, 0x81, 0x69, 0x9d, 0x68, 0xce, 0x1e, 0x8b, 0x51, 0xcf, + 0xb3, 0x30, 0x35, 0xa6, 0x93, 0xad, 0x3d, 0xf4, 0x26, 0xa4, 0xf7, 0xee, 0xf3, 0xb8, 0x9d, 0x13, + 0x28, 0xbf, 0xd2, 0x9f, 0x3d, 0xee, 0xe7, 0x53, 0x77, 0x3e, 0x64, 0x83, 0xa5, 0xf4, 0xa9, 0xa6, + 0xf6, 0xee, 0x07, 0x1f, 0xe8, 0x26, 0xcc, 0x75, 0xf4, 0x47, 0xda, 0x9e, 0xab, 0x1b, 0x22, 0x50, + 0x6d, 0x73, 0x56, 0x90, 0xd4, 0xd9, 0x8e, 0xfe, 0xe8, 0x8e, 0x90, 0xd7, 0xcd, 0x36, 0xce, 0xfd, + 0x97, 0x04, 0x09, 0x31, 0x22, 0xd4, 0x05, 0x10, 0xd3, 0x63, 0x99, 0x3c, 0x14, 0x4a, 0x97, 0x3f, + 0x3c, 0xee, 0xe7, 0x93, 0x15, 0x26, 0xad, 0x57, 0xc9, 0xd3, 0x7e, 0xfe, 0x83, 0x67, 0x25, 0x2d, + 0x1f, 0x44, 0x4d, 0xf2, 0x46, 0xea, 0x26, 0x4b, 0x0d, 0x1e, 0xe8, 0x44, 0x3b, 0xb0, 0x88, 0xe7, + 0xec, 0xbb, 0x7a, 0x87, 0x2d, 0xae, 0xac, 0xce, 0x1c, 0xe8, 0x64, 0xcd, 0x97, 0xa1, 0x1c, 0x0d, + 0x39, 0x1e, 0xf0, 0xab, 0x7f, 0xbe, 0xa5, 0x82, 0x6f, 0xb4, 0x02, 0x57, 0x82, 0xca, 0x1a, 0x1d, + 0x74, 0xab, 0x67, 0x1c, 0x62, 0xe6, 0x09, 0x28, 0x67, 0xcd, 0x07, 0x85, 0x1b, 0xfa, 0xa3, 0x32, + 0x2f, 0x2a, 0x5c, 0x81, 0xf9, 0xd0, 0xb2, 0x06, 0x01, 0xe2, 0x5f, 0xcd, 0x40, 0x62, 0x5b, 0x3f, + 0x6a, 0x3b, 0xba, 0x89, 0x96, 0x20, 0xe5, 0x5f, 0xd9, 0xd3, 0x10, 0x9b, 0xdb, 0x61, 0x58, 0x84, + 0x2c, 0xc8, 0xf4, 0x08, 0x76, 0xe9, 0x9a, 0x68, 0xec, 0x11, 0x2a, 0xe7, 0xaa, 0x72, 0xf9, 0x69, + 0x3f, 0x7f, 0x7b, 0xb2, 0x29, 0xc2, 0x46, 0xcf, 0xb5, 0xbc, 0xa3, 0x62, 0xf3, 0xc3, 0xbb, 0xbb, + 0x02, 0x8a, 0x6e, 0x24, 0x47, 0x4d, 0xf7, 0xc2, 0x9f, 0xe2, 0x01, 0x04, 0x1d, 0xae, 0xd6, 0xb1, + 0x0c, 0xd7, 0x21, 0x7e, 0x9e, 0x5c, 0x48, 0x37, 0x98, 0x10, 0xbd, 0x02, 0xb3, 0x7b, 0x96, 0xcd, + 0xee, 0x68, 0x7c, 0x3d, 0x9e, 0x22, 0xcf, 0xf8, 0x62, 0xa1, 0xf8, 0x00, 0x32, 0xa1, 0x47, 0x0f, + 0x74, 0xa9, 0xe3, 0x6c, 0xa9, 0xb7, 0x8e, 0xfb, 0xf9, 0xf4, 0x60, 0xeb, 0xf0, 0xe5, 0xbe, 0x8c, + 0x8f, 0x4a, 0x0f, 0x9a, 0xa1, 0x8b, 0xbd, 0x00, 0xd3, 0xec, 0xf1, 0x31, 0x7f, 0xe7, 0xa4, 0xf2, + 0x0f, 0xf4, 0x16, 0x4c, 0xb7, 0xb1, 0x4e, 0xb0, 0x78, 0xc2, 0xb4, 0x74, 0xc6, 0x66, 0x64, 0x6f, + 0x7c, 0x55, 0xae, 0x8e, 0xca, 0x10, 0xe7, 0xb7, 0x6e, 0xec, 0xae, 0x6c, 0x3c, 0x35, 0x79, 0xea, + 0x6b, 0xb5, 0xb5, 0x29, 0x55, 0xd4, 0x44, 0x35, 0x48, 0xb8, 0xfc, 0xa2, 0x95, 0xdd, 0xa0, 0x9d, + 0x7b, 0xd8, 0x0e, 0xdd, 0xe3, 0xae, 0x4d, 0xa9, 0x7e, 0x5d, 0xb4, 0xe3, 0xbf, 0x76, 0xe0, 0xac, + 0x2e, 0x1e, 0x73, 0x14, 0x27, 0x0c, 0xc8, 0x06, 0x80, 0x43, 0x28, 0x74, 0x80, 0x16, 0xcb, 0xba, + 0xb3, 0x8b, 0xb8, 0xb3, 0x07, 0x38, 0x74, 0x83, 0x4b, 0x07, 0xc8, 0x6b, 0xa2, 0x4d, 0x00, 0x23, + 0xf0, 0x34, 0xec, 0x8a, 0x2e, 0xb5, 0xf2, 0xda, 0x45, 0xa2, 0x99, 0xb5, 0x29, 0x35, 0x84, 0x80, + 0x3e, 0x84, 0x94, 0x31, 0xd8, 0x3a, 0xd9, 0x59, 0x06, 0xf8, 0xfa, 0x85, 0xf8, 0x73, 0x8d, 0x72, + 0xe6, 0x40, 0x3a, 0xcc, 0x99, 0xca, 0x28, 0x67, 0xd6, 0x20, 0x2d, 0xf2, 0x22, 0xfc, 0xdd, 0x7a, + 0x76, 0x8e, 0x51, 0x76, 0xd8, 0x4a, 0xfc, 0x97, 0xed, 0xc5, 0x9a, 0x6d, 0x38, 0x26, 0x36, 0x6b, + 0xf4, 0x5b, 0x15, 0x69, 0x60, 0xf6, 0x41, 0xd0, 0x2a, 0x64, 0x8c, 0x36, 0xd6, 0xed, 0x5e, 0xd7, + 0xc7, 0x41, 0x13, 0xe2, 0xa4, 0x45, 0x3d, 0x01, 0xb4, 0x09, 0x68, 0x8f, 0xbd, 0x90, 0x08, 0xf7, + 0x8a, 0xdd, 0xf4, 0x4d, 0x02, 0xa6, 0xb0, 0xba, 0xea, 0xa0, 0x67, 0xe8, 0x45, 0x48, 0xdb, 0x8e, + 0x6d, 0xe8, 0xb6, 0x81, 0xdb, 0xcc, 0xbb, 0xf1, 0xcb, 0xc1, 0x61, 0x21, 0xfa, 0x14, 0x32, 0x64, + 0x28, 0x84, 0xcf, 0x5e, 0x61, 0x2d, 0xbe, 0x71, 0xd1, 0xa4, 0xdf, 0xda, 0x94, 0x3a, 0x82, 0x84, + 0x7e, 0x0d, 0x14, 0x6f, 0xe4, 0x66, 0x80, 0x5d, 0x33, 0x9e, 0xfd, 0x1a, 0xe9, 0x94, 0xab, 0x8c, + 0xb5, 0x29, 0x75, 0x0c, 0x0d, 0x7d, 0x06, 0xb3, 0x64, 0xf8, 0xbd, 0x6d, 0xf6, 0x1a, 0x6b, 0xe0, + 0x3b, 0x67, 0xa6, 0xb7, 0x4f, 0x7a, 0xa2, 0xbc, 0x36, 0xa5, 0x8e, 0x62, 0x51, 0x78, 0x7b, 0xf8, + 0x82, 0x21, 0x9b, 0x3d, 0x17, 0xfe, 0xe4, 0x0b, 0x0f, 0x0a, 0x3f, 0x82, 0x55, 0x4e, 0x42, 0x42, + 0xdc, 0x02, 0x05, 0x77, 0xab, 0x09, 0x45, 0x2e, 0xfc, 0xa5, 0x0c, 0x72, 0x10, 0xba, 0x2d, 0x03, + 0x0a, 0x9c, 0xed, 0xe0, 0x41, 0x1b, 0xf5, 0x1a, 0x91, 0xb5, 0x29, 0x75, 0xce, 0x2f, 0x1b, 0xbc, + 0x69, 0x7b, 0x05, 0x66, 0x3b, 0x8e, 0x69, 0xed, 0x59, 0x03, 0xae, 0xe6, 0x99, 0xc2, 0x8c, 0x2f, + 0x16, 0x5c, 0x7d, 0x7b, 0xe8, 0xdd, 0xc7, 0x24, 0xef, 0x9d, 0xd7, 0xa6, 0x42, 0x0f, 0x43, 0xa8, + 0xef, 0x70, 0x7b, 0xb6, 0x6d, 0xd9, 0xfb, 0x9a, 0x38, 0x2d, 0xf2, 0xb0, 0x24, 0x2d, 0xa4, 0xe2, + 0xc0, 0x57, 0x19, 0x21, 0xd3, 0x1b, 0xe7, 0x92, 0xa9, 0x3f, 0xf6, 0x35, 0x29, 0x60, 0xd3, 0x3b, + 0xa3, 0x6c, 0x7a, 0xf3, 0x7c, 0x36, 0x0d, 0xc1, 0x04, 0x74, 0xba, 0x7b, 0x22, 0x9d, 0x2e, 0x4f, + 0x68, 0xeb, 0x21, 0xc4, 0x61, 0x3e, 0xad, 0x8c, 0xf0, 0xe9, 0x8d, 0x73, 0xf9, 0x34, 0x3c, 0x46, + 0x41, 0xa8, 0x5b, 0x27, 0x10, 0xea, 0xeb, 0x13, 0x11, 0x6a, 0x08, 0x2c, 0xcc, 0xa8, 0xea, 0x49, + 0x8c, 0x5a, 0x9c, 0x8c, 0x51, 0x43, 0x90, 0x43, 0x94, 0xfa, 0xfd, 0x31, 0xba, 0x50, 0xce, 0xdf, + 0x6f, 0x27, 0xa6, 0x08, 0xd6, 0xa4, 0x31, 0xbe, 0xd0, 0x4f, 0xe0, 0x8b, 0x39, 0x06, 0xff, 0xe6, + 0x05, 0xf8, 0x22, 0xd4, 0xc0, 0x38, 0x61, 0x7c, 0x0c, 0x33, 0xe1, 0x4d, 0xce, 0xde, 0x57, 0x9c, + 0x4d, 0x47, 0xa7, 0xbc, 0xe7, 0x67, 0x36, 0x10, 0x2a, 0x42, 0x3f, 0x18, 0xe7, 0x8a, 0xf9, 0x73, + 0xc1, 0x4f, 0xb9, 0xbe, 0x5c, 0x93, 0xc6, 0xc9, 0x02, 0x40, 0xf6, 0x6f, 0xa6, 0x43, 0xc4, 0x51, + 0xf8, 0x43, 0x09, 0xa2, 0x0d, 0xa7, 0x85, 0x32, 0x83, 0x6c, 0x13, 0xcb, 0x13, 0xbd, 0x3f, 0x50, + 0x17, 0xc7, 0x9a, 0x17, 0xce, 0xe8, 0x47, 0x90, 0x9d, 0x0b, 0x2a, 0xa1, 0x77, 0x21, 0xd1, 0xe5, + 0x21, 0xab, 0xe0, 0x86, 0xc2, 0x59, 0xf5, 0xb9, 0xa6, 0xea, 0x57, 0xb9, 0x79, 0x23, 0xfc, 0x9f, + 0x71, 0x36, 0x1c, 0x13, 0xa3, 0x0c, 0xc0, 0xb6, 0x4e, 0x48, 0xf7, 0xc0, 0xd5, 0x09, 0x56, 0xa6, + 0x50, 0x02, 0xa2, 0xeb, 0x1b, 0x4d, 0x45, 0xba, 0xf9, 0x71, 0x38, 0x37, 0x54, 0x55, 0x4b, 0xf5, + 0xcd, 0xfa, 0xe6, 0xaa, 0xb6, 0x59, 0xda, 0xa8, 0x35, 0x95, 0x29, 0x94, 0x85, 0x85, 0x8f, 0x4a, + 0xf5, 0x1d, 0x91, 0x2c, 0xd2, 0xea, 0x9b, 0x3b, 0x35, 0xf5, 0x5e, 0xe9, 0xae, 0x22, 0xa1, 0xab, + 0x80, 0xd4, 0xad, 0xca, 0x7a, 0xb3, 0x5a, 0xd6, 0x2a, 0x5b, 0x1b, 0xdb, 0xa5, 0xca, 0x4e, 0x7d, + 0x6b, 0x53, 0x89, 0x20, 0x19, 0x62, 0xd5, 0xad, 0xcd, 0x9a, 0x02, 0x37, 0x7f, 0x11, 0x85, 0x18, + 0xb5, 0x0e, 0xf4, 0x22, 0xa4, 0x76, 0x37, 0x9b, 0xdb, 0xb5, 0x4a, 0xfd, 0x4e, 0xbd, 0x56, 0x55, + 0xa6, 0x72, 0xf3, 0x8f, 0x9f, 0x2c, 0xcd, 0xd2, 0xa2, 0x5d, 0x9b, 0x74, 0xb1, 0xc1, 0x68, 0x11, + 0xe5, 0x20, 0x5e, 0x2e, 0x55, 0xd6, 0x77, 0xb7, 0x15, 0x29, 0x97, 0x79, 0xfc, 0x64, 0x09, 0xa8, + 0x02, 0xa7, 0x24, 0xf4, 0x1c, 0x24, 0xd4, 0x5a, 0x73, 0x67, 0x4b, 0xad, 0x29, 0x91, 0xdc, 0xec, + 0xe3, 0x27, 0x4b, 0x29, 0x5a, 0x28, 0x98, 0x06, 0xbd, 0x02, 0xe9, 0x66, 0x65, 0xad, 0xb6, 0x51, + 0xd2, 0x2a, 0x6b, 0xa5, 0xcd, 0xd5, 0x9a, 0x12, 0xcd, 0x2d, 0x3c, 0x7e, 0xb2, 0xa4, 0x8c, 0x9a, + 0x26, 0x6d, 0xa2, 0xbe, 0xb1, 0xbd, 0xa5, 0xee, 0x28, 0xb1, 0x41, 0x13, 0x9c, 0x11, 0x50, 0x01, + 0x80, 0xd7, 0xbe, 0x53, 0xab, 0x55, 0x95, 0xe9, 0x1c, 0x7a, 0xfc, 0x64, 0x29, 0x43, 0xcb, 0x07, + 0x1b, 0x1d, 0xbd, 0x04, 0x33, 0x15, 0xb5, 0x56, 0xda, 0xa9, 0x69, 0xcd, 0x9d, 0xd2, 0x4e, 0x53, + 0x89, 0x0f, 0x46, 0x12, 0xda, 0xbc, 0xa8, 0x08, 0x73, 0xa5, 0xdd, 0x9d, 0x2d, 0x6d, 0x48, 0x37, + 0x91, 0xbb, 0xf6, 0xf8, 0xc9, 0xd2, 0x3c, 0xd5, 0x2d, 0xf5, 0x3c, 0x27, 0xac, 0xff, 0x1a, 0x28, + 0x43, 0xfd, 0xd7, 0x56, 0x2b, 0x8a, 0x9c, 0xbb, 0xfa, 0xf8, 0xc9, 0x12, 0x1a, 0x1d, 0xc2, 0x6a, + 0x05, 0xfd, 0x7f, 0xb8, 0xba, 0xf3, 0xc9, 0x76, 0xad, 0x5a, 0x6b, 0x56, 0xb4, 0xe1, 0x61, 0x27, + 0x73, 0xd9, 0xc7, 0x4f, 0x96, 0x16, 0x68, 0x9d, 0xb1, 0xa1, 0xbf, 0x0e, 0x4a, 0x73, 0x47, 0xad, + 0x95, 0x36, 0xb4, 0xfa, 0xe6, 0x6a, 0xad, 0xc9, 0x16, 0x0b, 0x06, 0x5d, 0x1a, 0xd9, 0x66, 0x74, + 0x08, 0x9b, 0xb5, 0x8f, 0x46, 0xf0, 0x53, 0x03, 0xfd, 0x91, 0x9d, 0x93, 0x93, 0x7f, 0xfc, 0xe7, + 0x8b, 0x53, 0x7f, 0xf1, 0x93, 0xc5, 0xa9, 0xf2, 0xf5, 0x2f, 0xfe, 0x63, 0x71, 0xea, 0x8b, 0xe3, + 0x45, 0xe9, 0x67, 0xc7, 0x8b, 0xd2, 0x97, 0xc7, 0x8b, 0xd2, 0xbf, 0x1f, 0x2f, 0x4a, 0xbf, 0xff, + 0xd5, 0xe2, 0xd4, 0xcf, 0xbe, 0x5a, 0x9c, 0xfa, 0xf2, 0xab, 0xc5, 0xa9, 0x4f, 0xe3, 0xdc, 0x6a, + 0x5b, 0x71, 0x76, 0xba, 0x7a, 0xf3, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x3f, 0x14, 0x3b, 0x78, + 0xf5, 0x38, 0x00, 0x00, } diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index ce2d7dea1d48..c556ccaf9a0f 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -19,6 +19,7 @@ import "roachpb/io-formats.proto"; import "sql/catalog/descpb/structured.proto"; import "sql/catalog/descpb/tenant.proto"; import "util/hlc/timestamp.proto"; +import "sql/schemachanger/scpb/scpb.proto"; message Lease { option (gogoproto.equal) = true; @@ -302,6 +303,16 @@ message TypeSchemaChangeProgress { } +// TypeSchemaChangeDetails is the job detail information for the new schema change job. +message NewSchemaChangeDetails { + repeated cockroach.sql.schemachanger.scpb.Target targets = 1; +} + +// NewSchemaChangeProgress is the persisted progress for the new schema change job. +message NewSchemaChangeProgress { + repeated cockroach.sql.schemachanger.scpb.State states = 1; +} + message ResumeSpanList { repeated roachpb.Span resume_spans = 1 [(gogoproto.nullable) = false]; } @@ -598,6 +609,7 @@ message Payload { SchemaChangeGCDetails schemaChangeGC = 21; TypeSchemaChangeDetails typeSchemaChange = 22; StreamIngestionDetails streamIngestion = 23; + NewSchemaChangeDetails newSchemaChange = 24; } } @@ -619,6 +631,7 @@ message Progress { SchemaChangeGCProgress schemaChangeGC = 16; TypeSchemaChangeProgress typeSchemaChange = 17; StreamIngestionProgress streamIngest = 18; + NewSchemaChangeProgress newSchemaChange = 19; } } @@ -639,6 +652,7 @@ enum Type { // names for this enum, which cause a conflict with the SCHEMA_CHANGE entry. TYPEDESC_SCHEMA_CHANGE = 9 [(gogoproto.enumvalue_customname) = "TypeTypeSchemaChange"]; STREAM_INGESTION = 10 [(gogoproto.enumvalue_customname) = "TypeStreamIngestion"]; + NEW_SCHEMA_CHANGE = 11 [(gogoproto.enumvalue_customname) = "TypeNewSchemaChange"]; } message Job { diff --git a/pkg/jobs/jobspb/wrap.go b/pkg/jobs/jobspb/wrap.go index a55bf1c3356d..44809875b621 100644 --- a/pkg/jobs/jobspb/wrap.go +++ b/pkg/jobs/jobspb/wrap.go @@ -29,6 +29,7 @@ var _ Details = ChangefeedDetails{} var _ Details = CreateStatsDetails{} var _ Details = SchemaChangeGCDetails{} var _ Details = StreamIngestionDetails{} +var _ Details = NewSchemaChangeDetails{} // ProgressDetails is a marker interface for job progress details proto structs. type ProgressDetails interface{} @@ -40,6 +41,7 @@ var _ ProgressDetails = ChangefeedProgress{} var _ ProgressDetails = CreateStatsProgress{} var _ ProgressDetails = SchemaChangeGCProgress{} var _ ProgressDetails = StreamIngestionProgress{} +var _ ProgressDetails = NewSchemaChangeProgress{} // Type returns the payload's job type. func (p *Payload) Type() Type { @@ -71,6 +73,8 @@ func DetailsType(d isPayload_Details) Type { return TypeTypeSchemaChange case *Payload_StreamIngestion: return TypeStreamIngestion + case *Payload_NewSchemaChange: + return TypeNewSchemaChange default: panic(errors.AssertionFailedf("Payload.Type called on a payload with an unknown details type: %T", d)) } @@ -103,6 +107,8 @@ func WrapProgressDetails(details ProgressDetails) interface { return &Progress_TypeSchemaChange{TypeSchemaChange: &d} case StreamIngestionProgress: return &Progress_StreamIngest{StreamIngest: &d} + case NewSchemaChangeProgress: + return &Progress_NewSchemaChange{NewSchemaChange: &d} default: panic(errors.AssertionFailedf("WrapProgressDetails: unknown details type %T", d)) } @@ -130,6 +136,8 @@ func (p *Payload) UnwrapDetails() Details { return *d.TypeSchemaChange case *Payload_StreamIngestion: return *d.StreamIngestion + case *Payload_NewSchemaChange: + return *d.NewSchemaChange default: return nil } @@ -157,6 +165,8 @@ func (p *Progress) UnwrapDetails() ProgressDetails { return *d.TypeSchemaChange case *Progress_StreamIngest: return *d.StreamIngest + case *Progress_NewSchemaChange: + return *d.NewSchemaChange default: return nil } @@ -197,6 +207,8 @@ func WrapPayloadDetails(details Details) interface { return &Payload_TypeSchemaChange{TypeSchemaChange: &d} case StreamIngestionDetails: return &Payload_StreamIngestion{StreamIngestion: &d} + case NewSchemaChangeDetails: + return &Payload_NewSchemaChange{NewSchemaChange: &d} default: panic(errors.AssertionFailedf("jobs.WrapPayloadDetails: unknown details type %T", d)) } @@ -232,7 +244,7 @@ const ( func (Type) SafeValue() {} // NumJobTypes is the number of jobs types. -const NumJobTypes = 11 +const NumJobTypes = 12 func init() { if len(Type_name) != NumJobTypes { diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 7e706fb35d7c..113322dd0854 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -113,6 +113,7 @@ go_library( "//pkg/sql/roleoption", "//pkg/sql/row", "//pkg/sql/rowenc", + "//pkg/sql/schemachanger/scjob", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/sessiondatapb", diff --git a/pkg/server/server.go b/pkg/server/server.go index a3e3e64ecfa3..deb3dc2fae65 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -64,6 +64,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/sql/gcjob" // register jobs declared outside of pkg/sql "github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness" "github.com/cockroachdb/cockroach/pkg/sql/pgwire" + _ "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scjob" // register jobs declared outside of pkg/sql "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/cloud" "github.com/cockroachdb/cockroach/pkg/storage/cloudimpl" diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index d355bbf88c13..c1eb24d1299d 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1016,6 +1016,19 @@ func WaitToUpdateLeases(ctx context.Context, leaseMgr *lease.Manager, descID des return err } +// WaitToUpdateLeasesMultiple waits until the entire cluster has been updated to +// the latest versions of all the specified descriptors. +func WaitToUpdateLeasesMultiple( + ctx context.Context, leaseMgr *lease.Manager, ids []lease.IDVersion, +) error { + for _, idVer := range ids { + if err := WaitToUpdateLeases(ctx, leaseMgr, idVer.ID); err != nil { + return err + } + } + return nil +} + // done finalizes the mutations (adds new cols/indexes to the table). // It ensures that all nodes are on the current (pre-update) version of // sc.descID and that all nodes are on the new (post-update) version of diff --git a/pkg/sql/schemachanger/scjob/BUILD.bazel b/pkg/sql/schemachanger/scjob/BUILD.bazel new file mode 100644 index 000000000000..51a35c8f594d --- /dev/null +++ b/pkg/sql/schemachanger/scjob/BUILD.bazel @@ -0,0 +1,51 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "job", + srcs = ["job.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/job", + visibility = ["//visibility:public"], + deps = [ + "//pkg/jobs", + "//pkg/jobs/jobspb", + "//pkg/keys", + "//pkg/kv", + "//pkg/roachpb", + "//pkg/settings/cluster", + "//pkg/sql", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/lease", + "//pkg/sql/schemachanger/compiler", + "//pkg/sql/schemachanger/executor", + "//pkg/sql/schemachanger/targets", + "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", + "//pkg/util/log/logcrash", + ], +) + +go_library( + name = "scjob", + srcs = ["job.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scjob", + visibility = ["//visibility:public"], + deps = [ + "//pkg/jobs", + "//pkg/jobs/jobspb", + "//pkg/keys", + "//pkg/kv", + "//pkg/roachpb", + "//pkg/settings/cluster", + "//pkg/sql", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/lease", + "//pkg/sql/schemachanger/scexec", + "//pkg/sql/schemachanger/scpb", + "//pkg/sql/schemachanger/scplan", + "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", + "//pkg/util/log/logcrash", + ], +) diff --git a/pkg/sql/schemachanger/scjob/job.go b/pkg/sql/schemachanger/scjob/job.go new file mode 100644 index 000000000000..79cf13ff2256 --- /dev/null +++ b/pkg/sql/schemachanger/scjob/job.go @@ -0,0 +1,168 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scjob + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" +) + +func init() { + jobs.RegisterConstructor(jobspb.TypeNewSchemaChange, func( + job *jobs.Job, settings *cluster.Settings, + ) jobs.Resumer { + pl := job.Payload() + return &newSchemaChangeResumer{ + job: job, + targets: pl.GetNewSchemaChange().Targets, + } + }) +} + +type newSchemaChangeResumer struct { + job *jobs.Job + targets []*scpb.Target +} + +type badJobTracker struct { + txn *kv.Txn + descriptors *descs.Collection + codec keys.SQLCodec +} + +func (b badJobTracker) GetResumeSpans( + ctx context.Context, tableID descpb.ID, indexID descpb.IndexID, +) ([]roachpb.Span, error) { + table, err := b.descriptors.GetImmutableTableByID(ctx, b.txn, tableID, tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, + AvoidCached: true, + }, + }) + if err != nil { + return nil, err + } + return []roachpb.Span{table.IndexSpan(b.codec, indexID)}, nil +} + +func (b badJobTracker) SetResumeSpans( + ctx context.Context, tableID descpb.ID, indexID descpb.IndexID, total, done []roachpb.Span, +) error { + panic("implement me") +} + +var _ scexec.JobProgressTracker = (*badJobTracker)(nil) + +func (n *newSchemaChangeResumer) Resume(ctx context.Context, execCtxI interface{}) (err error) { + execCtx := execCtxI.(sql.JobExecContext) + if err := n.job.WithTxn(nil).Update(ctx, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + return nil + }); err != nil { + // TODO(ajwerner): Detect transient errors and classify as retriable here or + // in the jobs package. + return err + } + // TODO(ajwerner): Wait for leases on all descriptors before starting to + // avoid restarts. + + progress := n.job.Progress() + states := progress.GetNewSchemaChange().States + + settings := execCtx.ExtendedEvalContext().Settings + lm := execCtx.LeaseMgr() + db := lm.DB() + ie := execCtx.ExtendedEvalContext().InternalExecutor.(sqlutil.InternalExecutor) + sc, err := scplan.MakePlan(makeTargetStates(ctx, settings, n.targets, states), scplan.Params{ + ExecutionPhase: scplan.PostCommitPhase, + }) + if err != nil { + return err + } + + for _, s := range sc.Stages { + var descriptorsWithUpdatedVersions []lease.IDVersion + if err := descs.Txn(ctx, settings, lm, ie, db, func(ctx context.Context, txn *kv.Txn, descriptors *descs.Collection) error { + jt := badJobTracker{ + txn: txn, + descriptors: descriptors, + codec: execCtx.ExecCfg().Codec, + } + if err := scexec.NewExecutor(txn, descriptors, execCtx.ExecCfg().Codec, execCtx.ExecCfg().IndexBackfiller, jt).ExecuteOps(ctx, s.Ops); err != nil { + return err + } + descriptorsWithUpdatedVersions = descriptors.GetDescriptorsWithNewVersion() + defer n.job.WithTxn(nil) + return n.job.WithTxn(txn).Update(ctx, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + pg := md.Progress.GetNewSchemaChange() + pg.States = makeStates(s.After) + ju.UpdateProgress(md.Progress) + return nil + }) + }); err != nil { + return err + } + + // Wait for new versions. + if err := sql.WaitToUpdateLeasesMultiple( + ctx, + lm, + descriptorsWithUpdatedVersions, + ); err != nil { + return err + } + } + return nil +} + +func makeStates(next []*scpb.Node) []scpb.State { + states := make([]scpb.State, len(next)) + for i := range next { + states[i] = next[i].State + } + return states +} + +func makeTargetStates( + ctx context.Context, sv *cluster.Settings, protos []*scpb.Target, states []scpb.State, +) []*scpb.Node { + if len(protos) != len(states) { + logcrash.ReportOrPanic(ctx, &sv.SV, "unexpected slice size mismatch %d and %d", + len(protos), len(states)) + } + ts := make([]*scpb.Node, len(protos)) + for i := range protos { + ts[i] = &scpb.Node{ + Target: protos[i], + State: states[i], + } + } + return ts +} + +func (n *newSchemaChangeResumer) OnFailOrCancel(ctx context.Context, execCtx interface{}) error { + panic("unimplemented") +} diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 8f4f20541d91..e81826278a8b 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -2316,6 +2316,7 @@ var charts = []sectionDescription{ "jobs.import.currently_running", "jobs.restore.currently_running", "jobs.schema_change.currently_running", + "jobs.new_schema_change.currently_running", "jobs.schema_change_gc.currently_running", "jobs.typedesc_schema_change.currently_running", "jobs.stream_ingestion.currently_running", @@ -2405,6 +2406,18 @@ var charts = []sectionDescription{ }, Rate: DescribeDerivative_NON_NEGATIVE_DERIVATIVE, }, + { + Title: "Schema Change (New Implementation)", + Metrics: []string{ + "jobs.new_schema_change.fail_or_cancel_completed", + "jobs.new_schema_change.fail_or_cancel_failed", + "jobs.new_schema_change.fail_or_cancel_retry_error", + "jobs.new_schema_change.resume_completed", + "jobs.new_schema_change.resume_failed", + "jobs.new_schema_change.resume_retry_error", + }, + Rate: DescribeDerivative_NON_NEGATIVE_DERIVATIVE, + }, { Title: "Schema Change GC", Metrics: []string{ From c0bb7c0fff35fe9b16efd83def55c5df8dadafb2 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:03 -0500 Subject: [PATCH 10/15] sql: hook up new schema change to connExecutor This commit adds support for execution for the statement and pre-commit phases of the schema changer to the `connExecutor`. Also adds a session/cluster setting `sql.defaults.experimental_new_schema_changer.mode`, which is `'off'` default. The setting takes on the following three values: * `'off'` - never use the new schema changer * `'on'` - use the new schema changer in supported, implicit transaction * `'unsafe_always'` - always try to use the new schema changer, return errors where not supported This code also hooks up the planning logic for the above settings. As of this commit, only `ALTER TABLE ... ADD COLUMN` is supported, and even then, only for a subset of its behaviors. Release note: None --- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 7 +- pkg/clusterversion/key_string.go | 5 +- pkg/sql/BUILD.bazel | 5 + pkg/sql/conn_executor.go | 72 ++++++++ pkg/sql/conn_executor_exec.go | 8 + pkg/sql/exec_util.go | 17 ++ .../testdata/logic_test/information_schema | 1 + .../logictest/testdata/logic_test/pg_catalog | 3 + .../logictest/testdata/logic_test/show_source | 1 + pkg/sql/opaque.go | 168 ++++++++++-------- pkg/sql/planner.go | 2 + pkg/sql/schema_change_plan_node.go | 74 ++++++++ pkg/sql/schema_changer_state.go | 22 +++ .../scexec/executor_external_test.go | 4 +- pkg/sql/schemachanger/scplan/plan.go | 6 +- pkg/sql/schemachanger/scplan/rules.go | 8 +- pkg/sql/sem/tree/stmt.go | 1 + pkg/sql/sessiondata/session_data.go | 47 +++++ pkg/sql/vars.go | 19 ++ pkg/sql/walk.go | 1 + 21 files changed, 383 insertions(+), 90 deletions(-) create mode 100644 pkg/sql/schema_change_plan_node.go create mode 100644 pkg/sql/schema_changer_state.go diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 0236916accf6..81c75d1ecda7 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -100,6 +100,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -versionversion20.2-16set the active cluster version in the format '.' +versionversion20.2-18set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 12ef97af0d53..e15974015e4d 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -217,6 +217,8 @@ const ( // using the replicated legacy TruncatedState. It's also used in asserting // that no replicated truncated state representation is found. PostTruncatedAndRangeAppliedStateMigration + // NewSchemaChanger enables the new schema changer. + NewSchemaChanger // Step (1): Add new versions here. ) @@ -355,7 +357,10 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: PostTruncatedAndRangeAppliedStateMigration, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 16}, }, - + { + Key: NewSchemaChanger, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 18}, + }, // Step (2): Add new versions here. }) diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 552c79047387..140c4bc46ddd 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -36,11 +36,12 @@ func _() { _ = x[ReplicaVersions-25] _ = x[TruncatedAndRangeAppliedStateMigration-26] _ = x[PostTruncatedAndRangeAppliedStateMigration-27] + _ = x[NewSchemaChanger-28] } -const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigration" +const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationNewSchemaChanger" -var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 200, 217, 231, 295, 312, 321, 346, 366, 378, 383, 392, 420, 449, 471, 481, 496, 534, 576} +var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 200, 217, 231, 295, 312, 321, 346, 366, 378, 383, 392, 420, 449, 471, 481, 496, 534, 576, 592} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index f2eabe46c22c..8b829e7119a6 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -155,8 +155,10 @@ go_library( "scatter.go", "schema.go", "schema_change_cluster_setting.go", + "schema_change_plan_node.go", "schema_changer.go", "schema_changer_metrics.go", + "schema_changer_state.go", "scrub.go", "scrub_constraint.go", "scrub_fk.go", @@ -316,7 +318,10 @@ go_library( "//pkg/sql/rowenc", "//pkg/sql/rowexec", "//pkg/sql/schemachange", + "//pkg/sql/schemachanger/scbuild", "//pkg/sql/schemachanger/scexec", + "//pkg/sql/schemachanger/scpb", + "//pkg/sql/schemachanger/scplan", "//pkg/sql/scrub", "//pkg/sql/sem/builtins", "//pkg/sql/sem/transform", diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 7fcfd37f0c1b..ade398b5bf24 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -21,7 +21,9 @@ import ( "time" "unicode/utf8" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" @@ -37,6 +39,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" @@ -1034,6 +1039,8 @@ type connExecutor struct { // still need the statementID hash to disambiguate beyond the capped // statements. transactionStatementsHash util.FNV64 + + schemaChangerState SchemaChangerState } // sessionData contains the user-configurable connection variables. @@ -1224,6 +1231,11 @@ func (ns *prepStmtNamespace) resetTo( // commits, rolls back or restarts. func (ex *connExecutor) resetExtraTxnState(ctx context.Context, ev txnEvent) error { ex.extraTxnState.jobs = nil + if ex.server.cfg.Settings.Version.IsActive(ctx, clusterversion.NewSchemaChanger) { + ex.extraTxnState.schemaChangerState = SchemaChangerState{ + mode: ex.sessionData.NewSchemaChangerMode, + } + } for k := range ex.extraTxnState.schemaChangeJobsCache { delete(ex.extraTxnState.schemaChangeJobsCache, k) @@ -2166,6 +2178,7 @@ func (ex *connExecutor) resetEvalCtx(evalCtx *extendedEvalContext, txn *kv.Txn, evalCtx.Mon = ex.state.mon evalCtx.PrepareOnly = false evalCtx.SkipNormalize = false + evalCtx.SchemaChangerState = &ex.extraTxnState.schemaChangerState } // getTransactionState retrieves a text representation of the given state. @@ -2504,6 +2517,65 @@ func (ex *connExecutor) notifyStatsRefresherOfNewTables(ctx context.Context) { } } +// runPreCommitStages is part of the new schema changer infrastructure to +// mutate descriptors prior to committing a SQL transaction. +func (ex *connExecutor) runPreCommitStages(ctx context.Context) error { + if len(ex.extraTxnState.schemaChangerState.nodes) == 0 { + return nil + } + executor := scexec.NewExecutor( + ex.planner.txn, &ex.extraTxnState.descCollection, ex.server.cfg.Codec, + nil /* backfiller */, nil, /* jobTracker */ + ) + after, err := runNewSchemaChanger( + ctx, scplan.PreCommitPhase, + ex.extraTxnState.schemaChangerState.nodes, + executor, + ) + if err != nil { + return err + } + scs := &ex.extraTxnState.schemaChangerState + scs.nodes = after + targetSlice := make([]*scpb.Target, len(scs.nodes)) + states := make([]scpb.State, len(scs.nodes)) + for i := range scs.nodes { + targetSlice[i] = scs.nodes[i].Target + states[i] = scs.nodes[i].State + } + _, err = ex.planner.extendedEvalCtx.QueueJob(jobs.Record{ + Description: "Schema change job", // TODO(ajwerner): use const + Statement: "", // TODO(ajwerner): combine all of the DDL statements together + Username: ex.planner.User(), + DescriptorIDs: nil, // TODO(ajwerner): populate + Details: jobspb.NewSchemaChangeDetails{Targets: targetSlice}, + Progress: jobspb.NewSchemaChangeProgress{States: states}, + RunningStatus: "", + NonCancelable: false, + }) + return err +} + +func runNewSchemaChanger( + ctx context.Context, phase scplan.Phase, nodes []*scpb.Node, executor *scexec.Executor, +) (after []*scpb.Node, _ error) { + sc, err := scplan.MakePlan(nodes, scplan.Params{ + ExecutionPhase: phase, + // TODO(ajwerner): Populate the set of new descriptors + }) + if err != nil { + return nil, err + } + after = nodes + for _, s := range sc.Stages { + if err := executor.ExecuteOps(ctx, s.Ops); err != nil { + return nil, err + } + after = s.After + } + return after, nil +} + // StatementCounters groups metrics for counting different types of // statements. type StatementCounters struct { diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 1105be388173..2e40f696142f 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -682,6 +683,7 @@ func (ex *connExecutor) execStmtInOpenState( } log.VEventf(ctx, 2, "push detected for non-refreshable txn but auto-retry not possible") } + // No event was generated. return nil, nil, nil } @@ -730,6 +732,12 @@ func (ex *connExecutor) commitSQLTransaction( func (ex *connExecutor) commitSQLTransactionInternal( ctx context.Context, ast tree.Statement, ) error { + if ex.extraTxnState.schemaChangerState.mode != sessiondata.UseNewSchemaChangerOff { + if err := ex.runPreCommitStages(ctx); err != nil { + return err + } + } + if err := validatePrimaryKeys(&ex.extraTxnState.descCollection); err != nil { return err } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index b6ff99bc70ce..b42049597661 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -319,6 +319,19 @@ var experimentalUniqueWithoutIndexConstraintsMode = settings.RegisterBoolSetting false, ) +// DistSQLClusterExecMode controls the cluster default for when DistSQL is used. +var experimentalUseNewSchemaChanger = settings.RegisterEnumSetting( + "sql.defaults.experimental_new_schema_changer.enabled", + "default value for experimental_use_new_schema_changer session setting;"+ + "disables new schema changer by default", + "off", + map[int64]string{ + int64(sessiondata.UseNewSchemaChangerOff): "off", + int64(sessiondata.UseNewSchemaChangerOn): "on", + int64(sessiondata.UseNewSchemaChangerUnsafeAlways): "unsafe_always", + }, +) + // ExperimentalDistSQLPlanningClusterSettingName is the name for the cluster // setting that controls experimentalDistSQLPlanningClusterMode below. const ExperimentalDistSQLPlanningClusterSettingName = "sql.defaults.experimental_distsql_planning" @@ -2239,6 +2252,10 @@ func (m *sessionDataMutator) SetUniqueWithoutIndexConstraints(val bool) { m.data.EnableUniqueWithoutIndexConstraints = val } +func (m *sessionDataMutator) SetUseNewSchemaChanger(val sessiondata.NewSchemaChangerMode) { + m.data.NewSchemaChangerMode = val +} + // RecordLatestSequenceValue records that value to which the session incremented // a sequence. func (m *sessionDataMutator) RecordLatestSequenceVal(seqID uint32, val int64) { diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index f58a9b10ff1e..24c8e1bdb9ea 100755 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -3412,6 +3412,7 @@ experimental_enable_implicit_column_partitioning off experimental_enable_temp_tables off experimental_enable_unique_without_index_constraints on experimental_enable_virtual_columns off +experimental_use_new_schema_changer off extra_float_digits 0 force_savepoint_restart off foreign_key_cascades_limit 10000 diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index e4afea6c78f9..7c082b4fa5c9 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1997,6 +1997,7 @@ experimental_enable_implicit_column_partitioning off NULL experimental_enable_temp_tables off NULL NULL NULL string experimental_enable_unique_without_index_constraints on NULL NULL NULL string experimental_enable_virtual_columns off NULL NULL NULL string +experimental_use_new_schema_changer off NULL NULL NULL string extra_float_digits 0 NULL NULL NULL string force_savepoint_restart off NULL NULL NULL string foreign_key_cascades_limit 10000 NULL NULL NULL string @@ -2073,6 +2074,7 @@ experimental_enable_implicit_column_partitioning off NULL experimental_enable_temp_tables off NULL user NULL off off experimental_enable_unique_without_index_constraints on NULL user NULL off off experimental_enable_virtual_columns off NULL user NULL off off +experimental_use_new_schema_changer off NULL user NULL off off extra_float_digits 0 NULL user NULL 0 2 force_savepoint_restart off NULL user NULL off off foreign_key_cascades_limit 10000 NULL user NULL 10000 10000 @@ -2145,6 +2147,7 @@ experimental_enable_implicit_column_partitioning NULL NULL NULL experimental_enable_temp_tables NULL NULL NULL NULL NULL experimental_enable_unique_without_index_constraints NULL NULL NULL NULL NULL experimental_enable_virtual_columns NULL NULL NULL NULL NULL +experimental_use_new_schema_changer NULL NULL NULL NULL NULL extra_float_digits NULL NULL NULL NULL NULL force_savepoint_restart NULL NULL NULL NULL NULL foreign_key_cascades_limit NULL NULL NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index 16a9f95f62d6..7bc7be1535f1 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -53,6 +53,7 @@ experimental_enable_implicit_column_partitioning off experimental_enable_temp_tables off experimental_enable_unique_without_index_constraints off experimental_enable_virtual_columns off +experimental_use_new_schema_changer off extra_float_digits 0 force_savepoint_restart off foreign_key_cascades_limit 10000 diff --git a/pkg/sql/opaque.go b/pkg/sql/opaque.go index 84dad81b0b5f..fcf15cf86b53 100644 --- a/pkg/sql/opaque.go +++ b/pkg/sql/opaque.go @@ -47,153 +47,169 @@ func buildOpaque( scalarProps.Require(stmt.StatementTag(), tree.RejectSubqueries) var plan planNode - var err error + if tree.CanModifySchema(stmt) { + scPlan, usePlan, err := p.SchemaChange(ctx, stmt) + if err != nil { + return nil, err + } + if usePlan { + plan = scPlan + } + } + if plan == nil { + var err error + plan, err = planOpaque(ctx, p, stmt) + if err != nil { + return nil, err + } + } + if plan == nil { + return nil, errors.AssertionFailedf("planNode cannot be nil for %T", stmt) + } + res := &opaqueMetadata{ + info: stmt.StatementTag(), + plan: plan, + columns: planColumns(plan), + } + return res, nil +} + +func planOpaque(ctx context.Context, p *planner, stmt tree.Statement) (planNode, error) { switch n := stmt.(type) { case *tree.AlterDatabaseOwner: - plan, err = p.AlterDatabaseOwner(ctx, n) + return p.AlterDatabaseOwner(ctx, n) case *tree.AlterDatabaseAddRegion: - plan, err = p.AlterDatabaseAddRegion(ctx, n) + return p.AlterDatabaseAddRegion(ctx, n) case *tree.AlterDatabaseDropRegion: - plan, err = p.AlterDatabaseDropRegion(ctx, n) + return p.AlterDatabaseDropRegion(ctx, n) case *tree.AlterDatabasePrimaryRegion: - plan, err = p.AlterDatabasePrimaryRegion(ctx, n) + return p.AlterDatabasePrimaryRegion(ctx, n) case *tree.AlterDatabaseSurvivalGoal: - plan, err = p.AlterDatabaseSurvivalGoal(ctx, n) + return p.AlterDatabaseSurvivalGoal(ctx, n) case *tree.AlterIndex: - plan, err = p.AlterIndex(ctx, n) + return p.AlterIndex(ctx, n) case *tree.AlterSchema: - plan, err = p.AlterSchema(ctx, n) + return p.AlterSchema(ctx, n) case *tree.AlterTable: - plan, err = p.AlterTable(ctx, n) + return p.AlterTable(ctx, n) case *tree.AlterTableLocality: - plan, err = p.AlterTableLocality(ctx, n) + return p.AlterTableLocality(ctx, n) case *tree.AlterTableOwner: - plan, err = p.AlterTableOwner(ctx, n) + return p.AlterTableOwner(ctx, n) case *tree.AlterTableSetSchema: - plan, err = p.AlterTableSetSchema(ctx, n) + return p.AlterTableSetSchema(ctx, n) case *tree.AlterType: - plan, err = p.AlterType(ctx, n) + return p.AlterType(ctx, n) case *tree.AlterRole: - plan, err = p.AlterRole(ctx, n) + return p.AlterRole(ctx, n) case *tree.AlterSequence: - plan, err = p.AlterSequence(ctx, n) + return p.AlterSequence(ctx, n) case *tree.CommentOnColumn: - plan, err = p.CommentOnColumn(ctx, n) + return p.CommentOnColumn(ctx, n) case *tree.CommentOnDatabase: - plan, err = p.CommentOnDatabase(ctx, n) + return p.CommentOnDatabase(ctx, n) case *tree.CommentOnIndex: - plan, err = p.CommentOnIndex(ctx, n) + return p.CommentOnIndex(ctx, n) case *tree.CommentOnTable: - plan, err = p.CommentOnTable(ctx, n) + return p.CommentOnTable(ctx, n) case *tree.CreateDatabase: - plan, err = p.CreateDatabase(ctx, n) + return p.CreateDatabase(ctx, n) case *tree.CreateIndex: - plan, err = p.CreateIndex(ctx, n) + return p.CreateIndex(ctx, n) case *tree.CreateSchema: - plan, err = p.CreateSchema(ctx, n) + return p.CreateSchema(ctx, n) case *tree.CreateType: - plan, err = p.CreateType(ctx, n) + return p.CreateType(ctx, n) case *tree.CreateRole: - plan, err = p.CreateRole(ctx, n) + return p.CreateRole(ctx, n) case *tree.CreateSequence: - plan, err = p.CreateSequence(ctx, n) + return p.CreateSequence(ctx, n) case *tree.CreateExtension: - plan, err = p.CreateExtension(ctx, n) + return p.CreateExtension(ctx, n) case *tree.Deallocate: - plan, err = p.Deallocate(ctx, n) + return p.Deallocate(ctx, n) case *tree.Discard: - plan, err = p.Discard(ctx, n) + return p.Discard(ctx, n) case *tree.DropDatabase: - plan, err = p.DropDatabase(ctx, n) + return p.DropDatabase(ctx, n) case *tree.DropIndex: - plan, err = p.DropIndex(ctx, n) + return p.DropIndex(ctx, n) case *tree.DropOwnedBy: - plan, err = p.DropOwnedBy(ctx) + return p.DropOwnedBy(ctx) case *tree.DropRole: - plan, err = p.DropRole(ctx, n) + return p.DropRole(ctx, n) case *tree.DropSchema: - plan, err = p.DropSchema(ctx, n) + return p.DropSchema(ctx, n) case *tree.DropSequence: - plan, err = p.DropSequence(ctx, n) + return p.DropSequence(ctx, n) case *tree.DropTable: - plan, err = p.DropTable(ctx, n) + return p.DropTable(ctx, n) case *tree.DropType: - plan, err = p.DropType(ctx, n) + return p.DropType(ctx, n) case *tree.DropView: - plan, err = p.DropView(ctx, n) + return p.DropView(ctx, n) case *tree.Grant: - plan, err = p.Grant(ctx, n) + return p.Grant(ctx, n) case *tree.GrantRole: - plan, err = p.GrantRole(ctx, n) + return p.GrantRole(ctx, n) case *tree.ReassignOwnedBy: - plan, err = p.ReassignOwnedBy(ctx, n) + return p.ReassignOwnedBy(ctx, n) case *tree.RefreshMaterializedView: - plan, err = p.RefreshMaterializedView(ctx, n) + return p.RefreshMaterializedView(ctx, n) case *tree.RenameColumn: - plan, err = p.RenameColumn(ctx, n) + return p.RenameColumn(ctx, n) case *tree.RenameDatabase: - plan, err = p.RenameDatabase(ctx, n) + return p.RenameDatabase(ctx, n) case *tree.ReparentDatabase: - plan, err = p.ReparentDatabase(ctx, n) + return p.ReparentDatabase(ctx, n) case *tree.RenameIndex: - plan, err = p.RenameIndex(ctx, n) + return p.RenameIndex(ctx, n) case *tree.RenameTable: - plan, err = p.RenameTable(ctx, n) + return p.RenameTable(ctx, n) case *tree.Revoke: - plan, err = p.Revoke(ctx, n) + return p.Revoke(ctx, n) case *tree.RevokeRole: - plan, err = p.RevokeRole(ctx, n) + return p.RevokeRole(ctx, n) case *tree.Scatter: - plan, err = p.Scatter(ctx, n) + return p.Scatter(ctx, n) case *tree.Scrub: - plan, err = p.Scrub(ctx, n) + return p.Scrub(ctx, n) case *tree.SetClusterSetting: - plan, err = p.SetClusterSetting(ctx, n) + return p.SetClusterSetting(ctx, n) case *tree.SetZoneConfig: - plan, err = p.SetZoneConfig(ctx, n) + return p.SetZoneConfig(ctx, n) case *tree.SetVar: - plan, err = p.SetVar(ctx, n) + return p.SetVar(ctx, n) case *tree.SetTransaction: - plan, err = p.SetTransaction(ctx, n) + return p.SetTransaction(ctx, n) case *tree.SetSessionAuthorizationDefault: - plan, err = p.SetSessionAuthorizationDefault() + return p.SetSessionAuthorizationDefault() case *tree.SetSessionCharacteristics: - plan, err = p.SetSessionCharacteristics(n) + return p.SetSessionCharacteristics(n) case *tree.ShowClusterSetting: - plan, err = p.ShowClusterSetting(ctx, n) + return p.ShowClusterSetting(ctx, n) case *tree.ShowHistogram: - plan, err = p.ShowHistogram(ctx, n) + return p.ShowHistogram(ctx, n) case *tree.ShowTableStats: - plan, err = p.ShowTableStats(ctx, n) + return p.ShowTableStats(ctx, n) case *tree.ShowTraceForSession: - plan, err = p.ShowTrace(ctx, n) + return p.ShowTrace(ctx, n) case *tree.ShowZoneConfig: - plan, err = p.ShowZoneConfig(ctx, n) + return p.ShowZoneConfig(ctx, n) case *tree.ShowFingerprints: - plan, err = p.ShowFingerprints(ctx, n) + return p.ShowFingerprints(ctx, n) case *tree.Truncate: - plan, err = p.Truncate(ctx, n) + return p.Truncate(ctx, n) case tree.CCLOnlyStatement: - plan, err = p.maybePlanHook(ctx, stmt) + plan, err := p.maybePlanHook(ctx, stmt) if plan == nil && err == nil { return nil, pgerror.Newf(pgcode.CCLRequired, "a CCL binary is required to use this statement type: %T", stmt) } + return plan, err default: return nil, errors.AssertionFailedf("unknown opaque statement %T", stmt) } - if err != nil { - return nil, err - } - if plan == nil { - return nil, errors.AssertionFailedf("planNode cannot be nil for %T", stmt) - } - res := &opaqueMetadata{ - info: stmt.StatementTag(), - plan: plan, - columns: planColumns(plan), - } - return res, nil } func init() { diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index c387992bbbe4..3f18d1178786 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -95,6 +95,8 @@ type extendedEvalContext struct { schemaAccessors *schemaInterface sqlStatsCollector *sqlStatsCollector + + SchemaChangerState *SchemaChangerState } // copy returns a deep copy of ctx. diff --git a/pkg/sql/schema_change_plan_node.go b/pkg/sql/schema_change_plan_node.go new file mode 100644 index 000000000000..27a516e49c9c --- /dev/null +++ b/pkg/sql/schema_change_plan_node.go @@ -0,0 +1,74 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" +) + +// SchemaChange provides the planNode for the new schema changer. +func (p *planner) SchemaChange(ctx context.Context, stmt tree.Statement) (planNode, bool, error) { + // TODO(ajwerner): Call featureflag.CheckEnabled appropriately. + mode := p.extendedEvalCtx.SchemaChangerState.mode + if mode == sessiondata.UseNewSchemaChangerOff || + (mode == sessiondata.UseNewSchemaChangerOn && !p.extendedEvalCtx.TxnImplicit) { + return nil, false, nil + } + b := scbuild.NewBuilder(p, p.SemaCtx(), p.EvalContext()) + updated, err := b.Build(ctx, p.extendedEvalCtx.SchemaChangerState.nodes, stmt) + if scbuild.HasNotImplemented(err) && mode == sessiondata.UseNewSchemaChangerOn { + return nil, false, nil + } + if err != nil { + return nil, false, err + } + return &schemaChangePlanNode{ + plannedState: updated, + }, true, nil +} + +// schemaChangePlanNode is the planNode utilized by the new schema changer to +// perform all schema changes, unified in the new schema changer. +type schemaChangePlanNode struct { + // plannedState contains the set of states produced by the builder combining + // the nodes that existed preceding the current statement with the output of + // the built current statement. + // + // TODO(ajwerner): Give this a better name. + plannedState []*scpb.Node +} + +func (s *schemaChangePlanNode) startExec(params runParams) error { + executor := scexec.NewExecutor(params.p.txn, params.p.Descriptors(), params.p.EvalContext().Codec, + nil /* backfiller */, nil /* jobTracker */) + after, err := runNewSchemaChanger( + params.ctx, scplan.StatementPhase, s.plannedState, executor, + ) + if err != nil { + return err + } + scs := params.p.extendedEvalCtx.SchemaChangerState + scs.nodes = after + return nil +} + +func (s schemaChangePlanNode) Next(params runParams) (bool, error) { return false, nil } +func (s schemaChangePlanNode) Values() tree.Datums { return tree.Datums{} } +func (s schemaChangePlanNode) Close(ctx context.Context) {} + +var _ (planNode) = (*schemaChangePlanNode)(nil) diff --git a/pkg/sql/schema_changer_state.go b/pkg/sql/schema_changer_state.go new file mode 100644 index 000000000000..462c921080d3 --- /dev/null +++ b/pkg/sql/schema_changer_state.go @@ -0,0 +1,22 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" +) + +// SchemaChangerState is state associated with the new schema changer. +type SchemaChangerState struct { + mode sessiondata.NewSchemaChangerMode + nodes []*scpb.Node +} diff --git a/pkg/sql/schemachanger/scexec/executor_external_test.go b/pkg/sql/schemachanger/scexec/executor_external_test.go index 108b1b4fa116..67aba6af8414 100644 --- a/pkg/sql/schemachanger/scexec/executor_external_test.go +++ b/pkg/sql/schemachanger/scexec/executor_external_test.go @@ -297,7 +297,7 @@ func TestSchemaChanger(t *testing.T) { } for _, phase := range []scplan.Phase{ - scplan.PostStatementPhase, + scplan.StatementPhase, scplan.PreCommitPhase, } { sc, err := scplan.MakePlan(targetStates, scplan.Params{ @@ -391,7 +391,7 @@ func TestSchemaChanger(t *testing.T) { require.NoError(t, err) for _, phase := range []scplan.Phase{ - scplan.PostStatementPhase, + scplan.StatementPhase, scplan.PreCommitPhase, } { sc, err := scplan.MakePlan(targetStates, scplan.Params{ diff --git a/pkg/sql/schemachanger/scplan/plan.go b/pkg/sql/schemachanger/scplan/plan.go index eae3b84ac59a..176aab2d41a9 100644 --- a/pkg/sql/schemachanger/scplan/plan.go +++ b/pkg/sql/schemachanger/scplan/plan.go @@ -31,11 +31,9 @@ import ( type Phase int const ( - // PostStatementPhase refers to execution of ops occurring after statement + // StatementPhase refers to execution of ops occurring during statement // execution during the user transaction. - // TODO(ajwerner,lucy-zhang): Move the execution for these ops to actual - // statement execution for the planNodes and update the name. - PostStatementPhase Phase = iota + StatementPhase Phase = iota // PreCommitPhase refers to execution of ops occurring during the user // transaction immediately before commit. PreCommitPhase diff --git a/pkg/sql/schemachanger/scplan/rules.go b/pkg/sql/schemachanger/scplan/rules.go index 7665f2f75d62..19891c6e078b 100644 --- a/pkg/sql/schemachanger/scplan/rules.go +++ b/pkg/sql/schemachanger/scplan/rules.go @@ -89,7 +89,7 @@ var rules = map[scpb.Element]targetRules{ scpb.State_ABSENT: { { predicate: func(this *scpb.Column, flags Params) bool { - return flags.ExecutionPhase == PostStatementPhase && + return flags.ExecutionPhase == StatementPhase && !flags.CreatedDescriptorIDs.Contains(this.TableID) }, }, @@ -150,7 +150,7 @@ var rules = map[scpb.Element]targetRules{ { predicate: func(this *scpb.Column, flags Params) bool { return !flags.CreatedDescriptorIDs.Contains(this.TableID) && - (flags.ExecutionPhase == PostStatementPhase || + (flags.ExecutionPhase == StatementPhase || flags.ExecutionPhase == PreCommitPhase) }, }, @@ -203,7 +203,7 @@ var rules = map[scpb.Element]targetRules{ scpb.State_ABSENT: { { predicate: func(this *scpb.PrimaryIndex, flags Params) bool { - return flags.ExecutionPhase == PostStatementPhase && + return flags.ExecutionPhase == StatementPhase && !flags.CreatedDescriptorIDs.Contains(this.TableID) }, }, @@ -294,7 +294,7 @@ var rules = map[scpb.Element]targetRules{ scpb.State_PUBLIC: { { predicate: func(this *scpb.PrimaryIndex, flags Params) bool { - return flags.ExecutionPhase == PostStatementPhase && + return flags.ExecutionPhase == StatementPhase && !flags.CreatedDescriptorIDs.Contains(this.TableID) }, }, diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index d8df71b3213b..7c926c1c4a0c 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -69,6 +69,7 @@ const ( type Statement interface { fmt.Stringer NodeFormatter + StatementType() StatementType // StatementTag is a short string identifying the type of statement // (usually a single verb). This is different than the Stringer output, diff --git a/pkg/sql/sessiondata/session_data.go b/pkg/sql/sessiondata/session_data.go index c655ade603e4..d2589f1fb992 100644 --- a/pkg/sql/sessiondata/session_data.go +++ b/pkg/sql/sessiondata/session_data.go @@ -233,6 +233,9 @@ type LocalOnlySessionData struct { // TODO(rytaft): remove this once unique without index constraints are fully // supported. EnableUniqueWithoutIndexConstraints bool + + // NewSchemaChangerMode indicates whether to use the new schema changer. + NewSchemaChangerMode NewSchemaChangerMode /////////////////////////////////////////////////////////////////////////// // WARNING: consider whether a session parameter you're adding needs to // // be propagated to the remote nodes. If so, that parameter should live // @@ -396,3 +399,47 @@ func SerialNormalizationModeFromString(val string) (_ SerialNormalizationMode, o return 0, false } } + +// NewSchemaChangerMode controls if and when the new schema changer (in +// sql/schemachanger) is in use. +type NewSchemaChangerMode int64 + +const ( + // UseNewSchemaChangerOff means that we never use the new schema changer. + UseNewSchemaChangerOff NewSchemaChangerMode = iota + // UseNewSchemaChangerOn means that we use the new schema changer for + // supported statements in implicit transactions, but fall back to the old + // schema changer otherwise. + UseNewSchemaChangerOn + // UseNewSchemaChangerUnsafeAlways means that we attempt to use the new schema + // changer for all statements and return errors for unsupported statements. + // Used for testing/development. + UseNewSchemaChangerUnsafeAlways +) + +func (m NewSchemaChangerMode) String() string { + switch m { + case UseNewSchemaChangerOff: + return "off" + case UseNewSchemaChangerOn: + return "on" + case UseNewSchemaChangerUnsafeAlways: + return "unsafe_always" + default: + return fmt.Sprintf("invalid (%d)", m) + } +} + +// NewSchemaChangerModeFromString converts a string into a NewSchemaChangerMode +func NewSchemaChangerModeFromString(val string) (_ NewSchemaChangerMode, ok bool) { + switch strings.ToUpper(val) { + case "OFF": + return UseNewSchemaChangerOff, true + case "ON": + return UseNewSchemaChangerOn, true + case "UNSAFE_ALWAYS": + return UseNewSchemaChangerUnsafeAlways, true + default: + return 0, false + } +} diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index f6d43b322d6b..26101dd8ae0b 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -1206,6 +1206,25 @@ var varGen = map[string]sessionVar{ return formatBoolAsPostgresSetting(experimentalUniqueWithoutIndexConstraintsMode.Get(sv)) }, }, + + `experimental_use_new_schema_changer`: { + GetStringVal: makePostgresBoolGetStringValFn(`experimental_use_new_schema_changer`), + Set: func(_ context.Context, m *sessionDataMutator, s string) error { + mode, ok := sessiondata.NewSchemaChangerModeFromString(s) + if !ok { + return newVarValueError(`experimental_user_new_schema_changer`, s, + "off", "on", "unsafe_always") + } + m.SetUseNewSchemaChanger(mode) + return nil + }, + Get: func(evalCtx *extendedEvalContext) string { + return evalCtx.SessionData.NewSchemaChangerMode.String() + }, + GlobalDefault: func(sv *settings.Values) string { + return sessiondata.NewSchemaChangerMode(experimentalUseNewSchemaChanger.Get(sv)).String() + }, + }, } const compatErrMsg = "this parameter is currently recognized only for compatibility and has no effect in CockroachDB." diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index 8641e770f83c..e1cac78b9017 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -431,4 +431,5 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&windowNode{}): "window", reflect.TypeOf(&zeroNode{}): "norows", reflect.TypeOf(&zigzagJoinNode{}): "zigzag join", + reflect.TypeOf(&schemaChangePlanNode{}): "schema change", } From 249b43e30981ee18007968f830a18f024f408cc3 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:03 -0500 Subject: [PATCH 11/15] sql: explicitly pass a context to QueueJob The context in the evalContext after statement execution is somehow canceled which is highly problematic for the post statement phase. Instead, pass through the context from the current frame. Release note: None --- pkg/jobs/jobs_test.go | 6 ++++-- pkg/sql/conn_executor.go | 2 +- pkg/sql/drop_schema.go | 2 +- pkg/sql/execinfra/processorsbase.go | 1 + pkg/sql/planner.go | 22 ++++++++++++---------- pkg/sql/schema.go | 2 +- pkg/sql/table.go | 6 +++--- pkg/sql/type_change.go | 2 +- 8 files changed, 24 insertions(+), 19 deletions(-) diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index f83532535a22..e8cc170a4dd2 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -1963,9 +1963,10 @@ func TestJobInTxn(t *testing.T) { if !ok { return nil, nil, nil, false, nil } - fn := func(_ context.Context, _ []sql.PlanNode, _ chan<- tree.Datums) error { + fn := func(ctx context.Context, _ []sql.PlanNode, _ chan<- tree.Datums) error { var err error job, err = execCtx.ExtendedEvalContext().QueueJob( + ctx, jobs.Record{ Description: st.String(), Details: jobspb.BackupDetails{}, @@ -1998,9 +1999,10 @@ func TestJobInTxn(t *testing.T) { if !ok { return nil, nil, nil, false, nil } - fn := func(_ context.Context, _ []sql.PlanNode, _ chan<- tree.Datums) error { + fn := func(ctx context.Context, _ []sql.PlanNode, _ chan<- tree.Datums) error { var err error job, err = execCtx.ExtendedEvalContext().QueueJob( + ctx, jobs.Record{ Description: "RESTORE", Details: jobspb.RestoreDetails{}, diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index ade398b5bf24..51a2e2165dd9 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -2543,7 +2543,7 @@ func (ex *connExecutor) runPreCommitStages(ctx context.Context) error { targetSlice[i] = scs.nodes[i].Target states[i] = scs.nodes[i].State } - _, err = ex.planner.extendedEvalCtx.QueueJob(jobs.Record{ + _, err = ex.planner.extendedEvalCtx.QueueJob(ctx, jobs.Record{ Description: "Schema change job", // TODO(ajwerner): use const Statement: "", // TODO(ajwerner): combine all of the DDL statements together Username: ex.planner.User(), diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index 78bfd0dd7c18..f845eb765944 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -220,7 +220,7 @@ func (p *planner) createDropSchemaJob( typeIDs = append(typeIDs, t.ID) } - _, err := p.extendedEvalCtx.QueueJob(jobs.Record{ + _, err := p.extendedEvalCtx.QueueJob(p.EvalContext().Ctx(), jobs.Record{ Description: jobDesc, Username: p.User(), DescriptorIDs: schemas, diff --git a/pkg/sql/execinfra/processorsbase.go b/pkg/sql/execinfra/processorsbase.go index 0ac102870f33..5e090c26422b 100644 --- a/pkg/sql/execinfra/processorsbase.go +++ b/pkg/sql/execinfra/processorsbase.go @@ -893,6 +893,7 @@ func (pb *ProcessorBase) InternalClose() bool { // Reset the context so that any incidental uses after this point do not // access the finished span. pb.Ctx = pb.origCtx + pb.EvalCtx.Context = pb.origCtx // This prevents Next() from returning more rows. pb.Out.consumerClosed() diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 3f18d1178786..ab3c9a840f67 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -100,24 +100,26 @@ type extendedEvalContext struct { } // copy returns a deep copy of ctx. -func (ctx *extendedEvalContext) copy() *extendedEvalContext { - cpy := *ctx - cpy.EvalContext = *ctx.EvalContext.Copy() +func (evalCtx *extendedEvalContext) copy() *extendedEvalContext { + cpy := *evalCtx + cpy.EvalContext = *evalCtx.EvalContext.Copy() return &cpy } // QueueJob creates a new job from record and queues it for execution after // the transaction commits. -func (ctx *extendedEvalContext) QueueJob(record jobs.Record) (*jobs.Job, error) { - job, err := ctx.ExecCfg.JobRegistry.CreateJobWithTxn( - ctx.Context, +func (evalCtx *extendedEvalContext) QueueJob( + ctx context.Context, record jobs.Record, +) (*jobs.Job, error) { + job, err := evalCtx.ExecCfg.JobRegistry.CreateJobWithTxn( + ctx, record, - ctx.Txn, + evalCtx.Txn, ) if err != nil { return nil, err } - *ctx.Jobs = append(*ctx.Jobs, *job.ID()) + *evalCtx.Jobs = append(*evalCtx.Jobs, *job.ID()) return job, nil } @@ -222,8 +224,8 @@ type planner struct { contextDatabaseID descpb.ID } -func (ctx *extendedEvalContext) setSessionID(sessionID ClusterWideID) { - ctx.SessionID = sessionID +func (evalCtx *extendedEvalContext) setSessionID(sessionID ClusterWideID) { + evalCtx.SessionID = sessionID } // noteworthyInternalMemoryUsageBytes is the minimum size tracked by each diff --git a/pkg/sql/schema.go b/pkg/sql/schema.go index c667a8d6df17..3e3b515fee3d 100644 --- a/pkg/sql/schema.go +++ b/pkg/sql/schema.go @@ -81,7 +81,7 @@ func (p *planner) writeSchemaDescChange( }, Progress: jobspb.SchemaChangeProgress{}, } - newJob, err := p.extendedEvalCtx.QueueJob(jobRecord) + newJob, err := p.extendedEvalCtx.QueueJob(ctx, jobRecord) if err != nil { return err } diff --git a/pkg/sql/table.go b/pkg/sql/table.go index 06e82fbd240a..be7a19738b0e 100644 --- a/pkg/sql/table.go +++ b/pkg/sql/table.go @@ -63,7 +63,7 @@ func (p *planner) createDropDatabaseJob( }, Progress: jobspb.SchemaChangeProgress{}, } - newJob, err := p.extendedEvalCtx.QueueJob(jobRecord) + newJob, err := p.extendedEvalCtx.QueueJob(ctx, jobRecord) if err != nil { return err } @@ -87,7 +87,7 @@ func (p *planner) createNonDropDatabaseChangeJob( }, Progress: jobspb.SchemaChangeProgress{}, } - newJob, err := p.extendedEvalCtx.QueueJob(jobRecord) + newJob, err := p.extendedEvalCtx.QueueJob(ctx, jobRecord) if err != nil { return err } @@ -140,7 +140,7 @@ func (p *planner) createOrUpdateSchemaChangeJob( }, Progress: jobspb.SchemaChangeProgress{}, } - newJob, err := p.extendedEvalCtx.QueueJob(jobRecord) + newJob, err := p.extendedEvalCtx.QueueJob(ctx, jobRecord) if err != nil { return err } diff --git a/pkg/sql/type_change.go b/pkg/sql/type_change.go index c7eda7875974..6ee4662a0f42 100644 --- a/pkg/sql/type_change.go +++ b/pkg/sql/type_change.go @@ -61,7 +61,7 @@ func (p *planner) writeTypeSchemaChange( // Type change jobs are not cancellable. NonCancelable: true, } - newJob, err := p.extendedEvalCtx.QueueJob(jobRecord) + newJob, err := p.extendedEvalCtx.QueueJob(ctx, jobRecord) if err != nil { return err } From f157c66d829452a4022fcc31bf2d687eac726e67 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:03 -0500 Subject: [PATCH 12/15] sql/rowexec: move code used only for testing to test file Release note: None --- pkg/sql/rowexec/backfiller.go | 30 --------------------------- pkg/sql/rowexec/backfiller_test.go | 33 +++++++++++++++++++++++++++++- 2 files changed, 32 insertions(+), 31 deletions(-) diff --git a/pkg/sql/rowexec/backfiller.go b/pkg/sql/rowexec/backfiller.go index 5b3eb27b23db..5c2f9a252ffe 100644 --- a/pkg/sql/rowexec/backfiller.go +++ b/pkg/sql/rowexec/backfiller.go @@ -29,7 +29,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" ) @@ -259,32 +258,3 @@ func SetResumeSpansInJob( details.ResumeSpanList[mutationIdx].ResumeSpans = spans return job.WithTxn(txn).SetDetails(ctx, details) } - -// WriteResumeSpan writes a checkpoint for the backfill work on origSpan. -// origSpan is the span of keys that were assigned to be backfilled, -// resume is the left over work from origSpan. -func WriteResumeSpan( - ctx context.Context, - db *kv.DB, - codec keys.SQLCodec, - id descpb.ID, - mutationID descpb.MutationID, - filter backfill.MutationFilter, - finished roachpb.Spans, - jobsRegistry *jobs.Registry, -) error { - ctx, traceSpan := tracing.ChildSpan(ctx, "checkpoint") - defer traceSpan.Finish() - - return db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - resumeSpans, job, mutationIdx, error := GetResumeSpans( - ctx, jobsRegistry, txn, codec, id, mutationID, filter, - ) - if error != nil { - return error - } - - resumeSpans = roachpb.SubtractSpans(resumeSpans, finished) - return SetResumeSpansInJob(ctx, resumeSpans, mutationIdx, txn, job) - }) -} diff --git a/pkg/sql/rowexec/backfiller_test.go b/pkg/sql/rowexec/backfiller_test.go index f8301ecc7fae..272bd6f3337c 100644 --- a/pkg/sql/rowexec/backfiller_test.go +++ b/pkg/sql/rowexec/backfiller_test.go @@ -24,14 +24,45 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/backfill" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sqlmigrations" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) +// WriteResumeSpan writes a checkpoint for the backfill work on origSpan. +// origSpan is the span of keys that were assigned to be backfilled, +// resume is the left over work from origSpan. +func WriteResumeSpan( + ctx context.Context, + db *kv.DB, + codec keys.SQLCodec, + id descpb.ID, + mutationID descpb.MutationID, + filter backfill.MutationFilter, + finished roachpb.Spans, + jobsRegistry *jobs.Registry, +) error { + ctx, traceSpan := tracing.ChildSpan(ctx, "checkpoint") + defer traceSpan.Finish() + + return db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + resumeSpans, job, mutationIdx, error := rowexec.GetResumeSpans( + ctx, jobsRegistry, txn, codec, id, mutationID, filter, + ) + if error != nil { + return error + } + + resumeSpans = roachpb.SubtractSpans(resumeSpans, finished) + return rowexec.SetResumeSpansInJob(ctx, resumeSpans, mutationIdx, txn, job) + }) +} + func TestWriteResumeSpan(t *testing.T) { defer leaktest.AfterTest(t)() @@ -150,7 +181,7 @@ func TestWriteResumeSpan(t *testing.T) { if test.resume.Key != nil { finished.EndKey = test.resume.Key } - if err := rowexec.WriteResumeSpan( + if err := WriteResumeSpan( ctx, kvDB, keys.SystemSQLCodec, tableDesc.ID, mutationID, backfill.IndexMutationFilter, roachpb.Spans{finished}, registry, ); err != nil { t.Error(err) From 75426520229e4a12d1da9c5ca3741e271e676c95 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:03 -0500 Subject: [PATCH 13/15] sql/logictest: add new_schema_changer test file Release note: None --- .../testdata/logic_test/new_schema_changer | 177 ++++++++++++++++++ 1 file changed, 177 insertions(+) create mode 100644 pkg/sql/logictest/testdata/logic_test/new_schema_changer diff --git a/pkg/sql/logictest/testdata/logic_test/new_schema_changer b/pkg/sql/logictest/testdata/logic_test/new_schema_changer new file mode 100644 index 000000000000..0369b73d5dd2 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/new_schema_changer @@ -0,0 +1,177 @@ +subtest add_column + +statement ok +SET experimental_use_new_schema_changer = 'on' + +statement ok +CREATE TABLE foo (i INT PRIMARY KEY) + +statement ok +ALTER TABLE foo ADD COLUMN j INT + +statement ok +INSERT INTO foo VALUES (1, 1) + +query II rowsort +SELECT * FROM foo +---- +1 1 + +statement ok +DROP TABLE foo; + +subtest multi_add_column + +statement ok +CREATE TABLE foo (i INT PRIMARY KEY) + +statement ok +SET experimental_use_new_schema_changer = 'unsafe_always' + +statement ok +BEGIN + +statement ok +ALTER TABLE foo ADD COLUMN j INT + +statement ok +ALTER TABLE foo ADD COLUMN k INT + +statement ok +COMMIT + +statement ok +INSERT INTO foo VALUES (1, 2, 3) + +query III rowsort +SELECT * FROM foo +---- +1 2 3 + +statement ok +SET experimental_use_new_schema_changer = 'on' + +statement ok +DROP TABLE foo; + +subtest add_column_default + +statement ok +CREATE TABLE foo (i INT PRIMARY KEY) + +statement ok +INSERT INTO foo(i) VALUES (0) + +query I rowsort +SELECT * FROM foo +---- +0 + +statement ok +ALTER TABLE foo ADD COLUMN j INT DEFAULT 1 + +statement ok +INSERT INTO foo VALUES (1, 1) + +statement ok +INSERT INTO foo(i) VALUES (2) + +query II rowsort +SELECT * FROM foo +---- +0 1 +1 1 +2 1 + +statement ok +DROP TABLE foo + +subtest add_column_computed + +statement ok +CREATE TABLE foo (i INT PRIMARY KEY) + +statement ok +INSERT INTO foo VALUES (0); + +statement ok +ALTER TABLE foo ADD COLUMN j INT AS (i+1) STORED + +statement ok +INSERT INTO foo(i) VALUES (1) + +query II rowsort +SELECT * FROM foo +---- +0 1 +1 2 + +statement ok +DROP TABLE foo + +subtest add_column_families + +statement ok +CREATE TABLE foo (i INT PRIMARY KEY) + +statement ok +ALTER TABLE foo ADD COLUMN j INT CREATE FAMILY f2 + +statement ok +ALTER TABLE foo ADD COLUMN k INT FAMILY f2 + +statement ok +INSERT INTO foo VALUES (1, 2, 3) + +query III rowsort +SELECT * FROM foo +---- +1 2 3 + +statement ok +DROP TABLE foo + +subtest multi_table + +statement ok +CREATE TABLE foo (i INT PRIMARY KEY); +CREATE TABLE bar (j INT PRIMARY KEY); + +statement ok +SET experimental_use_new_schema_changer = 'unsafe_always' + +statement ok +BEGIN + +statement ok +ALTER TABLE foo ADD COLUMN a INT + +statement ok +ALTER TABLE bar ADD COLUMN b INT + +statement ok +COMMIT + +statement ok +INSERT INTO foo VALUES (1, 2) + +query II colnames,rowsort +SELECT * FROM foo +---- +i a +1 2 + +statement ok +INSERT INTO bar VALUES (3, 4) + +query II colnames,rowsort +SELECT * FROM bar +---- +j b +3 4 + +statement ok +SET experimental_use_new_schema_changer = 'on' + +statement ok +DROP TABLE foo, bar From b09e19a76060b6975d9d3a7a8f392ad1e2ac1e3e Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:03 -0500 Subject: [PATCH 14/15] schemachanger/scgraphviz: library to generate graphviz graphs Release note: None --- DEPS.bzl | 8 + go.mod | 1 + go.sum | 2 + pkg/sql/schemachanger/scgraphviz/BUILD.bazel | 17 ++ pkg/sql/schemachanger/scgraphviz/graphviz.go | 284 +++++++++++++++++++ pkg/testutils/lint/lint_test.go | 1 + vendor | 2 +- 7 files changed, 314 insertions(+), 1 deletion(-) create mode 100644 pkg/sql/schemachanger/scgraphviz/BUILD.bazel create mode 100644 pkg/sql/schemachanger/scgraphviz/graphviz.go diff --git a/DEPS.bzl b/DEPS.bzl index e18f9c13a6dd..25203a5a768f 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -757,6 +757,14 @@ def go_deps(): sum = "h1:G/bYguwHIzWq9ZoyUQqrjTmJbbYn3j3CKKpKinvZLFk=", version = "v1.0.0", ) + go_repository( + name = "com_github_emicklei_dot", + build_file_proto_mode = "disable_global", + importpath = "github.com/emicklei/dot", + sum = "h1:XDBW0Xco1QNyRb33cqLe10cT04yMWL1XpCZfa98Q6Og=", + version = "v0.15.0", + ) + go_repository( name = "com_github_envoyproxy_go_control_plane", build_file_proto_mode = "disable_global", diff --git a/go.mod b/go.mod index 0c00746480d1..058b98619328 100644 --- a/go.mod +++ b/go.mod @@ -58,6 +58,7 @@ require ( github.com/edsrzf/mmap-go v1.0.0 github.com/elastic/gosigar v0.10.0 github.com/elazarl/go-bindata-assetfs v1.0.0 + github.com/emicklei/dot v0.15.0 github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a github.com/frankban/quicktest v1.7.3 // indirect github.com/ghemawat/stream v0.0.0-20171120220530-696b145b53b9 diff --git a/go.sum b/go.sum index fd0d4deb4a34..9fb688da42fa 100644 --- a/go.sum +++ b/go.sum @@ -244,6 +244,8 @@ github.com/elastic/gosigar v0.10.0 h1:bPIzW1Qkut7n9uwvPAXbnLDVEd45TV5ZwxYZAVX/zE github.com/elastic/gosigar v0.10.0/go.mod h1:cdorVVzy1fhmEqmtgqkoE3bYtCfSCkVyjTyCIo22xvs= github.com/elazarl/go-bindata-assetfs v1.0.0 h1:G/bYguwHIzWq9ZoyUQqrjTmJbbYn3j3CKKpKinvZLFk= github.com/elazarl/go-bindata-assetfs v1.0.0/go.mod h1:v+YaWX3bdea5J/mo8dSETolEo7R71Vk1u8bnjau5yw4= +github.com/emicklei/dot v0.15.0 h1:XDBW0Xco1QNyRb33cqLe10cT04yMWL1XpCZfa98Q6Og= +github.com/emicklei/dot v0.15.0/go.mod h1:DeV7GvQtIw4h2u73RKBkkFdvVAz0D9fzeJrgPW6gy/s= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= diff --git a/pkg/sql/schemachanger/scgraphviz/BUILD.bazel b/pkg/sql/schemachanger/scgraphviz/BUILD.bazel new file mode 100644 index 000000000000..eeaa2d16b5ed --- /dev/null +++ b/pkg/sql/schemachanger/scgraphviz/BUILD.bazel @@ -0,0 +1,17 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "scgraphviz", + srcs = ["graphviz.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraphviz", + visibility = ["//visibility:public"], + deps = [ + "//pkg/sql/schemachanger/scgraph", + "//pkg/sql/schemachanger/scpb", + "//pkg/sql/schemachanger/scplan", + "//pkg/util/protoutil", + "@com_github_cockroachdb_errors//:errors", + "@com_github_emicklei_dot//:dot", + "@com_github_gogo_protobuf//jsonpb", + ], +) diff --git a/pkg/sql/schemachanger/scgraphviz/graphviz.go b/pkg/sql/schemachanger/scgraphviz/graphviz.go new file mode 100644 index 000000000000..902bba4b493a --- /dev/null +++ b/pkg/sql/schemachanger/scgraphviz/graphviz.go @@ -0,0 +1,284 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scgraphviz + +import ( + "bytes" + "encoding/json" + "fmt" + "html/template" + "reflect" + "strconv" + "strings" + + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" + "github.com/emicklei/dot" + "github.com/gogo/protobuf/jsonpb" +) + +// DrawStages returns a graphviz string of the stages of the Plan. +func DrawStages(p *scplan.Plan) (string, error) { + gv, err := drawStages(p) + if err != nil { + return "", err + } + return gv.String(), nil +} + +// DrawDependencies returns a graphviz string of graph used to build the Plan. +func DrawDependencies(p *scplan.Plan) (string, error) { + gv, err := drawDeps(p) + if err != nil { + return "", err + } + return gv.String(), nil +} + +func drawStages(p *scplan.Plan) (*dot.Graph, error) { + + dg := dot.NewGraph() + stagesSubgraph := dg.Subgraph("stages", dot.ClusterOption{}) + targetsSubgraph := stagesSubgraph.Subgraph("targets", dot.ClusterOption{}) + targetNodes := make(map[*scpb.Target]dot.Node, len(p.InitialNodes)) + for idx, n := range p.InitialNodes { + t := n.Target + tn := targetsSubgraph.Node(strconv.Itoa(idx)) + tn.Attr("label", htmlLabel(t.Element())) + tn.Attr("fontsize", "9") + tn.Attr("shape", "none") + targetNodes[t] = tn + } + + // Want to draw an edge to the initial target states with some dots + // or something. + curNodes := make([]dot.Node, len(p.InitialNodes)) + cur := p.InitialNodes + for i, n := range p.InitialNodes { + label := targetStateID(i, n.State) + tsn := stagesSubgraph.Node(fmt.Sprintf("initial %d", i)) + tsn.Attr("label", label) + tn := targetNodes[n.Target] + e := tn.Edge(tsn) + e.Dashed() + e.Label(n.Target.Direction.String()) + curNodes[i] = tsn + } + for id, st := range p.Stages { + stage := fmt.Sprintf("stage %d", id) + sg := stagesSubgraph.Subgraph(stage, dot.ClusterOption{}) + next := st.After + nextNodes := make([]dot.Node, len(curNodes)) + for i, st := range next { + cst := sg.Node(fmt.Sprintf("stage %d: %d", id, i)) + cst.Attr("label", targetStateID(i, st.State)) + if st != cur[i] { + ge := curNodes[i].Edge(cst) + oe, ok := p.Graph.GetOpEdgeFrom(cur[i]) + if ok { + ge.Attr("label", htmlLabel(oe.Op())) + ge.Attr("fontsize", "9") + } + } else { + ge := curNodes[i].Edge(cst) + ge.Dotted() + } + nextNodes[i] = cst + } + cur, curNodes = next, nextNodes + } + + return dg, nil +} + +func drawDeps(p *scplan.Plan) (*dot.Graph, error) { + dg := dot.NewGraph() + + depsSubgraph := dg.Subgraph("deps", dot.ClusterOption{}) + targetsSubgraph := depsSubgraph.Subgraph("targets", dot.ClusterOption{}) + targetNodes := make(map[*scpb.Target]dot.Node, len(p.InitialNodes)) + targetIdxMap := make(map[*scpb.Target]int) + for idx, n := range p.InitialNodes { + t := n.Target + tn := targetsSubgraph.Node(strconv.Itoa(idx)) + tn.Attr("label", htmlLabel(t.Element())) + tn.Attr("fontsize", "9") + tn.Attr("shape", "none") + targetNodes[t] = tn + targetIdxMap[t] = idx + } + + nodeNodes := make(map[*scpb.Node]dot.Node) + _ = p.Graph.ForEachNode(func(n *scpb.Node) error { + nodeNodes[n] = depsSubgraph.Node(targetStateID(targetIdxMap[n.Target], n.State)) + return nil + }) + + for _, n := range p.InitialNodes { + nn := nodeNodes[n] + tn := targetNodes[n.Target] + e := tn.Edge(nn) + e.Label(n.Target.Direction.String()) + e.Dashed() + } + + _ = p.Graph.ForEachEdge(func(e scgraph.Edge) error { + from := nodeNodes[e.From()] + to := nodeNodes[e.To()] + ge := from.Edge(to) + switch e := e.(type) { + case *scgraph.OpEdge: + ge.Attr("label", htmlLabel(e.Op())) + ge.Attr("fontsize", "9") + case *scgraph.DepEdge: + ge.Attr("color", "red") + } + return nil + }) + return dg, nil +} + +func targetStateID(targetID int, state scpb.State) string { + return fmt.Sprintf("%d:%s", targetID, state) +} + +func htmlLabel(o interface{}) dot.HTML { + var buf strings.Builder + if err := objectTemplate.Execute(&buf, o); err != nil { + panic(err) + } + return dot.HTML(buf.String()) +} + +// toMap converts a struct to a map, field by field. If at any point a protobuf +// message is encountered, it is converted to a map using jsonpb to marshal it +// to json and then marshaling it back to a map. This approach allows zero +// values to be effectively omitted. +func toMap(v interface{}) (interface{}, error) { + if v == nil { + return nil, nil + } + if msg, ok := v.(protoutil.Message); ok { + var buf bytes.Buffer + jsonEncoder := jsonpb.Marshaler{EmitDefaults: false} + if err := jsonEncoder.Marshal(&buf, msg); err != nil { + return nil, errors.Wrapf(err, "%T %v", v, v) + } + var m map[string]interface{} + if err := json.NewDecoder(&buf).Decode(&m); err != nil { + return nil, err + } + return m, nil + } + vv := reflect.ValueOf(v) + vt := vv.Type() + switch vt.Kind() { + case reflect.Struct: + case reflect.Ptr: + if vt.Elem().Kind() != reflect.Struct { + return v, nil + } + vv = vv.Elem() + vt = vt.Elem() + default: + return v, nil + } + + m := make(map[string]interface{}, vt.NumField()) + for i := 0; i < vt.NumField(); i++ { + vvf := vv.Field(i) + if !vvf.CanInterface() || vvf.IsZero() { + continue + } + var err error + if m[vt.Field(i).Name], err = toMap(vvf.Interface()); err != nil { + return nil, err + } + } + return m, nil +} + +var objectTemplate = template.Must(template.New("obj").Funcs(template.FuncMap{ + "typeOf": func(v interface{}) string { + return fmt.Sprintf("%T", v) + }, + "isMap": func(v interface{}) bool { + _, ok := v.(map[string]interface{}) + return ok + }, + "isSlice": func(v interface{}) bool { + vv := reflect.ValueOf(v) + if !vv.IsValid() { + return false + } + return vv.Kind() == reflect.Slice + }, + "emptyMap": func(v interface{}) bool { + m, ok := v.(map[string]interface{}) + return ok && len(m) == 0 + }, + "toMap": toMap, +}).Parse(` +{{- define "key" -}} + +{{- . -}} + +{{- end -}} + +{{- define "val" -}} + +{{- if (isMap .) -}} +{{- template "mapVal" . -}} +{{- else if (isSlice .) -}} +{{- template "sliceVal" . -}} +{{- else -}} +{{- . -}} +{{- end -}} + +{{- end -}} + +{{- define "sliceVal" -}} + +{{- range . -}} +{{- template "val" . -}} +{{- end -}} +
+{{- end -}} + +{{- define "mapVal" -}} + +{{- range $k, $v := . -}} +{{- if not (emptyMap $v) -}} + +{{- template "key" $k -}} +{{- template "val" $v -}} + +{{- end -}} +{{- end -}} +
+{{- end -}} + +{{- define "header" -}} + +{{- typeOf . -}} + +{{- end -}} + + +{{- template "header" . -}} + +
+{{- template "mapVal" (toMap .) -}} +
+`)) diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 3694d3fc05a0..5393b0777c43 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -983,6 +983,7 @@ func TestLint(t *testing.T) { ":!settings/settings_test.go", ":!sql/types/types_jsonpb.go", ":!sql/schemachanger/scbuild/builder_test.go", + ":!sql/schemachanger/scgraphviz/graphviz.go", ) if err != nil { t.Fatal(err) diff --git a/vendor b/vendor index e02b9082b8fb..26b10b3041ed 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit e02b9082b8fb2d06fa9cc04bf85b081f0ee47c45 +Subproject commit 26b10b3041ed0fb4fca39907a91f5010bdc56b81 From a719fda321c2073f33e95370f0ea10966c9b6bcd Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 25 Jan 2021 18:49:03 -0500 Subject: [PATCH 15/15] sql: hook up graphviz for schema changes to EXPLAIN (DDL) This commit implements the `EXPLAIN (DDL)` and `EXPLAIN (DDL, DEPS)` statements, both of which generate the graph of a provided DDL statement and produce a URL with an encoded graphviz graph, similar to `EXPLAIN (DISTSQL)`. Just `(DDL)` produces the stages; `(DDL, DEPS)` provides the dep edges. Release note: None --- pkg/sql/BUILD.bazel | 2 + pkg/sql/distsql_physical_planner.go | 2 +- pkg/sql/distsql_spec_exec_factory.go | 5 + pkg/sql/explain_ddl.go | 101 ++++++++++++++++++ .../testdata/logic_test/new_schema_changer | 6 ++ pkg/sql/opt/optbuilder/explain.go | 6 ++ pkg/sql/opt_exec_factory.go | 7 ++ pkg/sql/plan.go | 2 +- pkg/sql/plan_columns.go | 2 + pkg/sql/sem/tree/explain.go | 12 ++- pkg/sql/sqltelemetry/planning.go | 6 ++ pkg/sql/walk.go | 11 ++ 12 files changed, 158 insertions(+), 4 deletions(-) create mode 100644 pkg/sql/explain_ddl.go diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 8b829e7119a6..19a7df843cf9 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -91,6 +91,7 @@ go_library( "execute.go", "executor_statement_metrics.go", "explain_bundle.go", + "explain_ddl.go", "explain_plan.go", "explain_vec.go", "export.go", @@ -320,6 +321,7 @@ go_library( "//pkg/sql/schemachange", "//pkg/sql/schemachanger/scbuild", "//pkg/sql/schemachanger/scexec", + "//pkg/sql/schemachanger/scgraphviz", "//pkg/sql/schemachanger/scpb", "//pkg/sql/schemachanger/scplan", "//pkg/sql/scrub", diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 11c87bdae09e..2f536ccf9d41 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -2798,7 +2798,7 @@ func (dsp *DistSQLPlanner) wrapPlan(planCtx *PlanningCtx, n planNode) (*Physical if err := walkPlan(planCtx.ctx, n, planObserver{ enterNode: func(ctx context.Context, nodeName string, plan planNode) (bool, error) { switch plan.(type) { - case *explainVecNode, *explainPlanNode: + case *explainVecNode, *explainPlanNode, *explainDDLNode: // Don't continue recursing into explain nodes - they need to be left // alone since they handle their own planning later. return false, nil diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index b4ca6996fe19..9497585b55b3 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -767,6 +767,11 @@ func (e *distSQLSpecExecFactory) ConstructExplain( options: options, plan: *p, } + } else if options.Mode == tree.ExplainDDL { + explainNode = &explainDDLNode{ + options: options, + plan: *p, + } } else { explainNode = &explainPlanNode{ options: options, diff --git a/pkg/sql/explain_ddl.go b/pkg/sql/explain_ddl.go new file mode 100644 index 000000000000..ed2434c725d8 --- /dev/null +++ b/pkg/sql/explain_ddl.go @@ -0,0 +1,101 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "bytes" + "compress/gzip" + "context" + "encoding/base64" + "io" + "net/url" + + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraphviz" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +type explainDDLNode struct { + optColumnsSlot + options *tree.ExplainOptions + plan planComponents + run bool + values tree.Datums +} + +func (n *explainDDLNode) Next(params runParams) (bool, error) { + if n.run { + return false, nil + } + n.run = true + return true, nil +} + +func (n *explainDDLNode) Values() tree.Datums { + return n.values +} + +func (n *explainDDLNode) Close(ctx context.Context) { +} + +var _ planNode = (*explainDDLNode)(nil) + +func (n *explainDDLNode) startExec(params runParams) error { + b := scbuild.NewBuilder(params.p, params.p.SemaCtx(), params.p.EvalContext()) + var ts []*scpb.Node + var err error + switch n := params.p.stmt.AST.(*tree.Explain).Statement.(type) { + case *tree.AlterTable: + ts, err = b.AlterTable(params.ctx, params.extendedEvalCtx.SchemaChangerState.nodes, n) + default: + + } + if err != nil { + return err + } + sc, err := scplan.MakePlan(ts, scplan.Params{ + ExecutionPhase: scplan.PostCommitPhase, + // TODO(ajwerner): Populate created descriptors. + }) + if err != nil { + return err + } + var out string + if n.options.Flags[tree.ExplainFlagDeps] { + if out, err = scgraphviz.DrawDependencies(&sc); err != nil { + return err + } + } else { + if out, err = scgraphviz.DrawStages(&sc); err != nil { + return err + } + } + var buf bytes.Buffer + w := gzip.NewWriter(&buf) + if _, err := io.WriteString(w, out); err != nil { + return err + } + if err := w.Close(); err != nil { + return err + } + vizURL := (&url.URL{ + Scheme: "https", + Host: "cockroachdb.github.io", + Path: "scplan/viz.html", + Fragment: base64.StdEncoding.EncodeToString(buf.Bytes()), + }).String() + n.values = tree.Datums{ + tree.NewDString(vizURL), + } + return nil +} diff --git a/pkg/sql/logictest/testdata/logic_test/new_schema_changer b/pkg/sql/logictest/testdata/logic_test/new_schema_changer index 0369b73d5dd2..4fb6dd20ef6f 100644 --- a/pkg/sql/logictest/testdata/logic_test/new_schema_changer +++ b/pkg/sql/logictest/testdata/logic_test/new_schema_changer @@ -6,6 +6,12 @@ SET experimental_use_new_schema_changer = 'on' statement ok CREATE TABLE foo (i INT PRIMARY KEY) +statement ok +EXPLAIN (DDL) ALTER TABLE foo ADD COLUMN j INT + +statement ok +EXPLAIN (DDL, DEPS) ALTER TABLE foo ADD COLUMN j INT + statement ok ALTER TABLE foo ADD COLUMN j INT diff --git a/pkg/sql/opt/optbuilder/explain.go b/pkg/sql/opt/optbuilder/explain.go index 45cd11d1c7c2..c05c6abf1dcb 100644 --- a/pkg/sql/opt/optbuilder/explain.go +++ b/pkg/sql/opt/optbuilder/explain.go @@ -45,6 +45,12 @@ func (b *Builder) buildExplain(explain *tree.Explain, inScope *scope) (outScope case tree.ExplainVec: telemetry.Inc(sqltelemetry.ExplainVecUseCounter) + case tree.ExplainDDL: + if explain.Flags[tree.ExplainFlagDeps] { + telemetry.Inc(sqltelemetry.ExplainDDLDeps) + } else { + telemetry.Inc(sqltelemetry.ExplainDDLStages) + } default: panic(errors.Errorf("EXPLAIN mode %s not supported", explain.Mode)) diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index c7055b7e4c90..517daac12cb6 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -1893,6 +1893,13 @@ func (ef *execFactory) ConstructExplain( plan: *wrappedPlan, }, nil } + if options.Mode == tree.ExplainDDL { + wrappedPlan := plan.(*explain.Plan).WrappedPlan.(*planComponents) + return &explainDDLNode{ + options: options, + plan: *wrappedPlan, + }, nil + } flags := explain.MakeFlags(options) n := &explainPlanNode{ options: options, diff --git a/pkg/sql/plan.go b/pkg/sql/plan.go index dad792a9f885..de3bffa5d787 100644 --- a/pkg/sql/plan.go +++ b/pkg/sql/plan.go @@ -480,7 +480,7 @@ func startExec(params runParams, plan planNode) error { o := planObserver{ enterNode: func(ctx context.Context, _ string, p planNode) (bool, error) { switch p.(type) { - case *explainVecNode: + case *explainVecNode, *explainDDLNode: // Do not recurse: we're not starting the plan if we just show its structure with EXPLAIN. return false, nil case *showTraceNode: diff --git a/pkg/sql/plan_columns.go b/pkg/sql/plan_columns.go index 437b124e1b7e..99187e742dd6 100644 --- a/pkg/sql/plan_columns.go +++ b/pkg/sql/plan_columns.go @@ -97,6 +97,8 @@ func getPlanColumns(plan planNode, mut bool) colinfo.ResultColumns { // Nodes with a fixed schema. case *scrubNode: return n.getColumns(mut, colinfo.ScrubColumns) + case *explainDDLNode: + return n.getColumns(mut, colinfo.ExplainPlanColumns) case *explainPlanNode: return n.getColumns(mut, colinfo.ExplainPlanColumns) case *explainVecNode: diff --git a/pkg/sql/sem/tree/explain.go b/pkg/sql/sem/tree/explain.go index c68a66d40973..ba49dd358260 100644 --- a/pkg/sql/sem/tree/explain.go +++ b/pkg/sql/sem/tree/explain.go @@ -43,8 +43,7 @@ type ExplainOptions struct { Flags [numExplainFlags + 1]bool } -// ExplainMode indicates the mode of the explain. Currently there are two modes: -// PLAN (the default) and DISTSQL. +// ExplainMode indicates the mode of the explain. The default is ExplainPlan. type ExplainMode uint8 const ( @@ -69,6 +68,10 @@ const ( // EXPLAIN ANALYZE. ExplainDebug + // ExplainDDL generates a DDL plan diagram for the statement. Not allowed with + // + ExplainDDL + numExplainModes = iota ) @@ -78,6 +81,7 @@ var explainModeStrings = [...]string{ ExplainOpt: "OPT", ExplainVec: "VEC", ExplainDebug: "DEBUG", + ExplainDDL: "DDL", } var explainModeStringMap = func() map[string]ExplainMode { @@ -105,6 +109,8 @@ const ( ExplainFlagEnv ExplainFlagCatalog ExplainFlagJSON + ExplainFlagStages + ExplainFlagDeps numExplainFlags = iota ) @@ -114,6 +120,8 @@ var explainFlagStrings = [...]string{ ExplainFlagEnv: "ENV", ExplainFlagCatalog: "CATALOG", ExplainFlagJSON: "JSON", + ExplainFlagStages: "STAGES", + ExplainFlagDeps: "DEPS", } var explainFlagStringMap = func() map[string]ExplainFlag { diff --git a/pkg/sql/sqltelemetry/planning.go b/pkg/sql/sqltelemetry/planning.go index 02e28654ca6c..f056119cc953 100644 --- a/pkg/sql/sqltelemetry/planning.go +++ b/pkg/sql/sqltelemetry/planning.go @@ -106,6 +106,12 @@ var ExplainOptUseCounter = telemetry.GetCounterOnce("sql.plan.explain-opt") // ExplainVecUseCounter is to be incremented whenever EXPLAIN (VEC) is run. var ExplainVecUseCounter = telemetry.GetCounterOnce("sql.plan.explain-vec") +// ExplainDDLStages is to be incremented whenever EXPLAIN (DDL, STAGES) is run. +var ExplainDDLStages = telemetry.GetCounterOnce("sql.plan.explain-ddl-stages") + +// ExplainDDLDeps is to be incremented whenever EXPLAIN (DDL, DEPS) is run. +var ExplainDDLDeps = telemetry.GetCounterOnce("sql.plan.explain-ddl-deps") + // ExplainOptVerboseUseCounter is to be incremented whenever // EXPLAIN (OPT, VERBOSE) is run. var ExplainOptVerboseUseCounter = telemetry.GetCounterOnce("sql.plan.explain-opt-verbose") diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index e1cac78b9017..de6d110e03ef 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -235,6 +235,16 @@ func (v *planVisitor) visitInternal(plan planNode, name string) { } n.plan.main.planNode = v.visit(n.plan.main.planNode) + case *explainDDLNode: + // We check whether planNode is nil because the plan might be + // represented physically. We don't yet have a walker over such + // representation, so we simply short-circuit. + // TODO(yuzefovich): implement that walker and use it here. + if n.plan.main.planNode == nil { + return + } + n.plan.main.planNode = v.visit(n.plan.main.planNode) + case *ordinalityNode: n.source = v.visit(n.source) @@ -372,6 +382,7 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&errorIfRowsNode{}): "error if rows", reflect.TypeOf(&explainPlanNode{}): "explain plan", reflect.TypeOf(&explainVecNode{}): "explain vectorized", + reflect.TypeOf(&explainDDLNode{}): "explain ddl", reflect.TypeOf(&exportNode{}): "export", reflect.TypeOf(&filterNode{}): "filter", reflect.TypeOf(&GrantRoleNode{}): "grant role",