From 755fa250f5f5b12e820cfc4a7b7b9969671ddf28 Mon Sep 17 00:00:00 2001 From: arulajmani Date: Thu, 11 Feb 2021 13:36:13 -0500 Subject: [PATCH 1/2] sql: ensure type schema change cleanup job is resilient to retries Previously if the type schema changer ran into a non-permanent error, it wouldn't retry transparently. Instead, manual cleanup would be required. This patch fixes this behavior. This patch also adds a testing knob, `RunAfterOnFailOrCancel` to test the afformentioned bug. Fixes #60489 Release note (bug fix): Previosly, retryable errors in the cleanup phase of the type schema changer wouldn't be retried automatically in the background. This is now fixed. --- pkg/sql/type_change.go | 42 +++++++++++++++++++---- pkg/sql/type_change_test.go | 66 +++++++++++++++++++++++++++++++++++++ 2 files changed, 102 insertions(+), 6 deletions(-) diff --git a/pkg/sql/type_change.go b/pkg/sql/type_change.go index 2f9c8490a5ec..7380bc2dc96e 100644 --- a/pkg/sql/type_change.go +++ b/pkg/sql/type_change.go @@ -162,6 +162,9 @@ type TypeSchemaChangerTestingKnobs struct { // RunBeforeEnumMemberPromotion runs before enum members are promoted from // readable to all permissions in the typeSchemaChanger. RunBeforeEnumMemberPromotion func() + // RunAfterOnFailOrCancel runs after OnFailOrCancel completes, if + // OnFailOrCancel is triggered. + RunAfterOnFailOrCancel func() error } // ModuleTestingKnobs implements the ModuleTestingKnobs interface. @@ -612,14 +615,41 @@ func (t *typeChangeResumer) OnFailOrCancel(ctx context.Context, execCtx interfac execCfg: execCtx.(JobExecContext).ExecCfg(), } - if err := tc.cleanupEnumValues(ctx); err != nil { - return err + if rollbackErr := func() error { + if err := tc.cleanupEnumValues(ctx); err != nil { + return err + } + + if err := drainNamesForDescriptor( + ctx, tc.execCfg.Settings, tc.typeID, tc.execCfg.DB, + tc.execCfg.InternalExecutor, tc.execCfg.LeaseManager, tc.execCfg.Codec, nil, + ); err != nil { + return err + } + + if fn := tc.execCfg.TypeSchemaChangerTestingKnobs.RunAfterOnFailOrCancel; fn != nil { + return fn() + } + + return nil + }(); rollbackErr != nil { + switch { + case errors.Is(rollbackErr, catalog.ErrDescriptorNotFound): + // If the descriptor for the ID can't be found, we assume that another + // job executed already and dropped the type. + log.Infof( + ctx, + "descriptor %d not found for type change job; assuming it was dropped, and exiting", + tc.typeID, + ) + case !isPermanentSchemaChangeError(rollbackErr): + return jobs.NewRetryJobError(rollbackErr.Error()) + default: + return rollbackErr + } } - return drainNamesForDescriptor( - ctx, tc.execCfg.Settings, tc.typeID, tc.execCfg.DB, - tc.execCfg.InternalExecutor, tc.execCfg.LeaseManager, tc.execCfg.Codec, nil, - ) + return nil } func init() { diff --git a/pkg/sql/type_change_test.go b/pkg/sql/type_change_test.go index 2851f3fc1910..466e77f8e1f9 100644 --- a/pkg/sql/type_change_test.go +++ b/pkg/sql/type_change_test.go @@ -12,6 +12,7 @@ package sql_test import ( "context" + "fmt" "testing" "github.com/cockroachdb/cockroach/pkg/keys" @@ -26,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" ) // TestDrainingNamesAreCleanedTypeChangeOnFailure ensures that draining names @@ -158,6 +160,70 @@ CREATE TYPE d.t AS ENUM(); } } +// TestFailedTypeSchemaChangeRetriesTransparently fails the initial type schema +// change operation and then tests that if the cleanup job runs into a +// non-permanent error, it is retried transparently. +func TestFailedTypeSchemaChangeRetriesTransparently(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Decrease the adopt loop interval so that retries happen quickly. + defer sqltestutils.SetTestJobsAdoptInterval()() + + ctx := context.Background() + // Protects errReturned. + var mu syncutil.Mutex + // Ensures just the first try to cleanup returns a retryable error. + errReturned := false + params, _ := tests.CreateTestServerParams() + cleanupSuccessfullyFinished := make(chan struct{}) + params.Knobs.SQLTypeSchemaChanger = &sql.TypeSchemaChangerTestingKnobs{ + RunBeforeExec: func() error { + return errors.New("yikes") + }, + RunAfterOnFailOrCancel: func() error { + mu.Lock() + defer mu.Unlock() + if errReturned { + return nil + } + errReturned = true + close(cleanupSuccessfullyFinished) + return context.DeadlineExceeded + }, + } + + s, sqlDB, _ := serverutils.StartServer(t, params) + defer s.Stopper().Stop(ctx) + + // Create a type. + _, err := sqlDB.Exec(` +CREATE DATABASE d; +CREATE TYPE d.t AS ENUM(); +`) + require.NoError(t, err) + + // The initial drop should fail. + _, err = sqlDB.Exec(`DROP TYPE d.t`) + testutils.IsError(err, "yikes") + + // The cleanup job, which is expected to drain names, should retry + // transparently. + <-cleanupSuccessfullyFinished + + // type descriptor name + array alias name. + namespaceEntries := []string{"t", "_t"} + for _, name := range namespaceEntries { + rows := sqlDB.QueryRow(fmt.Sprintf(`SELECT count(*) FROM system.namespace WHERE name= '%s'`, name)) + var count int + err = rows.Scan(&count) + require.NoError(t, err) + if count != 0 { + t.Fatalf("expected namespace entries to be cleaned up for type desc %q", name) + } + } +} + func TestAddDropValuesInTransaction(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) From 84948227f9ee8fd6de5cd03b2257d7306b172885 Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Fri, 12 Feb 2021 16:09:54 -0500 Subject: [PATCH 2/2] builtins: change complete_stream builtin to take a timestamp This change adds a ts parameter to crdb_internal.complete_stream_ingestion_job builtin. This ts will be the ts as of which the cluster being ingested into will be considered in a consistent state. Release note: None --- docs/generated/sql/functions.md | 2 +- pkg/ccl/BUILD.bazel | 1 + pkg/ccl/ccl_init.go | 1 + .../streamingccl/streamingutils/BUILD.bazel | 1 + pkg/ccl/streamingccl/streamingutils/utils.go | 7 +- .../streamingccl/streamingutils/utils_test.go | 12 +- pkg/jobs/jobspb/jobs.pb.go | 1073 +++++++++-------- pkg/jobs/jobspb/jobs.proto | 10 +- pkg/sql/sem/builtins/BUILD.bazel | 1 + pkg/sql/sem/builtins/builtins.go | 15 +- pkg/streaming/BUILD.bazel | 1 + pkg/streaming/utils.go | 3 +- 12 files changed, 577 insertions(+), 550 deletions(-) diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 56c135732769..93497b21bd88 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -2274,7 +2274,7 @@ The swap_ordinate_string parameter is a 2-character string naming the ordinates -
Function → ReturnsDescription
crdb_internal.complete_stream_ingestion_job(job_id: int) → int

This function can be used to signal a running stream ingestion job to complete. The job will eventually stop ingesting, revert to the latest resolved timestamp and leave the cluster in a consistent state. This function does not wait for the job to reach a terminal state, but instead returns the job id as soon as it has signaled the job to complete. This builtin can be used in conjunction with SHOW JOBS WHEN COMPLETE to ensure that the job has left the cluster in a consistent state.

+
crdb_internal.complete_stream_ingestion_job(job_id: int, cutover_ts: timestamptz) → int

This function can be used to signal a running stream ingestion job to complete. The job will eventually stop ingesting, revert to the specified timestamp and leave the cluster in a consistent state. The specified timestamp can only be specified up to the microsecond. This function does not wait for the job to reach a terminal state, but instead returns the job id as soon as it has signaled the job to complete. This builtin can be used in conjunction with SHOW JOBS WHEN COMPLETE to ensure that the job has left the cluster in a consistent state.

diff --git a/pkg/ccl/BUILD.bazel b/pkg/ccl/BUILD.bazel index 2eabdbd72da4..f6190ee414b8 100644 --- a/pkg/ccl/BUILD.bazel +++ b/pkg/ccl/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/ccl/storageccl", "//pkg/ccl/storageccl/engineccl", "//pkg/ccl/streamingccl/streamingest", + "//pkg/ccl/streamingccl/streamingutils", "//pkg/ccl/utilccl", "//pkg/ccl/workloadccl", ], diff --git a/pkg/ccl/ccl_init.go b/pkg/ccl/ccl_init.go index 226a3e1fad2c..224be65c492d 100644 --- a/pkg/ccl/ccl_init.go +++ b/pkg/ccl/ccl_init.go @@ -26,6 +26,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamingest" + _ "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamingutils" _ "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/workloadccl" ) diff --git a/pkg/ccl/streamingccl/streamingutils/BUILD.bazel b/pkg/ccl/streamingccl/streamingutils/BUILD.bazel index 7b0a96fb789d..4bf25f325dd5 100644 --- a/pkg/ccl/streamingccl/streamingutils/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamingutils/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/kv", "//pkg/sql/sem/tree", "//pkg/streaming", + "//pkg/util/hlc", "//pkg/util/protoutil", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/ccl/streamingccl/streamingutils/utils.go b/pkg/ccl/streamingccl/streamingutils/utils.go index 00ec099a850a..2056bac36bdd 100644 --- a/pkg/ccl/streamingccl/streamingutils/utils.go +++ b/pkg/ccl/streamingccl/streamingutils/utils.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/streaming" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -23,7 +24,9 @@ func init() { streaming.CompleteIngestionHook = doCompleteIngestion } -func doCompleteIngestion(evalCtx *tree.EvalContext, txn *kv.Txn, jobID int) error { +func doCompleteIngestion( + evalCtx *tree.EvalContext, txn *kv.Txn, jobID int, cutoverTimestamp hlc.Timestamp, +) error { // Get the job payload for job_id. const jobsQuery = `SELECT progress FROM system.jobs WHERE id=$1 FOR UPDATE` row, err := evalCtx.InternalExecutor.QueryRow(evalCtx.Context, @@ -49,7 +52,7 @@ func doCompleteIngestion(evalCtx *tree.EvalContext, txn *kv.Txn, jobID int) erro // Update the sentinel being polled by the stream ingestion job to // check if a complete has been signaled. - sp.StreamIngest.MarkedForCompletion = true + sp.StreamIngest.CutoverTime = cutoverTimestamp progress.ModifiedMicros = timeutil.ToUnixMicros(txn.ReadTimestamp().GoTime()) progressBytes, err := protoutil.Marshal(progress) if err != nil { diff --git a/pkg/ccl/streamingccl/streamingutils/utils_test.go b/pkg/ccl/streamingccl/streamingutils/utils_test.go index 6fe2e857892d..f4b89fd1100e 100644 --- a/pkg/ccl/streamingccl/streamingutils/utils_test.go +++ b/pkg/ccl/streamingccl/streamingutils/utils_test.go @@ -57,13 +57,14 @@ func TestCutoverBuiltin(t *testing.T) { progress := job.Progress() sp, ok := progress.GetDetails().(*jobspb.Progress_StreamIngest) require.True(t, ok) - require.False(t, sp.StreamIngest.MarkedForCompletion) + require.True(t, sp.StreamIngest.CutoverTime.IsEmpty()) + cutoverTime := timeutil.Now() var jobID int err = db.QueryRowContext( ctx, - `SELECT crdb_internal.complete_stream_ingestion_job($1)`, - *job.ID()).Scan(&jobID) + `SELECT crdb_internal.complete_stream_ingestion_job($1, $2)`, + *job.ID(), cutoverTime).Scan(&jobID) require.NoError(t, err) require.Equal(t, *job.ID(), int64(jobID)) @@ -73,5 +74,8 @@ func TestCutoverBuiltin(t *testing.T) { progress = sj.Progress() sp, ok = progress.GetDetails().(*jobspb.Progress_StreamIngest) require.True(t, ok) - require.True(t, sp.StreamIngest.MarkedForCompletion) + // The builtin only offers microsecond precision and so we must account for + // that when comparing against our chosen time. + cutoverTime = cutoverTime.Round(time.Microsecond) + require.Equal(t, hlc.Timestamp{WallTime: cutoverTime.UnixNano()}, sp.StreamIngest.CutoverTime) } diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index eeaa3c29517c..518840216947 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -58,7 +58,7 @@ func (x EncryptionMode) String() string { return proto.EnumName(EncryptionMode_name, int32(x)) } func (EncryptionMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_4b9f65bb42f26fe6, []int{0} + return fileDescriptor_jobs_98841b04a5cce477, []int{0} } type Status int32 @@ -87,7 +87,7 @@ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } func (Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_4b9f65bb42f26fe6, []int{1} + return fileDescriptor_jobs_98841b04a5cce477, []int{1} } type Type int32 @@ -142,7 +142,7 @@ var Type_value = map[string]int32{ } func (Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_4b9f65bb42f26fe6, []int{2} + return fileDescriptor_jobs_98841b04a5cce477, []int{2} } type EncryptionInfo_Scheme int32 @@ -162,7 +162,7 @@ func (x EncryptionInfo_Scheme) String() string { return proto.EnumName(EncryptionInfo_Scheme_name, int32(x)) } func (EncryptionInfo_Scheme) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_4b9f65bb42f26fe6, []int{2, 0} + return fileDescriptor_jobs_98841b04a5cce477, []int{2, 0} } type SchemaChangeGCProgress_Status int32 @@ -192,7 +192,7 @@ func (x SchemaChangeGCProgress_Status) String() string { return proto.EnumName(SchemaChangeGCProgress_Status_name, int32(x)) } func (SchemaChangeGCProgress_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_4b9f65bb42f26fe6, []int{22, 0} + return fileDescriptor_jobs_98841b04a5cce477, []int{22, 0} } type Lease struct { @@ -206,7 +206,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_4b9f65bb42f26fe6, []int{0} + return fileDescriptor_jobs_98841b04a5cce477, []int{0} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -247,7 +247,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_4b9f65bb42f26fe6, []int{1} + return fileDescriptor_jobs_98841b04a5cce477, []int{1} } func (m *BackupEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -281,7 +281,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_4b9f65bb42f26fe6, []int{1, 0} + return fileDescriptor_jobs_98841b04a5cce477, []int{1, 0} } func (m *BackupEncryptionOptions_KMSInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -321,7 +321,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_4b9f65bb42f26fe6, []int{2} + return fileDescriptor_jobs_98841b04a5cce477, []int{2} } func (m *EncryptionInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -364,7 +364,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_4b9f65bb42f26fe6, []int{3} + return fileDescriptor_jobs_98841b04a5cce477, []int{3} } func (m *StreamIngestionDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -390,18 +390,18 @@ func (m *StreamIngestionDetails) XXX_DiscardUnknown() { var xxx_messageInfo_StreamIngestionDetails proto.InternalMessageInfo type StreamIngestionProgress struct { - // MarkedForCompletion is used to signal to the stream ingestion job to - // complete its ingestion. This involves stopping any subsequent ingestion, - // and rolling back to the latest resolved ts to bring the ingested cluster to - // a consistent state. - MarkedForCompletion bool `protobuf:"varint,1,opt,name=markedForCompletion,proto3" json:"markedForCompletion,omitempty"` + // CutoverTime is set to signal to the stream ingestion job to complete its + // ingestion. This involves stopping any subsequent ingestion, and rolling + // back any additional ingested data, to bring the ingested cluster to a + // consistent state as of the CutoverTime. + CutoverTime hlc.Timestamp `protobuf:"bytes,1,opt,name=cutover_time,json=cutoverTime,proto3" json:"cutover_time"` } 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_4b9f65bb42f26fe6, []int{4} + return fileDescriptor_jobs_98841b04a5cce477, []int{4} } func (m *StreamIngestionProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -456,7 +456,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_4b9f65bb42f26fe6, []int{5} + return fileDescriptor_jobs_98841b04a5cce477, []int{5} } func (m *BackupDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -488,7 +488,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_4b9f65bb42f26fe6, []int{6} + return fileDescriptor_jobs_98841b04a5cce477, []int{6} } func (m *BackupProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -550,7 +550,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_4b9f65bb42f26fe6, []int{7} + return fileDescriptor_jobs_98841b04a5cce477, []int{7} } func (m *RestoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -587,7 +587,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_4b9f65bb42f26fe6, []int{7, 0} + return fileDescriptor_jobs_98841b04a5cce477, []int{7, 0} } func (m *RestoreDetails_DescriptorRewrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -620,7 +620,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_4b9f65bb42f26fe6, []int{7, 1} + return fileDescriptor_jobs_98841b04a5cce477, []int{7, 1} } func (m *RestoreDetails_BackupLocalityInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -653,7 +653,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_4b9f65bb42f26fe6, []int{8} + return fileDescriptor_jobs_98841b04a5cce477, []int{8} } func (m *RestoreProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -714,7 +714,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_4b9f65bb42f26fe6, []int{9} + return fileDescriptor_jobs_98841b04a5cce477, []int{9} } func (m *ImportDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -752,7 +752,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_4b9f65bb42f26fe6, []int{9, 0} + return fileDescriptor_jobs_98841b04a5cce477, []int{9, 0} } func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -794,7 +794,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_4b9f65bb42f26fe6, []int{10} + return fileDescriptor_jobs_98841b04a5cce477, []int{10} } func (m *SequenceValChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -830,7 +830,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_4b9f65bb42f26fe6, []int{11} + return fileDescriptor_jobs_98841b04a5cce477, []int{11} } func (m *SequenceDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -865,7 +865,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_4b9f65bb42f26fe6, []int{11, 0} + return fileDescriptor_jobs_98841b04a5cce477, []int{11, 0} } func (m *SequenceDetails_SequenceChunks) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -911,7 +911,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_4b9f65bb42f26fe6, []int{12} + return fileDescriptor_jobs_98841b04a5cce477, []int{12} } func (m *ImportProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -954,7 +954,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_4b9f65bb42f26fe6, []int{13} + return fileDescriptor_jobs_98841b04a5cce477, []int{13} } func (m *TypeSchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -987,7 +987,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_4b9f65bb42f26fe6, []int{14} + return fileDescriptor_jobs_98841b04a5cce477, []int{14} } func (m *TypeSchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1021,7 +1021,7 @@ 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_4b9f65bb42f26fe6, []int{15} + return fileDescriptor_jobs_98841b04a5cce477, []int{15} } func (m *NewSchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1055,7 +1055,7 @@ 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_4b9f65bb42f26fe6, []int{16} + return fileDescriptor_jobs_98841b04a5cce477, []int{16} } func (m *NewSchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1088,7 +1088,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_4b9f65bb42f26fe6, []int{17} + return fileDescriptor_jobs_98841b04a5cce477, []int{17} } func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1123,7 +1123,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_4b9f65bb42f26fe6, []int{18} + return fileDescriptor_jobs_98841b04a5cce477, []int{18} } func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1186,7 +1186,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_4b9f65bb42f26fe6, []int{19} + return fileDescriptor_jobs_98841b04a5cce477, []int{19} } func (m *SchemaChangeGCDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1220,7 +1220,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_4b9f65bb42f26fe6, []int{19, 0} + return fileDescriptor_jobs_98841b04a5cce477, []int{19, 0} } func (m *SchemaChangeGCDetails_DroppedIndex) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1254,7 +1254,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_4b9f65bb42f26fe6, []int{19, 1} + return fileDescriptor_jobs_98841b04a5cce477, []int{19, 1} } func (m *SchemaChangeGCDetails_DroppedID) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1288,7 +1288,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_4b9f65bb42f26fe6, []int{19, 2} + return fileDescriptor_jobs_98841b04a5cce477, []int{19, 2} } func (m *SchemaChangeGCDetails_DroppedTenant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1352,7 +1352,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_4b9f65bb42f26fe6, []int{20} + return fileDescriptor_jobs_98841b04a5cce477, []int{20} } func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1384,7 +1384,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_4b9f65bb42f26fe6, []int{21} + return fileDescriptor_jobs_98841b04a5cce477, []int{21} } func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1422,7 +1422,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_4b9f65bb42f26fe6, []int{22} + return fileDescriptor_jobs_98841b04a5cce477, []int{22} } func (m *SchemaChangeGCProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1456,7 +1456,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_4b9f65bb42f26fe6, []int{22, 0} + return fileDescriptor_jobs_98841b04a5cce477, []int{22, 0} } func (m *SchemaChangeGCProgress_IndexProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1490,7 +1490,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_4b9f65bb42f26fe6, []int{22, 1} + return fileDescriptor_jobs_98841b04a5cce477, []int{22, 1} } func (m *SchemaChangeGCProgress_TableProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1523,7 +1523,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_4b9f65bb42f26fe6, []int{22, 2} + return fileDescriptor_jobs_98841b04a5cce477, []int{22, 2} } func (m *SchemaChangeGCProgress_TenantProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1556,7 +1556,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_4b9f65bb42f26fe6, []int{23} + return fileDescriptor_jobs_98841b04a5cce477, []int{23} } func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1609,7 +1609,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_4b9f65bb42f26fe6, []int{24} + return fileDescriptor_jobs_98841b04a5cce477, []int{24} } func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1644,7 +1644,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_4b9f65bb42f26fe6, []int{25} + return fileDescriptor_jobs_98841b04a5cce477, []int{25} } func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1687,7 +1687,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_4b9f65bb42f26fe6, []int{26} + return fileDescriptor_jobs_98841b04a5cce477, []int{26} } func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1731,7 +1731,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_4b9f65bb42f26fe6, []int{27} + return fileDescriptor_jobs_98841b04a5cce477, []int{27} } func (m *CreateStatsDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1772,7 +1772,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_4b9f65bb42f26fe6, []int{27, 0} + return fileDescriptor_jobs_98841b04a5cce477, []int{27, 0} } func (m *CreateStatsDetails_ColStat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1804,7 +1804,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_4b9f65bb42f26fe6, []int{28} + return fileDescriptor_jobs_98841b04a5cce477, []int{28} } func (m *CreateStatsProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1837,7 +1837,7 @@ func (m *MigrationDetails) Reset() { *m = MigrationDetails{} } func (m *MigrationDetails) String() string { return proto.CompactTextString(m) } func (*MigrationDetails) ProtoMessage() {} func (*MigrationDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4b9f65bb42f26fe6, []int{29} + return fileDescriptor_jobs_98841b04a5cce477, []int{29} } func (m *MigrationDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1869,7 +1869,7 @@ func (m *MigrationProgress) Reset() { *m = MigrationProgress{} } func (m *MigrationProgress) String() string { return proto.CompactTextString(m) } func (*MigrationProgress) ProtoMessage() {} func (*MigrationProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_4b9f65bb42f26fe6, []int{30} + return fileDescriptor_jobs_98841b04a5cce477, []int{30} } func (m *MigrationProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1939,7 +1939,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_4b9f65bb42f26fe6, []int{31} + return fileDescriptor_jobs_98841b04a5cce477, []int{31} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2371,7 +2371,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_4b9f65bb42f26fe6, []int{32} + return fileDescriptor_jobs_98841b04a5cce477, []int{32} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2867,7 +2867,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_4b9f65bb42f26fe6, []int{33} + return fileDescriptor_jobs_98841b04a5cce477, []int{33} } func (m *Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3280,16 +3280,14 @@ func (m *StreamIngestionProgress) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.MarkedForCompletion { - dAtA[i] = 0x8 - i++ - if m.MarkedForCompletion { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ + dAtA[i] = 0xa + i++ + i = encodeVarintJobs(dAtA, i, uint64(m.CutoverTime.Size())) + n4, err := m.CutoverTime.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } + i += n4 return i, nil } @@ -3311,19 +3309,19 @@ func (m *BackupDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintJobs(dAtA, i, uint64(m.StartTime.Size())) - n4, err := m.StartTime.MarshalTo(dAtA[i:]) + n5, err := m.StartTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n4 + i += n5 dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.EndTime.Size())) - n5, err := m.EndTime.MarshalTo(dAtA[i:]) + n6, err := m.EndTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n5 + i += n6 if len(m.URI) > 0 { dAtA[i] = 0x1a i++ @@ -3362,21 +3360,21 @@ func (m *BackupDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintJobs(dAtA, i, uint64(m.EncryptionOptions.Size())) - n6, err := m.EncryptionOptions.MarshalTo(dAtA[i:]) + n7, err := m.EncryptionOptions.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n6 + i += n7 } if m.ProtectedTimestampRecord != nil { dAtA[i] = 0x3a i++ i = encodeVarintJobs(dAtA, i, uint64(m.ProtectedTimestampRecord.Size())) - n7, err := m.ProtectedTimestampRecord.MarshalTo(dAtA[i:]) + n8, err := m.ProtectedTimestampRecord.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n7 + i += n8 } if len(m.CollectionURI) > 0 { dAtA[i] = 0x42 @@ -3388,11 +3386,11 @@ func (m *BackupDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintJobs(dAtA, i, uint64(m.EncryptionInfo.Size())) - n8, err := m.EncryptionInfo.MarshalTo(dAtA[i:]) + n9, err := m.EncryptionInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n8 + i += n9 } return i, nil } @@ -3454,11 +3452,11 @@ func (m *RestoreDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(v.Size())) - n9, err := v.MarshalTo(dAtA[i:]) + n10, err := v.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n9 + i += n10 } } } @@ -3480,11 +3478,11 @@ func (m *RestoreDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintJobs(dAtA, i, uint64(m.EndTime.Size())) - n10, err := m.EndTime.MarshalTo(dAtA[i:]) + n11, err := m.EndTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n10 + i += n11 if len(m.TableDescs) > 0 { for _, msg := range m.TableDescs { dAtA[i] = 0x2a @@ -3554,11 +3552,11 @@ func (m *RestoreDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Encryption.Size())) - n11, err := m.Encryption.MarshalTo(dAtA[i:]) + n12, err := m.Encryption.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n12 } if len(m.Tenants) > 0 { for _, msg := range m.Tenants { @@ -3760,11 +3758,11 @@ func (m *ImportDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Format.Size())) - n12, err := m.Format.MarshalTo(dAtA[i:]) + n13, err := m.Format.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n12 + i += n13 if m.SSTSize != 0 { dAtA[i] = 0x20 i++ @@ -3855,11 +3853,11 @@ func (m *ImportDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.ProtectedTimestampRecord.Size())) - n13, err := m.ProtectedTimestampRecord.MarshalTo(dAtA[i:]) + n14, err := m.ProtectedTimestampRecord.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n13 + i += n14 } return i, nil } @@ -3883,11 +3881,11 @@ func (m *ImportDetails_Table) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintJobs(dAtA, i, uint64(m.Desc.Size())) - n14, err := m.Desc.MarshalTo(dAtA[i:]) + n15, err := m.Desc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n14 + i += n15 } if len(m.Name) > 0 { dAtA[i] = 0x92 @@ -4025,11 +4023,11 @@ func (m *SequenceDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(v.Size())) - n15, err := v.MarshalTo(dAtA[i:]) + n16, err := v.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n15 + i += n16 } } } @@ -4086,8 +4084,8 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintJobs(dAtA, i, uint64(len(m.SamplingProgress)*4)) for _, num := range m.SamplingProgress { - f16 := math.Float32bits(float32(num)) - encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f16)) + f17 := math.Float32bits(float32(num)) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f17)) i += 4 } } @@ -4096,8 +4094,8 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintJobs(dAtA, i, uint64(len(m.ReadProgress)*4)) for _, num := range m.ReadProgress { - f17 := math.Float32bits(float32(num)) - encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f17)) + f18 := math.Float32bits(float32(num)) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f18)) i += 4 } } @@ -4106,8 +4104,8 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintJobs(dAtA, i, uint64(len(m.WriteProgress)*4)) for _, num := range m.WriteProgress { - f18 := math.Float32bits(float32(num)) - encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f18)) + f19 := math.Float32bits(float32(num)) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f19)) i += 4 } } @@ -4124,22 +4122,22 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { } } if len(m.ResumePos) > 0 { - dAtA20 := make([]byte, len(m.ResumePos)*10) - var j19 int + dAtA21 := make([]byte, len(m.ResumePos)*10) + var j20 int for _, num1 := range m.ResumePos { num := uint64(num1) for num >= 1<<7 { - dAtA20[j19] = uint8(uint64(num)&0x7f | 0x80) + dAtA21[j20] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j19++ + j20++ } - dAtA20[j19] = uint8(num) - j19++ + dAtA21[j20] = uint8(num) + j20++ } dAtA[i] = 0x2a i++ - i = encodeVarintJobs(dAtA, i, uint64(j19)) - i += copy(dAtA[i:], dAtA20[:j19]) + i = encodeVarintJobs(dAtA, i, uint64(j20)) + i += copy(dAtA[i:], dAtA21[:j20]) } if len(m.SequenceDetails) > 0 { for _, msg := range m.SequenceDetails { @@ -4251,21 +4249,21 @@ func (m *NewSchemaChangeProgress) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.States) > 0 { - dAtA22 := make([]byte, len(m.States)*10) - var j21 int + dAtA23 := make([]byte, len(m.States)*10) + var j22 int for _, num := range m.States { for num >= 1<<7 { - dAtA22[j21] = uint8(uint64(num)&0x7f | 0x80) + dAtA23[j22] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j21++ + j22++ } - dAtA22[j21] = uint8(num) - j21++ + dAtA23[j22] = uint8(num) + j22++ } dAtA[i] = 0xa i++ - i = encodeVarintJobs(dAtA, i, uint64(j21)) - i += copy(dAtA[i:], dAtA22[:j21]) + i = encodeVarintJobs(dAtA, i, uint64(j22)) + i += copy(dAtA[i:], dAtA23[:j22]) } return i, nil } @@ -4382,11 +4380,11 @@ func (m *SchemaChangeGCDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintJobs(dAtA, i, uint64(m.InterleavedTable.Size())) - n23, err := m.InterleavedTable.MarshalTo(dAtA[i:]) + n24, err := m.InterleavedTable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n23 + i += n24 } if len(m.InterleavedIndexes) > 0 { for _, msg := range m.InterleavedIndexes { @@ -4404,11 +4402,11 @@ func (m *SchemaChangeGCDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Tenant.Size())) - n24, err := m.Tenant.MarshalTo(dAtA[i:]) + n25, err := m.Tenant.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n24 + i += n25 } return i, nil } @@ -4557,38 +4555,38 @@ func (m *SchemaChangeDetails) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(m.FormatVersion)) } if len(m.DroppedTypes) > 0 { - dAtA26 := make([]byte, len(m.DroppedTypes)*10) - var j25 int + dAtA27 := make([]byte, len(m.DroppedTypes)*10) + var j26 int for _, num := range m.DroppedTypes { for num >= 1<<7 { - dAtA26[j25] = uint8(uint64(num)&0x7f | 0x80) + dAtA27[j26] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j25++ + j26++ } - dAtA26[j25] = uint8(num) - j25++ + dAtA27[j26] = uint8(num) + j26++ } dAtA[i] = 0x42 i++ - i = encodeVarintJobs(dAtA, i, uint64(j25)) - i += copy(dAtA[i:], dAtA26[:j25]) + i = encodeVarintJobs(dAtA, i, uint64(j26)) + i += copy(dAtA[i:], dAtA27[:j26]) } if len(m.DroppedSchemas) > 0 { - dAtA28 := make([]byte, len(m.DroppedSchemas)*10) - var j27 int + dAtA29 := make([]byte, len(m.DroppedSchemas)*10) + var j28 int for _, num := range m.DroppedSchemas { for num >= 1<<7 { - dAtA28[j27] = uint8(uint64(num)&0x7f | 0x80) + dAtA29[j28] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j27++ + j28++ } - dAtA28[j27] = uint8(num) - j27++ + dAtA29[j28] = uint8(num) + j28++ } dAtA[i] = 0x4a i++ - i = encodeVarintJobs(dAtA, i, uint64(j27)) - i += copy(dAtA[i:], dAtA28[:j27]) + i = encodeVarintJobs(dAtA, i, uint64(j28)) + i += copy(dAtA[i:], dAtA29[:j28]) } return i, nil } @@ -4654,11 +4652,11 @@ func (m *SchemaChangeGCProgress) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Tenant.Size())) - n29, err := m.Tenant.MarshalTo(dAtA[i:]) + n30, err := m.Tenant.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n30 } return i, nil } @@ -4832,21 +4830,21 @@ func (m *ChangefeedDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64((&v).Size())) - n30, err := (&v).MarshalTo(dAtA[i:]) + n31, err := (&v).MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n30 + i += n31 } } dAtA[i] = 0x3a i++ i = encodeVarintJobs(dAtA, i, uint64(m.StatementTime.Size())) - n31, err := m.StatementTime.MarshalTo(dAtA[i:]) + n32, err := m.StatementTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n31 + i += n32 return i, nil } @@ -4868,19 +4866,19 @@ func (m *ResolvedSpan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintJobs(dAtA, i, uint64(m.Span.Size())) - n32, err := m.Span.MarshalTo(dAtA[i:]) + n33, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n32 + i += n33 dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Timestamp.Size())) - n33, err := m.Timestamp.MarshalTo(dAtA[i:]) + n34, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n33 + i += n34 if m.BoundaryReached { dAtA[i] = 0x18 i++ @@ -4924,11 +4922,11 @@ func (m *ChangefeedProgress) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.ProtectedTimestampRecord.Size())) - n34, err := m.ProtectedTimestampRecord.MarshalTo(dAtA[i:]) + n35, err := m.ProtectedTimestampRecord.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n34 + i += n35 return i, nil } @@ -4956,11 +4954,11 @@ func (m *CreateStatsDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Table.Size())) - n35, err := m.Table.MarshalTo(dAtA[i:]) + n36, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n35 + i += n36 if len(m.ColumnStats) > 0 { for _, msg := range m.ColumnStats { dAtA[i] = 0x1a @@ -4983,11 +4981,11 @@ func (m *CreateStatsDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintJobs(dAtA, i, uint64(m.AsOf.Size())) - n36, err := m.AsOf.MarshalTo(dAtA[i:]) + n37, err := m.AsOf.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n36 + i += n37 } if len(m.FQTableName) > 0 { dAtA[i] = 0x32 @@ -5020,21 +5018,21 @@ func (m *CreateStatsDetails_ColStat) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.ColumnIDs) > 0 { - dAtA38 := make([]byte, len(m.ColumnIDs)*10) - var j37 int + dAtA39 := make([]byte, len(m.ColumnIDs)*10) + var j38 int for _, num := range m.ColumnIDs { for num >= 1<<7 { - dAtA38[j37] = uint8(uint64(num)&0x7f | 0x80) + dAtA39[j38] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j37++ + j38++ } - dAtA38[j37] = uint8(num) - j37++ + dAtA39[j38] = uint8(num) + j38++ } dAtA[i] = 0xa i++ - i = encodeVarintJobs(dAtA, i, uint64(j37)) - i += copy(dAtA[i:], dAtA38[:j37]) + i = encodeVarintJobs(dAtA, i, uint64(j38)) + i += copy(dAtA[i:], dAtA39[:j38]) } if m.HasHistogram { dAtA[i] = 0x10 @@ -5101,11 +5099,11 @@ func (m *MigrationDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintJobs(dAtA, i, uint64(m.ClusterVersion.Size())) - n39, err := m.ClusterVersion.MarshalTo(dAtA[i:]) + n40, err := m.ClusterVersion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n39 + i += n40 } return i, nil } @@ -5166,21 +5164,21 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(m.FinishedMicros)) } if len(m.DescriptorIDs) > 0 { - dAtA41 := make([]byte, len(m.DescriptorIDs)*10) - var j40 int + dAtA42 := make([]byte, len(m.DescriptorIDs)*10) + var j41 int for _, num := range m.DescriptorIDs { for num >= 1<<7 { - dAtA41[j40] = uint8(uint64(num)&0x7f | 0x80) + dAtA42[j41] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j40++ + j41++ } - dAtA41[j40] = uint8(num) - j40++ + dAtA42[j41] = uint8(num) + j41++ } dAtA[i] = 0x32 i++ - i = encodeVarintJobs(dAtA, i, uint64(j40)) - i += copy(dAtA[i:], dAtA41[:j40]) + i = encodeVarintJobs(dAtA, i, uint64(j41)) + i += copy(dAtA[i:], dAtA42[:j41]) } if len(m.Error) > 0 { dAtA[i] = 0x42 @@ -5192,18 +5190,18 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Lease.Size())) - n42, err := m.Lease.MarshalTo(dAtA[i:]) + n43, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n43 } if m.Details != nil { - nn43, err := m.Details.MarshalTo(dAtA[i:]) + nn44, err := m.Details.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn43 + i += nn44 } if len(m.Statement) > 0 { dAtA[i] = 0x82 @@ -5247,11 +5245,11 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.FinalResumeError.Size())) - n44, err := m.FinalResumeError.MarshalTo(dAtA[i:]) + n45, err := m.FinalResumeError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n44 + i += n45 } if m.Noncancelable { dAtA[i] = 0xa0 @@ -5274,11 +5272,11 @@ func (m *Payload_Backup) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Backup.Size())) - n45, err := m.Backup.MarshalTo(dAtA[i:]) + n46, err := m.Backup.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n45 + i += n46 } return i, nil } @@ -5288,11 +5286,11 @@ func (m *Payload_Restore) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Restore.Size())) - n46, err := m.Restore.MarshalTo(dAtA[i:]) + n47, err := m.Restore.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n46 + i += n47 } return i, nil } @@ -5302,11 +5300,11 @@ func (m *Payload_SchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChange.Size())) - n47, err := m.SchemaChange.MarshalTo(dAtA[i:]) + n48, err := m.SchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n47 + i += n48 } return i, nil } @@ -5316,11 +5314,11 @@ func (m *Payload_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Import.Size())) - n48, err := m.Import.MarshalTo(dAtA[i:]) + n49, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n48 + i += n49 } return i, nil } @@ -5330,11 +5328,11 @@ func (m *Payload_Changefeed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Changefeed.Size())) - n49, err := m.Changefeed.MarshalTo(dAtA[i:]) + n50, err := m.Changefeed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n50 } return i, nil } @@ -5344,11 +5342,11 @@ func (m *Payload_CreateStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x7a i++ i = encodeVarintJobs(dAtA, i, uint64(m.CreateStats.Size())) - n50, err := m.CreateStats.MarshalTo(dAtA[i:]) + n51, err := m.CreateStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n51 } return i, nil } @@ -5360,11 +5358,11 @@ func (m *Payload_SchemaChangeGC) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChangeGC.Size())) - n51, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) + n52, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n52 } return i, nil } @@ -5376,11 +5374,11 @@ func (m *Payload_TypeSchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.TypeSchemaChange.Size())) - n52, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) + n53, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n53 } return i, nil } @@ -5392,11 +5390,11 @@ func (m *Payload_StreamIngestion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.StreamIngestion.Size())) - n53, err := m.StreamIngestion.MarshalTo(dAtA[i:]) + n54, err := m.StreamIngestion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n54 } return i, nil } @@ -5408,11 +5406,11 @@ func (m *Payload_NewSchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.NewSchemaChange.Size())) - n54, err := m.NewSchemaChange.MarshalTo(dAtA[i:]) + n55, err := m.NewSchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n55 } return i, nil } @@ -5424,11 +5422,11 @@ func (m *Payload_Migration) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Migration.Size())) - n55, err := m.Migration.MarshalTo(dAtA[i:]) + n56, err := m.Migration.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n56 } return i, nil } @@ -5448,11 +5446,11 @@ func (m *Progress) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Progress != nil { - nn56, err := m.Progress.MarshalTo(dAtA[i:]) + nn57, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn56 + i += nn57 } if m.ModifiedMicros != 0 { dAtA[i] = 0x10 @@ -5466,11 +5464,11 @@ func (m *Progress) MarshalTo(dAtA []byte) (int, error) { i += copy(dAtA[i:], m.RunningStatus) } if m.Details != nil { - nn57, err := m.Details.MarshalTo(dAtA[i:]) + nn58, err := m.Details.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn57 + i += nn58 } return i, nil } @@ -5489,11 +5487,11 @@ func (m *Progress_HighWater) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.HighWater.Size())) - n58, err := m.HighWater.MarshalTo(dAtA[i:]) + n59, err := m.HighWater.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n58 + i += n59 } return i, nil } @@ -5503,11 +5501,11 @@ func (m *Progress_Backup) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Backup.Size())) - n59, err := m.Backup.MarshalTo(dAtA[i:]) + n60, err := m.Backup.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n59 + i += n60 } return i, nil } @@ -5517,11 +5515,11 @@ func (m *Progress_Restore) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Restore.Size())) - n60, err := m.Restore.MarshalTo(dAtA[i:]) + n61, err := m.Restore.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n60 + i += n61 } return i, nil } @@ -5531,11 +5529,11 @@ func (m *Progress_SchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChange.Size())) - n61, err := m.SchemaChange.MarshalTo(dAtA[i:]) + n62, err := m.SchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n61 + i += n62 } return i, nil } @@ -5545,11 +5543,11 @@ func (m *Progress_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Import.Size())) - n62, err := m.Import.MarshalTo(dAtA[i:]) + n63, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n62 + i += n63 } return i, nil } @@ -5559,11 +5557,11 @@ func (m *Progress_Changefeed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Changefeed.Size())) - n63, err := m.Changefeed.MarshalTo(dAtA[i:]) + n64, err := m.Changefeed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n63 + i += n64 } return i, nil } @@ -5573,11 +5571,11 @@ func (m *Progress_CreateStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x7a i++ i = encodeVarintJobs(dAtA, i, uint64(m.CreateStats.Size())) - n64, err := m.CreateStats.MarshalTo(dAtA[i:]) + n65, err := m.CreateStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n64 + i += n65 } return i, nil } @@ -5589,11 +5587,11 @@ func (m *Progress_SchemaChangeGC) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChangeGC.Size())) - n65, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) + n66, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n65 + i += n66 } return i, nil } @@ -5605,11 +5603,11 @@ func (m *Progress_TypeSchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.TypeSchemaChange.Size())) - n66, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) + n67, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n66 + i += n67 } return i, nil } @@ -5621,11 +5619,11 @@ func (m *Progress_StreamIngest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.StreamIngest.Size())) - n67, err := m.StreamIngest.MarshalTo(dAtA[i:]) + n68, err := m.StreamIngest.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n67 + i += n68 } return i, nil } @@ -5637,11 +5635,11 @@ func (m *Progress_NewSchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.NewSchemaChange.Size())) - n68, err := m.NewSchemaChange.MarshalTo(dAtA[i:]) + n69, err := m.NewSchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n68 + i += n69 } return i, nil } @@ -5653,11 +5651,11 @@ func (m *Progress_Migration) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Migration.Size())) - n69, err := m.Migration.MarshalTo(dAtA[i:]) + n70, err := m.Migration.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n69 + i += n70 } return i, nil } @@ -5685,21 +5683,21 @@ func (m *Job) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Progress.Size())) - n70, err := m.Progress.MarshalTo(dAtA[i:]) + n71, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n70 + i += n71 } if m.Payload != nil { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Payload.Size())) - n71, err := m.Payload.MarshalTo(dAtA[i:]) + n72, err := m.Payload.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n71 + i += n72 } return i, nil } @@ -5816,9 +5814,8 @@ func (m *StreamIngestionProgress) Size() (n int) { } var l int _ = l - if m.MarkedForCompletion { - n += 2 - } + l = m.CutoverTime.Size() + n += 1 + l + sovJobs(uint64(l)) return n } @@ -7792,10 +7789,10 @@ func (m *StreamIngestionProgress) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MarkedForCompletion", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CutoverTime", wireType) } - var v int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowJobs @@ -7805,12 +7802,22 @@ func (m *StreamIngestionProgress) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - m.MarkedForCompletion = bool(v != 0) + if msglen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CutoverTime.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipJobs(dAtA[iNdEx:]) @@ -15100,311 +15107,311 @@ var ( ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_4b9f65bb42f26fe6) } +func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_98841b04a5cce477) } -var fileDescriptor_jobs_4b9f65bb42f26fe6 = []byte{ - // 4845 bytes of a gzipped FileDescriptorProto +var fileDescriptor_jobs_98841b04a5cce477 = []byte{ + // 4840 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5b, 0x4b, 0x8c, 0x1b, 0x47, - 0x7a, 0x9e, 0x26, 0x39, 0x64, 0xf3, 0x9f, 0x21, 0xd9, 0xac, 0x19, 0x49, 0x34, 0x63, 0x8b, 0xb3, + 0x7a, 0x9e, 0x26, 0x39, 0x64, 0xf3, 0xe7, 0x90, 0xd3, 0xac, 0x19, 0x49, 0x34, 0x63, 0x8b, 0xb3, 0xf4, 0x4b, 0x92, 0x6d, 0x8e, 0x57, 0xce, 0x7a, 0x6d, 0xc5, 0x96, 0xcd, 0x97, 0x66, 0xc8, 0xd1, 0x3c, 0xd4, 0x9c, 0x91, 0x1f, 0x1b, 0x6f, 0xa7, 0xd9, 0x5d, 0x33, 0xd3, 0x19, 0xb2, 0x9b, 0xea, 0x6a, 0x4a, 0x9a, 0x0d, 0x90, 0x04, 0x1b, 0x04, 0x58, 0xe8, 0x94, 0x00, 0xd9, 0x5c, 0x12, 0x01, 0x01, 0xb2, 0x0b, 0xe4, 0x10, 0x20, 0x88, 0x11, 0x24, 0x39, 0xe4, 0x96, 0x8b, 0x0f, 0x09, 0xb0, 0x97, 0x00, 0x46, 0x0e, 0xdc, 0x64, 0x7c, 0xc9, 0x31, 0x48, 0x2e, 0x0b, 0x5d, 0x12, 0xd4, 0xa3, - 0x9b, 0x4d, 0x72, 0x1e, 0x1c, 0xc9, 0xde, 0xbd, 0x48, 0xec, 0xbf, 0xfe, 0xfa, 0xaa, 0xea, 0xaf, - 0xbf, 0xbe, 0xff, 0xaf, 0xc7, 0xc0, 0xc5, 0xdf, 0x76, 0xda, 0x64, 0x99, 0xfe, 0xd3, 0x6b, 0xb3, - 0xff, 0x4a, 0x3d, 0xd7, 0xf1, 0x1c, 0xf4, 0x9c, 0xe1, 0x18, 0x07, 0xae, 0xa3, 0x1b, 0xfb, 0x25, - 0x72, 0xaf, 0x53, 0x62, 0x25, 0x5c, 0x2b, 0x7f, 0x01, 0xbb, 0xae, 0xe3, 0x52, 0x7d, 0xfe, 0x83, - 0xd7, 0xc8, 0x2f, 0xee, 0x39, 0x7b, 0x0e, 0xfb, 0xb9, 0x4c, 0x7f, 0x09, 0x29, 0x62, 0x18, 0xbd, - 0xf6, 0xb2, 0xa9, 0x7b, 0xba, 0x90, 0xe5, 0x7c, 0x99, 0xe5, 0xbc, 0xb1, 0xeb, 0xb8, 0x5d, 0xdd, - 0xf3, 0x31, 0x5e, 0x24, 0xf7, 0x3a, 0xcb, 0x86, 0xee, 0xe9, 0x1d, 0x67, 0x6f, 0xd9, 0xc4, 0xc4, - 0xe8, 0xb5, 0x97, 0x89, 0xe7, 0xf6, 0x0d, 0xaf, 0xef, 0x62, 0x53, 0x28, 0x15, 0x8e, 0x51, 0xf2, - 0xb0, 0xad, 0xdb, 0x9e, 0x8f, 0xdf, 0xf7, 0xac, 0xce, 0xf2, 0x7e, 0xc7, 0x58, 0xf6, 0xac, 0x2e, - 0x26, 0x9e, 0xde, 0xed, 0x89, 0x92, 0x6f, 0xd1, 0xaa, 0xc4, 0xd8, 0xc7, 0x5d, 0xdd, 0xd8, 0xd7, - 0xed, 0x3d, 0xec, 0x2e, 0xf3, 0x36, 0x8c, 0x5e, 0x5b, 0xa8, 0xbc, 0x64, 0x74, 0xfa, 0xc4, 0xc3, - 0xee, 0x7d, 0xec, 0x12, 0xcb, 0xb1, 0x97, 0xc5, 0xa7, 0x26, 0xbe, 0xb9, 0x56, 0xf1, 0xf7, 0x60, - 0xf6, 0x36, 0xd6, 0x09, 0x46, 0x9f, 0x42, 0xc2, 0x76, 0x4c, 0xac, 0x59, 0x66, 0x4e, 0x5a, 0x92, - 0xae, 0xa4, 0x2a, 0xe5, 0xa3, 0x41, 0x21, 0xbe, 0xe1, 0x98, 0xb8, 0x51, 0x7b, 0x32, 0x28, 0xbc, - 0xb5, 0x67, 0x79, 0xfb, 0xfd, 0x76, 0xc9, 0x70, 0xba, 0xcb, 0x81, 0x45, 0xcd, 0xf6, 0xf0, 0xf7, - 0x72, 0xef, 0x60, 0x6f, 0x59, 0xd8, 0xa3, 0xc4, 0xab, 0xa9, 0x71, 0x8a, 0xd8, 0x30, 0xd1, 0x22, - 0xcc, 0xe2, 0x9e, 0x63, 0xec, 0xe7, 0x22, 0x4b, 0xd2, 0x95, 0xa8, 0xca, 0x3f, 0x6e, 0xc4, 0xfe, - 0xeb, 0x2f, 0x0a, 0x52, 0xf1, 0xa7, 0x11, 0xb8, 0x54, 0xd1, 0x8d, 0x83, 0x7e, 0xaf, 0x6e, 0x1b, - 0xee, 0x61, 0xcf, 0xb3, 0x1c, 0x7b, 0x93, 0xfd, 0x4b, 0x90, 0x02, 0xd1, 0x03, 0x7c, 0xc8, 0xfa, - 0x33, 0xaf, 0xd2, 0x9f, 0xe8, 0x7d, 0x88, 0x75, 0x1d, 0x13, 0x33, 0xa0, 0xf4, 0xf5, 0xab, 0xa5, - 0x13, 0x27, 0xb7, 0x34, 0x44, 0x5b, 0x77, 0x4c, 0xac, 0xb2, 0x6a, 0xa8, 0x0d, 0xf2, 0x41, 0x97, - 0x68, 0x96, 0xbd, 0xeb, 0xe4, 0xa2, 0x4b, 0xd2, 0x95, 0xb9, 0xeb, 0x37, 0x4e, 0x81, 0x38, 0xa1, - 0x5b, 0xa5, 0xb5, 0xf5, 0x56, 0xc3, 0xde, 0x75, 0x2a, 0x73, 0x47, 0x83, 0x42, 0x42, 0x7c, 0xa8, - 0x89, 0x83, 0x2e, 0xa1, 0x3f, 0xf2, 0x9b, 0xe0, 0xcb, 0x68, 0xff, 0xfb, 0xae, 0xc5, 0xfa, 0x9f, - 0x54, 0xe9, 0x4f, 0xf4, 0x3a, 0x20, 0xcc, 0xf1, 0xb0, 0xa9, 0x51, 0x4f, 0xd2, 0xe8, 0x00, 0x23, - 0x6c, 0x80, 0x4a, 0x50, 0x52, 0xd3, 0x3d, 0x7d, 0x0d, 0x1f, 0x72, 0x0b, 0x09, 0x3b, 0xfd, 0x7e, - 0x14, 0xd2, 0xc3, 0xae, 0x30, 0xf8, 0x55, 0x88, 0x33, 0x17, 0xc0, 0xac, 0x85, 0xf4, 0xf5, 0x37, - 0xa7, 0x32, 0x07, 0xad, 0x5a, 0x6a, 0xb1, 0x7a, 0xaa, 0xa8, 0x8f, 0x10, 0xc4, 0x88, 0xde, 0xf1, - 0x44, 0x47, 0xd8, 0x6f, 0xf4, 0x67, 0x12, 0x2c, 0x8d, 0xf7, 0xa8, 0x72, 0xb8, 0xb6, 0xde, 0x5a, - 0xd7, 0xa9, 0x1f, 0xad, 0xe1, 0xc3, 0x46, 0x2d, 0x17, 0x5d, 0x8a, 0x5e, 0x99, 0xbb, 0xbe, 0x39, - 0x7d, 0xc3, 0xf5, 0x33, 0x10, 0xeb, 0xb6, 0xe7, 0x1e, 0xaa, 0x67, 0x36, 0x9c, 0x6f, 0xc1, 0xcb, - 0x53, 0x41, 0x85, 0x7d, 0x28, 0xc9, 0x7d, 0x68, 0x11, 0x66, 0xef, 0xeb, 0x9d, 0x3e, 0x16, 0xa3, - 0xe5, 0x1f, 0x37, 0x22, 0xef, 0x48, 0xc5, 0x4b, 0x10, 0xe7, 0x86, 0x41, 0x29, 0x48, 0x96, 0xeb, - 0xad, 0xeb, 0xdf, 0x79, 0x7b, 0xa5, 0xba, 0xae, 0xcc, 0x88, 0x29, 0xf8, 0x3f, 0x09, 0x2e, 0xb6, - 0x3c, 0x17, 0xeb, 0xdd, 0x86, 0xbd, 0x87, 0x09, 0x1d, 0x53, 0x0d, 0x7b, 0xba, 0xd5, 0x21, 0xc8, - 0x86, 0x34, 0x61, 0x25, 0x9a, 0x6e, 0x9a, 0x2e, 0x26, 0x84, 0x37, 0x58, 0x59, 0x79, 0x32, 0x28, - 0x54, 0xa7, 0x5a, 0x3a, 0x86, 0xd1, 0x59, 0xe6, 0x10, 0x96, 0xbd, 0x67, 0x18, 0x9d, 0x12, 0x6f, - 0xa9, 0xcc, 0xe1, 0xd4, 0x14, 0x09, 0x7f, 0xa2, 0x6f, 0x43, 0x8c, 0xf4, 0x74, 0x9b, 0x0d, 0x61, - 0xee, 0xfa, 0xa5, 0x90, 0xfd, 0xfd, 0x25, 0xd8, 0xea, 0xe9, 0x76, 0x25, 0xf6, 0xc5, 0xa0, 0x30, - 0xa3, 0x32, 0x55, 0x54, 0x01, 0x20, 0x9e, 0xee, 0x7a, 0x1a, 0xe5, 0x12, 0xe1, 0xfd, 0x2f, 0x84, - 0x2a, 0x52, 0xae, 0x29, 0xed, 0x77, 0x8c, 0xd2, 0xb6, 0xcf, 0x35, 0xa2, 0x7a, 0x92, 0x55, 0xa3, - 0xd2, 0xe2, 0x1a, 0x5c, 0x1a, 0x33, 0xc0, 0x96, 0xeb, 0xec, 0xb1, 0x1e, 0xbd, 0x09, 0x0b, 0x5d, - 0xdd, 0x3d, 0xc0, 0xe6, 0x2d, 0xc7, 0xad, 0x3a, 0xdd, 0x5e, 0x07, 0xd3, 0x62, 0x66, 0x06, 0x59, - 0x3d, 0xae, 0xa8, 0xf8, 0x4f, 0x71, 0x48, 0xf1, 0x25, 0xe6, 0x5b, 0x71, 0xb4, 0x8b, 0xd2, 0xd3, - 0x74, 0x11, 0xdd, 0x04, 0x19, 0xdb, 0x26, 0x47, 0x88, 0x4c, 0x8f, 0x90, 0xc0, 0xb6, 0xc9, 0xea, - 0x3f, 0xc7, 0xd7, 0x6c, 0x94, 0x4d, 0x5f, 0xe2, 0x68, 0x50, 0x88, 0xee, 0xa8, 0x0d, 0xbe, 0x78, - 0xdf, 0x83, 0xbc, 0x89, 0x7b, 0x2e, 0x36, 0x74, 0xba, 0x7a, 0xdb, 0xac, 0xeb, 0x5a, 0x57, 0xb7, - 0xad, 0x5d, 0x4c, 0xbc, 0x5c, 0x8c, 0x79, 0x53, 0x6e, 0xa8, 0xc1, 0xc7, 0xb6, 0x2e, 0xca, 0xd1, - 0x1f, 0x48, 0xb0, 0xd0, 0x77, 0x2d, 0xa2, 0xb5, 0x0f, 0xb5, 0x8e, 0x63, 0xe8, 0x1d, 0xcb, 0x3b, - 0xd4, 0x0e, 0xee, 0xe7, 0x66, 0xd9, 0x12, 0xba, 0x79, 0x26, 0x0f, 0x09, 0x23, 0x95, 0x76, 0x5c, - 0x8b, 0x54, 0x0e, 0x6f, 0x0b, 0x84, 0xb5, 0xfb, 0xcc, 0xcd, 0x2b, 0x8b, 0x47, 0x83, 0x82, 0xb2, - 0xa3, 0x36, 0xc2, 0x45, 0x77, 0x55, 0xa5, 0x3f, 0xa6, 0x8c, 0xf4, 0x80, 0x80, 0x2c, 0xc7, 0xd6, - 0x1c, 0xce, 0x68, 0xb9, 0x38, 0x33, 0xd4, 0xf5, 0xf3, 0x73, 0xa1, 0x9a, 0xc5, 0x13, 0xac, 0xfd, - 0xc7, 0x12, 0xe4, 0x69, 0x70, 0xc1, 0x06, 0x35, 0x53, 0x10, 0xb9, 0x34, 0x17, 0x1b, 0x8e, 0x6b, - 0xe6, 0x12, 0xd4, 0x4e, 0x95, 0xd6, 0xbf, 0x4f, 0x1b, 0x53, 0x58, 0x0c, 0xec, 0xf7, 0x2d, 0xb3, - 0xb4, 0xb3, 0xd3, 0xa8, 0x1d, 0x0d, 0x0a, 0xb9, 0x2d, 0x1f, 0x3c, 0x98, 0x44, 0x95, 0x41, 0xab, - 0xb9, 0xde, 0x09, 0x25, 0xe8, 0x1d, 0x48, 0x1b, 0x4e, 0xa7, 0x83, 0x0d, 0x36, 0xec, 0x1d, 0xb5, - 0x91, 0x93, 0xd9, 0x04, 0x67, 0x8f, 0x06, 0x85, 0x54, 0x35, 0x28, 0xa1, 0x53, 0x9d, 0x32, 0xc2, - 0x9f, 0x48, 0x85, 0x4c, 0xc8, 0x60, 0x2c, 0x72, 0x24, 0x99, 0xb5, 0xae, 0x4e, 0x4d, 0x7a, 0x6a, - 0x1a, 0x8f, 0x7c, 0xe7, 0xab, 0x70, 0xe1, 0xd8, 0x59, 0x3c, 0x8b, 0xac, 0x92, 0x61, 0xb2, 0x52, - 0x20, 0xcd, 0x27, 0xc5, 0x5f, 0x82, 0xc5, 0xcf, 0xd3, 0x90, 0x56, 0x31, 0xf1, 0x1c, 0x17, 0xfb, - 0x2b, 0xea, 0x73, 0x09, 0x16, 0x68, 0x66, 0xe1, 0x5a, 0x3d, 0xcf, 0x71, 0x35, 0x17, 0x3f, 0x70, - 0x2d, 0x0f, 0x93, 0x5c, 0x84, 0x39, 0x5d, 0xf9, 0x94, 0x21, 0x8c, 0x02, 0x95, 0x6a, 0x01, 0x88, - 0x2a, 0x30, 0xb8, 0xdf, 0xdd, 0xfc, 0xe1, 0xcf, 0x0b, 0x37, 0xa6, 0x9a, 0xc7, 0xc9, 0x64, 0xa7, - 0xd4, 0xa8, 0xa9, 0xc8, 0x9c, 0x00, 0x46, 0xcf, 0x43, 0x8c, 0xfa, 0x2d, 0x0b, 0x2e, 0xc9, 0x8a, - 0x7c, 0x34, 0x28, 0xc4, 0xa8, 0x67, 0xab, 0x4c, 0x3a, 0xb2, 0xc0, 0x63, 0x4f, 0xb1, 0xc0, 0x57, - 0x60, 0xce, 0xd3, 0xdb, 0x1d, 0xac, 0xd1, 0x96, 0x89, 0x58, 0x7e, 0xaf, 0x8c, 0x59, 0x82, 0xdc, - 0xeb, 0xb4, 0x75, 0x82, 0x4b, 0xdb, 0x54, 0x33, 0x34, 0x76, 0xf0, 0x7c, 0x01, 0x41, 0xcb, 0x30, - 0xe7, 0xdc, 0xc7, 0xae, 0x6b, 0x99, 0x58, 0x33, 0xdb, 0x6c, 0x0d, 0x25, 0x2b, 0xe9, 0xa3, 0x41, - 0x01, 0x36, 0x85, 0xb8, 0x56, 0x51, 0xc1, 0x57, 0xa9, 0xb5, 0x91, 0x07, 0x8b, 0x82, 0x34, 0x82, - 0xf5, 0xcf, 0xfc, 0x29, 0xc1, 0xba, 0xf0, 0xde, 0xf4, 0x93, 0xc1, 0xe7, 0xdd, 0x77, 0x1e, 0x96, - 0x8b, 0xf0, 0x41, 0xa2, 0xf6, 0x44, 0x09, 0x7a, 0x0d, 0xb2, 0x3d, 0x17, 0xf7, 0x74, 0x17, 0x6b, - 0x06, 0x27, 0x5f, 0x6c, 0x32, 0xef, 0x97, 0x55, 0x45, 0x14, 0x54, 0x7d, 0x39, 0x7a, 0x99, 0xc6, - 0x31, 0xdd, 0xa3, 0x29, 0x12, 0xc1, 0x2e, 0xd5, 0x4c, 0x32, 0xcd, 0x14, 0x93, 0x36, 0x84, 0x10, - 0xbd, 0x05, 0x17, 0x86, 0xf3, 0x46, 0xb4, 0x5e, 0xbf, 0xdd, 0xb1, 0xc8, 0x3e, 0x36, 0x73, 0xc0, - 0xb4, 0x17, 0x43, 0x85, 0x5b, 0x7e, 0x19, 0x3a, 0x1c, 0x71, 0x45, 0x83, 0x1a, 0x46, 0xdf, 0xc3, - 0xb9, 0xb9, 0x25, 0xe9, 0xca, 0x6c, 0x65, 0xf5, 0xc9, 0xa0, 0x50, 0x9b, 0xda, 0x8f, 0x08, 0xee, - 0x2e, 0x7b, 0x2e, 0xc6, 0x21, 0xb7, 0xac, 0x0a, 0xbc, 0xb0, 0x47, 0xf9, 0x32, 0xa4, 0x02, 0x0c, - 0x97, 0x60, 0x6e, 0xfe, 0xa9, 0xd9, 0x2e, 0x84, 0x82, 0xca, 0x90, 0xe0, 0xc9, 0x3a, 0xc9, 0xa5, - 0xd8, 0x04, 0x7e, 0xeb, 0x24, 0x1f, 0x62, 0x5a, 0xa1, 0x59, 0xf2, 0xeb, 0xa1, 0x1a, 0x80, 0x77, - 0xd8, 0xf3, 0x3d, 0x31, 0xcd, 0x50, 0x5e, 0x3e, 0x09, 0xe5, 0xb0, 0x17, 0x76, 0xc4, 0xa4, 0x27, - 0xbe, 0x09, 0x6a, 0xc2, 0x3c, 0xdf, 0x09, 0x08, 0x9c, 0x0c, 0xc3, 0x79, 0xf5, 0x04, 0x1c, 0x96, - 0xe0, 0xe8, 0x21, 0xa4, 0x39, 0x12, 0x48, 0x08, 0xda, 0x82, 0x34, 0xcd, 0x4a, 0xa9, 0xa6, 0x40, - 0x53, 0x18, 0xda, 0xd5, 0x13, 0xd0, 0x6a, 0x42, 0x39, 0x84, 0x97, 0x32, 0x43, 0x32, 0x92, 0xff, - 0x5f, 0x09, 0xb2, 0x13, 0xe4, 0x81, 0xb6, 0x21, 0x12, 0x6c, 0x34, 0x28, 0xa7, 0x47, 0xd8, 0x26, - 0xe3, 0x59, 0x88, 0x24, 0x62, 0x99, 0x68, 0x0f, 0x92, 0xd4, 0x9d, 0x6d, 0x8f, 0xee, 0x62, 0x22, - 0x0c, 0xbc, 0x79, 0x34, 0x28, 0xc8, 0x5b, 0x4c, 0xf8, 0xcc, 0x4d, 0xc8, 0x1c, 0xbc, 0x61, 0xa2, - 0x02, 0xcc, 0x79, 0x8e, 0x86, 0x1f, 0x5a, 0xc4, 0xb3, 0xec, 0x3d, 0x96, 0x2c, 0xc8, 0x2a, 0x78, - 0x4e, 0x5d, 0x48, 0xf2, 0x7f, 0x1e, 0x01, 0x34, 0xb9, 0x4a, 0xd1, 0x3f, 0x4a, 0xf0, 0xbc, 0x9f, - 0x03, 0x38, 0xae, 0xb5, 0x67, 0xd9, 0x7a, 0x67, 0x24, 0x19, 0x90, 0x98, 0xb5, 0x3f, 0x7d, 0x16, - 0x2a, 0x10, 0x09, 0xc2, 0xa6, 0x80, 0x1f, 0x4f, 0x14, 0x9e, 0xa7, 0x11, 0x94, 0x27, 0x0a, 0x13, - 0x2a, 0x77, 0xd5, 0x5c, 0xff, 0x84, 0xca, 0xf9, 0x35, 0x78, 0xe1, 0x54, 0xe0, 0xf3, 0xc4, 0xae, - 0xfc, 0x0f, 0x25, 0xb8, 0x74, 0x42, 0x44, 0x09, 0xe3, 0xa4, 0x38, 0xce, 0x9d, 0x30, 0xce, 0xdc, - 0xf5, 0xdf, 0x78, 0x86, 0xa8, 0x15, 0xea, 0x44, 0x33, 0x26, 0x4b, 0x4a, 0xa4, 0xf8, 0x26, 0x64, - 0x44, 0xa5, 0x20, 0x95, 0x7d, 0x01, 0x60, 0xdf, 0xda, 0xdb, 0xd7, 0x1e, 0xe8, 0x1e, 0x76, 0xc5, - 0xee, 0x33, 0x49, 0x25, 0x1f, 0x51, 0x41, 0xf1, 0xdf, 0x64, 0x48, 0x35, 0xba, 0x3d, 0xc7, 0xf5, - 0xfc, 0x28, 0x7b, 0x1b, 0xe2, 0x2c, 0x2e, 0x10, 0x31, 0x7f, 0xa5, 0x53, 0x7a, 0x38, 0x52, 0x93, - 0xc7, 0x17, 0x41, 0x0b, 0x02, 0x23, 0x08, 0x7f, 0x91, 0x63, 0xc3, 0xdf, 0xfb, 0x10, 0xe7, 0x47, - 0x0d, 0x22, 0x85, 0x2f, 0x1c, 0x93, 0xfb, 0x37, 0x36, 0x6f, 0x59, 0x1d, 0x7c, 0x8b, 0xa9, 0xf9, - 0xe0, 0xbc, 0x12, 0x7a, 0x05, 0x64, 0x42, 0x3c, 0x8d, 0x58, 0x3f, 0xe0, 0xd1, 0x33, 0xca, 0x77, - 0xb1, 0xad, 0xd6, 0x76, 0xcb, 0xfa, 0x01, 0x56, 0x13, 0x84, 0x78, 0xf4, 0x07, 0xca, 0x83, 0xfc, - 0x40, 0xef, 0x74, 0x58, 0x94, 0x9d, 0x65, 0xbb, 0xf6, 0xe0, 0x7b, 0x74, 0x99, 0xc5, 0xbf, 0xd9, - 0x65, 0x26, 0x02, 0x66, 0x4f, 0xf7, 0xf6, 0x59, 0xe6, 0x98, 0x54, 0x81, 0x8b, 0xb6, 0x74, 0x6f, - 0x1f, 0xe5, 0x20, 0x41, 0x74, 0x1a, 0xbb, 0x48, 0x4e, 0x5e, 0x8a, 0x5e, 0x99, 0x57, 0xfd, 0x4f, - 0x74, 0x19, 0x58, 0xe4, 0xe5, 0x9f, 0x2c, 0x88, 0x45, 0xd5, 0x90, 0x84, 0xd9, 0xe1, 0xc0, 0xea, - 0x69, 0xbb, 0x07, 0x84, 0x07, 0x2d, 0x61, 0x87, 0x03, 0xab, 0x77, 0x6b, 0x8d, 0xa8, 0x09, 0x5a, - 0x78, 0xeb, 0x80, 0xa0, 0x57, 0x21, 0x63, 0xb1, 0xbd, 0x8e, 0x66, 0x5a, 0x2e, 0x36, 0xbc, 0xce, - 0x21, 0x0b, 0x58, 0xb2, 0x9a, 0xe6, 0xe2, 0x9a, 0x90, 0xa2, 0xab, 0xa0, 0x8c, 0x87, 0x59, 0x16, - 0x68, 0x64, 0x35, 0x33, 0x16, 0x65, 0xa9, 0x2a, 0x9f, 0xea, 0x50, 0xe0, 0x4c, 0x71, 0x55, 0x2e, - 0x1f, 0xc6, 0xcc, 0x12, 0x2c, 0xf4, 0x74, 0x97, 0x60, 0xad, 0xdd, 0xb7, 0xcd, 0x0e, 0xd6, 0x38, - 0x57, 0xe7, 0xd2, 0x4c, 0x3b, 0xcb, 0x8a, 0x2a, 0xac, 0x84, 0xd3, 0xfa, 0x59, 0xb9, 0xf7, 0xc5, - 0x5f, 0x41, 0xee, 0x9d, 0xff, 0x69, 0x04, 0x66, 0x99, 0x9f, 0xa3, 0x1b, 0x10, 0xa3, 0xd3, 0x2c, - 0x76, 0x76, 0xd3, 0xe6, 0x5c, 0xac, 0x0e, 0x42, 0x10, 0xb3, 0xf5, 0x2e, 0xce, 0x21, 0xe6, 0x04, - 0xec, 0x37, 0xba, 0x04, 0x09, 0x82, 0xef, 0x69, 0xf7, 0xf5, 0x4e, 0x6e, 0x81, 0xcd, 0x70, 0x9c, - 0xe0, 0x7b, 0x77, 0xf5, 0x0e, 0xba, 0x00, 0x71, 0x8b, 0x68, 0x36, 0x7e, 0x90, 0x5b, 0x64, 0x96, - 0x9a, 0xb5, 0xc8, 0x06, 0x7e, 0xc0, 0x68, 0x5b, 0x77, 0xf7, 0xb0, 0xa7, 0x19, 0x4e, 0x87, 0xe4, - 0x2e, 0xd0, 0x05, 0x46, 0x53, 0x3a, 0x2a, 0xaa, 0x3a, 0x1d, 0x82, 0x7e, 0x0d, 0x92, 0x0f, 0x74, - 0xa2, 0xe1, 0x6e, 0xcf, 0x3b, 0x64, 0xc6, 0x92, 0xa9, 0xdb, 0x93, 0x3a, 0xfd, 0x6e, 0xc6, 0xe4, - 0x88, 0x12, 0x6d, 0xc6, 0xe4, 0xa8, 0x12, 0x6b, 0xc6, 0xe4, 0x98, 0x32, 0xdb, 0x8c, 0xc9, 0xb3, - 0x4a, 0xbc, 0x19, 0x93, 0xe3, 0x4a, 0xa2, 0x19, 0x93, 0x13, 0x8a, 0xdc, 0x8c, 0xc9, 0xb2, 0x92, - 0x6c, 0xc6, 0xe4, 0xa4, 0x02, 0xcd, 0x98, 0x0c, 0xca, 0x5c, 0x33, 0x26, 0xcf, 0x29, 0xf3, 0xcd, - 0x98, 0x3c, 0xaf, 0xa4, 0x9a, 0x31, 0x39, 0xa5, 0xa4, 0x9b, 0x31, 0x39, 0xad, 0x64, 0x9a, 0x31, - 0x39, 0xa3, 0x28, 0xcd, 0x98, 0xac, 0x28, 0xd9, 0x66, 0x4c, 0xce, 0x2a, 0xa8, 0xf8, 0xb9, 0x04, - 0x4a, 0x0b, 0xdf, 0xeb, 0x63, 0xdb, 0xc0, 0x77, 0xf5, 0x4e, 0x75, 0xbf, 0x6f, 0x1f, 0xa0, 0x57, - 0x20, 0x63, 0xd0, 0x1f, 0x1a, 0xdf, 0x18, 0xd3, 0xa1, 0x4a, 0x6c, 0xa8, 0x29, 0x26, 0x6e, 0x51, - 0x29, 0x1d, 0xf1, 0x0b, 0x00, 0x42, 0x8f, 0xae, 0x6c, 0x7e, 0xce, 0x96, 0xe4, 0x2a, 0x74, 0x39, - 0x8f, 0xc1, 0xb8, 0xce, 0x03, 0x46, 0x1f, 0x23, 0x30, 0xaa, 0xf3, 0x00, 0x2d, 0xc3, 0xa2, 0x8d, - 0x1f, 0x7a, 0xda, 0xb8, 0x32, 0xa3, 0x0a, 0x35, 0x4b, 0xcb, 0xaa, 0xe1, 0x0a, 0xc5, 0x7f, 0x8d, - 0x40, 0xc6, 0xef, 0xb4, 0x4f, 0x87, 0xbb, 0xa0, 0xd0, 0x69, 0xb1, 0x4c, 0xcd, 0x73, 0x38, 0x92, - 0x4f, 0x8c, 0xef, 0x9f, 0x42, 0x8c, 0x63, 0x28, 0xf4, 0xbb, 0x61, 0x6e, 0x3b, 0xac, 0x39, 0x1e, - 0x1a, 0xd4, 0x14, 0x09, 0xcb, 0xf2, 0x3b, 0x90, 0xf6, 0x2b, 0x71, 0x09, 0xaa, 0x42, 0x7c, 0xa4, - 0xbd, 0xd7, 0xa6, 0x68, 0xcf, 0x37, 0xb5, 0x2a, 0xaa, 0xe6, 0x7f, 0x07, 0xd0, 0x64, 0xdb, 0xe1, - 0xb0, 0x34, 0xcb, 0xc3, 0xd2, 0xe6, 0x68, 0x58, 0x7a, 0xf7, 0x7c, 0x63, 0x0b, 0x75, 0x3b, 0xbc, - 0xab, 0xfb, 0xe7, 0x08, 0xa4, 0x79, 0x88, 0x08, 0xc2, 0xd1, 0x6b, 0x90, 0x65, 0xa4, 0x65, 0xd9, - 0x7b, 0x5a, 0x4f, 0x08, 0xd9, 0xf8, 0x22, 0xaa, 0xe2, 0x17, 0x04, 0xca, 0x2f, 0x42, 0xca, 0xc5, - 0xba, 0x39, 0x54, 0x8c, 0x30, 0xc5, 0x79, 0x2a, 0x0c, 0x94, 0x5e, 0x86, 0x34, 0x8b, 0x86, 0x43, - 0xad, 0x28, 0xd3, 0x4a, 0x31, 0x69, 0xa0, 0x56, 0x81, 0x14, 0xe9, 0xe9, 0xf6, 0x50, 0x2b, 0xc6, - 0x8c, 0x7a, 0xc6, 0x69, 0xd3, 0x3c, 0xad, 0x13, 0x8e, 0xa5, 0x2e, 0x26, 0xfd, 0x2e, 0xd6, 0x7a, - 0x0e, 0xdf, 0x6c, 0x45, 0xd5, 0x24, 0x97, 0x6c, 0x39, 0x04, 0xed, 0x30, 0x57, 0x61, 0xb6, 0xd0, - 0x4c, 0x6e, 0x9c, 0x5c, 0x9c, 0xb5, 0x72, 0x6d, 0x7a, 0x73, 0xaa, 0x19, 0x32, 0x2a, 0x28, 0xfe, - 0xad, 0x04, 0x97, 0x68, 0xc2, 0xcc, 0x59, 0xb1, 0xca, 0x4e, 0xc8, 0x7d, 0xef, 0xd4, 0x21, 0xc1, - 0x92, 0xee, 0x20, 0xff, 0x5c, 0x3d, 0x1a, 0x14, 0xe2, 0x54, 0xfb, 0x99, 0x23, 0x57, 0x9c, 0x02, - 0x37, 0xd8, 0xf6, 0xc8, 0x73, 0x75, 0x9b, 0x58, 0x74, 0xa3, 0x40, 0xa7, 0xad, 0x8b, 0xbb, 0x6d, - 0xec, 0xf2, 0xc9, 0x98, 0x57, 0x17, 0x47, 0x0a, 0xd7, 0x79, 0x59, 0x31, 0x0f, 0xb9, 0xf1, 0x2e, - 0x07, 0x3b, 0xfb, 0xdf, 0x84, 0x8b, 0x1b, 0xf8, 0xc1, 0x71, 0xa3, 0xa9, 0x40, 0x82, 0xf3, 0x97, - 0xef, 0xf2, 0x57, 0xc6, 0x59, 0x35, 0x7c, 0x49, 0x50, 0x62, 0x3d, 0xdd, 0x66, 0x15, 0x54, 0xbf, - 0x62, 0xf1, 0x53, 0xb8, 0x34, 0x86, 0x1e, 0x4c, 0xdf, 0x07, 0x10, 0xa7, 0x3b, 0x3f, 0x91, 0xd9, - 0xa4, 0x27, 0x77, 0x15, 0x93, 0xe8, 0x2d, 0xaa, 0xaf, 0x8a, 0x6a, 0x45, 0x95, 0x1d, 0x49, 0xf4, - 0xbb, 0x98, 0x7a, 0xc8, 0x6d, 0x8b, 0x78, 0xe8, 0x43, 0x98, 0x17, 0x1e, 0x41, 0x1d, 0xc5, 0xef, - 0xf6, 0x19, 0x4e, 0x35, 0xe7, 0x06, 0x20, 0xa4, 0xf8, 0x77, 0x12, 0x2c, 0xd4, 0x5c, 0xa7, 0xd7, - 0xc3, 0xa6, 0x88, 0x15, 0xdc, 0x16, 0x7e, 0x88, 0x90, 0x42, 0x21, 0x62, 0x03, 0x22, 0x8d, 0x9a, - 0xd8, 0x0b, 0xdc, 0x7c, 0xd6, 0x2d, 0x46, 0xa3, 0x86, 0xde, 0xe5, 0x06, 0xe9, 0x13, 0xc6, 0x9f, - 0xe9, 0x89, 0x4d, 0xdf, 0x88, 0x9b, 0x32, 0x45, 0x55, 0x54, 0x28, 0xfe, 0x24, 0x01, 0x17, 0xc2, - 0x46, 0x5e, 0xa9, 0xfa, 0x1d, 0xff, 0x0c, 0x12, 0x96, 0x6d, 0xe2, 0x87, 0x78, 0x2a, 0x9e, 0x3c, - 0x0e, 0xa2, 0x24, 0xec, 0xd1, 0xa0, 0x30, 0xfe, 0x36, 0x53, 0x60, 0xa2, 0x8f, 0x83, 0xf4, 0x94, - 0x1f, 0xfb, 0xdc, 0x78, 0x6a, 0xf4, 0xda, 0x58, 0xaa, 0x3a, 0x92, 0x09, 0xb2, 0x80, 0xf2, 0x0d, - 0x65, 0x82, 0x2d, 0xc8, 0x5a, 0xb6, 0x87, 0xdd, 0x0e, 0xd6, 0xef, 0xd3, 0xc4, 0x86, 0x36, 0x2f, - 0x4e, 0x7f, 0xa6, 0x4d, 0x23, 0x94, 0x10, 0x00, 0x4f, 0x47, 0x3e, 0x83, 0x85, 0x30, 0xa8, 0x3f, - 0x05, 0xa7, 0x9f, 0x08, 0x31, 0x0b, 0x0f, 0x61, 0xfd, 0x83, 0x97, 0x10, 0x50, 0x43, 0x98, 0xfd, - 0x2e, 0xc4, 0xf9, 0x46, 0x5f, 0x1c, 0xaf, 0xde, 0x7c, 0x5a, 0xb3, 0xf3, 0x03, 0x04, 0x55, 0xa0, - 0xe5, 0xff, 0x54, 0x82, 0xf9, 0xf0, 0x74, 0x23, 0x0b, 0x64, 0xd6, 0x77, 0x9f, 0xd2, 0xa2, 0x95, - 0x0d, 0x9a, 0xcb, 0xb2, 0x42, 0x36, 0x07, 0x1f, 0x3c, 0xf5, 0x1c, 0x70, 0x08, 0xe1, 0x4a, 0x0d, - 0x93, 0x26, 0x48, 0xa6, 0xeb, 0xf4, 0x86, 0xc7, 0xeb, 0x51, 0x55, 0xa6, 0x02, 0x9a, 0xf3, 0xe5, - 0x7f, 0x17, 0x92, 0x81, 0xa3, 0x84, 0x76, 0xf8, 0xd1, 0xaf, 0x71, 0x87, 0x7f, 0x6a, 0xfb, 0x35, - 0x48, 0x8d, 0x58, 0x0c, 0x5d, 0x0c, 0xfa, 0x10, 0xab, 0xc4, 0x79, 0x1f, 0xce, 0x44, 0x29, 0xfe, - 0x3c, 0x0e, 0x0b, 0xc7, 0x31, 0xed, 0x27, 0xa0, 0x84, 0x78, 0x4b, 0xeb, 0x58, 0xc4, 0x13, 0xeb, - 0xe9, 0xea, 0xe9, 0x1b, 0xd2, 0x10, 0xf9, 0x09, 0x6f, 0x49, 0xbb, 0xa3, 0x94, 0xf8, 0x3d, 0x48, - 0x9b, 0xbc, 0xe3, 0x9a, 0x58, 0xa8, 0xd1, 0x33, 0xf7, 0x91, 0xc7, 0x10, 0xa0, 0x40, 0x4f, 0x99, - 0xa1, 0x22, 0xc2, 0xee, 0x1d, 0x7c, 0xf4, 0xe0, 0x6c, 0xc7, 0x32, 0xd9, 0xea, 0x49, 0x55, 0x5a, - 0x47, 0x83, 0x42, 0x56, 0x60, 0xf9, 0x87, 0x39, 0xcf, 0x3c, 0x53, 0x59, 0x73, 0x0c, 0xd0, 0xa4, - 0x51, 0x97, 0x96, 0xd3, 0x86, 0x67, 0x87, 0x51, 0x97, 0xae, 0xa3, 0x67, 0x8f, 0xba, 0xf4, 0x67, - 0xc3, 0x44, 0x7f, 0x28, 0x41, 0x96, 0x9f, 0xec, 0x76, 0xfb, 0x9e, 0xce, 0x8f, 0xeb, 0xfd, 0xfd, - 0xe9, 0x27, 0x47, 0x83, 0x42, 0x86, 0x19, 0x64, 0x5d, 0x94, 0xb1, 0x66, 0x2b, 0x4f, 0xdb, 0xec, - 0x10, 0x45, 0xec, 0xd9, 0x02, 0x81, 0x89, 0xd6, 0x20, 0xcd, 0x37, 0xdb, 0xfe, 0x55, 0x3b, 0xdb, - 0xb8, 0xa6, 0x2a, 0x2f, 0x3d, 0x19, 0x14, 0x96, 0x8e, 0xf1, 0x2c, 0xbe, 0x4f, 0xbf, 0xcb, 0x75, - 0xd5, 0xd4, 0x6e, 0xf8, 0x13, 0x19, 0x90, 0x0a, 0x5c, 0xe3, 0xb0, 0x27, 0xf6, 0xb9, 0xcf, 0x1e, - 0xca, 0xe6, 0x7d, 0x1f, 0xa1, 0x98, 0x68, 0x0f, 0x32, 0x7e, 0x23, 0x3c, 0xa0, 0x93, 0x5c, 0xf2, - 0x6b, 0x69, 0xc6, 0x77, 0x6b, 0x3e, 0x6a, 0x22, 0x8e, 0x5c, 0x2e, 0xc2, 0xe2, 0xb1, 0x59, 0xce, - 0x97, 0x71, 0xb8, 0x38, 0x4a, 0x84, 0x41, 0x1e, 0xa2, 0x8d, 0x47, 0xc8, 0x0f, 0xa6, 0x26, 0x53, - 0x1f, 0x83, 0x93, 0x99, 0xff, 0x35, 0x1e, 0x23, 0x3f, 0x1b, 0x8b, 0x91, 0x4f, 0x81, 0xcf, 0xdc, - 0x6b, 0x0c, 0xdf, 0x0f, 0x94, 0x1f, 0x07, 0xb1, 0x80, 0x9f, 0xda, 0x7c, 0xf8, 0x14, 0xf0, 0xac, - 0xbe, 0xff, 0x19, 0x44, 0x83, 0x7f, 0x91, 0x20, 0x35, 0x32, 0xb2, 0x5f, 0x66, 0x38, 0xd8, 0x0a, - 0xb2, 0x21, 0xfe, 0x20, 0xe3, 0x9d, 0xf3, 0x0f, 0x6b, 0x34, 0x49, 0xca, 0xff, 0x83, 0x04, 0xa9, - 0x11, 0x43, 0x7e, 0x43, 0x81, 0xe4, 0xeb, 0xef, 0x79, 0x1b, 0xd2, 0xa3, 0x53, 0x14, 0x6a, 0x43, - 0xfa, 0x7a, 0xda, 0x28, 0x7e, 0x17, 0xe2, 0x5c, 0x82, 0x10, 0xa4, 0x3f, 0x2a, 0x37, 0xb6, 0x1b, - 0x1b, 0x2b, 0xda, 0xad, 0x4d, 0x55, 0x5b, 0xa9, 0x2a, 0x33, 0x68, 0x1e, 0xe4, 0x5a, 0xfd, 0x76, - 0x9d, 0x0a, 0x15, 0x09, 0xcd, 0x41, 0x82, 0x7d, 0xd5, 0x6b, 0x4a, 0xa4, 0x58, 0x01, 0x85, 0x63, - 0xef, 0x62, 0x1a, 0x18, 0x68, 0xde, 0x8f, 0x4a, 0xb0, 0xc0, 0x92, 0xf4, 0x2e, 0xcd, 0xdf, 0x68, - 0x28, 0xd4, 0x42, 0xd9, 0x73, 0x36, 0x28, 0xa2, 0x41, 0x71, 0x43, 0xef, 0xe2, 0xe2, 0xdf, 0xc7, - 0x20, 0x3b, 0x04, 0xf1, 0xc3, 0xe2, 0x2b, 0x20, 0x13, 0xcb, 0x3e, 0xd0, 0x86, 0x97, 0xe6, 0xfc, - 0x20, 0xcd, 0xb2, 0x0f, 0x76, 0xd4, 0x86, 0x9a, 0xa0, 0x85, 0x3b, 0xae, 0x85, 0x9a, 0x10, 0x73, - 0x7a, 0x9e, 0xbf, 0x87, 0x7c, 0xfb, 0x14, 0x53, 0x4c, 0xb4, 0x51, 0xda, 0xec, 0x79, 0xe2, 0x04, - 0x80, 0x61, 0xa0, 0xbf, 0x96, 0x86, 0xbb, 0x1e, 0xbe, 0x5b, 0x7c, 0xf7, 0x5c, 0x78, 0xdc, 0x00, - 0xe2, 0x06, 0xf3, 0x23, 0xba, 0x50, 0x9f, 0x0c, 0x0a, 0xd9, 0x71, 0x03, 0x91, 0x67, 0xbc, 0xda, - 0xf4, 0xbb, 0x88, 0x9a, 0xfc, 0x52, 0x6d, 0x68, 0x68, 0x16, 0x10, 0xa6, 0xbc, 0xb7, 0x4c, 0x8d, - 0x4c, 0x44, 0x7e, 0x0f, 0xe6, 0xc3, 0xbd, 0x3f, 0xe6, 0xb4, 0xbc, 0x3c, 0x7a, 0x2c, 0xf1, 0xda, - 0x54, 0x96, 0x11, 0x5b, 0xc2, 0xd0, 0x11, 0xfd, 0x77, 0x21, 0x19, 0x98, 0xfd, 0x3c, 0x67, 0xfb, - 0x9c, 0xe3, 0x83, 0xc3, 0xb2, 0x59, 0x25, 0x5e, 0xfc, 0x1b, 0x09, 0xe6, 0x55, 0x4c, 0x9c, 0xce, - 0x7d, 0x6c, 0xd2, 0x9c, 0x27, 0x78, 0xbd, 0x22, 0x4d, 0xff, 0x7a, 0xa5, 0x0c, 0xc9, 0xe0, 0x34, - 0xf3, 0x3c, 0xef, 0x3a, 0x86, 0xb5, 0xd0, 0x55, 0x50, 0xda, 0x4e, 0xdf, 0x36, 0x75, 0xf7, 0x50, - 0x73, 0xb1, 0x6e, 0xec, 0x63, 0x53, 0xdc, 0xdc, 0x64, 0x7c, 0xb9, 0xca, 0xc5, 0xc5, 0x1f, 0x45, - 0x00, 0x0d, 0x8d, 0x13, 0xa2, 0x22, 0x9a, 0xb9, 0xb1, 0x71, 0x88, 0xcd, 0x6b, 0xe4, 0xd8, 0xbb, - 0xb6, 0xb1, 0x04, 0x30, 0x18, 0xb8, 0x3f, 0xa5, 0x6e, 0x48, 0x46, 0xd0, 0x8f, 0x4f, 0x3f, 0xb3, - 0x8d, 0xb2, 0x33, 0x5b, 0xe6, 0xa5, 0xbf, 0xd4, 0x73, 0x5b, 0x11, 0xac, 0xff, 0x27, 0x06, 0xa8, - 0xea, 0x62, 0xdd, 0xc3, 0x94, 0x79, 0xc8, 0x69, 0x7b, 0xed, 0x0a, 0xcc, 0xf2, 0x8d, 0x59, 0xe4, - 0x3c, 0x1b, 0x33, 0x61, 0x14, 0x5e, 0x15, 0x7d, 0x1f, 0xe6, 0x0d, 0xa7, 0xd3, 0xef, 0xda, 0x1a, - 0xbb, 0x71, 0x16, 0x89, 0xf0, 0x77, 0x4e, 0x73, 0xe2, 0x89, 0xce, 0x95, 0xaa, 0x4e, 0x87, 0x7e, - 0xfb, 0x67, 0x07, 0x1c, 0x90, 0x69, 0xa0, 0xe7, 0x21, 0x19, 0x2c, 0x28, 0x96, 0x02, 0x27, 0xd5, - 0xa1, 0x00, 0x5d, 0x87, 0x59, 0x9d, 0x68, 0xce, 0x2e, 0xcb, 0x51, 0xcf, 0xf2, 0x30, 0x35, 0xa6, - 0x93, 0xcd, 0x5d, 0xf4, 0x16, 0xa4, 0x76, 0xef, 0xf1, 0xbc, 0x9d, 0x13, 0x28, 0x7f, 0x08, 0x90, - 0x39, 0x1a, 0x14, 0xe6, 0x6e, 0xdd, 0x61, 0x83, 0xa5, 0xf4, 0xa9, 0xce, 0xed, 0xde, 0x0b, 0x3e, - 0xd0, 0x35, 0xc8, 0x76, 0xf5, 0x87, 0xda, 0xae, 0xab, 0x1b, 0x22, 0x51, 0xed, 0x70, 0x56, 0x90, - 0xd4, 0x4c, 0x57, 0x7f, 0x78, 0x4b, 0xc8, 0x1b, 0x66, 0x07, 0xe7, 0xff, 0x5b, 0x82, 0x84, 0x18, - 0x11, 0xea, 0x01, 0x08, 0xf3, 0x58, 0x26, 0x4f, 0x85, 0x52, 0x95, 0x3b, 0x47, 0x83, 0x42, 0xb2, - 0xca, 0xa4, 0x8d, 0x1a, 0x79, 0x32, 0x28, 0x7c, 0xf8, 0xb4, 0xa4, 0xe5, 0x83, 0xa8, 0x49, 0xde, - 0x48, 0xc3, 0x64, 0x07, 0x8a, 0xfb, 0x3a, 0xd1, 0xf6, 0x2d, 0xe2, 0x39, 0x7b, 0xae, 0xde, 0x65, - 0x93, 0x2b, 0xab, 0xf3, 0xfb, 0x3a, 0x59, 0xf5, 0x65, 0x28, 0x4f, 0x53, 0x8e, 0xfb, 0xfc, 0xc1, - 0x00, 0x5f, 0x52, 0xc1, 0x37, 0xba, 0x0e, 0x17, 0x82, 0xca, 0x1a, 0x1d, 0x74, 0xbb, 0x6f, 0x1c, - 0x60, 0x16, 0x09, 0x28, 0x67, 0x2d, 0x04, 0x85, 0xeb, 0xfa, 0xc3, 0x0a, 0x2f, 0x2a, 0x5e, 0x80, - 0x85, 0xd0, 0xb4, 0x06, 0x09, 0x22, 0x06, 0x65, 0xdd, 0xda, 0x73, 0xf5, 0xf0, 0xcb, 0xbb, 0x3b, - 0x90, 0x19, 0x7b, 0xd9, 0x2a, 0x68, 0x25, 0x7c, 0x10, 0x36, 0xfa, 0x14, 0xb6, 0x54, 0xe5, 0x9f, - 0x7e, 0xca, 0x9d, 0x36, 0x46, 0xbe, 0x8b, 0x0b, 0x90, 0x0d, 0x9a, 0x09, 0xda, 0xfe, 0xc5, 0x3c, - 0x24, 0xb6, 0xf4, 0xc3, 0x8e, 0xa3, 0x9b, 0x68, 0x09, 0xe6, 0xfc, 0x47, 0x06, 0x7e, 0x7b, 0x49, - 0x35, 0x2c, 0x42, 0x16, 0xa4, 0xfb, 0x04, 0xbb, 0xd4, 0x1f, 0x34, 0xf6, 0xd0, 0x96, 0xf3, 0x64, - 0xa5, 0xf2, 0x64, 0x50, 0xb8, 0x39, 0xdd, 0xf4, 0x60, 0xa3, 0xef, 0x5a, 0xde, 0x61, 0xa9, 0x75, - 0xe7, 0xf6, 0x8e, 0x80, 0xa2, 0x8b, 0xd8, 0x51, 0x53, 0xfd, 0xf0, 0xa7, 0x78, 0xb2, 0x41, 0x4d, - 0xad, 0x75, 0x2d, 0xc3, 0x75, 0x88, 0x7f, 0xb2, 0x2f, 0xa4, 0xeb, 0x4c, 0x88, 0x5e, 0x85, 0xcc, - 0xae, 0x65, 0xb3, 0x5b, 0x25, 0x5f, 0x8f, 0x1f, 0xea, 0xa7, 0x7d, 0xb1, 0x50, 0xbc, 0x0f, 0xe9, - 0xd0, 0x33, 0x0d, 0xea, 0x66, 0x71, 0xe6, 0x66, 0x9b, 0x47, 0x83, 0x42, 0x6a, 0xb8, 0x6c, 0xb9, - 0xab, 0x3d, 0x4b, 0x7c, 0x4c, 0x0d, 0x9b, 0xa1, 0x8e, 0xb6, 0x08, 0xb3, 0xec, 0x19, 0x36, 0x7f, - 0x99, 0xa5, 0xf2, 0x0f, 0xf4, 0x36, 0xcc, 0x76, 0xb0, 0x4e, 0xb0, 0x78, 0x74, 0xb5, 0x74, 0x0a, - 0x11, 0xb0, 0x77, 0xcc, 0x2a, 0x57, 0x47, 0x15, 0x88, 0xf3, 0x7b, 0x42, 0x76, 0xbb, 0x37, 0x79, - 0x2c, 0x7a, 0xe2, 0xfb, 0xba, 0xd5, 0x19, 0x55, 0xd4, 0x44, 0x75, 0x48, 0xb8, 0xfc, 0x6a, 0x98, - 0xdd, 0xf9, 0x9d, 0xb9, 0xd1, 0x0f, 0xdd, 0x3c, 0xaf, 0xce, 0xa8, 0x7e, 0x5d, 0xb4, 0xed, 0xbf, - 0xcf, 0xe0, 0x11, 0x45, 0x3c, 0x3f, 0x29, 0x4d, 0x99, 0x0c, 0x0e, 0x01, 0x47, 0x50, 0xe8, 0x00, - 0x2d, 0x76, 0x4f, 0xc0, 0xae, 0x0e, 0x4f, 0x1f, 0xe0, 0xc8, 0x9d, 0x33, 0x1d, 0x20, 0xaf, 0x89, - 0x36, 0x00, 0x8c, 0x20, 0xca, 0xb1, 0x4b, 0xc5, 0xb9, 0xeb, 0xaf, 0x9f, 0x27, 0x93, 0x5a, 0x9d, - 0x51, 0x43, 0x08, 0xe8, 0x0e, 0xcc, 0x19, 0xc3, 0x65, 0x9b, 0xcb, 0x30, 0xc0, 0x37, 0xce, 0xc5, - 0xdd, 0xab, 0x94, 0xaf, 0x87, 0xd2, 0x51, 0xbe, 0x56, 0xc6, 0xf9, 0xba, 0x0e, 0x29, 0x71, 0x26, - 0xc3, 0x5f, 0xf0, 0xe7, 0xb2, 0x2c, 0x5c, 0x84, 0xbd, 0xc4, 0x7f, 0xe3, 0x5f, 0xaa, 0xdb, 0x86, - 0x63, 0x62, 0xb3, 0x4e, 0xbf, 0x55, 0x71, 0x04, 0xcd, 0x3e, 0x08, 0x5a, 0x81, 0xb4, 0xd1, 0xc1, - 0xba, 0xdd, 0xef, 0xf9, 0x38, 0x68, 0x4a, 0x9c, 0x94, 0xa8, 0x27, 0x80, 0x36, 0x00, 0xed, 0xb2, - 0x37, 0x1d, 0xe1, 0x5e, 0xb1, 0xbb, 0xc9, 0x69, 0xc0, 0x14, 0x56, 0x57, 0x1d, 0xf6, 0x0c, 0xbd, - 0x04, 0x29, 0xdb, 0xb1, 0x0d, 0xdd, 0x36, 0x70, 0x87, 0x45, 0x56, 0x7e, 0x9d, 0x39, 0x2a, 0x44, - 0x9f, 0x42, 0x9a, 0x8c, 0x6c, 0x1f, 0x72, 0x17, 0x58, 0x8b, 0x6f, 0x9e, 0xf7, 0xc0, 0x71, 0x75, - 0x46, 0x1d, 0x43, 0x42, 0xbf, 0x05, 0x8a, 0x37, 0x76, 0x2b, 0xc1, 0x2e, 0x46, 0x4f, 0x7f, 0x3f, - 0x75, 0xc2, 0xdd, 0xcb, 0xea, 0x8c, 0x3a, 0x81, 0x86, 0x3e, 0x83, 0x0c, 0x19, 0x7d, 0x53, 0x9c, - 0xbb, 0xc4, 0x1a, 0xf8, 0xf6, 0xa9, 0x47, 0xeb, 0xc7, 0x3d, 0xc3, 0x5e, 0x9d, 0x51, 0xc7, 0xb1, - 0x28, 0xbc, 0x3d, 0x7a, 0xb9, 0x91, 0xcb, 0x9d, 0x09, 0x7f, 0xfc, 0x65, 0x0b, 0x85, 0x1f, 0xc3, - 0x42, 0x6b, 0x90, 0xec, 0xfa, 0xb1, 0x22, 0xf7, 0xdc, 0x99, 0x19, 0xf7, 0x78, 0xf8, 0x5a, 0x9d, - 0x51, 0x87, 0xf5, 0x2b, 0x49, 0x48, 0x88, 0x4b, 0xb0, 0xe0, 0x6a, 0x39, 0xa1, 0xc8, 0xc5, 0x5f, - 0xc8, 0x20, 0x07, 0x39, 0xe8, 0x32, 0xa0, 0x20, 0x6b, 0x18, 0xbe, 0xe7, 0xa3, 0x21, 0x28, 0xb2, - 0x3a, 0xa3, 0x66, 0xfd, 0xb2, 0xe1, 0x93, 0xbe, 0x57, 0x21, 0xd3, 0x75, 0x4c, 0x6b, 0xd7, 0x1a, - 0x12, 0x3f, 0x3f, 0xf2, 0x4c, 0xfb, 0x62, 0x41, 0xfc, 0x37, 0x47, 0x9e, 0xbd, 0x4c, 0xf3, 0x40, - 0x9c, 0xf6, 0x3e, 0x78, 0x17, 0x43, 0x03, 0x91, 0xdb, 0xb7, 0xd9, 0x7d, 0x97, 0xd8, 0xf6, 0xf2, - 0xfc, 0x2a, 0x25, 0xa4, 0x62, 0xe7, 0x5a, 0x1d, 0x63, 0xe6, 0xab, 0x67, 0x32, 0xb3, 0x3f, 0xf6, - 0x55, 0x29, 0xa0, 0xe6, 0x5b, 0xe3, 0xd4, 0x7c, 0xed, 0x6c, 0x6a, 0x0e, 0xc1, 0x04, 0xdc, 0xbc, - 0x73, 0x2c, 0x37, 0x2f, 0x4f, 0xb9, 0x70, 0x42, 0x88, 0xa3, 0xe4, 0x5c, 0x1d, 0x23, 0xe7, 0xab, - 0x67, 0x92, 0x73, 0x78, 0x8c, 0x82, 0x9d, 0x37, 0x8f, 0x61, 0xe7, 0x37, 0xa6, 0x62, 0xe7, 0x10, - 0x58, 0x98, 0x9e, 0xd5, 0xe3, 0xe8, 0xb9, 0x34, 0x1d, 0x3d, 0x87, 0x20, 0x47, 0xf8, 0xf9, 0x7b, - 0x13, 0xdc, 0xa3, 0x9c, 0xbd, 0x78, 0x8f, 0x3d, 0xeb, 0x58, 0x95, 0x26, 0xc8, 0x47, 0x3f, 0x86, - 0x7c, 0xb2, 0x0c, 0xfe, 0xad, 0x73, 0x90, 0x4f, 0xa8, 0x81, 0x49, 0xf6, 0xf9, 0x18, 0xe6, 0xc3, - 0x8c, 0xc1, 0x9e, 0x97, 0x9c, 0xce, 0x6d, 0x27, 0xfc, 0x01, 0x04, 0xf3, 0x81, 0x50, 0x11, 0xfa, - 0xfe, 0x24, 0xf1, 0x2c, 0x9c, 0x09, 0x7e, 0xc2, 0x3d, 0xec, 0xaa, 0x34, 0xc9, 0x3c, 0xb7, 0xc3, - 0xcc, 0xb3, 0x78, 0x66, 0xec, 0x9e, 0xc8, 0x68, 0x57, 0xa5, 0x30, 0xf5, 0x00, 0xc8, 0xfe, 0x35, - 0x7f, 0x88, 0x86, 0x8a, 0x7f, 0x22, 0x41, 0xb4, 0xe9, 0xb4, 0x51, 0x7a, 0x78, 0x08, 0xc7, 0x8e, - 0xcf, 0x3e, 0x18, 0xaa, 0x8b, 0xdd, 0xde, 0x8b, 0xa7, 0xb4, 0x1d, 0x1c, 0x5a, 0x06, 0x95, 0xd0, - 0x7b, 0x90, 0xe8, 0xf1, 0x6c, 0x5a, 0x30, 0x4d, 0xf1, 0xb4, 0xfa, 0x5c, 0x53, 0xf5, 0xab, 0x5c, - 0xbb, 0x1a, 0xfe, 0x5b, 0xa8, 0x75, 0xc7, 0xc4, 0x28, 0x0d, 0xb0, 0xa5, 0x13, 0xd2, 0xdb, 0x77, - 0x75, 0x82, 0x95, 0x19, 0x94, 0x80, 0xe8, 0xda, 0x7a, 0x4b, 0x91, 0xae, 0x7d, 0x1c, 0x3e, 0x32, - 0xab, 0xa9, 0xe5, 0xc6, 0x46, 0x63, 0x63, 0x45, 0xdb, 0x28, 0xaf, 0xd7, 0x5b, 0xca, 0x0c, 0xca, - 0xc1, 0xe2, 0x47, 0xe5, 0xc6, 0xb6, 0x38, 0x43, 0xd3, 0x1a, 0x1b, 0xdb, 0x75, 0xf5, 0x6e, 0xf9, - 0xb6, 0x22, 0xa1, 0x8b, 0x80, 0xd4, 0xcd, 0xea, 0x5a, 0xab, 0x56, 0xd1, 0xaa, 0x9b, 0xeb, 0x5b, - 0xe5, 0xea, 0x76, 0x63, 0x73, 0x43, 0x89, 0x20, 0x19, 0x62, 0xb5, 0xcd, 0x8d, 0xba, 0x02, 0xd7, - 0x7e, 0x1c, 0x83, 0x18, 0xf5, 0x35, 0xf4, 0x12, 0xcc, 0xed, 0x6c, 0xb4, 0xb6, 0xea, 0xd5, 0xc6, - 0xad, 0x46, 0xbd, 0xa6, 0xcc, 0xe4, 0x17, 0x1e, 0x3d, 0x5e, 0xca, 0xd0, 0xa2, 0x1d, 0x9b, 0xf4, - 0xb0, 0xc1, 0x48, 0x16, 0xe5, 0x21, 0x5e, 0x29, 0x57, 0xd7, 0x76, 0xb6, 0x14, 0x29, 0x9f, 0x7e, - 0xf4, 0x78, 0x09, 0xa8, 0x02, 0x27, 0x38, 0xf4, 0x3c, 0x24, 0xd4, 0x7a, 0x6b, 0x7b, 0x53, 0xad, - 0x2b, 0x91, 0x7c, 0xe6, 0xd1, 0xe3, 0xa5, 0x39, 0x5a, 0x28, 0x78, 0x0b, 0xbd, 0x0a, 0xa9, 0x56, - 0x75, 0xb5, 0xbe, 0x5e, 0xd6, 0xaa, 0xab, 0xe5, 0x8d, 0x95, 0xba, 0x12, 0xcd, 0x2f, 0x3e, 0x7a, - 0xbc, 0xa4, 0x8c, 0x3b, 0x3a, 0x6d, 0xa2, 0xb1, 0xbe, 0xb5, 0xa9, 0x6e, 0x2b, 0xb1, 0x61, 0x13, - 0x9c, 0x5f, 0x50, 0x11, 0x80, 0xd7, 0xbe, 0x55, 0xaf, 0xd7, 0x94, 0xd9, 0x3c, 0x7a, 0xf4, 0x78, - 0x29, 0x4d, 0xcb, 0x87, 0xb4, 0x81, 0x5e, 0x86, 0xf9, 0xaa, 0x5a, 0x2f, 0x6f, 0xd7, 0xb5, 0xd6, - 0x76, 0x79, 0xbb, 0xa5, 0xc4, 0x87, 0x23, 0x09, 0x51, 0x01, 0x2a, 0x41, 0xb6, 0xbc, 0xb3, 0xbd, - 0xa9, 0x8d, 0xe8, 0x26, 0xf2, 0x97, 0x1e, 0x3d, 0x5e, 0x5a, 0xa0, 0xba, 0xe5, 0xbe, 0xe7, 0x84, - 0xf5, 0x5f, 0x07, 0x65, 0xa4, 0xff, 0xda, 0x4a, 0x55, 0x91, 0xf3, 0x17, 0x1f, 0x3d, 0x5e, 0x42, - 0xe3, 0x43, 0x58, 0xa9, 0xa2, 0x5f, 0x87, 0x8b, 0xdb, 0x9f, 0x6c, 0xd5, 0x6b, 0xf5, 0x56, 0x55, - 0x1b, 0x1d, 0x76, 0x32, 0x9f, 0x7b, 0xf4, 0x78, 0x69, 0x91, 0xd6, 0x99, 0x18, 0xfa, 0x1b, 0xa0, - 0xb4, 0xb6, 0xd5, 0x7a, 0x79, 0x5d, 0x6b, 0x6c, 0xac, 0xd4, 0x5b, 0x6c, 0xb2, 0x60, 0xd8, 0xa5, - 0xb1, 0x45, 0x4b, 0x87, 0xb0, 0x51, 0xff, 0x68, 0x0c, 0x7f, 0x6e, 0xa8, 0x3f, 0xb6, 0x0e, 0xd1, - 0x12, 0x24, 0xd7, 0x1b, 0x2b, 0x6a, 0x99, 0xe1, 0xce, 0xe7, 0xb3, 0x8f, 0x1e, 0x2f, 0xa5, 0xa8, - 0x5e, 0xb0, 0xaa, 0xf2, 0xf2, 0x8f, 0xfe, 0xf2, 0xf2, 0xcc, 0x5f, 0xfd, 0xe4, 0xf2, 0x4c, 0xe5, - 0xca, 0x17, 0xff, 0x79, 0x79, 0xe6, 0x8b, 0xa3, 0xcb, 0xd2, 0xcf, 0x8e, 0x2e, 0x4b, 0x5f, 0x1e, - 0x5d, 0x96, 0xfe, 0xe3, 0xe8, 0xb2, 0xf4, 0x47, 0x5f, 0x5d, 0x9e, 0xf9, 0xd9, 0x57, 0x97, 0x67, - 0xbe, 0xfc, 0xea, 0xf2, 0xcc, 0xa7, 0x71, 0xee, 0xd7, 0xed, 0x38, 0xdb, 0x1a, 0xbe, 0xf5, 0xff, - 0x01, 0x00, 0x00, 0xff, 0xff, 0xdb, 0x22, 0x5f, 0x7b, 0xbc, 0x3a, 0x00, 0x00, + 0x9b, 0x4d, 0x72, 0x1e, 0x1c, 0xc9, 0xde, 0xbd, 0x48, 0xec, 0xbf, 0xfe, 0xfa, 0xea, 0xf5, 0xd7, + 0xf7, 0xff, 0x7f, 0x55, 0x0d, 0x5c, 0xfc, 0x6d, 0xa7, 0x4d, 0x96, 0xe9, 0x3f, 0xbd, 0x36, 0xfb, + 0xaf, 0xd4, 0x73, 0x1d, 0xcf, 0x41, 0xcf, 0x19, 0x8e, 0x71, 0xe0, 0x3a, 0xba, 0xb1, 0x5f, 0x22, + 0xf7, 0x3a, 0x25, 0x56, 0xc2, 0xb5, 0xf2, 0x17, 0xb0, 0xeb, 0x3a, 0x2e, 0xd5, 0xe7, 0x3f, 0x78, + 0x8d, 0xfc, 0xe2, 0x9e, 0xb3, 0xe7, 0xb0, 0x9f, 0xcb, 0xf4, 0x97, 0x90, 0x22, 0x86, 0xd1, 0x6b, + 0x2f, 0x9b, 0xba, 0xa7, 0x0b, 0x59, 0xce, 0x97, 0x59, 0xce, 0x1b, 0xbb, 0x8e, 0xdb, 0xd5, 0x3d, + 0x1f, 0xe3, 0x45, 0x72, 0xaf, 0xb3, 0x6c, 0xe8, 0x9e, 0xde, 0x71, 0xf6, 0x96, 0x4d, 0x4c, 0x8c, + 0x5e, 0x7b, 0x99, 0x78, 0x6e, 0xdf, 0xf0, 0xfa, 0x2e, 0x36, 0x85, 0x52, 0xe1, 0x18, 0x25, 0x0f, + 0xdb, 0xba, 0xed, 0xf9, 0xf8, 0x7d, 0xcf, 0xea, 0x2c, 0xef, 0x77, 0x8c, 0x65, 0xcf, 0xea, 0x62, + 0xe2, 0xe9, 0xdd, 0x9e, 0x28, 0xf9, 0x16, 0xad, 0x4a, 0x8c, 0x7d, 0xdc, 0xd5, 0x8d, 0x7d, 0xdd, + 0xde, 0xc3, 0xee, 0x32, 0x6f, 0xc3, 0xe8, 0xb5, 0x85, 0xca, 0x4b, 0x46, 0xa7, 0x4f, 0x3c, 0xec, + 0xde, 0xc7, 0x2e, 0xb1, 0x1c, 0x7b, 0x59, 0x7c, 0x6a, 0xe2, 0x9b, 0x6b, 0x15, 0x7f, 0x0f, 0x66, + 0x6f, 0x63, 0x9d, 0x60, 0xf4, 0x29, 0x24, 0x6c, 0xc7, 0xc4, 0x9a, 0x65, 0xe6, 0xa4, 0x25, 0xe9, + 0x4a, 0xba, 0x52, 0x3e, 0x1a, 0x14, 0xe2, 0x1b, 0x8e, 0x89, 0x1b, 0xb5, 0x27, 0x83, 0xc2, 0x5b, + 0x7b, 0x96, 0xb7, 0xdf, 0x6f, 0x97, 0x0c, 0xa7, 0xbb, 0x1c, 0xcc, 0xa8, 0xd9, 0x1e, 0xfe, 0x5e, + 0xee, 0x1d, 0xec, 0x2d, 0x8b, 0xf9, 0x28, 0xf1, 0x6a, 0x6a, 0x9c, 0x22, 0x36, 0x4c, 0xb4, 0x08, + 0xb3, 0xb8, 0xe7, 0x18, 0xfb, 0xb9, 0xc8, 0x92, 0x74, 0x25, 0xaa, 0xf2, 0x8f, 0x1b, 0xb1, 0xff, + 0xfa, 0x8b, 0x82, 0x54, 0xfc, 0x69, 0x04, 0x2e, 0x55, 0x74, 0xe3, 0xa0, 0xdf, 0xab, 0xdb, 0x86, + 0x7b, 0xd8, 0xf3, 0x2c, 0xc7, 0xde, 0x64, 0xff, 0x12, 0xa4, 0x40, 0xf4, 0x00, 0x1f, 0xb2, 0xfe, + 0xcc, 0xa9, 0xf4, 0x27, 0x7a, 0x1f, 0x62, 0x5d, 0xc7, 0xc4, 0x0c, 0x28, 0x73, 0xfd, 0x6a, 0xe9, + 0xc4, 0xc5, 0x2d, 0x0d, 0xd1, 0xd6, 0x1d, 0x13, 0xab, 0xac, 0x1a, 0x6a, 0x83, 0x7c, 0xd0, 0x25, + 0x9a, 0x65, 0xef, 0x3a, 0xb9, 0xe8, 0x92, 0x74, 0x25, 0x75, 0xfd, 0xc6, 0x29, 0x10, 0x27, 0x74, + 0xab, 0xb4, 0xb6, 0xde, 0x6a, 0xd8, 0xbb, 0x4e, 0x25, 0x75, 0x34, 0x28, 0x24, 0xc4, 0x87, 0x9a, + 0x38, 0xe8, 0x12, 0xfa, 0x23, 0xbf, 0x09, 0xbe, 0x8c, 0xf6, 0xbf, 0xef, 0x5a, 0xac, 0xff, 0x49, + 0x95, 0xfe, 0x44, 0xaf, 0x03, 0xc2, 0x1c, 0x0f, 0x9b, 0x1a, 0xb5, 0x24, 0x8d, 0x0e, 0x30, 0xc2, + 0x06, 0xa8, 0x04, 0x25, 0x35, 0xdd, 0xd3, 0xd7, 0xf0, 0x21, 0x9f, 0x21, 0x31, 0x4f, 0xbf, 0x1f, + 0x85, 0xcc, 0xb0, 0x2b, 0x0c, 0x7e, 0x15, 0xe2, 0xcc, 0x04, 0x30, 0x6b, 0x21, 0x73, 0xfd, 0xcd, + 0xa9, 0xa6, 0x83, 0x56, 0x2d, 0xb5, 0x58, 0x3d, 0x55, 0xd4, 0x47, 0x08, 0x62, 0x44, 0xef, 0x78, + 0xa2, 0x23, 0xec, 0x37, 0xfa, 0x33, 0x09, 0x96, 0xc6, 0x7b, 0x54, 0x39, 0x5c, 0x5b, 0x6f, 0xad, + 0xeb, 0xd4, 0x8e, 0xd6, 0xf0, 0x61, 0xa3, 0x96, 0x8b, 0x2e, 0x45, 0xaf, 0xa4, 0xae, 0x6f, 0x4e, + 0xdf, 0x70, 0xfd, 0x0c, 0xc4, 0xba, 0xed, 0xb9, 0x87, 0xea, 0x99, 0x0d, 0xe7, 0x5b, 0xf0, 0xf2, + 0x54, 0x50, 0x61, 0x1b, 0x4a, 0x72, 0x1b, 0x5a, 0x84, 0xd9, 0xfb, 0x7a, 0xa7, 0x8f, 0xc5, 0x68, + 0xf9, 0xc7, 0x8d, 0xc8, 0x3b, 0x52, 0xf1, 0x12, 0xc4, 0xf9, 0xc4, 0xa0, 0x34, 0x24, 0xcb, 0xf5, + 0xd6, 0xf5, 0xef, 0xbc, 0xbd, 0x52, 0x5d, 0x57, 0x66, 0xc4, 0x12, 0xfc, 0x9f, 0x04, 0x17, 0x5b, + 0x9e, 0x8b, 0xf5, 0x6e, 0xc3, 0xde, 0xc3, 0x84, 0x8e, 0xa9, 0x86, 0x3d, 0xdd, 0xea, 0x10, 0x64, + 0x43, 0x86, 0xb0, 0x12, 0x4d, 0x37, 0x4d, 0x17, 0x13, 0xc2, 0x1b, 0xac, 0xac, 0x3c, 0x19, 0x14, + 0xaa, 0x53, 0x6d, 0x1d, 0xc3, 0xe8, 0x2c, 0x73, 0x08, 0xcb, 0xde, 0x33, 0x8c, 0x4e, 0x89, 0xb7, + 0x54, 0xe6, 0x70, 0x6a, 0x9a, 0x84, 0x3f, 0xd1, 0xb7, 0x21, 0x46, 0x7a, 0xba, 0xcd, 0x86, 0x90, + 0xba, 0x7e, 0x29, 0x34, 0xff, 0xfe, 0x16, 0x6c, 0xf5, 0x74, 0xbb, 0x12, 0xfb, 0x62, 0x50, 0x98, + 0x51, 0x99, 0x2a, 0xaa, 0x00, 0x10, 0x4f, 0x77, 0x3d, 0x8d, 0x72, 0x89, 0xb0, 0xfe, 0x17, 0x42, + 0x15, 0x29, 0xd7, 0x94, 0xf6, 0x3b, 0x46, 0x69, 0xdb, 0xe7, 0x1a, 0x51, 0x3d, 0xc9, 0xaa, 0x51, + 0x69, 0x51, 0x87, 0x4b, 0x63, 0x13, 0xb0, 0xe5, 0x3a, 0x7b, 0xac, 0x47, 0xb7, 0x60, 0xce, 0xe8, + 0x7b, 0xce, 0x7d, 0xec, 0xf2, 0x06, 0xa4, 0xe9, 0x1b, 0x48, 0x89, 0x8a, 0xac, 0x89, 0x7f, 0x8a, + 0x43, 0x9a, 0x6f, 0x3c, 0x7f, 0x6e, 0x47, 0x3b, 0x2e, 0x3d, 0x4d, 0xc7, 0xd1, 0x4d, 0x90, 0xb1, + 0x6d, 0x72, 0x84, 0xc8, 0xf4, 0x08, 0x09, 0x6c, 0x9b, 0xac, 0xfe, 0x73, 0x7c, 0x27, 0x47, 0xd9, + 0xa2, 0x26, 0x8e, 0x06, 0x85, 0xe8, 0x8e, 0xda, 0xe0, 0x5b, 0xfa, 0x3d, 0xc8, 0x9b, 0xb8, 0xe7, + 0x62, 0x43, 0xa7, 0x7b, 0xba, 0xcd, 0xba, 0xae, 0x75, 0x75, 0xdb, 0xda, 0xc5, 0xc4, 0xcb, 0xc5, + 0x98, 0x8d, 0xe5, 0x86, 0x1a, 0x7c, 0x6c, 0xeb, 0xa2, 0x1c, 0xfd, 0x81, 0x04, 0x0b, 0x7d, 0xd7, + 0x22, 0x5a, 0xfb, 0x50, 0xeb, 0x38, 0x86, 0xde, 0xb1, 0xbc, 0x43, 0xed, 0xe0, 0x7e, 0x6e, 0x96, + 0x6d, 0xac, 0x9b, 0x67, 0xb2, 0x93, 0x98, 0xa4, 0xd2, 0x8e, 0x6b, 0x91, 0xca, 0xe1, 0x6d, 0x81, + 0xb0, 0x76, 0x9f, 0x19, 0x7f, 0x65, 0xf1, 0x68, 0x50, 0x50, 0x76, 0xd4, 0x46, 0xb8, 0xe8, 0xae, + 0xaa, 0xf4, 0xc7, 0x94, 0x91, 0x1e, 0xd0, 0x92, 0xe5, 0xd8, 0x9a, 0xc3, 0x79, 0x2e, 0x17, 0x67, + 0x13, 0x75, 0xfd, 0xfc, 0x0c, 0xa9, 0x66, 0xf1, 0x04, 0x97, 0xff, 0xb1, 0x04, 0x79, 0xea, 0x72, + 0xb0, 0x41, 0xa7, 0x29, 0xf0, 0x67, 0x9a, 0x8b, 0x0d, 0xc7, 0x35, 0x73, 0x09, 0x3a, 0x4f, 0x95, + 0xd6, 0xbf, 0x4f, 0xeb, 0x69, 0x98, 0x67, 0xec, 0xf7, 0x2d, 0xb3, 0xb4, 0xb3, 0xd3, 0xa8, 0x1d, + 0x0d, 0x0a, 0xb9, 0x2d, 0x1f, 0x3c, 0x58, 0x44, 0x95, 0x41, 0xab, 0xb9, 0xde, 0x09, 0x25, 0xe8, + 0x1d, 0xc8, 0x18, 0x4e, 0xa7, 0x83, 0x0d, 0x36, 0xec, 0x1d, 0xb5, 0x91, 0x93, 0xd9, 0x02, 0x67, + 0x8f, 0x06, 0x85, 0x74, 0x35, 0x28, 0xa1, 0x4b, 0x9d, 0x36, 0xc2, 0x9f, 0x48, 0x85, 0xf9, 0xd0, + 0x84, 0x31, 0x7f, 0x92, 0x64, 0xb3, 0x75, 0x75, 0x6a, 0x2a, 0x54, 0x33, 0x78, 0xe4, 0x3b, 0x5f, + 0x85, 0x0b, 0xc7, 0xae, 0xe2, 0x59, 0x14, 0x96, 0x0c, 0x53, 0x98, 0x02, 0x19, 0xbe, 0x28, 0xfe, + 0xc6, 0x2c, 0x7e, 0x9e, 0x81, 0x8c, 0x8a, 0x89, 0xe7, 0xb8, 0xd8, 0xdf, 0x51, 0x9f, 0x4b, 0xb0, + 0x40, 0xe3, 0x0d, 0xd7, 0xea, 0x79, 0x8e, 0xab, 0xb9, 0xf8, 0x81, 0x6b, 0x79, 0x98, 0xe4, 0x22, + 0xcc, 0xe8, 0xca, 0xa7, 0x0c, 0x61, 0x14, 0xa8, 0x54, 0x0b, 0x40, 0x54, 0x81, 0xc1, 0xed, 0xee, + 0xe6, 0x0f, 0x7f, 0x5e, 0xb8, 0x31, 0xd5, 0x3a, 0x4e, 0x86, 0x40, 0xa5, 0x46, 0x4d, 0x45, 0xe6, + 0x04, 0x30, 0x7a, 0x1e, 0x62, 0xd4, 0x6e, 0x99, 0xcb, 0x49, 0x56, 0xe4, 0xa3, 0x41, 0x21, 0x46, + 0x2d, 0x5b, 0x65, 0xd2, 0x91, 0x0d, 0x1e, 0x7b, 0x8a, 0x0d, 0xbe, 0x02, 0x29, 0x4f, 0x6f, 0x77, + 0xb0, 0x46, 0x5b, 0x26, 0x62, 0xfb, 0xbd, 0x32, 0x36, 0x13, 0xe4, 0x5e, 0xa7, 0xad, 0x13, 0x5c, + 0xda, 0xa6, 0x9a, 0xa1, 0xb1, 0x83, 0xe7, 0x0b, 0x08, 0x5a, 0x86, 0x14, 0xe5, 0x32, 0xd7, 0x32, + 0xb1, 0x66, 0xb6, 0xd9, 0x1e, 0x4a, 0x56, 0x32, 0x47, 0x83, 0x02, 0x6c, 0x0a, 0x71, 0xad, 0xa2, + 0x82, 0xaf, 0x52, 0x6b, 0x23, 0x0f, 0x16, 0x05, 0x69, 0x04, 0xfb, 0x9f, 0xd9, 0x53, 0x82, 0x75, + 0xe1, 0xbd, 0xe9, 0x17, 0x83, 0xaf, 0xbb, 0x6f, 0x3c, 0x2c, 0x42, 0xe1, 0x83, 0x44, 0xed, 0x89, + 0x12, 0xf4, 0x1a, 0x64, 0x7b, 0x2e, 0xee, 0xe9, 0x2e, 0xd6, 0x0c, 0xa7, 0xdb, 0xeb, 0x60, 0x0f, + 0x9b, 0xcc, 0xfa, 0x65, 0x55, 0x11, 0x05, 0x55, 0x5f, 0x8e, 0x5e, 0xa6, 0xde, 0x4d, 0xf7, 0x68, + 0xe0, 0x44, 0xb0, 0x4b, 0x35, 0x93, 0x4c, 0x33, 0xcd, 0xa4, 0x0d, 0x21, 0x44, 0x6f, 0xc1, 0x85, + 0xe1, 0xba, 0x11, 0xad, 0xd7, 0x6f, 0x77, 0x2c, 0xb2, 0x8f, 0xcd, 0x1c, 0x30, 0xed, 0xc5, 0x50, + 0xe1, 0x96, 0x5f, 0x86, 0x0e, 0x47, 0x4c, 0xd1, 0xa0, 0x13, 0xa3, 0xef, 0xe1, 0x5c, 0x6a, 0x49, + 0xba, 0x32, 0x5b, 0x59, 0x7d, 0x32, 0x28, 0xd4, 0xa6, 0xb6, 0x23, 0x82, 0xbb, 0xcb, 0x9e, 0x8b, + 0x71, 0xc8, 0x2c, 0xab, 0x02, 0x2f, 0x6c, 0x51, 0xbe, 0x0c, 0xa9, 0x00, 0xc3, 0x2d, 0x98, 0x9b, + 0x7b, 0x6a, 0xb6, 0x0b, 0xa1, 0xa0, 0x32, 0x24, 0x78, 0x08, 0x4f, 0x72, 0x69, 0xb6, 0x80, 0xdf, + 0x3a, 0xc9, 0x86, 0x98, 0x56, 0x68, 0x95, 0xfc, 0x7a, 0xa8, 0x06, 0xe0, 0x1d, 0xf6, 0x7c, 0x4b, + 0xcc, 0x30, 0x94, 0x97, 0x4f, 0x42, 0x39, 0xec, 0x85, 0x0d, 0x31, 0xe9, 0x89, 0x6f, 0x82, 0x9a, + 0x30, 0xc7, 0xf3, 0x03, 0x81, 0x33, 0xcf, 0x70, 0x5e, 0x3d, 0x01, 0x87, 0x85, 0x3d, 0x7a, 0x08, + 0x29, 0x45, 0x02, 0x09, 0x41, 0x5b, 0x90, 0xa1, 0xb1, 0x2a, 0xd5, 0x14, 0x68, 0x0a, 0x43, 0xbb, + 0x7a, 0x02, 0x5a, 0x4d, 0x28, 0x87, 0xf0, 0xd2, 0x66, 0x48, 0x46, 0xf2, 0xff, 0x2b, 0x41, 0x76, + 0x82, 0x3c, 0xd0, 0x36, 0x44, 0x82, 0xf4, 0x83, 0x72, 0x7a, 0x84, 0xa5, 0x1e, 0xcf, 0x42, 0x24, + 0x11, 0xcb, 0x44, 0x7b, 0x90, 0xa4, 0xe6, 0x6c, 0x7b, 0x34, 0xb7, 0x89, 0x30, 0xf0, 0xe6, 0xd1, + 0xa0, 0x20, 0x6f, 0x31, 0xe1, 0x33, 0x37, 0x21, 0x73, 0xf0, 0x86, 0x89, 0x0a, 0x90, 0xf2, 0x1c, + 0x0d, 0x3f, 0xb4, 0x88, 0x67, 0xd9, 0x7b, 0x2c, 0x58, 0x90, 0x55, 0xf0, 0x9c, 0xba, 0x90, 0xe4, + 0xff, 0x3c, 0x02, 0x68, 0x72, 0x97, 0xa2, 0x7f, 0x94, 0xe0, 0x79, 0x3f, 0x06, 0x70, 0x5c, 0x6b, + 0xcf, 0xb2, 0xf5, 0xce, 0x48, 0x30, 0x20, 0xb1, 0xd9, 0xfe, 0xf4, 0x59, 0xa8, 0x40, 0x04, 0x08, + 0x9b, 0x02, 0x7e, 0x3c, 0x50, 0x78, 0x9e, 0x7a, 0x50, 0x1e, 0x28, 0x4c, 0xa8, 0xdc, 0x55, 0x73, + 0xfd, 0x13, 0x2a, 0xe7, 0xd7, 0xe0, 0x85, 0x53, 0x81, 0xcf, 0xe3, 0xbb, 0xf2, 0x3f, 0x94, 0xe0, + 0xd2, 0x09, 0x1e, 0x25, 0x8c, 0x93, 0xe6, 0x38, 0x77, 0xc2, 0x38, 0xa9, 0xeb, 0xbf, 0xf1, 0x0c, + 0x5e, 0x2b, 0xd4, 0x89, 0x66, 0x4c, 0x96, 0x94, 0x48, 0xf1, 0x4d, 0x98, 0x17, 0x95, 0x82, 0x00, + 0xf7, 0x05, 0x80, 0x7d, 0x6b, 0x6f, 0x5f, 0x7b, 0xa0, 0x7b, 0xd8, 0x15, 0x39, 0x69, 0x92, 0x4a, + 0x3e, 0xa2, 0x82, 0xe2, 0xbf, 0xc9, 0x90, 0x6e, 0x74, 0x7b, 0x8e, 0xeb, 0xf9, 0x5e, 0xf6, 0x36, + 0xc4, 0x99, 0x5f, 0x20, 0x62, 0xfd, 0x4a, 0xa7, 0xf4, 0x70, 0xa4, 0x26, 0xf7, 0x2f, 0x82, 0x16, + 0x04, 0x46, 0xe0, 0xfe, 0x22, 0xc7, 0xba, 0xbf, 0xf7, 0x21, 0xce, 0x0f, 0x20, 0x44, 0x60, 0x5f, + 0x38, 0x26, 0x23, 0x68, 0x6c, 0xde, 0xb2, 0x3a, 0xf8, 0x16, 0x53, 0xf3, 0xc1, 0x79, 0x25, 0xf4, + 0x0a, 0xc8, 0x84, 0x78, 0x1a, 0xb1, 0x7e, 0xc0, 0xbd, 0x67, 0x94, 0xe7, 0xb6, 0xad, 0xd6, 0x76, + 0xcb, 0xfa, 0x01, 0x56, 0x13, 0x84, 0x78, 0xf4, 0x07, 0xca, 0x83, 0xfc, 0x40, 0xef, 0x74, 0x98, + 0x97, 0x9d, 0x65, 0xb9, 0x7c, 0xf0, 0x3d, 0xba, 0xcd, 0xe2, 0xdf, 0xec, 0x36, 0x13, 0x0e, 0xb3, + 0xa7, 0x7b, 0xfb, 0x2c, 0x72, 0x4c, 0xaa, 0xc0, 0x45, 0x5b, 0xba, 0xb7, 0x8f, 0x72, 0x90, 0x20, + 0x3a, 0xf5, 0x5d, 0x24, 0x27, 0x2f, 0x45, 0xaf, 0xcc, 0xa9, 0xfe, 0x27, 0xba, 0x0c, 0xcc, 0xf3, + 0xf2, 0x4f, 0xe6, 0xc4, 0xa2, 0x6a, 0x48, 0xc2, 0xe6, 0xe1, 0xc0, 0xea, 0x69, 0xbb, 0x07, 0x84, + 0x3b, 0x2d, 0x31, 0x0f, 0x07, 0x56, 0xef, 0xd6, 0x1a, 0x51, 0x13, 0xb4, 0xf0, 0xd6, 0x01, 0x41, + 0xaf, 0xc2, 0xbc, 0xc5, 0x32, 0x20, 0xcd, 0xb4, 0x5c, 0x6c, 0x78, 0x9d, 0x43, 0xe6, 0xb0, 0x64, + 0x35, 0xc3, 0xc5, 0x35, 0x21, 0x45, 0x57, 0x41, 0x19, 0x77, 0xb3, 0xcc, 0xd1, 0xc8, 0xea, 0xfc, + 0x98, 0x97, 0xa5, 0xaa, 0x7c, 0xa9, 0x43, 0x8e, 0x33, 0xcd, 0x55, 0xb9, 0x7c, 0xe8, 0x33, 0x4b, + 0xb0, 0xd0, 0xd3, 0x5d, 0x82, 0xb5, 0x76, 0xdf, 0x36, 0x3b, 0x58, 0xe3, 0x5c, 0x9d, 0xcb, 0x30, + 0xed, 0x2c, 0x2b, 0xaa, 0xb0, 0x12, 0x4e, 0xeb, 0x67, 0xc5, 0xde, 0x17, 0x7f, 0x05, 0xb1, 0x77, + 0xfe, 0xa7, 0x11, 0x98, 0x65, 0x76, 0x8e, 0x6e, 0x40, 0x8c, 0x2e, 0xb3, 0xc8, 0xec, 0xa6, 0x8d, + 0xb9, 0x58, 0x1d, 0x84, 0x20, 0x66, 0xeb, 0x5d, 0x9c, 0x43, 0xcc, 0x08, 0xd8, 0x6f, 0x74, 0x09, + 0x12, 0x04, 0xdf, 0xd3, 0xee, 0xeb, 0x9d, 0xdc, 0x02, 0x5b, 0xe1, 0x38, 0xc1, 0xf7, 0xee, 0xea, + 0x1d, 0x74, 0x01, 0xe2, 0x16, 0xd1, 0x6c, 0xfc, 0x20, 0xb7, 0xc8, 0x66, 0x6a, 0xd6, 0x22, 0x1b, + 0xf8, 0x01, 0xa3, 0x6d, 0xdd, 0xdd, 0xc3, 0x9e, 0x66, 0x38, 0x1d, 0x92, 0xbb, 0x40, 0x37, 0x18, + 0x0d, 0xe9, 0xa8, 0xa8, 0xea, 0x74, 0x08, 0xfa, 0x35, 0x48, 0x3e, 0xd0, 0x89, 0x86, 0xbb, 0x3d, + 0xef, 0x90, 0x4d, 0x96, 0x4c, 0xcd, 0x9e, 0xd4, 0xe9, 0x77, 0x33, 0x26, 0x47, 0x94, 0x68, 0x33, + 0x26, 0x47, 0x95, 0x58, 0x33, 0x26, 0xc7, 0x94, 0xd9, 0x66, 0x4c, 0x9e, 0x55, 0xe2, 0xcd, 0x98, + 0x1c, 0x57, 0x12, 0xcd, 0x98, 0x9c, 0x50, 0xe4, 0x66, 0x4c, 0x96, 0x95, 0x64, 0x33, 0x26, 0x27, + 0x15, 0x68, 0xc6, 0x64, 0x50, 0x52, 0xcd, 0x98, 0x9c, 0x52, 0xe6, 0x9a, 0x31, 0x79, 0x4e, 0x49, + 0x37, 0x63, 0x72, 0x5a, 0xc9, 0x34, 0x63, 0x72, 0x46, 0x99, 0x6f, 0xc6, 0xe4, 0x79, 0x45, 0x69, + 0xc6, 0x64, 0x45, 0xc9, 0x36, 0x63, 0x72, 0x56, 0x41, 0xc5, 0xcf, 0x25, 0x50, 0x5a, 0xf8, 0x5e, + 0x1f, 0xdb, 0x06, 0xbe, 0xab, 0x77, 0xaa, 0xfb, 0x7d, 0xfb, 0x00, 0xbd, 0x02, 0xf3, 0x06, 0xfd, + 0xa1, 0xf1, 0xc4, 0x98, 0x0e, 0x55, 0x62, 0x43, 0x4d, 0x33, 0x71, 0x8b, 0x4a, 0xe9, 0x88, 0x5f, + 0x00, 0x10, 0x7a, 0x74, 0x67, 0xf3, 0xd3, 0xb7, 0x24, 0x57, 0xa1, 0xdb, 0x79, 0x0c, 0xc6, 0x75, + 0x1e, 0x30, 0xfa, 0x18, 0x81, 0x51, 0x9d, 0x07, 0x68, 0x19, 0x16, 0x6d, 0xfc, 0xd0, 0xd3, 0xc6, + 0x95, 0x19, 0x55, 0xa8, 0x59, 0x5a, 0x56, 0x0d, 0x57, 0x28, 0xfe, 0x6b, 0x04, 0xe6, 0xfd, 0x4e, + 0xfb, 0x74, 0xb8, 0x0b, 0x0a, 0x5d, 0x16, 0xcb, 0xd4, 0x3c, 0x87, 0x23, 0xf9, 0xc4, 0xf8, 0xfe, + 0x29, 0xc4, 0x38, 0x86, 0x42, 0xbf, 0x1b, 0xe6, 0xb6, 0xc3, 0x9a, 0xe3, 0xae, 0x41, 0x4d, 0x93, + 0xb0, 0x2c, 0xbf, 0x03, 0x19, 0xbf, 0x12, 0x97, 0xa0, 0x2a, 0xc4, 0x47, 0xda, 0x7b, 0x6d, 0x8a, + 0xf6, 0xfc, 0xa9, 0x56, 0x45, 0xd5, 0xfc, 0xef, 0x00, 0x9a, 0x6c, 0x3b, 0xec, 0x96, 0x66, 0xb9, + 0x5b, 0xda, 0x1c, 0x75, 0x4b, 0xef, 0x9e, 0x6f, 0x6c, 0xa1, 0x6e, 0x87, 0xb3, 0xba, 0x7f, 0x8e, + 0x40, 0x86, 0xbb, 0x88, 0xc0, 0x1d, 0xbd, 0x06, 0x59, 0x46, 0x5a, 0x96, 0xbd, 0xa7, 0xf5, 0x84, + 0x90, 0x8d, 0x2f, 0xa2, 0x2a, 0x7e, 0x41, 0xa0, 0xfc, 0x22, 0xa4, 0x5d, 0xac, 0x9b, 0x43, 0xc5, + 0x08, 0x53, 0x9c, 0xa3, 0xc2, 0x40, 0xe9, 0x65, 0xc8, 0x30, 0x6f, 0x38, 0xd4, 0x8a, 0x32, 0xad, + 0x34, 0x93, 0x06, 0x6a, 0x15, 0x48, 0x93, 0x9e, 0x6e, 0x0f, 0xb5, 0x62, 0x6c, 0x52, 0xcf, 0x38, + 0x83, 0x9a, 0xa3, 0x75, 0xc2, 0xbe, 0xd4, 0xc5, 0xa4, 0xdf, 0xc5, 0x5a, 0xcf, 0xe1, 0xc9, 0x56, + 0x54, 0x4d, 0x72, 0xc9, 0x96, 0x43, 0xd0, 0x0e, 0x33, 0x15, 0x36, 0x17, 0x9a, 0xc9, 0x27, 0x27, + 0x17, 0x67, 0xad, 0x5c, 0x9b, 0x7e, 0x3a, 0xd5, 0x79, 0x32, 0x2a, 0x28, 0xfe, 0xad, 0x04, 0x97, + 0x68, 0xc0, 0xcc, 0x59, 0xb1, 0xca, 0xce, 0xcd, 0x7d, 0xeb, 0xd4, 0x21, 0xc1, 0x82, 0xee, 0x20, + 0xfe, 0x5c, 0x3d, 0x1a, 0x14, 0xe2, 0x54, 0xfb, 0x99, 0x3d, 0x57, 0x9c, 0x02, 0x37, 0x58, 0x7a, + 0xe4, 0xb9, 0xba, 0x4d, 0x2c, 0x9a, 0x28, 0xd0, 0x65, 0xeb, 0xe2, 0x6e, 0x1b, 0xbb, 0x7c, 0x31, + 0xe6, 0xd4, 0xc5, 0x91, 0xc2, 0x75, 0x5e, 0x56, 0xcc, 0x43, 0x6e, 0xbc, 0xcb, 0x41, 0x66, 0xff, + 0x9b, 0x70, 0x71, 0x03, 0x3f, 0x38, 0x6e, 0x34, 0x15, 0x48, 0x70, 0xfe, 0xf2, 0x4d, 0xfe, 0xca, + 0x38, 0xab, 0x86, 0xaf, 0x0e, 0x4a, 0xac, 0xa7, 0xdb, 0xac, 0x82, 0xea, 0x57, 0x2c, 0x7e, 0x0a, + 0x97, 0xc6, 0xd0, 0x83, 0xe5, 0xfb, 0x00, 0xe2, 0x34, 0xf3, 0x13, 0x91, 0x4d, 0x66, 0x32, 0xab, + 0x98, 0x44, 0x6f, 0x51, 0x7d, 0x55, 0x54, 0x2b, 0xaa, 0xec, 0x48, 0xa2, 0xdf, 0xc5, 0xd4, 0x42, + 0x6e, 0x5b, 0xc4, 0x43, 0x1f, 0xc2, 0x9c, 0xb0, 0x08, 0x6a, 0x28, 0x7e, 0xb7, 0xcf, 0x30, 0xaa, + 0x94, 0x1b, 0x80, 0x90, 0xe2, 0xdf, 0x49, 0xb0, 0x50, 0x73, 0x9d, 0x5e, 0x0f, 0x9b, 0xc2, 0x57, + 0xf0, 0xb9, 0xf0, 0x5d, 0x84, 0x14, 0x72, 0x11, 0x1b, 0x10, 0x69, 0xd4, 0x44, 0x2e, 0x70, 0xf3, + 0x59, 0x53, 0x8c, 0x46, 0x0d, 0xbd, 0xcb, 0x27, 0xa4, 0x4f, 0x18, 0x7f, 0x66, 0x26, 0x92, 0xbe, + 0x11, 0x33, 0x65, 0x8a, 0xaa, 0xa8, 0x50, 0xfc, 0x49, 0x02, 0x2e, 0x84, 0x27, 0x79, 0xa5, 0xea, + 0x77, 0xfc, 0x33, 0x48, 0x58, 0xb6, 0x89, 0x1f, 0xe2, 0xa9, 0x78, 0xf2, 0x38, 0x88, 0x92, 0x98, + 0x8f, 0x06, 0x85, 0xf1, 0xd3, 0x4c, 0x81, 0x89, 0x3e, 0x0e, 0xc2, 0x53, 0x7e, 0xec, 0x73, 0xe3, + 0xa9, 0xd1, 0x6b, 0x63, 0xa1, 0xea, 0x48, 0x24, 0xc8, 0x1c, 0xca, 0x37, 0x14, 0x09, 0xb6, 0x20, + 0x6b, 0xd9, 0x1e, 0x76, 0x3b, 0x58, 0xbf, 0x4f, 0x03, 0x1b, 0xda, 0xbc, 0x38, 0xfd, 0x99, 0x36, + 0x8c, 0x50, 0x42, 0x00, 0x3c, 0x1c, 0xf9, 0x0c, 0x16, 0xc2, 0xa0, 0xfe, 0x12, 0x9c, 0x7e, 0x22, + 0xc4, 0x66, 0x78, 0x08, 0xeb, 0x1f, 0xbc, 0x84, 0x80, 0x1a, 0x62, 0xda, 0xef, 0x42, 0x9c, 0x27, + 0xfa, 0xe2, 0x78, 0xf5, 0xe6, 0xd3, 0x4e, 0x3b, 0x3f, 0x40, 0x50, 0x05, 0x5a, 0xfe, 0x4f, 0x25, + 0x98, 0x0b, 0x2f, 0x37, 0xb2, 0x40, 0x66, 0x7d, 0xf7, 0x29, 0x2d, 0x5a, 0xd9, 0xa0, 0xb1, 0x2c, + 0x2b, 0x64, 0x6b, 0xf0, 0xc1, 0x53, 0xaf, 0x01, 0x87, 0x10, 0xa6, 0xd4, 0x30, 0x69, 0x80, 0x64, + 0xba, 0x4e, 0x6f, 0x78, 0xbc, 0x1e, 0x55, 0x65, 0x2a, 0xa0, 0x31, 0x5f, 0xfe, 0x77, 0x21, 0x19, + 0x18, 0x4a, 0x28, 0xc3, 0x8f, 0x7e, 0x8d, 0x19, 0xfe, 0xa9, 0xed, 0xd7, 0x20, 0x3d, 0x32, 0x63, + 0xe8, 0x62, 0xd0, 0x87, 0x58, 0x25, 0xce, 0xfb, 0x70, 0x26, 0x4a, 0xf1, 0xe7, 0x71, 0x58, 0x38, + 0x8e, 0x69, 0x3f, 0x01, 0x25, 0xc4, 0x5b, 0x5a, 0xc7, 0x22, 0x9e, 0xd8, 0x4f, 0x57, 0x4f, 0x4f, + 0x48, 0x43, 0xe4, 0x27, 0xac, 0x25, 0xe3, 0x8e, 0x52, 0xe2, 0xf7, 0x20, 0x63, 0xf2, 0x8e, 0x6b, + 0x62, 0xa3, 0x46, 0xcf, 0xcc, 0x23, 0x8f, 0x21, 0x40, 0x81, 0x9e, 0x36, 0x43, 0x45, 0x84, 0xdd, + 0x3b, 0xf8, 0xe8, 0xc1, 0xd9, 0x8e, 0x65, 0xb2, 0xdd, 0x93, 0xae, 0xb4, 0x8e, 0x06, 0x85, 0xac, + 0xc0, 0xf2, 0x0f, 0x73, 0x9e, 0x79, 0xa5, 0xb2, 0xe6, 0x18, 0xa0, 0x49, 0xbd, 0x2e, 0x2d, 0xa7, + 0x0d, 0xcf, 0x0e, 0xbd, 0x2e, 0xdd, 0x47, 0xcf, 0xee, 0x75, 0xe9, 0xcf, 0x86, 0x89, 0xfe, 0x50, + 0x82, 0x2c, 0x3f, 0xd9, 0xed, 0xf6, 0x3d, 0x9d, 0x1f, 0xd7, 0xfb, 0xf9, 0xe9, 0x27, 0x47, 0x83, + 0xc2, 0x3c, 0x9b, 0x90, 0x75, 0x51, 0xc6, 0x9a, 0xad, 0x3c, 0x6d, 0xb3, 0x43, 0x14, 0x91, 0xb3, + 0x05, 0x02, 0x13, 0xad, 0x41, 0x86, 0x27, 0xdb, 0xfe, 0x05, 0x3c, 0x4b, 0x5c, 0xd3, 0x95, 0x97, + 0x9e, 0x0c, 0x0a, 0x4b, 0xc7, 0x58, 0x16, 0xcf, 0xd3, 0xef, 0x72, 0x5d, 0x35, 0xbd, 0x1b, 0xfe, + 0x44, 0x06, 0xa4, 0x03, 0xd3, 0x38, 0xec, 0x89, 0x3c, 0xf7, 0xd9, 0x5d, 0xd9, 0x9c, 0x6f, 0x23, + 0x14, 0x13, 0xed, 0xc1, 0xbc, 0xdf, 0x08, 0x77, 0xe8, 0x24, 0x97, 0xfc, 0x5a, 0x9a, 0xf1, 0xcd, + 0x9a, 0x8f, 0x9a, 0x88, 0x23, 0x97, 0x8b, 0xb0, 0x78, 0x6c, 0x94, 0xf3, 0x65, 0x1c, 0x2e, 0x8e, + 0x12, 0x61, 0x10, 0x87, 0x68, 0xe3, 0x1e, 0xf2, 0x83, 0xa9, 0xc9, 0xd4, 0xc7, 0xe0, 0x64, 0xe6, + 0x7f, 0x8d, 0xfb, 0xc8, 0xcf, 0xc6, 0x7c, 0xe4, 0x53, 0xe0, 0x33, 0xf3, 0x1a, 0xc3, 0xf7, 0x1d, + 0xe5, 0xc7, 0x81, 0x2f, 0xe0, 0xa7, 0x36, 0x1f, 0x3e, 0x05, 0x3c, 0xab, 0xef, 0x7f, 0x06, 0xde, + 0xe0, 0x5f, 0x24, 0x48, 0x8f, 0x8c, 0xec, 0x97, 0xe9, 0x0e, 0xb6, 0x82, 0x68, 0x88, 0x3f, 0xd3, + 0x78, 0xe7, 0xfc, 0xc3, 0x1a, 0x0d, 0x92, 0xf2, 0xff, 0x20, 0x41, 0x7a, 0x64, 0x22, 0xbf, 0x21, + 0x47, 0xf2, 0xf5, 0xf7, 0xbc, 0x0d, 0x99, 0xd1, 0x25, 0x0a, 0xb5, 0x21, 0x7d, 0x3d, 0x6d, 0x14, + 0xbf, 0x0b, 0x71, 0x2e, 0x41, 0x08, 0x32, 0x1f, 0x95, 0x1b, 0xdb, 0x8d, 0x8d, 0x15, 0xed, 0xd6, + 0xa6, 0xaa, 0xad, 0x54, 0x95, 0x19, 0x34, 0x07, 0x72, 0xad, 0x7e, 0xbb, 0x4e, 0x85, 0x8a, 0x84, + 0x52, 0x90, 0x60, 0x5f, 0xf5, 0x9a, 0x12, 0x29, 0x56, 0x40, 0xe1, 0xd8, 0xbb, 0x98, 0x3a, 0x06, + 0x1a, 0xf7, 0xa3, 0x12, 0x2c, 0xb0, 0x20, 0xbd, 0x4b, 0xe3, 0x37, 0xea, 0x0a, 0xb5, 0x50, 0xf4, + 0x9c, 0x0d, 0x8a, 0xa8, 0x53, 0xdc, 0xd0, 0xbb, 0xb8, 0xf8, 0xf7, 0x31, 0xc8, 0x0e, 0x41, 0x7c, + 0xb7, 0xf8, 0x0a, 0xc8, 0xc4, 0xb2, 0x0f, 0xb4, 0xe1, 0xa5, 0x39, 0x3f, 0x48, 0xb3, 0xec, 0x83, + 0x1d, 0xb5, 0xa1, 0x26, 0x68, 0xe1, 0x8e, 0x6b, 0xa1, 0x26, 0xc4, 0x9c, 0x9e, 0xe7, 0xe7, 0x90, + 0x6f, 0x9f, 0x32, 0x15, 0x13, 0x6d, 0x94, 0x36, 0x7b, 0x9e, 0x38, 0x01, 0x60, 0x18, 0xe8, 0xaf, + 0xa5, 0x61, 0xd6, 0xc3, 0xb3, 0xc5, 0x77, 0xcf, 0x85, 0xc7, 0x27, 0x40, 0xdc, 0x60, 0x7e, 0x44, + 0x37, 0xea, 0x93, 0x41, 0x21, 0x3b, 0x3e, 0x41, 0xe4, 0x19, 0xaf, 0x36, 0xfd, 0x2e, 0xa2, 0x26, + 0xbf, 0x54, 0x1b, 0x4e, 0x34, 0x73, 0x08, 0x53, 0xde, 0x5b, 0xa6, 0x47, 0x16, 0x22, 0xbf, 0x07, + 0x73, 0xe1, 0xde, 0x1f, 0x73, 0x5a, 0x5e, 0x1e, 0x3d, 0x96, 0x78, 0x6d, 0xaa, 0x99, 0x11, 0x29, + 0x61, 0xe8, 0x88, 0xfe, 0xbb, 0x90, 0x0c, 0xa6, 0xfd, 0x3c, 0x67, 0xfb, 0x9c, 0xe3, 0x83, 0xc3, + 0xb2, 0x59, 0x25, 0x5e, 0xfc, 0x1b, 0x09, 0xe6, 0x54, 0x4c, 0x9c, 0xce, 0x7d, 0x6c, 0xd2, 0x98, + 0x27, 0x78, 0xd3, 0x22, 0x4d, 0xff, 0xa6, 0xa5, 0x0c, 0xc9, 0xe0, 0x34, 0xf3, 0x3c, 0xef, 0x3a, + 0x86, 0xb5, 0xd0, 0x55, 0x50, 0xda, 0x4e, 0xdf, 0x36, 0x75, 0xf7, 0x50, 0x73, 0xb1, 0x6e, 0xec, + 0x63, 0x53, 0xdc, 0xdc, 0xcc, 0xfb, 0x72, 0x95, 0x8b, 0x8b, 0x3f, 0x8a, 0x00, 0x1a, 0x4e, 0x4e, + 0x88, 0x8a, 0x68, 0xe4, 0xc6, 0xc6, 0x21, 0x92, 0xd7, 0xc8, 0xb1, 0x77, 0x6d, 0x63, 0x01, 0x60, + 0x30, 0x70, 0x7f, 0x49, 0xdd, 0x90, 0x8c, 0xa0, 0x1f, 0x9f, 0x7e, 0x66, 0x1b, 0x65, 0x67, 0xb6, + 0xcc, 0x4a, 0x7f, 0xa9, 0xe7, 0xb6, 0xc2, 0x59, 0xff, 0x4f, 0x0c, 0x50, 0xd5, 0xc5, 0xba, 0x87, + 0x29, 0xf3, 0x90, 0xd3, 0x72, 0xed, 0x0a, 0xcc, 0xf2, 0xc4, 0x2c, 0x72, 0x9e, 0xc4, 0x4c, 0x4c, + 0x0a, 0xaf, 0x8a, 0xbe, 0x0f, 0x73, 0x86, 0xd3, 0xe9, 0x77, 0x6d, 0x8d, 0xdd, 0x38, 0x8b, 0x40, + 0xf8, 0x3b, 0xa7, 0x19, 0xf1, 0x44, 0xe7, 0x4a, 0x55, 0xa7, 0x43, 0xbf, 0x83, 0x47, 0x47, 0x0c, + 0x90, 0x69, 0xa0, 0xe7, 0x21, 0x19, 0x6c, 0x28, 0x16, 0x02, 0x27, 0xd5, 0xa1, 0x00, 0x5d, 0x87, + 0x59, 0x9d, 0x68, 0xce, 0x2e, 0x8b, 0x51, 0xcf, 0xb2, 0x30, 0x35, 0xa6, 0x93, 0xcd, 0x5d, 0xf4, + 0x16, 0xa4, 0x77, 0xef, 0xf1, 0xb8, 0x9d, 0x13, 0x28, 0x7f, 0x08, 0x30, 0x7f, 0x34, 0x28, 0xa4, + 0x6e, 0xdd, 0x61, 0x83, 0xa5, 0xf4, 0xa9, 0xa6, 0x76, 0xef, 0x05, 0x1f, 0xe8, 0x1a, 0x64, 0xbb, + 0xfa, 0x43, 0x6d, 0xd7, 0xd5, 0x0d, 0x11, 0xa8, 0x76, 0x38, 0x2b, 0x48, 0xea, 0x7c, 0x57, 0x7f, + 0x78, 0x4b, 0xc8, 0x1b, 0x66, 0x07, 0xe7, 0xff, 0x5b, 0x82, 0x84, 0x18, 0x11, 0xea, 0x01, 0x88, + 0xe9, 0xb1, 0x4c, 0x1e, 0x0a, 0xa5, 0x2b, 0x77, 0x8e, 0x06, 0x85, 0x64, 0x95, 0x49, 0x1b, 0x35, + 0xf2, 0x64, 0x50, 0xf8, 0xf0, 0x69, 0x49, 0xcb, 0x07, 0x51, 0x93, 0xbc, 0x91, 0x86, 0xc9, 0x0e, + 0x14, 0xf7, 0x75, 0xa2, 0xed, 0x5b, 0xc4, 0x73, 0xf6, 0x5c, 0xbd, 0xcb, 0x16, 0x57, 0x56, 0xe7, + 0xf6, 0x75, 0xb2, 0xea, 0xcb, 0x50, 0x9e, 0x86, 0x1c, 0xf7, 0xf9, 0x83, 0x01, 0xbe, 0xa5, 0x82, + 0x6f, 0x74, 0x1d, 0x2e, 0x04, 0x95, 0x35, 0x3a, 0xe8, 0x76, 0xdf, 0x38, 0xc0, 0xcc, 0x13, 0x50, + 0xce, 0x5a, 0x08, 0x0a, 0xd7, 0xf5, 0x87, 0x15, 0x5e, 0x54, 0xbc, 0x00, 0x0b, 0xa1, 0x65, 0x0d, + 0x02, 0x44, 0x0c, 0xca, 0xba, 0xb5, 0xe7, 0xea, 0xe1, 0xf7, 0x78, 0x77, 0x60, 0x7e, 0xec, 0xbd, + 0xab, 0xa0, 0x95, 0xf0, 0x41, 0xd8, 0xe8, 0x03, 0xd9, 0x52, 0x95, 0x7f, 0xfa, 0x21, 0x77, 0xc6, + 0x18, 0xf9, 0x2e, 0x2e, 0x40, 0x36, 0x68, 0x26, 0x68, 0xfb, 0x17, 0x73, 0x90, 0xd8, 0xd2, 0x0f, + 0x3b, 0x8e, 0x6e, 0xa2, 0x25, 0x48, 0xf9, 0x8f, 0x0c, 0xfc, 0xf6, 0x92, 0x6a, 0x58, 0x84, 0x2c, + 0xc8, 0xf4, 0x09, 0x76, 0xa9, 0x3d, 0x68, 0xec, 0xf9, 0x2d, 0xe7, 0xc9, 0x4a, 0xe5, 0xc9, 0xa0, + 0x70, 0x73, 0xba, 0xe5, 0xc1, 0x46, 0xdf, 0xb5, 0xbc, 0xc3, 0x52, 0xeb, 0xce, 0xed, 0x1d, 0x01, + 0x45, 0x37, 0xb1, 0xa3, 0xa6, 0xfb, 0xe1, 0x4f, 0xf1, 0x64, 0x83, 0x4e, 0xb5, 0xd6, 0xb5, 0x0c, + 0xd7, 0x21, 0xfe, 0xc9, 0xbe, 0x90, 0xae, 0x33, 0x21, 0x7a, 0x15, 0xe6, 0x77, 0x2d, 0x9b, 0xdd, + 0x2a, 0xf9, 0x7a, 0xfc, 0x50, 0x3f, 0xe3, 0x8b, 0x85, 0xe2, 0x7d, 0xc8, 0x84, 0x9e, 0x69, 0x50, + 0x33, 0x8b, 0x33, 0x33, 0xdb, 0x3c, 0x1a, 0x14, 0xd2, 0xc3, 0x6d, 0xcb, 0x4d, 0xed, 0x59, 0xfc, + 0x63, 0x7a, 0xd8, 0x0c, 0x35, 0xb4, 0x45, 0x98, 0x65, 0x8f, 0xb3, 0xf9, 0xcb, 0x2c, 0x95, 0x7f, + 0xa0, 0xb7, 0x61, 0xb6, 0x83, 0x75, 0x82, 0xc5, 0xa3, 0xab, 0xa5, 0x53, 0x88, 0x80, 0xbd, 0x6e, + 0x56, 0xb9, 0x3a, 0xaa, 0x40, 0x9c, 0xdf, 0x13, 0xb2, 0xdb, 0xbd, 0xc9, 0x63, 0xd1, 0x13, 0xdf, + 0xd7, 0xad, 0xce, 0xa8, 0xa2, 0x26, 0xaa, 0x43, 0xc2, 0xe5, 0x57, 0xc3, 0xec, 0xce, 0xef, 0xcc, + 0x44, 0x3f, 0x74, 0xf3, 0xbc, 0x3a, 0xa3, 0xfa, 0x75, 0xd1, 0xb6, 0xff, 0x3e, 0x83, 0x7b, 0x14, + 0xf1, 0xfc, 0xa4, 0x34, 0x65, 0x30, 0x38, 0x04, 0x1c, 0x41, 0xa1, 0x03, 0xb4, 0xd8, 0x3d, 0x01, + 0xbb, 0x3a, 0x3c, 0x7d, 0x80, 0x23, 0x77, 0xce, 0x74, 0x80, 0xbc, 0x26, 0xda, 0x00, 0x30, 0x02, + 0x2f, 0xc7, 0x2e, 0x15, 0x53, 0xd7, 0x5f, 0x3f, 0x4f, 0x24, 0xb5, 0x3a, 0xa3, 0x86, 0x10, 0xd0, + 0x1d, 0x48, 0x19, 0xc3, 0x6d, 0x9b, 0x9b, 0x67, 0x80, 0x6f, 0x9c, 0x8b, 0xbb, 0x57, 0x29, 0x5f, + 0x0f, 0xa5, 0xa3, 0x7c, 0xad, 0x8c, 0xf3, 0x75, 0x1d, 0xd2, 0xe2, 0x4c, 0x86, 0xbf, 0xeb, 0xcf, + 0x65, 0x99, 0xbb, 0x08, 0x5b, 0x89, 0xff, 0xf2, 0xbf, 0x54, 0xb7, 0x0d, 0xc7, 0xc4, 0x66, 0x9d, + 0x7e, 0xab, 0xe2, 0x08, 0x9a, 0x7d, 0x10, 0xb4, 0x02, 0x19, 0xa3, 0x83, 0x75, 0xbb, 0xdf, 0xf3, + 0x71, 0xd0, 0x94, 0x38, 0x69, 0x51, 0x4f, 0x00, 0x6d, 0x00, 0xda, 0x65, 0x6f, 0x3a, 0xc2, 0xbd, + 0x62, 0x77, 0x93, 0xd3, 0x80, 0x29, 0xac, 0xae, 0x3a, 0xec, 0x19, 0x7a, 0x09, 0xd2, 0xb6, 0x63, + 0x1b, 0xba, 0x6d, 0xe0, 0x0e, 0xf3, 0xac, 0xfc, 0x3a, 0x73, 0x54, 0x88, 0x3e, 0x85, 0x0c, 0x19, + 0x49, 0x1f, 0x72, 0x17, 0x58, 0x8b, 0x6f, 0x9e, 0xf7, 0xc0, 0x71, 0x75, 0x46, 0x1d, 0x43, 0x42, + 0xbf, 0x05, 0x8a, 0x37, 0x76, 0x2b, 0xc1, 0x2e, 0x46, 0x4f, 0x7f, 0x3f, 0x75, 0xc2, 0xdd, 0xcb, + 0xea, 0x8c, 0x3a, 0x81, 0x86, 0x3e, 0x83, 0x79, 0x32, 0xfa, 0xd2, 0x38, 0x77, 0x89, 0x35, 0xf0, + 0xed, 0x53, 0x8f, 0xd6, 0x8f, 0x7b, 0x9c, 0xbd, 0x3a, 0xa3, 0x8e, 0x63, 0x51, 0x78, 0x7b, 0xf4, + 0x72, 0x23, 0x97, 0x3b, 0x13, 0xfe, 0xf8, 0xcb, 0x16, 0x0a, 0x3f, 0x86, 0x85, 0xd6, 0x20, 0xd9, + 0xf5, 0x7d, 0x45, 0xee, 0xb9, 0x33, 0x23, 0xee, 0x71, 0xf7, 0xb5, 0x3a, 0xa3, 0x0e, 0xeb, 0x57, + 0x92, 0x90, 0x10, 0x97, 0x60, 0xc1, 0xd5, 0x72, 0x42, 0x91, 0x8b, 0xbf, 0x90, 0x41, 0x0e, 0x62, + 0xd0, 0x65, 0x40, 0x41, 0xd4, 0x30, 0x7c, 0xcf, 0x47, 0x5d, 0x50, 0x64, 0x75, 0x46, 0xcd, 0xfa, + 0x65, 0xc3, 0x27, 0x7d, 0xaf, 0xc2, 0x7c, 0xd7, 0x31, 0xad, 0x5d, 0x6b, 0x48, 0xfc, 0xfc, 0xc8, + 0x33, 0xe3, 0x8b, 0x05, 0xf1, 0xdf, 0x1c, 0x79, 0xf6, 0x32, 0xcd, 0xb3, 0x71, 0xda, 0xfb, 0xe0, + 0x5d, 0x0c, 0x75, 0x44, 0x6e, 0xdf, 0x66, 0xf7, 0x5d, 0x22, 0xed, 0xe5, 0xf1, 0x55, 0x5a, 0x48, + 0x45, 0xe6, 0x5a, 0x1d, 0x63, 0xe6, 0xab, 0x67, 0x32, 0xb3, 0x3f, 0xf6, 0x55, 0x29, 0xa0, 0xe6, + 0x5b, 0xe3, 0xd4, 0x7c, 0xed, 0x6c, 0x6a, 0x0e, 0xc1, 0x04, 0xdc, 0xbc, 0x73, 0x2c, 0x37, 0x2f, + 0x4f, 0xb9, 0x71, 0x42, 0x88, 0xa3, 0xe4, 0x5c, 0x1d, 0x23, 0xe7, 0xab, 0x67, 0x92, 0x73, 0x78, + 0x8c, 0x82, 0x9d, 0x37, 0x8f, 0x61, 0xe7, 0x37, 0xa6, 0x62, 0xe7, 0x10, 0x58, 0x98, 0x9e, 0xd5, + 0xe3, 0xe8, 0xb9, 0x34, 0x1d, 0x3d, 0x87, 0x20, 0x47, 0xf8, 0xf9, 0x7b, 0x13, 0xdc, 0xa3, 0x9c, + 0xbd, 0x79, 0x8f, 0x3d, 0xeb, 0x58, 0x95, 0x26, 0xc8, 0x47, 0x3f, 0x86, 0x7c, 0xb2, 0x0c, 0xfe, + 0xad, 0x73, 0x90, 0x4f, 0xa8, 0x81, 0x49, 0xf6, 0xf9, 0x18, 0xe6, 0xc2, 0x8c, 0xc1, 0x9e, 0x97, + 0x9c, 0xce, 0x6d, 0x27, 0xfc, 0x59, 0x04, 0xb3, 0x81, 0x50, 0x11, 0xfa, 0xfe, 0x24, 0xf1, 0x2c, + 0x9c, 0x09, 0x7e, 0xc2, 0x3d, 0xec, 0xaa, 0x34, 0xc9, 0x3c, 0xb7, 0xc3, 0xcc, 0xb3, 0x78, 0xa6, + 0xef, 0x9e, 0x88, 0x68, 0x57, 0xa5, 0x30, 0xf5, 0x00, 0xc8, 0xfe, 0x35, 0x7f, 0x88, 0x86, 0x8a, + 0x7f, 0x22, 0x41, 0xb4, 0xe9, 0xb4, 0x51, 0x66, 0x78, 0x08, 0xc7, 0x8e, 0xcf, 0x3e, 0x18, 0xaa, + 0x8b, 0x6c, 0xef, 0xc5, 0x53, 0xda, 0x0e, 0x0e, 0x2d, 0x83, 0x4a, 0xe8, 0x3d, 0x48, 0xf4, 0x78, + 0x34, 0x2d, 0x98, 0xa6, 0x78, 0x5a, 0x7d, 0xae, 0xa9, 0xfa, 0x55, 0xae, 0x5d, 0x0d, 0xff, 0x85, + 0xd4, 0xba, 0x63, 0x62, 0x94, 0x01, 0xd8, 0xd2, 0x09, 0xe9, 0xed, 0xbb, 0x3a, 0xc1, 0xca, 0x0c, + 0x4a, 0x40, 0x74, 0x6d, 0xbd, 0xa5, 0x48, 0xd7, 0x3e, 0x0e, 0x1f, 0x99, 0xd5, 0xd4, 0x72, 0x63, + 0xa3, 0xb1, 0xb1, 0xa2, 0x6d, 0x94, 0xd7, 0xeb, 0x2d, 0x65, 0x06, 0xe5, 0x60, 0xf1, 0xa3, 0x72, + 0x63, 0x5b, 0x9c, 0xa1, 0x69, 0x8d, 0x8d, 0xed, 0xba, 0x7a, 0xb7, 0x7c, 0x5b, 0x91, 0xd0, 0x45, + 0x40, 0xea, 0x66, 0x75, 0xad, 0x55, 0xab, 0x68, 0xd5, 0xcd, 0xf5, 0xad, 0x72, 0x75, 0xbb, 0xb1, + 0xb9, 0xa1, 0x44, 0x90, 0x0c, 0xb1, 0xda, 0xe6, 0x46, 0x5d, 0x81, 0x6b, 0x3f, 0x8e, 0x41, 0x8c, + 0xda, 0x1a, 0x7a, 0x09, 0x52, 0x3b, 0x1b, 0xad, 0xad, 0x7a, 0xb5, 0x71, 0xab, 0x51, 0xaf, 0x29, + 0x33, 0xf9, 0x85, 0x47, 0x8f, 0x97, 0xe6, 0x69, 0xd1, 0x8e, 0x4d, 0x7a, 0xd8, 0x60, 0x24, 0x8b, + 0xf2, 0x10, 0xaf, 0x94, 0xab, 0x6b, 0x3b, 0x5b, 0x8a, 0x94, 0xcf, 0x3c, 0x7a, 0xbc, 0x04, 0x54, + 0x81, 0x13, 0x1c, 0x7a, 0x1e, 0x12, 0x6a, 0xbd, 0xb5, 0xbd, 0xa9, 0xd6, 0x95, 0x48, 0x7e, 0xfe, + 0xd1, 0xe3, 0xa5, 0x14, 0x2d, 0x14, 0xbc, 0x85, 0x5e, 0x85, 0x74, 0xab, 0xba, 0x5a, 0x5f, 0x2f, + 0x6b, 0xd5, 0xd5, 0xf2, 0xc6, 0x4a, 0x5d, 0x89, 0xe6, 0x17, 0x1f, 0x3d, 0x5e, 0x52, 0xc6, 0x0d, + 0x9d, 0x36, 0xd1, 0x58, 0xdf, 0xda, 0x54, 0xb7, 0x95, 0xd8, 0xb0, 0x09, 0xce, 0x2f, 0xa8, 0x08, + 0xc0, 0x6b, 0xdf, 0xaa, 0xd7, 0x6b, 0xca, 0x6c, 0x1e, 0x3d, 0x7a, 0xbc, 0x94, 0xa1, 0xe5, 0x43, + 0xda, 0x40, 0x2f, 0xc3, 0x5c, 0x55, 0xad, 0x97, 0xb7, 0xeb, 0x5a, 0x6b, 0xbb, 0xbc, 0xdd, 0x52, + 0xe2, 0xc3, 0x91, 0x84, 0xa8, 0x00, 0x95, 0x20, 0x5b, 0xde, 0xd9, 0xde, 0xd4, 0x46, 0x74, 0x13, + 0xf9, 0x4b, 0x8f, 0x1e, 0x2f, 0x2d, 0x50, 0xdd, 0x72, 0xdf, 0x73, 0xc2, 0xfa, 0xaf, 0x83, 0x32, + 0xd2, 0x7f, 0x6d, 0xa5, 0xaa, 0xc8, 0xf9, 0x8b, 0x8f, 0x1e, 0x2f, 0xa1, 0xf1, 0x21, 0xac, 0x54, + 0xd1, 0xaf, 0xc3, 0xc5, 0xed, 0x4f, 0xb6, 0xea, 0xb5, 0x7a, 0xab, 0xaa, 0x8d, 0x0e, 0x3b, 0x99, + 0xcf, 0x3d, 0x7a, 0xbc, 0xb4, 0x48, 0xeb, 0x4c, 0x0c, 0xfd, 0x0d, 0x50, 0x5a, 0xdb, 0x6a, 0xbd, + 0xbc, 0xae, 0x35, 0x36, 0x56, 0xea, 0x2d, 0xb6, 0x58, 0x30, 0xec, 0xd2, 0xd8, 0xa6, 0xa5, 0x43, + 0xd8, 0xa8, 0x7f, 0x34, 0x86, 0x9f, 0x1a, 0xea, 0x8f, 0xed, 0x43, 0xb4, 0x04, 0xc9, 0xf5, 0xc6, + 0x8a, 0x5a, 0x66, 0xb8, 0x73, 0xf9, 0xec, 0xa3, 0xc7, 0x4b, 0x69, 0xaa, 0x17, 0xec, 0xaa, 0xbc, + 0xfc, 0xa3, 0xbf, 0xbc, 0x3c, 0xf3, 0x57, 0x3f, 0xb9, 0x3c, 0x53, 0xb9, 0xf2, 0xc5, 0x7f, 0x5e, + 0x9e, 0xf9, 0xe2, 0xe8, 0xb2, 0xf4, 0xb3, 0xa3, 0xcb, 0xd2, 0x97, 0x47, 0x97, 0xa5, 0xff, 0x38, + 0xba, 0x2c, 0xfd, 0xd1, 0x57, 0x97, 0x67, 0x7e, 0xf6, 0xd5, 0xe5, 0x99, 0x2f, 0xbf, 0xba, 0x3c, + 0xf3, 0x69, 0x9c, 0xdb, 0x75, 0x3b, 0xce, 0x52, 0xc3, 0xb7, 0xfe, 0x3f, 0x00, 0x00, 0xff, 0xff, + 0x23, 0xef, 0xff, 0x88, 0xd2, 0x3a, 0x00, 0x00, } diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 0fb4cd1db289..84774244d678 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -94,11 +94,11 @@ message StreamIngestionDetails { } message StreamIngestionProgress { - // MarkedForCompletion is used to signal to the stream ingestion job to - // complete its ingestion. This involves stopping any subsequent ingestion, - // and rolling back to the latest resolved ts to bring the ingested cluster to - // a consistent state. - bool markedForCompletion = 1; + // CutoverTime is set to signal to the stream ingestion job to complete its + // ingestion. This involves stopping any subsequent ingestion, and rolling + // back any additional ingested data, to bring the ingested cluster to a + // consistent state as of the CutoverTime. + util.hlc.Timestamp cutover_time = 1 [(gogoproto.nullable) = false]; } message BackupDetails { diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index 4616b5eaa9e6..9c7b8bc3e8ef 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -68,6 +68,7 @@ go_library( "//pkg/util/errorutil", "//pkg/util/errorutil/unimplemented", "//pkg/util/fuzzystrmatch", + "//pkg/util/hlc", "//pkg/util/humanizeutil", "//pkg/util/ipaddr", "//pkg/util/json", diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 34fec7ae2c5a..dce6113a236a 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -61,6 +61,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/streaming" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/fuzzystrmatch" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/cockroach/pkg/util/json" @@ -4775,16 +4776,22 @@ may increase either contention or retry errors, or both.`, Category: categoryStreamIngestion, }, tree.Overload{ - Types: tree.ArgTypes{{"job_id", types.Int}}, + Types: tree.ArgTypes{ + {"job_id", types.Int}, + {"cutover_ts", types.TimestampTZ}, + }, ReturnType: tree.FixedReturnType(types.Int), Fn: func(evalCtx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { jobID := int(*args[0].(*tree.DInt)) - err := streaming.CompleteIngestionHook(evalCtx, evalCtx.Txn, jobID) + cutoverTime := args[1].(*tree.DTimestampTZ).Time + cutoverTimestamp := hlc.Timestamp{WallTime: cutoverTime.UnixNano()} + err := streaming.CompleteIngestionHook(evalCtx, evalCtx.Txn, jobID, cutoverTimestamp) return tree.NewDInt(tree.DInt(jobID)), err }, Info: "This function can be used to signal a running stream ingestion job to complete. " + - "The job will eventually stop ingesting, revert to the latest resolved timestamp and leave the " + - "cluster in a consistent state. " + + "The job will eventually stop ingesting, revert to the specified timestamp and leave the " + + "cluster in a consistent state. The specified timestamp can only be specified up to the" + + " microsecond. " + "This function does not wait for the job to reach a terminal state, " + "but instead returns the job id as soon as it has signaled the job to complete. " + "This builtin can be used in conjunction with SHOW JOBS WHEN COMPLETE to ensure that the" + diff --git a/pkg/streaming/BUILD.bazel b/pkg/streaming/BUILD.bazel index 0bf7b6a0a363..330bec823c06 100644 --- a/pkg/streaming/BUILD.bazel +++ b/pkg/streaming/BUILD.bazel @@ -8,5 +8,6 @@ go_library( deps = [ "//pkg/kv", "//pkg/sql/sem/tree", + "//pkg/util/hlc", ], ) diff --git a/pkg/streaming/utils.go b/pkg/streaming/utils.go index 3ba07474c61b..5520e83996a2 100644 --- a/pkg/streaming/utils.go +++ b/pkg/streaming/utils.go @@ -13,6 +13,7 @@ package streaming import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/hlc" ) // CompleteIngestionHook is the hook run by the @@ -20,4 +21,4 @@ import ( // It is used to signal to a running stream ingestion job to stop ingesting data // and eventually move to a consistent state as of the latest resolved // timestamp. -var CompleteIngestionHook func(*tree.EvalContext, *kv.Txn, int) error +var CompleteIngestionHook func(*tree.EvalContext, *kv.Txn, int, hlc.Timestamp) error