From ae32109a78f8afe10998b0c27d93be3df45e17fe Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Fri, 12 Feb 2021 16:09:54 -0500 Subject: [PATCH] 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 +- .../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 | 10 +- pkg/streaming/BUILD.bazel | 1 + pkg/streaming/utils.go | 3 +- 10 files changed, 572 insertions(+), 548 deletions(-) diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 56c135732769..24dde1fd2e71 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 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.

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..a9850f6911a6 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 := time.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..deaa961e1514 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,11 +4776,16 @@ 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. " + 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