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/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.enable | boolean | false | if set, traces for recent requests can be seen at https:///debug/requests |
trace.lightstep.token | string |
| if set, traces go to Lightstep using this token |
trace.zipkin.collector | string |
| if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set |
-version | version | 20.2-16 | set the active cluster version in the format '.' |
+version | version | 20.2-18 | set the active cluster version in the format '.' |
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/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/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/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/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..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",
@@ -98,6 +99,7 @@ go_library(
"grant_revoke.go",
"grant_role.go",
"group.go",
+ "index_backfiller.go",
"index_join.go",
"information_schema.go",
"insert.go",
@@ -154,8 +156,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",
@@ -315,6 +319,11 @@ go_library(
"//pkg/sql/rowenc",
"//pkg/sql/rowexec",
"//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",
"//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/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.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/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
+}
diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go
index 7fcfd37f0c1b..51a2e2165dd9 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(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(),
+ 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/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_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/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/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/exec_util.go b/pkg/sql/exec_util.go
index bc3d05bed842..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"
@@ -782,6 +795,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.
@@ -2235,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/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/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/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/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/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/new_schema_changer b/pkg/sql/logictest/testdata/logic_test/new_schema_changer
new file mode 100644
index 000000000000..4fb6dd20ef6f
--- /dev/null
+++ b/pkg/sql/logictest/testdata/logic_test/new_schema_changer
@@ -0,0 +1,183 @@
+subtest add_column
+
+statement ok
+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
+
+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
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/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/planner.go b/pkg/sql/planner.go
index c387992bbbe4..ab3c9a840f67 100644
--- a/pkg/sql/planner.go
+++ b/pkg/sql/planner.go
@@ -95,27 +95,31 @@ type extendedEvalContext struct {
schemaAccessors *schemaInterface
sqlStatsCollector *sqlStatsCollector
+
+ SchemaChangerState *SchemaChangerState
}
// 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
}
@@ -220,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/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)
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/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.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/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.
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/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/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..67aba6af8414
--- /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.StatementPhase,
+ 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.StatementPhase,
+ 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)
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
+}
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" -}}
+
+{{- end -}}
+
+
+{{- template "header" . -}}
+
+{{- template "mapVal" (toMap .) -}}
+ |
+
+`))
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/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