diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index 89736c7a078e..10ae90348883 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -636,7 +636,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { r.testingKnobs.duringSystemTableRestoration = func(systemTableName string) error { if !alreadyErrored && systemTableName == customRestoreSystemTable { alreadyErrored = true - return jobs.NewRetryJobError("injected error") + return jobs.MarkAsRetryJobError(errors.New("injected error")) } return nil } @@ -645,7 +645,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { } } // The initial restore will return an error, and restart. - sqlDBRestore.ExpectErr(t, `injected error: restarting in background`, `RESTORE FROM $1`, LocalFoo) + sqlDBRestore.ExpectErr(t, `running execution from '.*' to '.*' on \d+ failed: injected error`, `RESTORE FROM $1`, LocalFoo) // Reduce retry delays. sqlDBRestore.Exec(t, "SET CLUSTER SETTING jobs.registry.retry.initial_delay = '1ms'") // Expect the restore to succeed. diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index f2e26bca8f18..39af8f89bde0 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -711,7 +711,7 @@ func (b *changefeedResumer) resumeWithRetries( // retries will not help. // Instead, we want to make sure that the changefeed job is not marked failed // due to a transient, retryable error. - err = jobs.NewRetryJobError(fmt.Sprintf("retryable flow error: %+v", err)) + err = jobs.MarkAsRetryJobError(err) b.setJobRunningStatus(ctx, "retryable flow error: %s", err) } diff --git a/pkg/jobs/BUILD.bazel b/pkg/jobs/BUILD.bazel index c5357cc8cc88..0c5cc24c1b47 100644 --- a/pkg/jobs/BUILD.bazel +++ b/pkg/jobs/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "adopt.go", "config.go", + "errors.go", "executor_impl.go", "helpers.go", "job_scheduler.go", diff --git a/pkg/jobs/adopt.go b/pkg/jobs/adopt.go index 057ead3a5dd0..5418def3885d 100644 --- a/pkg/jobs/adopt.go +++ b/pkg/jobs/adopt.go @@ -411,6 +411,7 @@ func (r *Registry) runJob( } r.maybeDumpTrace(ctx, resumer, int64(job.ID()), int64(span.TraceID()), err) + r.maybeRecordExecutionFailure(ctx, err, job) if r.knobs.AfterJobStateMachine != nil { r.knobs.AfterJobStateMachine() } diff --git a/pkg/jobs/config.go b/pkg/jobs/config.go index f618ec2297af..2e41dbe662f2 100644 --- a/pkg/jobs/config.go +++ b/pkg/jobs/config.go @@ -21,17 +21,19 @@ import ( ) const ( - intervalBaseSettingKey = "jobs.registry.interval.base" - adoptIntervalSettingKey = "jobs.registry.interval.adopt" - cancelIntervalSettingKey = "jobs.registry.interval.cancel" - gcIntervalSettingKey = "jobs.registry.interval.gc" - retentionTimeSettingKey = "jobs.retention_time" - cancelUpdateLimitKey = "jobs.cancel_update_limit" - retryInitialDelaySettingKey = "jobs.registry.retry.initial_delay" - retryMaxDelaySettingKey = "jobs.registry.retry.max_delay" + intervalBaseSettingKey = "jobs.registry.interval.base" + adoptIntervalSettingKey = "jobs.registry.interval.adopt" + cancelIntervalSettingKey = "jobs.registry.interval.cancel" + gcIntervalSettingKey = "jobs.registry.interval.gc" + retentionTimeSettingKey = "jobs.retention_time" + cancelUpdateLimitKey = "jobs.cancel_update_limit" + retryInitialDelaySettingKey = "jobs.registry.retry.initial_delay" + retryMaxDelaySettingKey = "jobs.registry.retry.max_delay" + executionErrorsMaxEntriesKey = "jobs.execution_errors.max_entries" + executionErrorsMaxEntrySizeKey = "jobs.execution_errors.max_entry_size" ) -var ( +const ( // defaultAdoptInterval is the default adopt interval. defaultAdoptInterval = 30 * time.Second @@ -58,6 +60,15 @@ var ( // defaultRetryMaxDelay is the maximum delay to retry a failed job. defaultRetryMaxDelay = 24 * time.Hour + + // defaultExecutionErrorsMaxEntries is the default number of error entries + // which will be retained. + defaultExecutionErrorsMaxEntries = 3 + + // defaultExecutionErrorsMaxEntrySize is the maximum allowed size of an + // error. If this size is exceeded, the error will be formatted as a string + // and then truncated to fit the size. + defaultExecutionErrorsMaxEntrySize = 64 << 10 // 64 KiB ) var ( @@ -122,6 +133,21 @@ var ( defaultRetryMaxDelay, settings.PositiveDuration, ) + + executionErrorsMaxEntriesSetting = settings.RegisterIntSetting( + executionErrorsMaxEntriesKey, + "the maximum number of retriable error entries which will be stored for introspection", + defaultExecutionErrorsMaxEntries, + settings.NonNegativeInt, + ) + + executionErrorsMaxEntrySize = settings.RegisterByteSizeSetting( + executionErrorsMaxEntrySizeKey, + "the maximum byte size of individual error entries which will be stored"+ + " for introspection", + defaultExecutionErrorsMaxEntrySize, + settings.NonNegativeInt, + ) ) // jitter adds a small jitter in the given duration. diff --git a/pkg/jobs/errors.go b/pkg/jobs/errors.go new file mode 100644 index 000000000000..f0c72e99aeb9 --- /dev/null +++ b/pkg/jobs/errors.go @@ -0,0 +1,161 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package jobs + +import ( + "context" + "fmt" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" +) + +// errRetryJobSentinel exists so the errors returned from MarkAsRetryJobError can +// be marked with it, allowing more robust detection of retry errors even if +// they are wrapped, etc. This was originally introduced to deal with injected +// retry errors from testing knobs. +var errRetryJobSentinel = errors.New("retriable job error") + +// MarkAsRetryJobError marks an error as a retriable job error which +// indicates that the registry should retry the job. +func MarkAsRetryJobError(err error) error { + return errors.Mark(err, errRetryJobSentinel) +} + +// Registry does not retry a job that fails due to a permanent error. +var errJobPermanentSentinel = errors.New("permanent job error") + +// MarkAsPermanentJobError marks an error as a permanent job error, which indicates +// Registry to not retry the job when it fails due to this error. +func MarkAsPermanentJobError(err error) error { + return errors.Mark(err, errJobPermanentSentinel) +} + +// IsPermanentJobError checks whether the given error is a permanent error. +func IsPermanentJobError(err error) bool { + return errors.Is(err, errJobPermanentSentinel) +} + +// InvalidStatusError is the error returned when the desired operation is +// invalid given the job's current status. +type InvalidStatusError struct { + id jobspb.JobID + status Status + op string + err string +} + +func (e *InvalidStatusError) Error() string { + if e.err != "" { + return fmt.Sprintf("cannot %s %s job (id %d, err: %q)", e.op, e.status, e.id, e.err) + } + return fmt.Sprintf("cannot %s %s job (id %d)", e.op, e.status, e.id) +} + +// SimplifyInvalidStatusError unwraps an *InvalidStatusError into an error +// message suitable for users. Other errors are returned as passed. +func SimplifyInvalidStatusError(err error) error { + if ierr := (*InvalidStatusError)(nil); errors.As(err, &ierr) { + return errors.Errorf("job %s", ierr.status) + } + return err +} + +// retriableExecutionError captures metadata about retriable errors encountered +// during the execution of a job. These errors propagate information to be +// stored in the payload in Payload.RetriableExecutionErrorLog. +type retriableExecutionError struct { + instanceID base.SQLInstanceID + start, end time.Time + status Status + cause error +} + +func newRetriableExecutionError( + instanceID base.SQLInstanceID, status Status, start, end time.Time, cause error, +) *retriableExecutionError { + return &retriableExecutionError{ + instanceID: instanceID, + status: status, + start: start, + end: end, + cause: cause, + } +} + +// Error makes retriableExecutionError and error. +func (e *retriableExecutionError) Error() string { + return formatRetriableExecutionFailure( + e.instanceID, e.status, e.start, e.end, e.cause, + ) + +} + +func formatRetriableExecutionFailure( + instanceID base.SQLInstanceID, status Status, start, end time.Time, cause error, +) string { + mustTimestamp := func(ts time.Time) *tree.DTimestamp { + ret, _ := tree.MakeDTimestamp(ts, time.Microsecond) + return ret + } + return fmt.Sprintf( + "%s execution from %v to %v on %d failed: %v", + status, + mustTimestamp(start), + mustTimestamp(end), + instanceID, + cause, + ) +} + +// Cause exposes the underlying error. +func (e *retriableExecutionError) Cause() error { return e.cause } + +// Unwrap exposes the underlying error. +func (e *retriableExecutionError) Unwrap() error { return e.cause } + +// Format formats the error. +func (e *retriableExecutionError) Format(s fmt.State, verb rune) { errors.FormatError(e, s, verb) } + +// SafeFormatError formats the error safely. +func (e *retriableExecutionError) SafeFormatError(p errors.Printer) error { + if p.Detail() { + p.Printf("retriable execution error") + } + return e.cause +} + +func (e *retriableExecutionError) toRetriableExecutionFailure( + ctx context.Context, maxErrorSize int, +) *jobspb.RetriableExecutionFailure { + // If the cause is too large, we format it, losing all structure, and retain + // a prefix. + ef := &jobspb.RetriableExecutionFailure{ + Status: string(e.status), + ExecutionStartMicros: timeutil.ToUnixMicros(e.start), + ExecutionEndMicros: timeutil.ToUnixMicros(e.end), + InstanceID: e.instanceID, + } + if encodedCause := errors.EncodeError(ctx, e.cause); encodedCause.Size() < maxErrorSize { + ef.Error = &encodedCause + } else { + formatted := e.cause.Error() + if len(formatted) > maxErrorSize { + formatted = formatted[:maxErrorSize] + } + ef.TruncatedError = formatted + } + return ef +} diff --git a/pkg/jobs/helpers_test.go b/pkg/jobs/helpers_test.go index 8bcbf7b49d7f..6c8dde33b127 100644 --- a/pkg/jobs/helpers_test.go +++ b/pkg/jobs/helpers_test.go @@ -89,36 +89,26 @@ func (j *Job) Succeeded(ctx context.Context) error { return j.succeeded(ctx, nil /* txn */, nil /* fn */) } -var ( - AdoptQuery = claimQuery - - CancelQuery = pauseAndCancelUpdate - - GcQuery = expiredJobsQuery - - RemoveClaimsQuery = removeClaimsQuery - - ProcessJobsQuery = processQueryWithBackoff - - IntervalBaseSettingKey = intervalBaseSettingKey - - AdoptIntervalSettingKey = adoptIntervalSettingKey - - CancelIntervalSettingKey = cancelIntervalSettingKey - - GcIntervalSettingKey = gcIntervalSettingKey - - RetentionTimeSettingKey = retentionTimeSettingKey - - AdoptIntervalSetting = adoptIntervalSetting - - CancelIntervalSetting = cancelIntervalSetting +const ( + AdoptQuery = claimQuery + CancelQuery = pauseAndCancelUpdate + GcQuery = expiredJobsQuery + RemoveClaimsQuery = removeClaimsQuery + ProcessJobsQuery = processQueryWithBackoff + IntervalBaseSettingKey = intervalBaseSettingKey + AdoptIntervalSettingKey = adoptIntervalSettingKey + CancelIntervalSettingKey = cancelIntervalSettingKey + GcIntervalSettingKey = gcIntervalSettingKey + RetentionTimeSettingKey = retentionTimeSettingKey + DefaultAdoptInterval = defaultAdoptInterval + ExecutionErrorsMaxEntriesKey = executionErrorsMaxEntriesKey + ExecutionErrorsMaxEntrySizeKey = executionErrorsMaxEntrySizeKey +) +var ( + AdoptIntervalSetting = adoptIntervalSetting + CancelIntervalSetting = cancelIntervalSetting CancellationsUpdateLimitSetting = cancellationsUpdateLimitSetting - - GcIntervalSetting = gcIntervalSetting - - RetentionTimeSetting = retentionTimeSetting - - DefaultAdoptInterval = defaultAdoptInterval + GcIntervalSetting = gcIntervalSetting + RetentionTimeSetting = retentionTimeSetting ) diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go index 36b0ef1b0601..a21fefd30a1e 100644 --- a/pkg/jobs/jobs.go +++ b/pkg/jobs/jobs.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -76,6 +77,7 @@ type Job struct { syncutil.Mutex payload jobspb.Payload progress jobspb.Progress + runStats *RunStats } } @@ -232,31 +234,6 @@ func (s Status) Terminal() bool { return s == StatusFailed || s == StatusSucceeded || s == StatusCanceled || s == StatusRevertFailed } -// InvalidStatusError is the error returned when the desired operation is -// invalid given the job's current status. -type InvalidStatusError struct { - id jobspb.JobID - status Status - op string - err string -} - -func (e *InvalidStatusError) Error() string { - if e.err != "" { - return fmt.Sprintf("cannot %s %s job (id %d, err: %q)", e.op, e.status, e.id, e.err) - } - return fmt.Sprintf("cannot %s %s job (id %d)", e.op, e.status, e.id) -} - -// SimplifyInvalidStatusError unwraps an *InvalidStatusError into an error -// message suitable for users. Other errors are returned as passed. -func SimplifyInvalidStatusError(err error) error { - if ierr := (*InvalidStatusError)(nil); errors.As(err, &ierr) { - return errors.Errorf("job %s", ierr.status) - } - return err -} - // ID returns the ID of the job. func (j *Job) ID() jobspb.JobID { return j.id @@ -831,7 +808,7 @@ func UnmarshalProgress(datum tree.Datum) (*jobspb.Progress, error) { return progress, nil } -// unnarshalCreatedBy unrmarshals and returns created_by_type and created_by_id datums +// unmarshalCreatedBy unmarshals and returns created_by_type and created_by_id datums // which may be tree.DNull, or tree.DString and tree.DInt respectively. func unmarshalCreatedBy(createdByType, createdByID tree.Datum) (*CreatedByInfo, error) { if createdByType == tree.DNull || createdByID == tree.DNull { @@ -869,6 +846,17 @@ func (j *Job) CurrentStatus(ctx context.Context, txn *kv.Txn) (Status, error) { return Status(statusString), nil } +// getRunStats returns the RunStats for a job. If they are not set, it will +// return a zero-value. +func (j *Job) getRunStats() (rs RunStats) { + j.mu.Lock() + defer j.mu.Unlock() + if j.mu.runStats != nil { + rs = *j.mu.runStats + } + return rs +} + // Start will resume the job. The transaction used to create the StartableJob // must be committed. If a non-nil error is returned, the job was not started // and nothing will be send on errCh. Clients must not start jobs more than @@ -975,3 +963,34 @@ func (sj *StartableJob) Cancel(ctx context.Context) error { defer sj.registry.unregister(sj.ID()) return sj.registry.CancelRequested(ctx, nil, sj.ID()) } + +// FormatRetriableExecutionErrorLogToStringArray extracts the events +// stored in the payload, formats them into strings and returns them as an +// array of strings. This function is intended for use with crdb_internal.jobs. +func FormatRetriableExecutionErrorLogToStringArray( + ctx context.Context, pl *jobspb.Payload, +) *tree.DArray { + arr := tree.NewDArray(types.String) + for _, ev := range pl.RetriableExecutionFailureLog { + if ev == nil { // no reason this should happen, but be defensive + continue + } + var cause error + if ev.Error != nil { + cause = errors.DecodeError(ctx, *ev.Error) + } else { + cause = fmt.Errorf("(truncated) %s", ev.TruncatedError) + } + msg := formatRetriableExecutionFailure( + ev.InstanceID, + Status(ev.Status), + timeutil.FromUnixMicros(ev.ExecutionStartMicros), + timeutil.FromUnixMicros(ev.ExecutionEndMicros), + cause, + ) + // We really don't care about errors here. I'd much rather see nothing + // in my log than crash. + _ = arr.Append(tree.NewDString(msg)) + } + return arr +} diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index 871dc74153d7..bda0533d2a07 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -2928,7 +2928,7 @@ func TestMetrics(t *testing.T) { // Fail the Resume with a retriable error. errCh := <-resuming require.Equal(t, int64(1), importMetrics.CurrentlyRunning.Value()) - errCh <- jobs.NewRetryJobError("") + errCh <- jobs.MarkAsRetryJobError(errors.New("boom")) int64EqSoon(t, importMetrics.ResumeRetryError.Count, 1) // It will be retried. int64EqSoon(t, importMetrics.CurrentlyRunning.Value, 1) @@ -2992,7 +2992,7 @@ func TestMetrics(t *testing.T) { // We'll inject retriable errors in OnFailOrCancel. errCh := <-resuming require.Equal(t, int64(1), importMetrics.CurrentlyRunning.Value()) - errCh <- jobs.NewRetryJobError("boom") + errCh <- jobs.MarkAsRetryJobError(errors.New("boom")) int64EqSoon(t, importMetrics.FailOrCancelRetryError.Count, 1) } { @@ -3257,41 +3257,3 @@ func TestNonCancelableJobsRetry(t *testing.T) { close(rts.failOrCancelCheckCh) rts.check(t, jobs.StatusFailed) } - -// TestExecutionLogToJSON tests conversion of an executionLog in jobs payload -// to a JSON string. -func TestExecutionLogToJSON(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - for _, test := range []struct { - name string - executionLog []*jobspb.ExecutionEvent - expected string - }{ - { - "empty", - []*jobspb.ExecutionEvent{}, - `[]`, - }, - { - "with values", - []*jobspb.ExecutionEvent{ - { - InstanceId: 1, - Status: string(jobs.StatusRunning), - EventTimeMicros: timeutil.ToUnixMicros(timeutil.Unix(1, 0)), - ExecutionError: "error string", - Type: jobspb.JobStartEvent, - }, - }, - `[{"eventTimeMicros": "1000000", "executionError": "error string", "instanceId": 1, "status": "` + string(jobs.StatusRunning) + `", "type": "START"}]`, - }, - } { - t.Run(test.name, func(t *testing.T) { - encoded, err := jobspb.ExecutionLogToJSON(test.executionLog) - require.NoError(t, err) - require.Equal(t, test.expected, encoded.String()) - }) - } -} diff --git a/pkg/jobs/jobspb/BUILD.bazel b/pkg/jobs/jobspb/BUILD.bazel index 63d842164ddc..0a152099977d 100644 --- a/pkg/jobs/jobspb/BUILD.bazel +++ b/pkg/jobs/jobspb/BUILD.bazel @@ -4,18 +4,14 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "jobspb", - srcs = [ - "json_encoding.go", - "wrap.go", - ], + srcs = ["wrap.go"], embed = [":jobspb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/jobs/jobspb", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/cloud", "//pkg/sql/catalog/descpb", - "//pkg/sql/protoreflect", - "//pkg/util/json", "@com_github_cockroachdb_errors//:errors", "@com_github_gogo_protobuf//jsonpb", ], diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 8ca8d27594b9..f9d7354b8d02 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -7,6 +7,7 @@ import ( bytes "bytes" encoding_binary "encoding/binary" fmt "fmt" + github_com_cockroachdb_cockroach_pkg_base "github.com/cockroachdb/cockroach/pkg/base" clusterversion "github.com/cockroachdb/cockroach/pkg/clusterversion" roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security" @@ -273,39 +274,6 @@ func (ResolvedSpan_BoundaryType) EnumDescriptor() ([]byte, []int) { return fileDescriptor_6c315f3a2536c4ef, []int{27, 0} } -type ExecutionEvent_EventType int32 - -const ( - // JobEventUnspecified is the zero value for EventType and should be used. - JobEventUnspecified ExecutionEvent_EventType = 0 - // JobStartEvent indicates that this event was created at the beginning of - // job execution. - JobStartEvent ExecutionEvent_EventType = 1 - // JobEndEvent indicates that this event was created at the end of job - // execution. - JobEndEvent ExecutionEvent_EventType = 2 -) - -var ExecutionEvent_EventType_name = map[int32]string{ - 0: "UNSPECIFIED", - 1: "START", - 2: "END", -} - -var ExecutionEvent_EventType_value = map[string]int32{ - "UNSPECIFIED": 0, - "START": 1, - "END": 2, -} - -func (x ExecutionEvent_EventType) String() string { - return proto.EnumName(ExecutionEvent_EventType_name, int32(x)) -} - -func (ExecutionEvent_EventType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_6c315f3a2536c4ef, []int{34, 0} -} - // BackupEncryptionOptions stores information resolved during the BACKUP/RESTORE // planning stage, and by the BACKUP/RESTORE job to encrypt or decrypt BACKUP // data and manifest files. @@ -2313,55 +2281,6 @@ func (m *MigrationProgress) XXX_DiscardUnknown() { var xxx_messageInfo_MigrationProgress proto.InternalMessageInfo -// ExecutionEvent holds information about a job when it starts or -// ends an execution. It contains information about the event time, -// the node on which the job is executed, the status of the job,and -// any errors occurred during execution. -type ExecutionEvent struct { - // Type of event, start or end of execution when this event is created. - Type ExecutionEvent_EventType `protobuf:"varint,1,opt,name=type,proto3,enum=cockroach.sql.jobs.jobspb.ExecutionEvent_EventType" json:"type,omitempty"` - // ID of the node on which the job is executed. - InstanceId int32 `protobuf:"varint,2,opt,name=instance_id,json=instanceId,proto3" json:"instance_id,omitempty"` - // Status of the job in this execution. It should be a jobs.Status. - Status string `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"` - // Event timestamp when the event is created, represented as the number - // of microseconds elapsed since UnixEpoch. See timeutil.ToUnixMicros - // for more information about conversion to micros. - EventTimeMicros int64 `protobuf:"varint,4,opt,name=event_time_micros,json=eventTimeMicros,proto3" json:"event_time_micros,omitempty"` - // Errors during execution: it is empty for a JobStartEvent. For a JobEndEvent, - // it consists of errors encountered during the current job execution. - ExecutionError string `protobuf:"bytes,5,opt,name=execution_error,json=executionError,proto3" json:"execution_error,omitempty"` -} - -func (m *ExecutionEvent) Reset() { *m = ExecutionEvent{} } -func (m *ExecutionEvent) String() string { return proto.CompactTextString(m) } -func (*ExecutionEvent) ProtoMessage() {} -func (*ExecutionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_6c315f3a2536c4ef, []int{34} -} -func (m *ExecutionEvent) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *ExecutionEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil -} -func (m *ExecutionEvent) XXX_Merge(src proto.Message) { - xxx_messageInfo_ExecutionEvent.Merge(m, src) -} -func (m *ExecutionEvent) XXX_Size() int { - return m.Size() -} -func (m *ExecutionEvent) XXX_DiscardUnknown() { - xxx_messageInfo_ExecutionEvent.DiscardUnknown(m) -} - -var xxx_messageInfo_ExecutionEvent proto.InternalMessageInfo - type SQLStatsCompactionDetails struct { } @@ -2369,7 +2288,7 @@ func (m *SQLStatsCompactionDetails) Reset() { *m = SQLStatsCompactionDet func (m *SQLStatsCompactionDetails) String() string { return proto.CompactTextString(m) } func (*SQLStatsCompactionDetails) ProtoMessage() {} func (*SQLStatsCompactionDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_6c315f3a2536c4ef, []int{35} + return fileDescriptor_6c315f3a2536c4ef, []int{34} } func (m *SQLStatsCompactionDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2401,7 +2320,7 @@ func (m *SQLStatsCompactionProgress) Reset() { *m = SQLStatsCompactionPr func (m *SQLStatsCompactionProgress) String() string { return proto.CompactTextString(m) } func (*SQLStatsCompactionProgress) ProtoMessage() {} func (*SQLStatsCompactionProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_6c315f3a2536c4ef, []int{36} + return fileDescriptor_6c315f3a2536c4ef, []int{35} } func (m *SQLStatsCompactionProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2469,21 +2388,18 @@ type Payload struct { // PauseReason is used to describe the reason that the job is currently paused // or has been requested to be paused. PauseReason string `protobuf:"bytes,28,opt,name=pause_reason,json=pauseReason,proto3" json:"pause_reason,omitempty"` - // An ExecutionEvent is created whenever a job's Resumer starts and ends - // an execution. At the beginning of the execution of a job's Resumer, - // a EventType.START event is created and appended to executionLog in - // job's payload. When the Resumer completes execution, which can be with - // or without an error, an EventType.END event is created and appended to - // job's executionLog. The sequence of events in this log show the lifecycle - // of a job and the errors encountered during job execution. - ExecutionLog []*ExecutionEvent `protobuf:"bytes,29,rep,name=executionLog,proto3" json:"executionLog,omitempty"` + // RetriableExecutionFailureLog stores a history of retriable execution + // failures. These failures may occur in either the RUNNING or REVERTING + // status. A finite number of these entries will be kept, as governed by + // the jobs.execution_errors.max_entries cluster setting. + RetriableExecutionFailureLog []*RetriableExecutionFailure `protobuf:"bytes,32,rep,name=retriable_execution_failure_log,json=retriableExecutionFailureLog,proto3" json:"retriable_execution_failure_log,omitempty"` } 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_6c315f3a2536c4ef, []int{37} + return fileDescriptor_6c315f3a2536c4ef, []int{36} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2714,7 +2630,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_6c315f3a2536c4ef, []int{38} + return fileDescriptor_6c315f3a2536c4ef, []int{37} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2963,7 +2879,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_6c315f3a2536c4ef, []int{39} + return fileDescriptor_6c315f3a2536c4ef, []int{38} } func (m *Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2988,6 +2904,55 @@ func (m *Job) XXX_DiscardUnknown() { var xxx_messageInfo_Job proto.InternalMessageInfo +// RetriableExecutionFailure is used in Payload.RetriableExecutionFailureLog +// to store a history of executions which failed. +type RetriableExecutionFailure struct { + // Status is the status of the job when this failure occurred. + Status string `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + // ExecutionStartMicros is the timestamp at which this execution occurred. + ExecutionStartMicros int64 `protobuf:"varint,2,opt,name=execution_start_micros,json=executionStartMicros,proto3" json:"execution_start_micros,omitempty"` + // ExecutionEndMicros is the timestamp at which this execution concluded. + ExecutionEndMicros int64 `protobuf:"varint,3,opt,name=execution_end_micros,json=executionEndMicros,proto3" json:"execution_end_micros,omitempty"` + // InstanceID is the instance which coordinated the execution. + InstanceID github_com_cockroachdb_cockroach_pkg_base.SQLInstanceID `protobuf:"varint,4,opt,name=instance_id,json=instanceId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/base.SQLInstanceID" json:"instance_id"` + // Error stores the structured error which occurred. It might be nil if it + // was too large. In that case, the TruncatedError will be populated. + Error *errorspb.EncodedError `protobuf:"bytes,5,opt,name=error,proto3" json:"error,omitempty"` + // TruncatedError is a fragment of a error message populated in the case + // that the error was too large. While the structure may be lost, at least + // some information will be preserved. + TruncatedError string `protobuf:"bytes,6,opt,name=truncated_error,json=truncatedError,proto3" json:"truncated_error,omitempty"` +} + +func (m *RetriableExecutionFailure) Reset() { *m = RetriableExecutionFailure{} } +func (m *RetriableExecutionFailure) String() string { return proto.CompactTextString(m) } +func (*RetriableExecutionFailure) ProtoMessage() {} +func (*RetriableExecutionFailure) Descriptor() ([]byte, []int) { + return fileDescriptor_6c315f3a2536c4ef, []int{39} +} +func (m *RetriableExecutionFailure) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RetriableExecutionFailure) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *RetriableExecutionFailure) XXX_Merge(src proto.Message) { + xxx_messageInfo_RetriableExecutionFailure.Merge(m, src) +} +func (m *RetriableExecutionFailure) XXX_Size() int { + return m.Size() +} +func (m *RetriableExecutionFailure) XXX_DiscardUnknown() { + xxx_messageInfo_RetriableExecutionFailure.DiscardUnknown(m) +} + +var xxx_messageInfo_RetriableExecutionFailure proto.InternalMessageInfo + func init() { proto.RegisterEnum("cockroach.sql.jobs.jobspb.EncryptionMode", EncryptionMode_name, EncryptionMode_value) proto.RegisterEnum("cockroach.sql.jobs.jobspb.Status", Status_name, Status_value) @@ -2996,7 +2961,6 @@ func init() { proto.RegisterEnum("cockroach.sql.jobs.jobspb.SchedulePTSChainingRecord_PTSAction", SchedulePTSChainingRecord_PTSAction_name, SchedulePTSChainingRecord_PTSAction_value) proto.RegisterEnum("cockroach.sql.jobs.jobspb.SchemaChangeGCProgress_Status", SchemaChangeGCProgress_Status_name, SchemaChangeGCProgress_Status_value) proto.RegisterEnum("cockroach.sql.jobs.jobspb.ResolvedSpan_BoundaryType", ResolvedSpan_BoundaryType_name, ResolvedSpan_BoundaryType_value) - proto.RegisterEnum("cockroach.sql.jobs.jobspb.ExecutionEvent_EventType", ExecutionEvent_EventType_name, ExecutionEvent_EventType_value) proto.RegisterType((*BackupEncryptionOptions)(nil), "cockroach.sql.jobs.jobspb.BackupEncryptionOptions") proto.RegisterType((*BackupEncryptionOptions_KMSInfo)(nil), "cockroach.sql.jobs.jobspb.BackupEncryptionOptions.KMSInfo") proto.RegisterType((*EncryptionInfo)(nil), "cockroach.sql.jobs.jobspb.EncryptionInfo") @@ -3058,377 +3022,377 @@ func init() { proto.RegisterType((*CreateStatsProgress)(nil), "cockroach.sql.jobs.jobspb.CreateStatsProgress") proto.RegisterType((*MigrationDetails)(nil), "cockroach.sql.jobs.jobspb.MigrationDetails") proto.RegisterType((*MigrationProgress)(nil), "cockroach.sql.jobs.jobspb.MigrationProgress") - proto.RegisterType((*ExecutionEvent)(nil), "cockroach.sql.jobs.jobspb.ExecutionEvent") proto.RegisterType((*SQLStatsCompactionDetails)(nil), "cockroach.sql.jobs.jobspb.SQLStatsCompactionDetails") proto.RegisterType((*SQLStatsCompactionProgress)(nil), "cockroach.sql.jobs.jobspb.SQLStatsCompactionProgress") proto.RegisterType((*Payload)(nil), "cockroach.sql.jobs.jobspb.Payload") proto.RegisterType((*Progress)(nil), "cockroach.sql.jobs.jobspb.Progress") proto.RegisterType((*Job)(nil), "cockroach.sql.jobs.jobspb.Job") + proto.RegisterType((*RetriableExecutionFailure)(nil), "cockroach.sql.jobs.jobspb.RetriableExecutionFailure") } func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_6c315f3a2536c4ef) } var fileDescriptor_6c315f3a2536c4ef = []byte{ // 5743 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5c, 0x5d, 0x6c, 0x1b, 0xcb, - 0x75, 0xd6, 0x52, 0x14, 0x7f, 0x0e, 0x45, 0x6a, 0x39, 0x92, 0x65, 0x9a, 0xd7, 0x16, 0x15, 0x5e, - 0xfb, 0x5e, 0xfb, 0x26, 0x97, 0x4a, 0xec, 0xfc, 0xdc, 0x38, 0xb9, 0xbe, 0x97, 0x7f, 0x92, 0xa8, - 0x7f, 0x2f, 0x29, 0xdf, 0xbf, 0xde, 0x6c, 0x97, 0xdc, 0x91, 0xb4, 0x15, 0xb9, 0x4b, 0xef, 0x2c, - 0x65, 0x2b, 0x05, 0x82, 0x20, 0x69, 0x80, 0xc0, 0x4f, 0x0d, 0xd0, 0xe6, 0xa5, 0x35, 0x50, 0x20, - 0x09, 0xd0, 0x87, 0x06, 0x45, 0x83, 0x22, 0x2d, 0xfa, 0x54, 0xa0, 0x7d, 0xc8, 0x43, 0x0b, 0x04, - 0x28, 0x8a, 0xa6, 0x45, 0xa1, 0xb6, 0xca, 0x4b, 0x1f, 0x8b, 0xf6, 0xed, 0x3e, 0x15, 0xf3, 0xb3, - 0xcb, 0x5d, 0x4a, 0xa2, 0x28, 0xd9, 0x49, 0x5e, 0x64, 0xee, 0x99, 0x99, 0x6f, 0x66, 0xce, 0x9e, - 0xf9, 0xce, 0x99, 0x33, 0xb3, 0x86, 0xd9, 0xdf, 0xb1, 0x9a, 0x64, 0x81, 0xfe, 0xe9, 0x36, 0xd9, - 0x3f, 0x85, 0xae, 0x6d, 0x39, 0x16, 0xba, 0xd6, 0xb2, 0x5a, 0xfb, 0xb6, 0xa5, 0xb5, 0xf6, 0x0a, - 0xe4, 0x71, 0xbb, 0xc0, 0x4a, 0x78, 0xad, 0xec, 0x15, 0x6c, 0xdb, 0x96, 0x4d, 0xeb, 0xf3, 0x1f, - 0xbc, 0x45, 0x76, 0x66, 0xd7, 0xda, 0xb5, 0xd8, 0xcf, 0x05, 0xfa, 0x4b, 0x48, 0x11, 0xc3, 0xe8, - 0x36, 0x17, 0x74, 0xcd, 0xd1, 0x84, 0x2c, 0xe3, 0xca, 0x0c, 0xeb, 0xcd, 0x1d, 0xcb, 0xee, 0x68, - 0x8e, 0x8b, 0xf1, 0x2a, 0x79, 0xdc, 0x5e, 0x68, 0x69, 0x8e, 0xd6, 0xb6, 0x76, 0x17, 0x74, 0x4c, - 0x5a, 0xdd, 0xe6, 0x02, 0x71, 0xec, 0x5e, 0xcb, 0xe9, 0xd9, 0x58, 0x17, 0x95, 0x72, 0xa7, 0x54, - 0x72, 0xb0, 0xa9, 0x99, 0x8e, 0x8b, 0xdf, 0x73, 0x8c, 0xf6, 0xc2, 0x5e, 0xbb, 0xb5, 0xe0, 0x18, - 0x1d, 0x4c, 0x1c, 0xad, 0xd3, 0x15, 0x25, 0x9f, 0xa2, 0x4d, 0x49, 0x6b, 0x0f, 0x77, 0xb4, 0xd6, - 0x9e, 0x66, 0xee, 0x62, 0x7b, 0x81, 0xf7, 0xd1, 0xea, 0x36, 0x45, 0x95, 0x9b, 0xad, 0x76, 0x8f, - 0x38, 0xd8, 0x3e, 0xc0, 0x36, 0x31, 0x2c, 0x73, 0x41, 0x3c, 0xaa, 0xe2, 0x99, 0xd7, 0xca, 0xff, - 0x28, 0x04, 0x57, 0x4b, 0x5a, 0x6b, 0xbf, 0xd7, 0xad, 0x9a, 0x2d, 0xfb, 0xb0, 0xeb, 0x18, 0x96, - 0xb9, 0xc9, 0xfe, 0x12, 0x24, 0xc3, 0xf8, 0x3e, 0x3e, 0xcc, 0x48, 0xf3, 0xd2, 0xed, 0x49, 0x85, - 0xfe, 0x44, 0x6f, 0x43, 0xb8, 0x63, 0xe9, 0x38, 0x13, 0x9a, 0x97, 0x6e, 0xa7, 0xee, 0xde, 0x29, - 0x9c, 0xa9, 0xdb, 0x42, 0x1f, 0x6d, 0xdd, 0xd2, 0xb1, 0xc2, 0x9a, 0xa1, 0x26, 0xc4, 0xf6, 0x3b, - 0x44, 0x35, 0xcc, 0x1d, 0x2b, 0x33, 0x3e, 0x2f, 0xdd, 0x4e, 0xdc, 0xbd, 0x3f, 0x04, 0xe2, 0x8c, - 0x61, 0x15, 0x56, 0xd7, 0xeb, 0x35, 0x73, 0xc7, 0x2a, 0x25, 0x8e, 0x8f, 0x72, 0x51, 0xf1, 0xa0, - 0x44, 0xf7, 0x3b, 0x84, 0xfe, 0xc8, 0x6e, 0x82, 0x2b, 0xa3, 0xe3, 0xef, 0xd9, 0x06, 0x1b, 0x7f, - 0x5c, 0xa1, 0x3f, 0xd1, 0x67, 0x00, 0x61, 0x8e, 0x87, 0x75, 0x95, 0xbe, 0x48, 0x95, 0x4e, 0x30, - 0xc4, 0x26, 0x28, 0x7b, 0x25, 0x15, 0xcd, 0xd1, 0x56, 0xf1, 0xe1, 0xfd, 0xf0, 0x7f, 0xff, 0x49, - 0x4e, 0xe2, 0x7f, 0xf3, 0xdf, 0x1c, 0x87, 0x54, 0x7f, 0x28, 0x0c, 0x7e, 0x19, 0x22, 0xec, 0x0d, - 0x60, 0xd6, 0x43, 0xea, 0xee, 0x67, 0x47, 0x52, 0x07, 0x6d, 0x5a, 0xa8, 0xb3, 0x76, 0x8a, 0x68, - 0x8f, 0x10, 0x84, 0x89, 0xd6, 0x76, 0xc4, 0x40, 0xd8, 0x6f, 0xf4, 0x47, 0x12, 0xcc, 0x0f, 0x8e, - 0xa8, 0x74, 0xb8, 0xba, 0x5e, 0x5f, 0xd7, 0xe8, 0x6b, 0x5c, 0xc5, 0x87, 0xb5, 0x4a, 0x66, 0x7c, - 0x7e, 0xfc, 0x76, 0xe2, 0xee, 0xe6, 0xe8, 0x1d, 0x57, 0xcf, 0x41, 0xac, 0x9a, 0x8e, 0x7d, 0xa8, - 0x9c, 0xdb, 0x71, 0xb6, 0x0e, 0xb7, 0x46, 0x82, 0xf2, 0xdb, 0x50, 0x9c, 0xdb, 0xd0, 0x0c, 0x4c, - 0x1c, 0x68, 0xed, 0x1e, 0x16, 0xb3, 0xe5, 0x0f, 0xf7, 0x43, 0x6f, 0x49, 0xf9, 0xab, 0x10, 0xe1, - 0x8a, 0x41, 0x49, 0x88, 0x17, 0xab, 0xf5, 0xbb, 0x5f, 0xf8, 0xe2, 0x52, 0x79, 0x5d, 0x1e, 0x13, - 0xaf, 0xe0, 0xa7, 0x12, 0xcc, 0xd6, 0x1d, 0x1b, 0x6b, 0x9d, 0x9a, 0xb9, 0x8b, 0x09, 0x9d, 0x53, - 0x05, 0x3b, 0x9a, 0xd1, 0x26, 0xe8, 0x16, 0xa4, 0x08, 0x2b, 0x51, 0x35, 0x5d, 0xb7, 0x31, 0x21, - 0xa2, 0xc3, 0x24, 0x97, 0x16, 0xb9, 0x10, 0x7d, 0x0e, 0xc2, 0xa4, 0xab, 0x99, 0xac, 0xe7, 0xc4, - 0xdd, 0xab, 0x3e, 0xb5, 0x89, 0x85, 0x5c, 0xa8, 0x77, 0x35, 0xb3, 0x14, 0xfe, 0xd9, 0x51, 0x6e, - 0x4c, 0x61, 0x55, 0x51, 0x09, 0x80, 0x38, 0x9a, 0xed, 0xa8, 0x74, 0x05, 0x0a, 0xa3, 0xbd, 0xe1, - 0x6b, 0x48, 0x57, 0x68, 0x61, 0xaf, 0xdd, 0x2a, 0x34, 0xdc, 0x15, 0x2a, 0x9a, 0xc7, 0x59, 0x33, - 0x2a, 0xcd, 0xff, 0xcb, 0x38, 0x5c, 0x1d, 0x18, 0xf8, 0x96, 0x6d, 0xed, 0xb2, 0x21, 0x2d, 0xc2, - 0x64, 0xab, 0xe7, 0x58, 0x07, 0xd8, 0xe6, 0x3d, 0x48, 0xa3, 0xf7, 0x90, 0x10, 0x0d, 0xa9, 0x1c, - 0x7d, 0x03, 0x50, 0x57, 0xb3, 0x1d, 0x83, 0x82, 0xab, 0x5d, 0x81, 0x9e, 0x09, 0x31, 0xfb, 0xa8, - 0x0d, 0xb1, 0x8f, 0x33, 0xc6, 0x55, 0xd8, 0x72, 0xc1, 0x5c, 0x09, 0x7b, 0x9d, 0xa2, 0xe7, 0x74, - 0x77, 0xb0, 0x34, 0xbb, 0x0b, 0xe9, 0x13, 0x4d, 0x90, 0x02, 0xc8, 0x60, 0xc8, 0x58, 0x57, 0x3d, - 0x06, 0xbb, 0xc8, 0x14, 0xd3, 0x6e, 0x73, 0xaf, 0x20, 0xfb, 0x4c, 0x82, 0xd9, 0xd3, 0x07, 0x77, - 0x8a, 0xad, 0x7d, 0xe8, 0xb7, 0xb5, 0xc4, 0xdd, 0xca, 0xcb, 0x50, 0x84, 0xdf, 0x62, 0x7f, 0x1c, - 0x82, 0x6b, 0xd4, 0x64, 0xf5, 0x5e, 0x1b, 0x6f, 0x35, 0xea, 0xe5, 0x3d, 0xcd, 0x30, 0x0d, 0x73, - 0x57, 0xc1, 0x2d, 0xcb, 0xd6, 0xd1, 0xf7, 0x24, 0xc8, 0x52, 0x96, 0xc5, 0xad, 0x80, 0x02, 0x54, - 0x9b, 0x15, 0x73, 0x5e, 0x2d, 0xd5, 0xff, 0xed, 0x28, 0x77, 0x6f, 0xd7, 0x70, 0xf6, 0x7a, 0xcd, - 0x42, 0xcb, 0xea, 0x2c, 0x78, 0x23, 0xd4, 0x9b, 0xfd, 0xdf, 0x0b, 0xdd, 0xfd, 0xdd, 0x05, 0xe6, - 0x0c, 0x7a, 0x3d, 0x43, 0x2f, 0x6c, 0x6f, 0xd7, 0x2a, 0xc7, 0x47, 0xb9, 0xcc, 0x96, 0x0b, 0xee, - 0xa9, 0x87, 0xf7, 0xac, 0x64, 0xba, 0x67, 0x94, 0xa0, 0x47, 0x10, 0xd1, 0x5a, 0x74, 0x3a, 0x82, - 0xc3, 0x1f, 0x0c, 0x53, 0xc9, 0x59, 0x33, 0x2b, 0x6c, 0x35, 0xea, 0x45, 0x86, 0xa2, 0x08, 0xb4, - 0xfc, 0x4d, 0x88, 0x7b, 0x42, 0x04, 0x10, 0xd9, 0xde, 0xaa, 0x14, 0x1b, 0x55, 0x79, 0x0c, 0x25, - 0x20, 0xaa, 0x54, 0xd7, 0xaa, 0xc5, 0x7a, 0x55, 0x96, 0xf2, 0xff, 0x14, 0x85, 0x24, 0xa7, 0x75, - 0x77, 0xe5, 0x06, 0xd7, 0x97, 0x74, 0x99, 0xf5, 0x85, 0x1e, 0x40, 0x0c, 0x9b, 0x5c, 0xc1, 0xe2, - 0x45, 0x8f, 0x84, 0x10, 0xc5, 0x26, 0x53, 0x0f, 0xba, 0xc6, 0xfd, 0x04, 0x5d, 0xdc, 0xf1, 0x52, - 0xf4, 0xf8, 0x28, 0x37, 0xbe, 0xad, 0xd4, 0xb8, 0xc3, 0xf8, 0xb6, 0x04, 0xd3, 0x3d, 0xdb, 0x20, - 0x6a, 0xf3, 0x50, 0x6d, 0x5b, 0x2d, 0xad, 0x6d, 0x38, 0x87, 0xea, 0xfe, 0x41, 0x66, 0x82, 0x2d, - 0xac, 0x07, 0xe7, 0x7a, 0x2f, 0x31, 0xcd, 0xc2, 0xb6, 0x6d, 0x90, 0xd2, 0xe1, 0x9a, 0x40, 0x58, - 0x3d, 0xe0, 0xab, 0x69, 0xe6, 0xf8, 0x28, 0x27, 0x6f, 0x2b, 0x35, 0x7f, 0xd1, 0x23, 0x45, 0xee, - 0x0d, 0x54, 0x46, 0x5f, 0x85, 0xac, 0x8e, 0xbb, 0x36, 0x6e, 0x69, 0xd4, 0x90, 0x9a, 0x0c, 0x59, - 0xed, 0x68, 0xa6, 0xb1, 0x83, 0x89, 0x93, 0x09, 0x33, 0x1e, 0xcd, 0xf4, 0x6b, 0xf0, 0xae, 0xd7, - 0x45, 0x39, 0xd2, 0x3c, 0xa7, 0x47, 0xb9, 0xc1, 0xe2, 0x5e, 0x34, 0x13, 0x61, 0x8a, 0xba, 0x7b, - 0x71, 0xff, 0xab, 0xa4, 0xf1, 0x89, 0x48, 0x41, 0x81, 0x29, 0x5f, 0x17, 0xcc, 0xbf, 0xc7, 0x19, - 0xfe, 0x9d, 0x91, 0x5d, 0x93, 0x92, 0xc2, 0x41, 0xf7, 0x7a, 0xce, 0xea, 0x89, 0xfe, 0x26, 0x56, - 0xcf, 0x5b, 0x90, 0x6a, 0x59, 0xed, 0x36, 0x66, 0x66, 0xae, 0x6e, 0x2b, 0xb5, 0x4c, 0x8c, 0x19, - 0x4d, 0xfa, 0xf8, 0x28, 0x97, 0x2c, 0x7b, 0x25, 0xd4, 0x7c, 0x92, 0x2d, 0xff, 0x23, 0xfa, 0x03, - 0x09, 0xae, 0x13, 0xb1, 0x9e, 0xd4, 0xae, 0x43, 0xd4, 0x96, 0x58, 0x51, 0xee, 0x7c, 0x80, 0xe9, - 0xeb, 0xf3, 0x97, 0x59, 0x8e, 0xa5, 0x1b, 0xc7, 0x47, 0xb9, 0xb3, 0x79, 0x48, 0xb9, 0xe6, 0x76, - 0xbc, 0xe5, 0x90, 0x60, 0x51, 0xb6, 0x0c, 0x57, 0x4e, 0x35, 0xcd, 0xf3, 0xfc, 0x76, 0xdc, 0xcf, - 0x82, 0x32, 0xa4, 0xb8, 0xad, 0xb8, 0x14, 0x99, 0xff, 0x9b, 0x59, 0x48, 0x29, 0x98, 0x38, 0x96, - 0x8d, 0xdd, 0x85, 0xee, 0x5f, 0xa4, 0xe1, 0x4b, 0x2c, 0xd2, 0x9f, 0x48, 0x30, 0x4d, 0x63, 0x64, - 0xdb, 0xe8, 0x3a, 0x96, 0xad, 0xda, 0xf8, 0x89, 0x6d, 0x38, 0xd8, 0x75, 0x71, 0xc5, 0x21, 0x7a, - 0x0b, 0x0e, 0xa4, 0x50, 0xf1, 0x40, 0x14, 0x81, 0xc1, 0x17, 0xe3, 0x83, 0x6f, 0xfd, 0x47, 0xee, - 0xfe, 0x48, 0xa6, 0x74, 0x32, 0x6c, 0x2f, 0xd4, 0x2a, 0x0a, 0xd2, 0x4f, 0x00, 0xa3, 0xeb, 0x10, - 0xa6, 0x8b, 0x99, 0xc5, 0x69, 0xf1, 0x52, 0xec, 0xf8, 0x28, 0x17, 0xa6, 0xcb, 0x5d, 0x61, 0x52, - 0xe4, 0xc0, 0x8c, 0x58, 0xcb, 0x1e, 0xb5, 0xb0, 0xa5, 0x13, 0x65, 0x53, 0xfa, 0xea, 0xe8, 0x53, - 0xe2, 0xda, 0x77, 0x5f, 0x21, 0x0b, 0x8e, 0xb9, 0xf6, 0x50, 0xf3, 0x44, 0x09, 0xda, 0x82, 0x14, - 0x8d, 0x7c, 0x9b, 0x1a, 0xc1, 0x2a, 0x1d, 0x32, 0xc9, 0xc8, 0xac, 0xbf, 0xc1, 0xa5, 0x4a, 0x1e, - 0xb7, 0x69, 0x9d, 0x42, 0x45, 0x54, 0xf6, 0xe9, 0x2d, 0xa9, 0xfb, 0x64, 0x04, 0x2d, 0x41, 0xc2, - 0xd1, 0x9a, 0x6d, 0x17, 0x8e, 0x73, 0xe3, 0x6b, 0x67, 0xc0, 0x35, 0x68, 0x4d, 0x1f, 0x16, 0x38, - 0xae, 0x80, 0xa0, 0x0a, 0x80, 0x73, 0xd8, 0x75, 0x71, 0x52, 0x0c, 0xe7, 0xd6, 0x59, 0x38, 0x87, - 0x5d, 0x3f, 0x4c, 0xdc, 0x11, 0xcf, 0x04, 0xad, 0xc0, 0x24, 0xdf, 0x19, 0x09, 0x9c, 0x29, 0x86, - 0xf3, 0xfa, 0x19, 0x38, 0x2c, 0xe2, 0xd4, 0x7c, 0x48, 0x09, 0xe2, 0x49, 0x08, 0x2a, 0x42, 0x94, - 0xef, 0xc8, 0x48, 0x26, 0xc9, 0x60, 0x3e, 0x75, 0xd6, 0x70, 0x58, 0x2d, 0x9f, 0xea, 0xdd, 0x76, - 0x68, 0x01, 0x12, 0x34, 0x4a, 0xb3, 0x0d, 0x1d, 0xab, 0x7a, 0x93, 0xf1, 0x6e, 0xbc, 0x94, 0x3a, - 0x3e, 0xca, 0xc1, 0xa6, 0x10, 0x57, 0x4a, 0x0a, 0xb8, 0x55, 0x2a, 0x4d, 0xf4, 0x69, 0x48, 0x77, - 0x6d, 0xdc, 0xd5, 0x6c, 0xac, 0xb6, 0xac, 0x4e, 0xb7, 0x8d, 0x1d, 0xac, 0x33, 0x9e, 0x89, 0x29, - 0xb2, 0x28, 0x28, 0xbb, 0x72, 0x1e, 0xf9, 0x6a, 0x0e, 0xdd, 0x54, 0x11, 0x6c, 0xd3, 0x9a, 0x71, - 0x56, 0x33, 0xc9, 0xa4, 0x35, 0x21, 0x44, 0x87, 0x30, 0x4b, 0x0e, 0x89, 0x83, 0x3b, 0x2a, 0x53, - 0x37, 0x51, 0x3b, 0xc6, 0xae, 0x4d, 0x7d, 0x45, 0x26, 0xcd, 0xa6, 0x55, 0x1e, 0xdd, 0xd8, 0xea, - 0x0c, 0x87, 0xbd, 0x46, 0xb2, 0x2e, 0x50, 0xf8, 0xb6, 0x61, 0x86, 0x9c, 0x52, 0x84, 0xee, 0xc1, - 0x95, 0xfe, 0xca, 0x20, 0x6a, 0xb7, 0xd7, 0x6c, 0x1b, 0x64, 0x0f, 0x73, 0xc6, 0x8b, 0x29, 0x33, - 0xbe, 0xc2, 0x2d, 0xb7, 0x0c, 0x1d, 0x06, 0x16, 0x7b, 0x8b, 0x6a, 0x47, 0xdb, 0xc5, 0x99, 0xc4, - 0xbc, 0x74, 0x7b, 0xa2, 0xb4, 0xfc, 0xc9, 0x51, 0xae, 0x32, 0xf2, 0x4a, 0x25, 0xb8, 0xb3, 0xe0, - 0xd8, 0x18, 0xfb, 0x16, 0x7e, 0x59, 0xe0, 0xf9, 0xd7, 0xac, 0x2b, 0x43, 0x0a, 0x40, 0xdf, 0x13, - 0x65, 0x26, 0x2f, 0xed, 0x26, 0x7d, 0x28, 0xc8, 0x04, 0x64, 0xe3, 0x03, 0xad, 0x6d, 0xe8, 0x9a, - 0x83, 0x55, 0xc3, 0xd4, 0xf1, 0x53, 0x4c, 0x32, 0x88, 0xa9, 0xfe, 0xcb, 0xa3, 0xab, 0x5e, 0xf1, - 0x30, 0x6a, 0x14, 0xc2, 0x0d, 0x92, 0xed, 0xa0, 0x18, 0x13, 0xf4, 0xe7, 0x12, 0x20, 0x6f, 0x91, - 0x77, 0x2c, 0xdd, 0xd8, 0x31, 0xb0, 0x4d, 0x32, 0xd3, 0xac, 0xc3, 0x77, 0x2f, 0xc0, 0x95, 0x02, - 0x63, 0xdd, 0x85, 0x78, 0x39, 0x54, 0x99, 0xd6, 0x07, 0x71, 0xb3, 0xff, 0x27, 0x41, 0xfa, 0x04, - 0x33, 0xa3, 0x06, 0x84, 0x0c, 0x1e, 0x28, 0x27, 0x4b, 0xd4, 0x67, 0x87, 0x6a, 0x95, 0x4f, 0x8e, - 0x5e, 0xa8, 0xeb, 0x90, 0xa1, 0xa3, 0x5d, 0x88, 0xd3, 0x45, 0x64, 0x3a, 0xaa, 0xa1, 0x33, 0x6f, - 0x96, 0x2c, 0xad, 0x1c, 0x1f, 0xe5, 0x62, 0x5b, 0x4c, 0xf8, 0xc2, 0x5d, 0xc4, 0x38, 0x78, 0x4d, - 0x47, 0x39, 0x48, 0x38, 0x96, 0x8a, 0x9f, 0x1a, 0xc4, 0x31, 0xcc, 0x5d, 0x16, 0x60, 0xc6, 0x14, - 0x70, 0xac, 0xaa, 0x90, 0x64, 0xff, 0x38, 0x04, 0xe8, 0x24, 0x79, 0xa3, 0xbf, 0x96, 0xe0, 0xba, - 0x1b, 0x75, 0x5a, 0xb6, 0xb1, 0x6b, 0x98, 0x5a, 0x3b, 0x10, 0x7e, 0x4a, 0xec, 0x45, 0x7e, 0xf8, - 0x22, 0x1e, 0x42, 0x84, 0xa4, 0x9b, 0x02, 0x7e, 0x30, 0x34, 0xbd, 0x4e, 0x23, 0x24, 0x1e, 0x9a, - 0x9e, 0xa8, 0xf2, 0x48, 0xc9, 0xf4, 0xce, 0x68, 0x9c, 0x5d, 0x85, 0x1b, 0x43, 0x81, 0x2f, 0x12, - 0x58, 0x64, 0xbf, 0x25, 0xc1, 0xd5, 0x33, 0xdc, 0xb5, 0x1f, 0x27, 0xc9, 0x71, 0x1e, 0x06, 0x37, - 0x7b, 0x5f, 0x79, 0x81, 0x90, 0xc0, 0x3f, 0x88, 0x25, 0xb8, 0x76, 0x26, 0xe5, 0x9d, 0x37, 0x9b, - 0x98, 0x1f, 0xe8, 0x5f, 0x25, 0x98, 0x1a, 0x58, 0xc1, 0xe8, 0x03, 0x9f, 0x81, 0xd7, 0x8e, 0x8f, - 0x72, 0x51, 0xd6, 0xc9, 0x4b, 0xb1, 0xf2, 0xfd, 0x93, 0x56, 0xbe, 0x41, 0x7b, 0x60, 0x1d, 0xb3, - 0x1e, 0xde, 0xb9, 0x74, 0x0f, 0x1c, 0xa2, 0x6f, 0xe9, 0xd9, 0xbf, 0x95, 0x40, 0x1e, 0x24, 0x0b, - 0xb4, 0x09, 0x32, 0x7e, 0xea, 0xd8, 0x9a, 0xea, 0x73, 0xea, 0xd2, 0x45, 0x9c, 0x7a, 0x8a, 0x35, - 0x6f, 0x78, 0x9e, 0xfd, 0x23, 0x48, 0xda, 0x78, 0x97, 0x86, 0xde, 0x2d, 0xcb, 0xdc, 0x31, 0x76, - 0xc5, 0x9b, 0xfe, 0xe2, 0xc8, 0x91, 0x4b, 0x41, 0x61, 0xcd, 0xcb, 0xac, 0xb5, 0x32, 0x69, 0xfb, - 0x9e, 0xb2, 0xdf, 0x94, 0x60, 0xf6, 0x74, 0xbe, 0x3b, 0xc5, 0xd6, 0xb6, 0x82, 0xb6, 0x76, 0xff, - 0xf2, 0x94, 0xea, 0xb3, 0x90, 0x95, 0x70, 0x4c, 0x92, 0x43, 0xf9, 0xcf, 0x52, 0x33, 0x61, 0x6d, - 0xbc, 0x44, 0xca, 0x0d, 0x80, 0x3d, 0x63, 0x77, 0x4f, 0x7d, 0xa2, 0x39, 0xd8, 0x16, 0x09, 0xd9, - 0x38, 0x95, 0xbc, 0x47, 0x05, 0xf9, 0xbf, 0x07, 0x48, 0xd6, 0x3a, 0x5d, 0xcb, 0x76, 0xdc, 0x68, - 0x7b, 0x0d, 0x22, 0xdc, 0xd1, 0x0b, 0x85, 0x17, 0x86, 0x0c, 0x30, 0xd0, 0x92, 0xc7, 0x67, 0xc2, - 0xb3, 0x08, 0x0c, 0xb4, 0x09, 0x51, 0x1e, 0x14, 0x91, 0xcc, 0x55, 0x06, 0xb7, 0x30, 0x32, 0x1c, - 0x0f, 0xaf, 0xdc, 0x98, 0x48, 0xa0, 0x78, 0x71, 0x71, 0xe8, 0xd4, 0xb8, 0xf8, 0x6d, 0x88, 0xf0, - 0x6c, 0xba, 0xc8, 0xb7, 0xe5, 0x4e, 0x49, 0xd4, 0xd5, 0x36, 0x17, 0x8d, 0x36, 0x5e, 0x64, 0xd5, - 0xdc, 0xd1, 0xf2, 0x46, 0xe8, 0x35, 0x88, 0x11, 0xe2, 0xa8, 0xc4, 0xf8, 0x3a, 0xdf, 0x69, 0x8c, - 0xf3, 0x4c, 0x71, 0xbd, 0xde, 0xa8, 0x1b, 0x5f, 0xc7, 0x4a, 0x94, 0x10, 0x87, 0xfe, 0x40, 0x73, - 0xc0, 0xa2, 0x2e, 0xa2, 0xd1, 0x58, 0x8a, 0x85, 0x4d, 0xe3, 0x8a, 0x4f, 0xc2, 0x70, 0xf6, 0x8d, - 0xae, 0xba, 0xb3, 0x4f, 0x78, 0xac, 0x22, 0x70, 0xf6, 0x8d, 0xee, 0xe2, 0x2a, 0x51, 0xa2, 0xb4, - 0x70, 0x71, 0x9f, 0xa0, 0x2c, 0xc4, 0x9e, 0x68, 0xed, 0x36, 0xdb, 0xd9, 0x4c, 0x30, 0x14, 0xef, - 0x39, 0xe8, 0x6a, 0x22, 0xbf, 0x5a, 0x57, 0x23, 0xf6, 0x12, 0x5d, 0xcd, 0xd9, 0x63, 0xbb, 0xe3, - 0xb8, 0x02, 0x5c, 0xb4, 0xa5, 0x39, 0x7b, 0x28, 0x03, 0x51, 0x3e, 0x2f, 0x92, 0x89, 0xcd, 0x8f, - 0xdf, 0x9e, 0x54, 0xdc, 0x47, 0xf4, 0x3a, 0x4c, 0xf1, 0x34, 0x9b, 0xaa, 0x1b, 0x36, 0x6e, 0x39, - 0xed, 0x43, 0x16, 0x67, 0xc5, 0x94, 0x14, 0x17, 0x57, 0x84, 0x14, 0xdd, 0x01, 0x79, 0x30, 0x30, - 0x65, 0xf1, 0x51, 0x4c, 0x99, 0x1a, 0x88, 0x4b, 0x69, 0x0c, 0x2b, 0xde, 0xb5, 0x2f, 0xe0, 0xcb, - 0xf0, 0x18, 0x56, 0x14, 0xf4, 0x83, 0xbd, 0x3b, 0x20, 0x8b, 0xa8, 0xb4, 0x5f, 0x37, 0xc9, 0x71, - 0xb9, 0xbc, 0x5f, 0xb5, 0x00, 0xd3, 0x5d, 0xcd, 0x26, 0x58, 0x6d, 0xf6, 0x4c, 0xbd, 0x8d, 0x55, - 0x8e, 0x95, 0x49, 0xb1, 0xda, 0x69, 0x56, 0x54, 0x62, 0x25, 0xdc, 0xee, 0xce, 0x4b, 0x22, 0xcc, - 0xfe, 0x26, 0x92, 0x08, 0xb7, 0x41, 0xd6, 0xf1, 0x8e, 0xd6, 0x6b, 0x3b, 0xaa, 0x61, 0x0a, 0x3b, - 0xbd, 0x46, 0x03, 0x5b, 0x25, 0x25, 0xe4, 0x35, 0x93, 0x59, 0x68, 0xf6, 0x47, 0x21, 0x98, 0x60, - 0xeb, 0x11, 0xdd, 0x87, 0x30, 0x7d, 0xeb, 0x22, 0x41, 0x36, 0xea, 0xde, 0x8a, 0xb5, 0x41, 0x08, - 0xc2, 0xa6, 0xd6, 0xc1, 0x19, 0xc4, 0x6c, 0x82, 0xfd, 0x46, 0x57, 0x21, 0x4a, 0xf0, 0x63, 0xf5, - 0x40, 0x6b, 0x67, 0xa6, 0x99, 0xc9, 0x46, 0x08, 0x7e, 0xfc, 0x48, 0x6b, 0xa3, 0x2b, 0x10, 0x31, - 0x88, 0x6a, 0xe2, 0x27, 0x99, 0x19, 0xee, 0xbf, 0x0c, 0xb2, 0x81, 0x9f, 0xa0, 0x57, 0x20, 0xfe, - 0x44, 0x23, 0x2a, 0xee, 0x74, 0x9d, 0x43, 0xa6, 0xb5, 0x18, 0x35, 0x72, 0x52, 0xa5, 0xcf, 0x2c, - 0xcc, 0xd1, 0xec, 0x5d, 0xec, 0xa8, 0x2d, 0xab, 0x4d, 0x32, 0x57, 0xe8, 0xa2, 0xa6, 0xfb, 0x3a, - 0x2a, 0x2a, 0x5b, 0x6d, 0xb2, 0x12, 0x8e, 0x85, 0xe4, 0xf1, 0x95, 0x70, 0x6c, 0x5c, 0x0e, 0xaf, - 0x84, 0x63, 0x61, 0x79, 0x62, 0x25, 0x1c, 0x9b, 0x90, 0x23, 0x2b, 0xe1, 0x58, 0x44, 0x8e, 0xae, - 0x84, 0x63, 0x51, 0x39, 0xb6, 0x12, 0x8e, 0xc5, 0xe4, 0xf8, 0x4a, 0x38, 0x16, 0x97, 0x61, 0x25, - 0x1c, 0x03, 0x39, 0xb1, 0x12, 0x8e, 0x25, 0xe4, 0xc9, 0x95, 0x70, 0x6c, 0x52, 0x4e, 0xae, 0x84, - 0x63, 0x49, 0x39, 0xb5, 0x12, 0x8e, 0xa5, 0xe4, 0xa9, 0x95, 0x70, 0x6c, 0x4a, 0x96, 0x57, 0xc2, - 0x31, 0x59, 0x4e, 0xaf, 0x84, 0x63, 0x69, 0x19, 0x65, 0xab, 0xe2, 0xe0, 0x40, 0x43, 0x5f, 0x09, - 0xe8, 0x69, 0xe4, 0x3d, 0x1f, 0x6b, 0x94, 0xff, 0x89, 0x04, 0x72, 0x1d, 0x3f, 0xee, 0x61, 0xb3, - 0x85, 0x1f, 0x69, 0xed, 0xf2, 0x5e, 0xcf, 0xdc, 0x47, 0xaf, 0xc1, 0x54, 0x8b, 0xfe, 0x50, 0x79, - 0x9a, 0x92, 0x6a, 0x4c, 0x62, 0x1a, 0x4b, 0x32, 0x71, 0x9d, 0x4a, 0xa9, 0xe2, 0x6e, 0x00, 0x88, - 0x7a, 0xf4, 0x7d, 0x86, 0x58, 0x95, 0x38, 0xaf, 0x42, 0xc9, 0x66, 0x00, 0xc6, 0xb6, 0x9e, 0x30, - 0x72, 0x0b, 0xc0, 0x28, 0xd6, 0x13, 0xb4, 0x00, 0x33, 0x26, 0x7e, 0xea, 0xa8, 0x83, 0x95, 0x19, - 0x91, 0x29, 0x69, 0x5a, 0x56, 0xf6, 0x37, 0xc8, 0xff, 0x63, 0x08, 0xa6, 0xdc, 0x41, 0xbb, 0xec, - 0xbf, 0x03, 0x32, 0x7d, 0xbb, 0x86, 0xae, 0x3a, 0x16, 0x47, 0x72, 0xfd, 0xc0, 0xdb, 0xc3, 0xf2, - 0x4b, 0x41, 0x14, 0xfa, 0x5c, 0xd3, 0x1b, 0x16, 0xeb, 0x8e, 0x3b, 0x42, 0x25, 0x49, 0xfc, 0xb2, - 0xec, 0x36, 0xa4, 0xdc, 0x46, 0x5c, 0x82, 0xca, 0x10, 0x09, 0xf4, 0xf7, 0xe9, 0x11, 0xfa, 0x73, - 0x55, 0xad, 0x88, 0xa6, 0xd9, 0xdf, 0x05, 0x74, 0xb2, 0x6f, 0xbf, 0x13, 0x9e, 0xe0, 0x4e, 0x78, - 0x33, 0xe8, 0x84, 0xbf, 0x7c, 0xb1, 0xb9, 0xf9, 0x86, 0xed, 0x4f, 0x66, 0xfd, 0x5d, 0x08, 0x52, - 0xdc, 0x85, 0x79, 0xde, 0x97, 0x92, 0x19, 0xe5, 0x4a, 0xc3, 0xdc, 0xed, 0x1f, 0xad, 0xd0, 0xf9, - 0x85, 0x14, 0xd9, 0x2d, 0xf0, 0x2a, 0xbf, 0x4a, 0x63, 0x14, 0x4d, 0x0f, 0x9e, 0xc1, 0x84, 0x68, - 0xac, 0xa1, 0xe9, 0x5e, 0xa5, 0x5b, 0x90, 0x62, 0x71, 0x66, 0xbf, 0xd6, 0x38, 0xab, 0x95, 0x64, - 0x52, 0xaf, 0x5a, 0x09, 0x92, 0xa4, 0xab, 0xf9, 0xce, 0x73, 0xc2, 0x4c, 0xa9, 0xe7, 0x1c, 0x5c, - 0x4d, 0xd2, 0x36, 0xfe, 0xd0, 0xc1, 0xc6, 0xa4, 0xd7, 0xc1, 0x6a, 0xd7, 0xe2, 0xb9, 0x99, 0x71, - 0x25, 0xce, 0x25, 0x5b, 0x16, 0x41, 0xdb, 0xcc, 0x54, 0x98, 0x2e, 0x54, 0x9d, 0x2b, 0x27, 0x13, - 0x61, 0xbd, 0xbc, 0x31, 0xba, 0x3a, 0x95, 0x29, 0x12, 0x14, 0xe4, 0xff, 0x42, 0x82, 0xab, 0x34, - 0x6e, 0xe3, 0x2b, 0xad, 0xcc, 0x8e, 0xa8, 0x5d, 0xeb, 0xd4, 0x20, 0xca, 0x02, 0x42, 0x2f, 0xf0, - 0x5d, 0x3e, 0x3e, 0xca, 0x45, 0x68, 0xed, 0x17, 0xf6, 0x87, 0x11, 0x0a, 0x5c, 0x63, 0x39, 0x07, - 0xc7, 0xd6, 0x4c, 0xc2, 0x0e, 0x6e, 0xe8, 0x6b, 0xeb, 0xe0, 0x4e, 0x93, 0xee, 0x80, 0x43, 0xcc, - 0xf5, 0xcd, 0x04, 0x0a, 0xd7, 0x79, 0x59, 0x3e, 0x0b, 0x99, 0xc1, 0x21, 0x7b, 0x09, 0xcd, 0xdf, - 0x82, 0xd9, 0x0d, 0xfc, 0xe4, 0xb4, 0xd9, 0x94, 0x20, 0xca, 0x99, 0xce, 0x35, 0xf9, 0xdb, 0x83, - 0xa4, 0xe3, 0x3f, 0xa5, 0x2f, 0xb0, 0x91, 0x36, 0x58, 0x03, 0xc5, 0x6d, 0x98, 0xff, 0x08, 0xae, - 0x0e, 0xa0, 0x7b, 0xaf, 0xef, 0x5d, 0x88, 0x10, 0x47, 0x73, 0x44, 0x20, 0x97, 0x1a, 0x05, 0xbd, - 0xee, 0x68, 0x4e, 0x8f, 0x28, 0xa2, 0x5d, 0xfe, 0x16, 0xbc, 0x5a, 0xec, 0x39, 0x16, 0x35, 0x10, - 0x11, 0xf7, 0xe2, 0x96, 0x65, 0xb6, 0x8c, 0xb6, 0xa1, 0xf9, 0x8e, 0x50, 0xf3, 0xaf, 0xc1, 0xcd, - 0x61, 0xd5, 0x3c, 0x4d, 0x28, 0x2c, 0xb3, 0xdb, 0xeb, 0x60, 0x5a, 0x73, 0xcd, 0x20, 0x0e, 0x7a, - 0x17, 0x26, 0x85, 0x85, 0x51, 0xc3, 0x73, 0xd5, 0x70, 0x8e, 0x91, 0x26, 0x6c, 0x0f, 0x84, 0xe4, - 0xff, 0x52, 0x82, 0xe9, 0x8a, 0x6d, 0x75, 0xbb, 0x58, 0x17, 0x2e, 0x8c, 0xeb, 0xd6, 0xf5, 0x5c, - 0x92, 0xcf, 0x73, 0x6d, 0x40, 0xa8, 0x56, 0x11, 0xfb, 0x99, 0x07, 0x2f, 0xba, 0x4d, 0xaa, 0x55, - 0xd0, 0x97, 0xb9, 0x82, 0x7b, 0x84, 0xf1, 0x71, 0xea, 0x44, 0x82, 0x2f, 0x78, 0x46, 0xd8, 0xd7, - 0x6c, 0x8f, 0xe4, 0x7f, 0x18, 0x85, 0x2b, 0xfe, 0x97, 0xb6, 0x54, 0x76, 0x07, 0xfe, 0x31, 0x44, - 0xdd, 0x24, 0xcf, 0x08, 0xbc, 0x7b, 0x1a, 0x44, 0x41, 0xe8, 0xc3, 0x9f, 0xe8, 0x71, 0x31, 0x51, - 0x1d, 0xd2, 0x86, 0xe9, 0x60, 0xbb, 0x8d, 0xb5, 0x03, 0x1a, 0xd6, 0x50, 0x9d, 0x89, 0xa4, 0xfa, - 0xa8, 0xa1, 0x81, 0xec, 0x03, 0xe0, 0x21, 0xc6, 0xc7, 0x30, 0xed, 0x07, 0x75, 0xc7, 0x3f, 0x3c, - 0x9b, 0xcb, 0x86, 0xd7, 0x87, 0x75, 0xd3, 0xce, 0x3e, 0x20, 0x37, 0x25, 0xf5, 0xbe, 0xb7, 0x23, - 0xe1, 0x19, 0xfb, 0xfb, 0x97, 0xd6, 0x48, 0x65, 0x60, 0x77, 0x12, 0x88, 0xb1, 0x99, 0x53, 0xfd, - 0x15, 0xc5, 0xd8, 0x8f, 0x20, 0xc2, 0x93, 0xba, 0xe2, 0xf0, 0xec, 0xc1, 0x65, 0xa7, 0xc0, 0x93, - 0xc5, 0x8a, 0x40, 0xcb, 0x7e, 0x5f, 0x82, 0x49, 0xff, 0xeb, 0x46, 0x06, 0xc4, 0x98, 0xfa, 0x5d, - 0x8a, 0x1c, 0x7f, 0xe9, 0x3b, 0x77, 0x6e, 0x4a, 0x35, 0x9d, 0x06, 0x76, 0xba, 0x6d, 0x75, 0xfb, - 0x87, 0xa7, 0xe3, 0x4a, 0x8c, 0x0a, 0x68, 0xd0, 0x9a, 0xfd, 0x06, 0xc4, 0x3d, 0xa5, 0xfb, 0x72, - 0x71, 0xe3, 0x2f, 0x31, 0x17, 0x37, 0xb4, 0xff, 0x0a, 0x24, 0x03, 0x1a, 0x43, 0xb3, 0xde, 0x18, - 0xc2, 0xa5, 0x08, 0x1f, 0xc3, 0xb9, 0x28, 0xf9, 0x1f, 0x47, 0x61, 0xfa, 0x34, 0xe6, 0xfe, 0x00, - 0x64, 0x1f, 0x6f, 0xa9, 0x6d, 0x83, 0x38, 0xc2, 0x36, 0xef, 0x0c, 0xdf, 0xce, 0xfb, 0xc8, 0x4f, - 0x98, 0x62, 0xca, 0x0e, 0x52, 0xe2, 0x47, 0x90, 0xd2, 0xf9, 0xc0, 0x45, 0xbe, 0x5d, 0xdc, 0xd4, - 0x19, 0xb6, 0x0d, 0x3f, 0x85, 0x00, 0x05, 0x7a, 0x52, 0xf7, 0x15, 0x11, 0xd4, 0x82, 0xa4, 0x07, - 0x7e, 0xd8, 0x15, 0xfb, 0xb9, 0x17, 0x27, 0xc3, 0x49, 0xb7, 0x17, 0x8a, 0x89, 0x76, 0x61, 0xca, - 0xed, 0xc4, 0xdd, 0xfa, 0xc7, 0x5f, 0x4a, 0x37, 0xae, 0x62, 0xea, 0x22, 0x15, 0xf0, 0x6d, 0x09, - 0xa6, 0xdd, 0x9e, 0xbc, 0x94, 0xb5, 0xa1, 0x33, 0x3a, 0x4b, 0x96, 0xea, 0xc7, 0x47, 0xb9, 0xb4, - 0xd0, 0x8c, 0x9b, 0x39, 0x79, 0x61, 0xbb, 0x4b, 0xeb, 0x03, 0x80, 0x3a, 0x8d, 0x49, 0x68, 0x39, - 0xed, 0x78, 0xa2, 0x1f, 0x93, 0x50, 0x62, 0x7b, 0xf1, 0x98, 0x84, 0xfe, 0xac, 0xe9, 0xe8, 0x3b, - 0x12, 0xa4, 0xf9, 0x31, 0x59, 0xa7, 0xe7, 0x68, 0xfc, 0xa0, 0xdc, 0xcd, 0x09, 0x7c, 0x70, 0x7c, - 0x94, 0x9b, 0x62, 0xaf, 0x77, 0x5d, 0x94, 0xb1, 0x6e, 0x4b, 0x97, 0xed, 0xb6, 0x8f, 0x22, 0xb6, - 0xd0, 0x9e, 0x40, 0x47, 0xab, 0x90, 0xe2, 0x89, 0x12, 0xf7, 0x26, 0x20, 0x4b, 0x16, 0x24, 0x4b, - 0x37, 0x3f, 0x39, 0xca, 0xcd, 0x9f, 0xb2, 0x4e, 0x78, 0x8e, 0xe5, 0x11, 0xaf, 0xab, 0x24, 0x77, - 0xfc, 0x8f, 0x68, 0x0d, 0xa6, 0x78, 0x20, 0xdb, 0xbf, 0xde, 0x03, 0xa3, 0x1f, 0xee, 0xf2, 0x20, - 0xd8, 0x93, 0x8a, 0xfc, 0xd7, 0x2c, 0xcc, 0x9c, 0x1a, 0x83, 0xfd, 0x22, 0x02, 0xb3, 0x41, 0x5a, - 0xf5, 0xa2, 0x24, 0x75, 0xd0, 0xdf, 0xbe, 0x33, 0x32, 0x35, 0x7b, 0x17, 0x7d, 0x18, 0x35, 0xba, - 0x4f, 0x83, 0x1e, 0xf7, 0xe3, 0x01, 0xef, 0x75, 0x09, 0x7c, 0xf6, 0x7a, 0x07, 0xf0, 0x5d, 0x17, - 0xf6, 0xbe, 0xe7, 0x59, 0x78, 0xc6, 0xeb, 0xdd, 0x4b, 0xc0, 0xb3, 0xf6, 0xde, 0x25, 0x25, 0xd7, - 0xb7, 0xfc, 0x83, 0x04, 0xc9, 0xc0, 0xcc, 0x7e, 0x9d, 0xce, 0x65, 0xcb, 0x8b, 0xad, 0xf8, 0x65, - 0xa3, 0xb7, 0x2e, 0x3e, 0xad, 0x60, 0xc8, 0x95, 0xfd, 0x2b, 0x09, 0x92, 0x01, 0x45, 0xfe, 0x8a, - 0xdc, 0xd2, 0xcb, 0x1f, 0x79, 0x13, 0x52, 0xc1, 0x57, 0xe4, 0xeb, 0x43, 0x7a, 0x39, 0x7d, 0xe4, - 0xbf, 0x04, 0x11, 0x2e, 0x41, 0x08, 0x52, 0xef, 0x15, 0x6b, 0x8d, 0xda, 0xc6, 0x92, 0xba, 0xb8, - 0xa9, 0xa8, 0x4b, 0x65, 0x79, 0x0c, 0x4d, 0x42, 0xac, 0x52, 0x5d, 0xab, 0x52, 0xa1, 0x2c, 0xa1, - 0x04, 0x44, 0xd9, 0x53, 0xb5, 0x22, 0x87, 0xf2, 0x25, 0x90, 0x39, 0xf6, 0x0e, 0xa6, 0x6e, 0x86, - 0xee, 0x4a, 0x50, 0x01, 0xa6, 0xd9, 0x0e, 0xa2, 0x43, 0x23, 0x2b, 0xba, 0xbc, 0x55, 0x5f, 0x2c, - 0x9e, 0xf6, 0x8a, 0xe8, 0xea, 0xdd, 0xd0, 0x3a, 0x38, 0xff, 0xd3, 0x30, 0xa4, 0xfb, 0x20, 0xae, - 0x93, 0xfd, 0x33, 0xa9, 0xbf, 0x3f, 0x8a, 0x9c, 0x7b, 0xde, 0x79, 0xa2, 0xbd, 0xd8, 0x2a, 0x89, - 0x73, 0xc7, 0xf7, 0xe8, 0xa2, 0xf9, 0xe4, 0x28, 0x97, 0x1e, 0x1c, 0x2c, 0x79, 0xc1, 0x03, 0x49, - 0x77, 0x88, 0x2c, 0xe7, 0x6b, 0x98, 0xfb, 0x6a, 0xff, 0x3e, 0x18, 0xcf, 0xf9, 0x1a, 0xe6, 0xfe, - 0xb6, 0x52, 0x53, 0xa2, 0xb4, 0x70, 0xdb, 0x36, 0xd0, 0x0a, 0x84, 0xad, 0xae, 0xe3, 0x6e, 0xc8, - 0xbf, 0x78, 0xa1, 0x29, 0x6d, 0x76, 0xc5, 0x7c, 0x14, 0x86, 0x81, 0x56, 0xf8, 0x11, 0x7e, 0x5f, - 0xd1, 0x8c, 0x90, 0x47, 0xa4, 0xd0, 0x64, 0xe0, 0x45, 0x64, 0x77, 0x61, 0xd2, 0xaf, 0xb1, 0x53, - 0x4e, 0x2e, 0x8a, 0xc1, 0xa4, 0xc9, 0xa7, 0x47, 0x1a, 0xba, 0xd8, 0xb0, 0xfa, 0x0e, 0xb3, 0xbe, - 0x04, 0x71, 0x6f, 0x1e, 0x17, 0x39, 0xd3, 0xe3, 0x1c, 0xef, 0x65, 0x04, 0x27, 0xe4, 0x48, 0xfe, - 0x87, 0x21, 0x98, 0x54, 0x30, 0xb1, 0xda, 0x07, 0x58, 0xa7, 0x11, 0x94, 0x77, 0x4d, 0x57, 0x1a, - 0xfd, 0x9a, 0x6e, 0x11, 0xe2, 0x7d, 0x0f, 0x74, 0x81, 0x3b, 0x80, 0xfd, 0x56, 0xe8, 0x03, 0x48, - 0x36, 0xad, 0x9e, 0xa9, 0x6b, 0xf6, 0x21, 0x8b, 0xab, 0x58, 0x04, 0x92, 0x1a, 0x7a, 0x23, 0xcb, - 0x3f, 0xea, 0x42, 0x49, 0x34, 0xa6, 0xf1, 0x93, 0x32, 0xd9, 0xf4, 0x3d, 0xe5, 0xdf, 0x86, 0x49, - 0x7f, 0x29, 0x8a, 0x41, 0x78, 0x63, 0x73, 0xa3, 0xca, 0xd7, 0x64, 0xa9, 0x58, 0x5e, 0x5d, 0xac, - 0xad, 0xad, 0xc9, 0x12, 0x95, 0x57, 0xdf, 0xaf, 0x35, 0xe4, 0x10, 0xbf, 0x35, 0x59, 0x6f, 0x14, - 0x95, 0x86, 0x9b, 0x38, 0xcd, 0x63, 0x48, 0xfa, 0xfb, 0xa3, 0xcc, 0x47, 0xc3, 0x4e, 0x26, 0x08, - 0xec, 0xbc, 0x5f, 0x1f, 0x71, 0xc4, 0xae, 0x05, 0xd9, 0x7e, 0xd4, 0xfc, 0x3f, 0x87, 0x00, 0xf5, - 0x5f, 0xbc, 0x47, 0x56, 0xef, 0x03, 0xb4, 0xf6, 0x70, 0x6b, 0xbf, 0x6b, 0x19, 0xa6, 0x23, 0xf6, - 0x9a, 0x6f, 0x8d, 0x64, 0x3b, 0x1e, 0x59, 0x95, 0xbd, 0xf6, 0x8a, 0x0f, 0x0b, 0xfd, 0xe1, 0xf0, - 0x14, 0xfd, 0x38, 0x4b, 0xd1, 0xb3, 0x95, 0xff, 0x6b, 0x4d, 0xd3, 0x67, 0x8b, 0x00, 0xfd, 0x11, - 0xa3, 0x7b, 0x30, 0x71, 0x81, 0xec, 0x06, 0xaf, 0xeb, 0xb7, 0xf5, 0xfc, 0xff, 0x86, 0x01, 0x95, - 0x6d, 0xac, 0x39, 0x98, 0x52, 0x34, 0x19, 0x96, 0xe2, 0x28, 0xc1, 0x04, 0xdf, 0xd2, 0x87, 0x2e, - 0xb2, 0xa5, 0x77, 0xbb, 0x66, 0x4d, 0xd1, 0xd7, 0x60, 0xb2, 0x65, 0xb5, 0x7b, 0x1d, 0x53, 0x65, - 0x17, 0x81, 0xc4, 0xfe, 0xe3, 0x0b, 0xc3, 0xde, 0xd8, 0x89, 0xc1, 0x15, 0xca, 0x56, 0x9b, 0x3e, - 0x7b, 0xf7, 0xcd, 0x19, 0x20, 0xab, 0x81, 0xae, 0x43, 0xdc, 0x63, 0x1e, 0x66, 0x0e, 0x71, 0xa5, - 0x2f, 0x40, 0x77, 0x61, 0x42, 0x23, 0xaa, 0xb5, 0xc3, 0x82, 0xe9, 0xf3, 0x96, 0xa2, 0x12, 0xd6, - 0xc8, 0xe6, 0x0e, 0x7a, 0x03, 0xd2, 0x1d, 0xed, 0xa9, 0xba, 0x63, 0xf3, 0x1b, 0xc5, 0xaa, 0xa1, - 0xb7, 0x39, 0x13, 0x4a, 0xca, 0x54, 0x47, 0x7b, 0xba, 0x28, 0xe4, 0x35, 0xbd, 0x8d, 0xd1, 0x3d, - 0x48, 0xee, 0x3c, 0xe6, 0x5b, 0x2b, 0xee, 0x95, 0xf8, 0xad, 0xaa, 0xa9, 0xe3, 0xa3, 0x5c, 0x62, - 0xf1, 0x21, 0x53, 0x0c, 0xf5, 0x49, 0x4a, 0x62, 0xe7, 0xb1, 0xf7, 0x90, 0xfd, 0x1f, 0x09, 0xa2, - 0x62, 0x46, 0xa8, 0x0b, 0x20, 0xd4, 0x63, 0xe8, 0xfc, 0x9d, 0x26, 0x4b, 0x0f, 0x8f, 0x8f, 0x72, - 0xf1, 0x32, 0x93, 0xd6, 0x2a, 0xe4, 0x93, 0xa3, 0xdc, 0xbb, 0x97, 0xf5, 0x28, 0x2e, 0x88, 0x12, - 0xe7, 0x9d, 0xd4, 0x74, 0x96, 0x17, 0xde, 0xd3, 0x88, 0xba, 0x67, 0x10, 0xc7, 0xda, 0xb5, 0xb5, - 0x8e, 0xb8, 0x1f, 0x30, 0xb9, 0xa7, 0x91, 0x65, 0x57, 0x86, 0xb2, 0x34, 0x36, 0x3b, 0xe0, 0xf7, - 0xb8, 0xf8, 0x6d, 0x11, 0xef, 0x19, 0xdd, 0x85, 0x2b, 0x5e, 0x63, 0x95, 0x6a, 0xaa, 0xd9, 0x6b, - 0xed, 0x63, 0xe6, 0x83, 0x28, 0xb9, 0x4f, 0x7b, 0x85, 0xeb, 0xda, 0xd3, 0x12, 0x2f, 0xca, 0x5f, - 0x81, 0x69, 0xdf, 0x6b, 0xf5, 0x22, 0x69, 0x0c, 0x32, 0xbf, 0xc6, 0xe0, 0xfb, 0x84, 0xe2, 0x21, - 0x4c, 0x0d, 0x7c, 0x21, 0x24, 0xf8, 0xd7, 0x9f, 0x71, 0x0c, 0x7e, 0x52, 0x54, 0x28, 0xf3, 0x47, - 0x77, 0x6f, 0x90, 0x6a, 0x05, 0x9e, 0xf3, 0xd3, 0x90, 0xf6, 0xba, 0xf1, 0xfa, 0xfe, 0xf7, 0x10, - 0xa4, 0xaa, 0x4f, 0x71, 0xab, 0x47, 0xa5, 0xd5, 0x03, 0x6a, 0x2d, 0x4b, 0x10, 0x66, 0x84, 0xcb, - 0xc3, 0xa0, 0x7b, 0xc3, 0xae, 0x0c, 0x07, 0x1a, 0x16, 0xd8, 0x5f, 0xc6, 0xb7, 0x0c, 0x00, 0xe5, - 0x20, 0x61, 0x98, 0xc4, 0xd1, 0xcc, 0x16, 0x76, 0x2f, 0x3d, 0x4c, 0x28, 0xe0, 0x8a, 0x6a, 0x3a, - 0x9a, 0x0d, 0x24, 0xfb, 0xe2, 0x6e, 0xe0, 0x44, 0x6d, 0x0f, 0x1f, 0x78, 0x71, 0x4e, 0xc7, 0x68, - 0xd9, 0x16, 0x11, 0x47, 0x2e, 0x53, 0xac, 0x80, 0x1a, 0xeb, 0x3a, 0x13, 0xa3, 0xd7, 0x61, 0x0a, - 0xbb, 0xc3, 0x50, 0xd9, 0x87, 0x63, 0xcc, 0xca, 0xe3, 0x4a, 0xca, 0x13, 0x57, 0xa9, 0x34, 0x4f, - 0x20, 0xee, 0x0d, 0x10, 0xdd, 0x86, 0xc4, 0xf6, 0x46, 0x7d, 0xab, 0x5a, 0xae, 0x2d, 0xd6, 0xaa, - 0x15, 0x79, 0x2c, 0x7b, 0xf5, 0xd9, 0xf3, 0xf9, 0xe9, 0x15, 0xab, 0xc9, 0xaa, 0x6c, 0x9b, 0xa4, - 0x8b, 0x5b, 0xc6, 0x8e, 0x81, 0x75, 0x74, 0x1d, 0x26, 0x38, 0xf1, 0x4b, 0xd9, 0xf4, 0xb3, 0xe7, - 0xf3, 0xc9, 0x15, 0xab, 0xc9, 0x0e, 0x7b, 0xb8, 0xae, 0x32, 0x30, 0x5e, 0xdd, 0xa8, 0xc8, 0xa1, - 0xec, 0xd4, 0xb3, 0xe7, 0xf3, 0x09, 0xda, 0xde, 0xd4, 0x59, 0x49, 0x36, 0xfc, 0xdd, 0x1f, 0xcc, - 0x8d, 0xe5, 0x5f, 0x81, 0x6b, 0xf5, 0x87, 0x6b, 0xec, 0x75, 0x97, 0xad, 0x4e, 0x97, 0xaf, 0x19, - 0x37, 0xc7, 0x7b, 0x1d, 0xb2, 0x27, 0x0b, 0xbd, 0x37, 0xf3, 0xe3, 0x29, 0x88, 0x6e, 0x69, 0x87, - 0x6d, 0x4b, 0xd3, 0xd1, 0x3c, 0x24, 0xdc, 0xab, 0x71, 0xae, 0x25, 0xc4, 0x15, 0xbf, 0x28, 0x48, - 0x00, 0x32, 0x3b, 0xf2, 0xf3, 0x11, 0x80, 0x01, 0xa9, 0x1e, 0xc1, 0x36, 0x5d, 0x9b, 0x2a, 0xfb, - 0xd0, 0x8c, 0x07, 0x02, 0xa5, 0xd2, 0x27, 0x47, 0xb9, 0x07, 0xa3, 0x2d, 0x2b, 0xdc, 0xea, 0xd9, - 0x86, 0x73, 0x58, 0xa8, 0x3f, 0x5c, 0xdb, 0x16, 0x50, 0x94, 0xc5, 0x2d, 0x25, 0xd9, 0xf3, 0x3f, - 0x8a, 0x1b, 0x90, 0x74, 0x89, 0xb8, 0x2f, 0x4e, 0x1c, 0xac, 0x09, 0x69, 0xff, 0xb5, 0xed, 0x18, - 0x26, 0x3b, 0x45, 0x0e, 0xbe, 0xe0, 0x94, 0x2b, 0x16, 0x15, 0x0f, 0x20, 0xe5, 0xbb, 0x7a, 0x48, - 0xe9, 0x21, 0xc2, 0xe8, 0x61, 0xf3, 0xf8, 0x28, 0x97, 0xec, 0xd3, 0x2d, 0xa7, 0x88, 0x17, 0x09, - 0x3a, 0x93, 0xfd, 0x6e, 0x28, 0x41, 0xcc, 0xc0, 0x04, 0xb7, 0xa6, 0x18, 0x0f, 0x99, 0xd8, 0x03, - 0xaa, 0x42, 0x52, 0x24, 0xa9, 0xf8, 0x37, 0x8a, 0xe2, 0xbe, 0xe6, 0xbc, 0x6f, 0x91, 0xb8, 0x5f, - 0x31, 0x16, 0xaa, 0x66, 0xcb, 0xd2, 0xb1, 0xce, 0xac, 0x4f, 0x11, 0x39, 0x79, 0xf6, 0x40, 0xd0, - 0x12, 0xa4, 0x5a, 0x6d, 0xac, 0x99, 0xbd, 0xae, 0x8b, 0x83, 0x46, 0xc4, 0x49, 0x8a, 0x76, 0x02, - 0x68, 0x03, 0xd0, 0x0e, 0xbb, 0x8e, 0xe6, 0x1f, 0x15, 0x3b, 0x43, 0x1e, 0x05, 0x4c, 0x66, 0x6d, - 0x95, 0xfe, 0xc8, 0xd0, 0x4d, 0x48, 0x9a, 0x96, 0xd9, 0xa2, 0xeb, 0xb3, 0xcd, 0x7c, 0x1e, 0x3f, - 0x76, 0x0e, 0x0a, 0x51, 0x09, 0x22, 0xfc, 0x2a, 0x83, 0xc8, 0x2e, 0xdc, 0x1e, 0xf5, 0xc3, 0x8b, - 0xe5, 0x31, 0x45, 0xb4, 0x44, 0x55, 0x88, 0xda, 0xfc, 0x5a, 0x0d, 0xbb, 0xde, 0x70, 0x6e, 0x96, - 0xcf, 0x77, 0x69, 0x67, 0x79, 0x4c, 0x71, 0xdb, 0xa2, 0x86, 0x7b, 0xbb, 0x98, 0x47, 0x38, 0xe2, - 0x82, 0x68, 0x61, 0xc4, 0xbd, 0x5b, 0x1f, 0x30, 0x80, 0x42, 0x27, 0x68, 0xb0, 0x43, 0x47, 0x76, - 0xf1, 0x61, 0xf8, 0x04, 0x03, 0x17, 0x6c, 0xe8, 0x04, 0x79, 0x4b, 0xb4, 0x41, 0x43, 0x34, 0x37, - 0xea, 0x62, 0x57, 0x22, 0x12, 0x77, 0x3f, 0x73, 0x91, 0x9d, 0xc9, 0xf2, 0x98, 0xe2, 0x43, 0x40, - 0x0f, 0x21, 0xd1, 0xea, 0x3b, 0x8f, 0xcc, 0x14, 0x03, 0x7c, 0xf3, 0x42, 0x11, 0xc4, 0x32, 0x8d, - 0x1a, 0xfa, 0x52, 0xf4, 0x21, 0xa4, 0x48, 0x60, 0x27, 0x9b, 0xb9, 0xc2, 0x50, 0x3f, 0x7b, 0xd1, - 0x4c, 0xfa, 0xf2, 0x98, 0x32, 0x80, 0x84, 0x7e, 0x1b, 0x64, 0x67, 0xe0, 0xf8, 0x8e, 0xdd, 0x54, - 0x18, 0x7e, 0x7b, 0xf7, 0x8c, 0x43, 0xca, 0xe5, 0x31, 0xe5, 0x04, 0x1a, 0xfa, 0x18, 0xa6, 0x48, - 0xf0, 0x2b, 0xb1, 0xcc, 0x55, 0xd6, 0xc1, 0xe7, 0x46, 0xff, 0xae, 0xac, 0x8f, 0x3f, 0x88, 0x45, - 0xe1, 0xcd, 0xe0, 0x29, 0x20, 0xbb, 0x31, 0x33, 0x1c, 0xfe, 0xf4, 0x53, 0x49, 0x0a, 0x3f, 0x80, - 0x85, 0x56, 0x21, 0xde, 0x71, 0xbd, 0x31, 0xbb, 0x6f, 0x32, 0x7c, 0xf3, 0x37, 0x18, 0x20, 0x2c, - 0x8f, 0x29, 0xfd, 0xf6, 0xe8, 0xf7, 0x24, 0xb8, 0xae, 0x0d, 0x39, 0x2e, 0xcc, 0xbc, 0x72, 0xee, - 0x09, 0xc9, 0x08, 0x87, 0x92, 0xcb, 0x63, 0xca, 0xd0, 0x5e, 0xd0, 0x0e, 0x20, 0xf2, 0xb8, 0x3d, - 0xe0, 0xd0, 0x32, 0x73, 0xe7, 0x7f, 0x4a, 0x73, 0x96, 0x8b, 0x5c, 0x1e, 0x53, 0x4e, 0x41, 0x44, - 0x9f, 0x82, 0xc9, 0xae, 0xd6, 0x23, 0x58, 0xb5, 0xb1, 0x46, 0x2c, 0x33, 0x73, 0x9d, 0xfb, 0x43, - 0x26, 0x53, 0x98, 0x08, 0xad, 0xc3, 0xa4, 0xe7, 0xff, 0xd7, 0xac, 0xdd, 0xcc, 0x8d, 0x73, 0x4f, - 0x12, 0x82, 0xc1, 0x8c, 0x12, 0x68, 0x5e, 0x8a, 0x43, 0x54, 0x1c, 0xc7, 0x7b, 0x77, 0x65, 0xf8, - 0x2d, 0x19, 0x7e, 0x3f, 0x26, 0x2b, 0xbf, 0x92, 0xff, 0x5e, 0x02, 0x62, 0xde, 0xfe, 0x6c, 0x01, - 0x90, 0x17, 0x39, 0xf7, 0xbf, 0x1a, 0xa0, 0x7e, 0x3b, 0xb4, 0x3c, 0xa6, 0xa4, 0xdd, 0xb2, 0xfe, - 0x87, 0x03, 0x0f, 0x02, 0x57, 0x0a, 0x47, 0xf9, 0xb0, 0x95, 0x5a, 0x80, 0x77, 0xe7, 0x90, 0xfa, - 0x53, 0x71, 0xb1, 0xdc, 0xf3, 0xa7, 0xfc, 0xe0, 0x25, 0xe5, 0x8a, 0x85, 0x3f, 0xbd, 0x05, 0x29, - 0xbb, 0x67, 0xb2, 0x53, 0x78, 0x11, 0x7b, 0xf1, 0xed, 0x42, 0x52, 0x48, 0x45, 0xc6, 0xaa, 0x3c, - 0x40, 0xf1, 0x77, 0xce, 0xa5, 0x78, 0x77, 0xee, 0xcb, 0x92, 0xc7, 0xf1, 0x8b, 0x83, 0x1c, 0xff, - 0xc6, 0xf9, 0x1c, 0xef, 0x83, 0xf1, 0x48, 0x7e, 0xfb, 0x54, 0x92, 0x5f, 0x18, 0x91, 0xa5, 0x7c, - 0x88, 0x41, 0x96, 0x2f, 0x0f, 0xb0, 0xfc, 0x9d, 0x73, 0x59, 0xde, 0x3f, 0x47, 0x41, 0xf3, 0x9b, - 0xa7, 0xd0, 0xfc, 0x9b, 0x17, 0xda, 0x89, 0x2f, 0x4b, 0x01, 0x9e, 0x57, 0x4e, 0xe3, 0xf9, 0xc2, - 0x68, 0x3c, 0xef, 0x83, 0x0c, 0x10, 0xfd, 0x47, 0x27, 0x88, 0x5e, 0x3e, 0x9f, 0x29, 0x4f, 0xcd, - 0x71, 0x2e, 0x4b, 0x27, 0x98, 0x5e, 0x3b, 0x85, 0xe9, 0xd3, 0x0c, 0xfe, 0xde, 0x05, 0x98, 0xde, - 0xd7, 0xc1, 0x49, 0xaa, 0x7f, 0x1f, 0x26, 0xfd, 0xf4, 0xcc, 0xee, 0xce, 0x0d, 0x77, 0x24, 0x67, - 0x7c, 0x3f, 0xcc, 0x6c, 0xc0, 0x57, 0x84, 0xbe, 0x76, 0x92, 0xe5, 0xa7, 0xcf, 0x05, 0x3f, 0xe3, - 0x76, 0xc8, 0xb2, 0x74, 0x92, 0xe6, 0xd7, 0xfc, 0x34, 0x3f, 0x73, 0x6e, 0x10, 0x70, 0x62, 0x83, - 0xb6, 0x2c, 0xf9, 0x79, 0xfe, 0x3b, 0x12, 0x5c, 0x1f, 0x46, 0xd4, 0xc2, 0xc3, 0xbe, 0x73, 0x49, - 0x9e, 0xf7, 0x75, 0x3a, 0xb4, 0x1b, 0xb4, 0x7b, 0x2a, 0xd1, 0x73, 0xef, 0xfb, 0x85, 0x0b, 0x11, - 0xbd, 0xaf, 0xcb, 0xd3, 0x98, 0xfe, 0x35, 0x88, 0x39, 0xb6, 0xc6, 0xf7, 0x8f, 0x57, 0xd8, 0x39, - 0x33, 0x4b, 0x00, 0x37, 0xa8, 0x8c, 0x25, 0x8a, 0xd9, 0x0f, 0xbd, 0x04, 0x10, 0x73, 0x6f, 0x65, - 0xf9, 0xb8, 0x3a, 0xff, 0x03, 0x09, 0xc6, 0x57, 0xac, 0x26, 0xba, 0xe1, 0x3b, 0x95, 0x48, 0x8a, - 0xdc, 0xd5, 0xc4, 0x8a, 0xd5, 0x14, 0xc7, 0x0b, 0xef, 0xf4, 0x5b, 0x8b, 0x24, 0xcf, 0xab, 0x43, - 0x26, 0xe1, 0x1d, 0xea, 0x78, 0x8d, 0xd0, 0x57, 0x21, 0xda, 0xe5, 0x5b, 0x35, 0x41, 0xdd, 0xf9, - 0x61, 0xed, 0x79, 0x4d, 0xc5, 0x6d, 0xf2, 0xc6, 0x1d, 0xff, 0xff, 0x65, 0xb1, 0x6e, 0xe9, 0x18, - 0xa5, 0x00, 0xb6, 0x34, 0x42, 0xba, 0x7b, 0xb6, 0x46, 0xb0, 0x3c, 0x86, 0xa2, 0x30, 0xbe, 0xba, - 0x5e, 0x97, 0xa5, 0x37, 0xde, 0xf7, 0x1f, 0x29, 0x54, 0x94, 0x62, 0x6d, 0xa3, 0xb6, 0xb1, 0xa4, - 0x6e, 0x14, 0xd7, 0xab, 0x75, 0x79, 0x0c, 0x65, 0x60, 0xe6, 0xbd, 0x62, 0xad, 0x21, 0xce, 0x18, - 0xd4, 0xda, 0x46, 0xa3, 0xaa, 0x3c, 0x2a, 0xae, 0xc9, 0x12, 0x9a, 0x05, 0xa4, 0x6c, 0x96, 0x57, - 0xeb, 0x95, 0x92, 0x5a, 0xde, 0x5c, 0xdf, 0x2a, 0x96, 0x1b, 0xb5, 0xcd, 0x0d, 0x39, 0x84, 0x62, - 0x10, 0xae, 0x6c, 0x6e, 0x54, 0x65, 0x78, 0xe3, 0xfb, 0x13, 0x10, 0x66, 0x5b, 0xe3, 0x9b, 0x83, - 0x5b, 0xe3, 0xe9, 0x67, 0xcf, 0xe7, 0xa7, 0x68, 0x91, 0x7f, 0x5b, 0x9c, 0x85, 0x48, 0xa9, 0x58, - 0x5e, 0xdd, 0xde, 0x92, 0xa5, 0x6c, 0xea, 0xd9, 0xf3, 0x79, 0xa0, 0x15, 0xb8, 0x23, 0x40, 0xd7, - 0x79, 0xb6, 0x74, 0x53, 0xa9, 0xba, 0x1b, 0x63, 0x96, 0x14, 0x10, 0x64, 0xfe, 0x3a, 0x24, 0xeb, - 0xe5, 0xe5, 0xea, 0x7a, 0x51, 0x2d, 0x2f, 0x17, 0x37, 0x96, 0xaa, 0xf2, 0x78, 0x76, 0xe6, 0xd9, - 0xf3, 0x79, 0x79, 0x90, 0x10, 0x68, 0x17, 0xb5, 0xf5, 0xad, 0x4d, 0xa5, 0x21, 0x87, 0xfb, 0x5d, - 0x70, 0x1e, 0x46, 0x79, 0x00, 0xde, 0x7a, 0xb1, 0x5a, 0xad, 0xc8, 0x13, 0x59, 0xf4, 0xec, 0xf9, - 0x7c, 0x8a, 0x96, 0xf7, 0xe9, 0x15, 0xdd, 0x82, 0xc9, 0xb2, 0x52, 0x2d, 0x36, 0xaa, 0x6a, 0xbd, - 0x51, 0x6c, 0xd4, 0xe5, 0x48, 0x7f, 0x26, 0x3e, 0xca, 0x44, 0x05, 0x48, 0x17, 0xb7, 0x1b, 0x9b, - 0x6a, 0xa0, 0x6e, 0x94, 0x27, 0x04, 0x68, 0x5d, 0xba, 0x10, 0xfc, 0xf5, 0x3f, 0x03, 0x72, 0x60, - 0xfc, 0xea, 0x52, 0x59, 0x8e, 0x65, 0x67, 0x9f, 0x3d, 0x9f, 0x47, 0x83, 0x53, 0x58, 0x2a, 0xa3, - 0xcf, 0xc3, 0x6c, 0xe3, 0x83, 0xad, 0x6a, 0xa5, 0x5a, 0x2f, 0xab, 0xc1, 0x69, 0xc7, 0xb3, 0x99, - 0x67, 0xcf, 0xe7, 0x67, 0x68, 0x9b, 0x13, 0x53, 0x7f, 0x13, 0xe4, 0x7a, 0x43, 0xa9, 0x16, 0xd7, - 0xd5, 0xda, 0xc6, 0x52, 0xb5, 0xce, 0x5e, 0x16, 0xf4, 0x87, 0x34, 0x40, 0x6e, 0x74, 0x0a, 0x1b, - 0xd5, 0xf7, 0x06, 0xf0, 0x13, 0xfd, 0xfa, 0x03, 0x7c, 0x85, 0xe6, 0x21, 0xbe, 0x5e, 0x5b, 0x52, - 0x8a, 0x0c, 0x77, 0x92, 0xe7, 0x35, 0x68, 0x3d, 0x8f, 0x7d, 0x50, 0x0d, 0x72, 0x4c, 0x29, 0xf5, - 0xad, 0xe2, 0x86, 0x5a, 0xde, 0xdc, 0x58, 0xac, 0x2d, 0xa9, 0x4a, 0xb5, 0xbc, 0xb9, 0x51, 0xae, - 0xad, 0xd5, 0x78, 0xbb, 0x64, 0xf6, 0xe6, 0xb3, 0xe7, 0xf3, 0xf3, 0xae, 0x8a, 0xce, 0xe4, 0x8a, - 0xcf, 0xc3, 0x4c, 0xfd, 0xe1, 0x1a, 0xd7, 0xab, 0xdf, 0xf8, 0x52, 0xd9, 0xec, 0xb3, 0xe7, 0xf3, - 0xb3, 0x6c, 0x3e, 0x27, 0x68, 0x21, 0x1b, 0xfb, 0xee, 0x0f, 0xe6, 0xc6, 0xfe, 0xf4, 0x87, 0x73, - 0x63, 0xa5, 0xdb, 0x3f, 0xfb, 0xaf, 0xb9, 0xb1, 0x9f, 0x1d, 0xcf, 0x49, 0x3f, 0x3f, 0x9e, 0x93, - 0x7e, 0x71, 0x3c, 0x27, 0xfd, 0xe7, 0xf1, 0x9c, 0xf4, 0xfb, 0xbf, 0x9c, 0x1b, 0xfb, 0xf9, 0x2f, - 0xe7, 0xc6, 0x7e, 0xf1, 0xcb, 0xb9, 0xb1, 0x0f, 0x23, 0x7c, 0x61, 0x35, 0x23, 0x2c, 0xb5, 0x71, - 0xef, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0x9b, 0xdc, 0xbc, 0x3d, 0x7c, 0x48, 0x00, 0x00, + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x7c, 0x4d, 0x6c, 0x23, 0xc9, + 0x75, 0xbf, 0x9a, 0xa2, 0xf8, 0xf1, 0x28, 0x52, 0xcd, 0x92, 0x46, 0xc3, 0xe1, 0xce, 0x0e, 0x65, + 0xee, 0xcc, 0xee, 0xcc, 0xda, 0x4b, 0xad, 0x67, 0xed, 0xf5, 0x7a, 0xec, 0x9d, 0x5d, 0x7e, 0x49, + 0xa2, 0xbe, 0xa7, 0x49, 0xcd, 0x7e, 0xfd, 0xd7, 0xfd, 0x6f, 0xb1, 0x4b, 0x54, 0x47, 0x64, 0x37, + 0xa7, 0xab, 0x39, 0x33, 0xb2, 0x01, 0xc3, 0xb0, 0x63, 0x20, 0x98, 0x53, 0x0c, 0x24, 0xbe, 0x24, + 0x03, 0x04, 0xb0, 0x0d, 0xe4, 0x10, 0x23, 0x88, 0x11, 0x38, 0x41, 0x4e, 0x01, 0x92, 0x83, 0x0f, + 0x09, 0x60, 0x20, 0x08, 0xe2, 0x24, 0x80, 0x92, 0xc8, 0x97, 0x1c, 0x72, 0x08, 0x92, 0xdb, 0x9c, + 0x82, 0xfa, 0xe8, 0x66, 0x37, 0x45, 0x51, 0x94, 0x66, 0x6c, 0x5f, 0x34, 0xec, 0x57, 0x55, 0xbf, + 0xaa, 0x7a, 0xfd, 0xea, 0xfd, 0x5e, 0xbd, 0xaa, 0x1e, 0x98, 0xff, 0x2d, 0x6b, 0x97, 0x2c, 0xd2, + 0x3f, 0xdd, 0x5d, 0xf6, 0x4f, 0xa1, 0x6b, 0x5b, 0x8e, 0x85, 0xae, 0x34, 0xad, 0xe6, 0x81, 0x6d, + 0x69, 0xcd, 0xfd, 0x02, 0x79, 0xd0, 0x2e, 0xb0, 0x12, 0x5e, 0x2b, 0x7b, 0x09, 0xdb, 0xb6, 0x65, + 0xd3, 0xfa, 0xfc, 0x07, 0x6f, 0x91, 0x9d, 0x6b, 0x59, 0x2d, 0x8b, 0xfd, 0x5c, 0xa4, 0xbf, 0x84, + 0x14, 0x31, 0x8c, 0xee, 0xee, 0xa2, 0xae, 0x39, 0x9a, 0x90, 0x65, 0x5c, 0x99, 0x61, 0xbd, 0xb1, + 0x67, 0xd9, 0x1d, 0xcd, 0x71, 0x31, 0x5e, 0x21, 0x0f, 0xda, 0x8b, 0x4d, 0xcd, 0xd1, 0xda, 0x56, + 0x6b, 0x51, 0xc7, 0xa4, 0xd9, 0xdd, 0x5d, 0x24, 0x8e, 0xdd, 0x6b, 0x3a, 0x3d, 0x1b, 0xeb, 0xa2, + 0x52, 0x6e, 0x48, 0x25, 0x07, 0x9b, 0x9a, 0xe9, 0xb8, 0xf8, 0x3d, 0xc7, 0x68, 0x2f, 0xee, 0xb7, + 0x9b, 0x8b, 0x8e, 0xd1, 0xc1, 0xc4, 0xd1, 0x3a, 0x5d, 0x51, 0xf2, 0x19, 0xda, 0x94, 0x34, 0xf7, + 0x71, 0x47, 0x6b, 0xee, 0x6b, 0x66, 0x0b, 0xdb, 0x8b, 0xbc, 0x8f, 0x66, 0x77, 0x57, 0x54, 0xb9, + 0xde, 0x6c, 0xf7, 0x88, 0x83, 0xed, 0x87, 0xd8, 0x26, 0x86, 0x65, 0x2e, 0x8a, 0x47, 0x55, 0x3c, + 0xf3, 0x5a, 0xf9, 0x1f, 0x85, 0xe0, 0x72, 0x49, 0x6b, 0x1e, 0xf4, 0xba, 0x55, 0xb3, 0x69, 0x1f, + 0x76, 0x1d, 0xc3, 0x32, 0xb7, 0xd8, 0x5f, 0x82, 0x64, 0x98, 0x3c, 0xc0, 0x87, 0x19, 0x69, 0x41, + 0xba, 0x39, 0xad, 0xd0, 0x9f, 0xe8, 0x5d, 0x08, 0x77, 0x2c, 0x1d, 0x67, 0x42, 0x0b, 0xd2, 0xcd, + 0xd4, 0xed, 0x5b, 0x85, 0x53, 0x75, 0x5b, 0xe8, 0xa3, 0x6d, 0x58, 0x3a, 0x56, 0x58, 0x33, 0xb4, + 0x0b, 0xb1, 0x83, 0x0e, 0x51, 0x0d, 0x73, 0xcf, 0xca, 0x4c, 0x2e, 0x48, 0x37, 0x13, 0xb7, 0xef, + 0x8c, 0x80, 0x38, 0x65, 0x58, 0x85, 0xb5, 0x8d, 0x7a, 0xcd, 0xdc, 0xb3, 0x4a, 0x89, 0xe3, 0xa3, + 0x5c, 0x54, 0x3c, 0x28, 0xd1, 0x83, 0x0e, 0xa1, 0x3f, 0xb2, 0x5b, 0xe0, 0xca, 0xe8, 0xf8, 0x7b, + 0xb6, 0xc1, 0xc6, 0x1f, 0x57, 0xe8, 0x4f, 0xf4, 0x39, 0x40, 0x98, 0xe3, 0x61, 0x5d, 0xa5, 0x2f, + 0x52, 0xa5, 0x13, 0x0c, 0xb1, 0x09, 0xca, 0x5e, 0x49, 0x45, 0x73, 0xb4, 0x35, 0x7c, 0x78, 0x27, + 0xfc, 0x9f, 0x7f, 0x94, 0x93, 0xf8, 0xdf, 0xfc, 0xb7, 0x26, 0x21, 0xd5, 0x1f, 0x0a, 0x83, 0x5f, + 0x81, 0x08, 0x7b, 0x03, 0x98, 0xf5, 0x90, 0xba, 0xfd, 0xe6, 0x58, 0xea, 0xa0, 0x4d, 0x0b, 0x75, + 0xd6, 0x4e, 0x11, 0xed, 0x11, 0x82, 0x30, 0xd1, 0xda, 0x8e, 0x18, 0x08, 0xfb, 0x8d, 0xfe, 0x40, + 0x82, 0x85, 0xc1, 0x11, 0x95, 0x0e, 0xd7, 0x36, 0xea, 0x1b, 0x1a, 0x7d, 0x8d, 0x6b, 0xf8, 0xb0, + 0x56, 0xc9, 0x4c, 0x2e, 0x4c, 0xde, 0x4c, 0xdc, 0xde, 0x1a, 0xbf, 0xe3, 0xea, 0x19, 0x88, 0x55, + 0xd3, 0xb1, 0x0f, 0x95, 0x33, 0x3b, 0xce, 0xd6, 0xe1, 0xc6, 0x58, 0x50, 0x7e, 0x1b, 0x8a, 0x73, + 0x1b, 0x9a, 0x83, 0xa9, 0x87, 0x5a, 0xbb, 0x87, 0xc5, 0x6c, 0xf9, 0xc3, 0x9d, 0xd0, 0x3b, 0x52, + 0xfe, 0x32, 0x44, 0xb8, 0x62, 0x50, 0x12, 0xe2, 0xc5, 0x6a, 0xfd, 0xf6, 0x17, 0xdf, 0x5e, 0x2e, + 0x6f, 0xc8, 0x13, 0xe2, 0x15, 0xfc, 0x54, 0x82, 0xf9, 0xba, 0x63, 0x63, 0xad, 0x53, 0x33, 0x5b, + 0x98, 0xd0, 0x39, 0x55, 0xb0, 0xa3, 0x19, 0x6d, 0x82, 0x6e, 0x40, 0x8a, 0xb0, 0x12, 0x55, 0xd3, + 0x75, 0x1b, 0x13, 0x22, 0x3a, 0x4c, 0x72, 0x69, 0x91, 0x0b, 0xd1, 0xe7, 0x21, 0x4c, 0xba, 0x9a, + 0xc9, 0x7a, 0x4e, 0xdc, 0xbe, 0xec, 0x53, 0x9b, 0x58, 0xc8, 0x85, 0x7a, 0x57, 0x33, 0x4b, 0xe1, + 0x9f, 0x1d, 0xe5, 0x26, 0x14, 0x56, 0x15, 0x95, 0x00, 0x88, 0xa3, 0xd9, 0x8e, 0x4a, 0x57, 0xa0, + 0x30, 0xda, 0x97, 0x7d, 0x0d, 0xe9, 0x0a, 0x2d, 0xec, 0xb7, 0x9b, 0x85, 0x86, 0xbb, 0x42, 0x45, + 0xf3, 0x38, 0x6b, 0x46, 0xa5, 0xf9, 0x7f, 0x9a, 0x84, 0xcb, 0x03, 0x03, 0xdf, 0xb6, 0xad, 0x16, + 0x1b, 0xd2, 0x12, 0x4c, 0x37, 0x7b, 0x8e, 0xf5, 0x10, 0xdb, 0xbc, 0x07, 0x69, 0xfc, 0x1e, 0x12, + 0xa2, 0x21, 0x95, 0xa3, 0x6f, 0x02, 0xea, 0x6a, 0xb6, 0x63, 0x50, 0x70, 0xb5, 0x2b, 0xd0, 0x33, + 0x21, 0x66, 0x1f, 0xb5, 0x11, 0xf6, 0x71, 0xca, 0xb8, 0x0a, 0xdb, 0x2e, 0x98, 0x2b, 0x61, 0xaf, + 0x53, 0xf4, 0x9c, 0xee, 0x0e, 0x96, 0x66, 0x5b, 0x90, 0x3e, 0xd1, 0x04, 0x29, 0x80, 0x0c, 0x86, + 0x8c, 0x75, 0xd5, 0xf3, 0x60, 0xe7, 0x99, 0x62, 0xda, 0x6d, 0xee, 0x15, 0x64, 0x9f, 0x48, 0x30, + 0x3f, 0x7c, 0x70, 0x43, 0x6c, 0xed, 0x63, 0xbf, 0xad, 0x25, 0x6e, 0x57, 0x5e, 0x84, 0x22, 0xfc, + 0x16, 0xfb, 0xe3, 0x10, 0x5c, 0xa1, 0x26, 0xab, 0xf7, 0xda, 0x78, 0xbb, 0x51, 0x2f, 0xef, 0x6b, + 0x86, 0x69, 0x98, 0x2d, 0x05, 0x37, 0x2d, 0x5b, 0x47, 0xdf, 0x93, 0x20, 0x4b, 0xbd, 0x2c, 0x6e, + 0x06, 0x14, 0xa0, 0xda, 0xac, 0x98, 0xfb, 0xd5, 0x52, 0xfd, 0x5f, 0x8e, 0x72, 0x6f, 0xb5, 0x0c, + 0x67, 0xbf, 0xb7, 0x5b, 0x68, 0x5a, 0x9d, 0x45, 0x6f, 0x84, 0xfa, 0x6e, 0xff, 0xf7, 0x62, 0xf7, + 0xa0, 0xb5, 0xc8, 0xc8, 0xa0, 0xd7, 0x33, 0xf4, 0xc2, 0xce, 0x4e, 0xad, 0x72, 0x7c, 0x94, 0xcb, + 0x6c, 0xbb, 0xe0, 0x9e, 0x7a, 0x78, 0xcf, 0x4a, 0xa6, 0x7b, 0x4a, 0x09, 0xba, 0x0f, 0x11, 0xad, + 0x49, 0xa7, 0x23, 0x7c, 0xf8, 0xdd, 0x51, 0x2a, 0x39, 0x6d, 0x66, 0x85, 0xed, 0x46, 0xbd, 0xc8, + 0x50, 0x14, 0x81, 0x96, 0xbf, 0x0e, 0x71, 0x4f, 0x88, 0x00, 0x22, 0x3b, 0xdb, 0x95, 0x62, 0xa3, + 0x2a, 0x4f, 0xa0, 0x04, 0x44, 0x95, 0xea, 0x7a, 0xb5, 0x58, 0xaf, 0xca, 0x52, 0xfe, 0x1f, 0xa2, + 0x90, 0xe4, 0x6e, 0xdd, 0x5d, 0xb9, 0xc1, 0xf5, 0x25, 0x5d, 0x64, 0x7d, 0xa1, 0xbb, 0x10, 0xc3, + 0x26, 0x57, 0xb0, 0x78, 0xd1, 0x63, 0x21, 0x44, 0xb1, 0xc9, 0xd4, 0x83, 0xae, 0x70, 0x9e, 0xa0, + 0x8b, 0x3b, 0x5e, 0x8a, 0x1e, 0x1f, 0xe5, 0x26, 0x77, 0x94, 0x1a, 0x27, 0x8c, 0xef, 0x48, 0x30, + 0xdb, 0xb3, 0x0d, 0xa2, 0xee, 0x1e, 0xaa, 0x6d, 0xab, 0xa9, 0xb5, 0x0d, 0xe7, 0x50, 0x3d, 0x78, + 0x98, 0x99, 0x62, 0x0b, 0xeb, 0xee, 0x99, 0xec, 0x25, 0xa6, 0x59, 0xd8, 0xb1, 0x0d, 0x52, 0x3a, + 0x5c, 0x17, 0x08, 0x6b, 0x0f, 0xf9, 0x6a, 0x9a, 0x3b, 0x3e, 0xca, 0xc9, 0x3b, 0x4a, 0xcd, 0x5f, + 0x74, 0x5f, 0x91, 0x7b, 0x03, 0x95, 0xd1, 0x57, 0x21, 0xab, 0xe3, 0xae, 0x8d, 0x9b, 0x1a, 0x35, + 0xa4, 0x5d, 0x86, 0xac, 0x76, 0x34, 0xd3, 0xd8, 0xc3, 0xc4, 0xc9, 0x84, 0x99, 0x1f, 0xcd, 0xf4, + 0x6b, 0xf0, 0xae, 0x37, 0x44, 0x39, 0xd2, 0x3c, 0xd2, 0xa3, 0xbe, 0xc1, 0xe2, 0x2c, 0x9a, 0x89, + 0x30, 0x45, 0xdd, 0x3e, 0x3f, 0xff, 0x2a, 0x69, 0x7c, 0x22, 0x52, 0x50, 0x60, 0xc6, 0xd7, 0x05, + 0xe3, 0xf7, 0x38, 0xc3, 0xbf, 0x35, 0x36, 0x35, 0x29, 0x29, 0x1c, 0xa4, 0xd7, 0x33, 0x56, 0x4f, + 0xf4, 0x37, 0xb1, 0x7a, 0xde, 0x81, 0x54, 0xd3, 0x6a, 0xb7, 0x31, 0x33, 0x73, 0x75, 0x47, 0xa9, + 0x65, 0x62, 0xcc, 0x68, 0xd2, 0xc7, 0x47, 0xb9, 0x64, 0xd9, 0x2b, 0xa1, 0xe6, 0x93, 0x6c, 0xfa, + 0x1f, 0xd1, 0xef, 0x49, 0x70, 0x95, 0x88, 0xf5, 0xa4, 0x76, 0x1d, 0xa2, 0x36, 0xc5, 0x8a, 0x72, + 0xe7, 0x03, 0x4c, 0x5f, 0x5f, 0xb8, 0xc8, 0x72, 0x2c, 0xbd, 0x7c, 0x7c, 0x94, 0x3b, 0xdd, 0x0f, + 0x29, 0x57, 0xdc, 0x8e, 0xb7, 0x1d, 0x12, 0x2c, 0xca, 0x96, 0xe1, 0xd2, 0x50, 0xd3, 0x3c, 0x8b, + 0xb7, 0xe3, 0x7e, 0x2f, 0x28, 0x43, 0x8a, 0xdb, 0x8a, 0xeb, 0x22, 0xf3, 0x7f, 0x35, 0x0f, 0x29, + 0x05, 0x13, 0xc7, 0xb2, 0xb1, 0xbb, 0xd0, 0xfd, 0x8b, 0x34, 0x7c, 0x81, 0x45, 0xfa, 0x13, 0x09, + 0x66, 0x69, 0x8c, 0x6c, 0x1b, 0x5d, 0xc7, 0xb2, 0x55, 0x1b, 0x3f, 0xb2, 0x0d, 0x07, 0xbb, 0x14, + 0x57, 0x1c, 0xa1, 0xb7, 0xe0, 0x40, 0x0a, 0x15, 0x0f, 0x44, 0x11, 0x18, 0x7c, 0x31, 0xde, 0xfd, + 0xf6, 0xbf, 0xe5, 0xee, 0x8c, 0x65, 0x4a, 0x27, 0xc3, 0xf6, 0x42, 0xad, 0xa2, 0x20, 0xfd, 0x04, + 0x30, 0xba, 0x0a, 0x61, 0xba, 0x98, 0x59, 0x9c, 0x16, 0x2f, 0xc5, 0x8e, 0x8f, 0x72, 0x61, 0xba, + 0xdc, 0x15, 0x26, 0x45, 0x0e, 0xcc, 0x89, 0xb5, 0xec, 0xb9, 0x16, 0xb6, 0x74, 0xa2, 0x6c, 0x4a, + 0x5f, 0x1d, 0x7f, 0x4a, 0x5c, 0xfb, 0xee, 0x2b, 0x64, 0xc1, 0x31, 0xd7, 0x1e, 0xda, 0x3d, 0x51, + 0x82, 0xb6, 0x21, 0x45, 0x23, 0xdf, 0x5d, 0x8d, 0x60, 0x95, 0x0e, 0x99, 0x64, 0x64, 0xd6, 0xdf, + 0xe0, 0x52, 0x25, 0x0f, 0xda, 0xb4, 0x4e, 0xa1, 0x22, 0x2a, 0xfb, 0xf4, 0x96, 0xd4, 0x7d, 0x32, + 0x82, 0x96, 0x21, 0xe1, 0x68, 0xbb, 0x6d, 0x17, 0x8e, 0xfb, 0xc6, 0x57, 0x4f, 0x81, 0x6b, 0xd0, + 0x9a, 0x3e, 0x2c, 0x70, 0x5c, 0x01, 0x41, 0x15, 0x00, 0xe7, 0xb0, 0xeb, 0xe2, 0xa4, 0x18, 0xce, + 0x8d, 0xd3, 0x70, 0x0e, 0xbb, 0x7e, 0x98, 0xb8, 0x23, 0x9e, 0x09, 0x5a, 0x85, 0x69, 0xbe, 0x33, + 0x12, 0x38, 0x33, 0x0c, 0xe7, 0xb5, 0x53, 0x70, 0x58, 0xc4, 0xa9, 0xf9, 0x90, 0x12, 0xc4, 0x93, + 0x10, 0x54, 0x84, 0x28, 0xdf, 0x91, 0x91, 0x4c, 0x92, 0xc1, 0x7c, 0xe6, 0xb4, 0xe1, 0xb0, 0x5a, + 0x3e, 0xd5, 0xbb, 0xed, 0xd0, 0x22, 0x24, 0x68, 0x94, 0x66, 0x1b, 0x3a, 0x56, 0xf5, 0x5d, 0xe6, + 0x77, 0xe3, 0xa5, 0xd4, 0xf1, 0x51, 0x0e, 0xb6, 0x84, 0xb8, 0x52, 0x52, 0xc0, 0xad, 0x52, 0xd9, + 0x45, 0x9f, 0x85, 0x74, 0xd7, 0xc6, 0x5d, 0xcd, 0xc6, 0x6a, 0xd3, 0xea, 0x74, 0xdb, 0xd8, 0xc1, + 0x3a, 0xf3, 0x33, 0x31, 0x45, 0x16, 0x05, 0x65, 0x57, 0xce, 0x23, 0x5f, 0xcd, 0xa1, 0x9b, 0x2a, + 0x82, 0x6d, 0x5a, 0x33, 0xce, 0x6a, 0x26, 0x99, 0xb4, 0x26, 0x84, 0xe8, 0x10, 0xe6, 0xc9, 0x21, + 0x71, 0x70, 0x47, 0x65, 0xea, 0x26, 0x6a, 0xc7, 0x68, 0xd9, 0x94, 0x2b, 0x32, 0x69, 0x36, 0xad, + 0xf2, 0xf8, 0xc6, 0x56, 0x67, 0x38, 0xec, 0x35, 0x92, 0x0d, 0x81, 0xc2, 0xb7, 0x0d, 0x73, 0x64, + 0x48, 0x11, 0x7a, 0x0b, 0x2e, 0xf5, 0x57, 0x06, 0x51, 0xbb, 0xbd, 0xdd, 0xb6, 0x41, 0xf6, 0x31, + 0xf7, 0x78, 0x31, 0x65, 0xce, 0x57, 0xb8, 0xed, 0x96, 0xa1, 0xc3, 0xc0, 0x62, 0x6f, 0x52, 0xed, + 0x68, 0x2d, 0x9c, 0x49, 0x2c, 0x48, 0x37, 0xa7, 0x4a, 0x2b, 0xcf, 0x8e, 0x72, 0x95, 0xb1, 0x57, + 0x2a, 0xc1, 0x9d, 0x45, 0xc7, 0xc6, 0xd8, 0xb7, 0xf0, 0xcb, 0x02, 0xcf, 0xbf, 0x66, 0x5d, 0x19, + 0x52, 0x00, 0xfa, 0x4c, 0x94, 0x99, 0xbe, 0x30, 0x4d, 0xfa, 0x50, 0x90, 0x09, 0xc8, 0xc6, 0x0f, + 0xb5, 0xb6, 0xa1, 0x6b, 0x0e, 0x56, 0x0d, 0x53, 0xc7, 0x8f, 0x31, 0xc9, 0x20, 0xa6, 0xfa, 0x2f, + 0x8f, 0xaf, 0x7a, 0xc5, 0xc3, 0xa8, 0x51, 0x08, 0x37, 0x48, 0xb6, 0x83, 0x62, 0x4c, 0xd0, 0x9f, + 0x4a, 0x80, 0xbc, 0x45, 0xde, 0xb1, 0x74, 0x63, 0xcf, 0xc0, 0x36, 0xc9, 0xcc, 0xb2, 0x0e, 0xdf, + 0x3f, 0x87, 0xaf, 0x14, 0x18, 0x1b, 0x2e, 0xc4, 0x8b, 0x71, 0x95, 0x69, 0x7d, 0x10, 0x37, 0xfb, + 0xbf, 0x12, 0xa4, 0x4f, 0x78, 0x66, 0xd4, 0x80, 0x90, 0xc1, 0x03, 0xe5, 0x64, 0x89, 0x72, 0x76, + 0xa8, 0x56, 0x79, 0x76, 0xf4, 0x5c, 0x5d, 0x87, 0x0c, 0x1d, 0xb5, 0x20, 0x4e, 0x17, 0x91, 0xe9, + 0xa8, 0x86, 0xce, 0xd8, 0x2c, 0x59, 0x5a, 0x3d, 0x3e, 0xca, 0xc5, 0xb6, 0x99, 0xf0, 0xb9, 0xbb, + 0x88, 0x71, 0xf0, 0x9a, 0x8e, 0x72, 0x90, 0x70, 0x2c, 0x15, 0x3f, 0x36, 0x88, 0x63, 0x98, 0x2d, + 0x16, 0x60, 0xc6, 0x14, 0x70, 0xac, 0xaa, 0x90, 0x64, 0xff, 0x30, 0x04, 0xe8, 0xa4, 0xf3, 0x46, + 0x7f, 0x29, 0xc1, 0x55, 0x37, 0xea, 0xb4, 0x6c, 0xa3, 0x65, 0x98, 0x5a, 0x3b, 0x10, 0x7e, 0x4a, + 0xec, 0x45, 0x7e, 0xfc, 0x3c, 0x0c, 0x21, 0x42, 0xd2, 0x2d, 0x01, 0x3f, 0x18, 0x9a, 0x5e, 0xa5, + 0x11, 0x12, 0x0f, 0x4d, 0x4f, 0x54, 0xb9, 0xaf, 0x64, 0x7a, 0xa7, 0x34, 0xce, 0xae, 0xc1, 0xcb, + 0x23, 0x81, 0xcf, 0x13, 0x58, 0x64, 0xbf, 0x2d, 0xc1, 0xe5, 0x53, 0xe8, 0xda, 0x8f, 0x93, 0xe4, + 0x38, 0xf7, 0x82, 0x9b, 0xbd, 0xaf, 0x3c, 0x47, 0x48, 0xe0, 0x1f, 0xc4, 0x32, 0x5c, 0x39, 0xd5, + 0xe5, 0x9d, 0x35, 0x9b, 0x98, 0x1f, 0xe8, 0x9f, 0x25, 0x98, 0x19, 0x58, 0xc1, 0xe8, 0x23, 0x9f, + 0x81, 0xd7, 0x8e, 0x8f, 0x72, 0x51, 0xd6, 0xc9, 0x0b, 0xb1, 0xf2, 0x83, 0x93, 0x56, 0xbe, 0x49, + 0x7b, 0x60, 0x1d, 0xb3, 0x1e, 0xde, 0xbb, 0x70, 0x0f, 0x1c, 0xa2, 0x6f, 0xe9, 0xd9, 0xbf, 0x96, + 0x40, 0x1e, 0x74, 0x16, 0x68, 0x0b, 0x64, 0xfc, 0xd8, 0xb1, 0x35, 0xd5, 0x47, 0xea, 0xd2, 0x79, + 0x48, 0x3d, 0xc5, 0x9a, 0x37, 0x3c, 0x66, 0xff, 0x04, 0x92, 0x36, 0x6e, 0xd1, 0xd0, 0xbb, 0x69, + 0x99, 0x7b, 0x46, 0x4b, 0xbc, 0xe9, 0xb7, 0xc7, 0x8e, 0x5c, 0x0a, 0x0a, 0x6b, 0x5e, 0x66, 0xad, + 0x95, 0x69, 0xdb, 0xf7, 0x94, 0xfd, 0x96, 0x04, 0xf3, 0xc3, 0xfd, 0xdd, 0x10, 0x5b, 0xdb, 0x0e, + 0xda, 0xda, 0x9d, 0x8b, 0xbb, 0x54, 0x9f, 0x85, 0xac, 0x86, 0x63, 0x92, 0x1c, 0xca, 0xbf, 0x49, + 0xcd, 0x84, 0xb5, 0xf1, 0x12, 0x29, 0x2f, 0x03, 0xec, 0x1b, 0xad, 0x7d, 0xf5, 0x91, 0xe6, 0x60, + 0x5b, 0x24, 0x64, 0xe3, 0x54, 0xf2, 0x01, 0x15, 0xe4, 0xff, 0x16, 0x20, 0x59, 0xeb, 0x74, 0x2d, + 0xdb, 0x71, 0xa3, 0xed, 0x75, 0x88, 0x70, 0xa2, 0x17, 0x0a, 0x2f, 0x8c, 0x18, 0x60, 0xa0, 0x25, + 0x8f, 0xcf, 0x04, 0xb3, 0x08, 0x0c, 0xb4, 0x05, 0x51, 0x1e, 0x14, 0x91, 0xcc, 0x65, 0x06, 0xb7, + 0x38, 0x36, 0x1c, 0x0f, 0xaf, 0xdc, 0x98, 0x48, 0xa0, 0x78, 0x71, 0x71, 0x68, 0x68, 0x5c, 0xfc, + 0x2e, 0x44, 0x78, 0x36, 0x5d, 0xe4, 0xdb, 0x72, 0x43, 0x12, 0x75, 0xb5, 0xad, 0x25, 0xa3, 0x8d, + 0x97, 0x58, 0x35, 0x77, 0xb4, 0xbc, 0x11, 0x7a, 0x15, 0x62, 0x84, 0x38, 0x2a, 0x31, 0xbe, 0xce, + 0x77, 0x1a, 0x93, 0x3c, 0x53, 0x5c, 0xaf, 0x37, 0xea, 0xc6, 0xd7, 0xb1, 0x12, 0x25, 0xc4, 0xa1, + 0x3f, 0xd0, 0x35, 0x60, 0x51, 0x17, 0xd1, 0x68, 0x2c, 0xc5, 0xc2, 0xa6, 0x49, 0xc5, 0x27, 0x61, + 0x38, 0x07, 0x46, 0x57, 0xdd, 0x3b, 0x20, 0x3c, 0x56, 0x11, 0x38, 0x07, 0x46, 0x77, 0x69, 0x8d, + 0x28, 0x51, 0x5a, 0xb8, 0x74, 0x40, 0x50, 0x16, 0x62, 0x8f, 0xb4, 0x76, 0x9b, 0xed, 0x6c, 0xa6, + 0x18, 0x8a, 0xf7, 0x1c, 0xa4, 0x9a, 0xc8, 0xaf, 0x96, 0x6a, 0xc4, 0x5e, 0xa2, 0xab, 0x39, 0xfb, + 0x6c, 0x77, 0x1c, 0x57, 0x80, 0x8b, 0xb6, 0x35, 0x67, 0x1f, 0x65, 0x20, 0xca, 0xe7, 0x45, 0x32, + 0xb1, 0x85, 0xc9, 0x9b, 0xd3, 0x8a, 0xfb, 0x88, 0x5e, 0x83, 0x19, 0x9e, 0x66, 0x53, 0x75, 0xc3, + 0xc6, 0x4d, 0xa7, 0x7d, 0xc8, 0xe2, 0xac, 0x98, 0x92, 0xe2, 0xe2, 0x8a, 0x90, 0xa2, 0x5b, 0x20, + 0x0f, 0x06, 0xa6, 0x2c, 0x3e, 0x8a, 0x29, 0x33, 0x03, 0x71, 0x29, 0x8d, 0x61, 0xc5, 0xbb, 0xf6, + 0x05, 0x7c, 0x19, 0x1e, 0xc3, 0x8a, 0x82, 0x7e, 0xb0, 0x77, 0x0b, 0x64, 0x11, 0x95, 0xf6, 0xeb, + 0x26, 0x39, 0x2e, 0x97, 0xf7, 0xab, 0x16, 0x60, 0xb6, 0xab, 0xd9, 0x04, 0xab, 0xbb, 0x3d, 0x53, + 0x6f, 0x63, 0x95, 0x63, 0x65, 0x52, 0xac, 0x76, 0x9a, 0x15, 0x95, 0x58, 0x09, 0xb7, 0xbb, 0xb3, + 0x92, 0x08, 0xf3, 0xbf, 0x89, 0x24, 0xc2, 0x4d, 0x90, 0x75, 0xbc, 0xa7, 0xf5, 0xda, 0x8e, 0x6a, + 0x98, 0xc2, 0x4e, 0xaf, 0xd0, 0xc0, 0x56, 0x49, 0x09, 0x79, 0xcd, 0x64, 0x16, 0x9a, 0xfd, 0x51, + 0x08, 0xa6, 0xd8, 0x7a, 0x44, 0x77, 0x20, 0x4c, 0xdf, 0xba, 0x48, 0x90, 0x8d, 0xbb, 0xb7, 0x62, + 0x6d, 0x10, 0x82, 0xb0, 0xa9, 0x75, 0x70, 0x06, 0x31, 0x9b, 0x60, 0xbf, 0xd1, 0x65, 0x88, 0x12, + 0xfc, 0x40, 0x7d, 0xa8, 0xb5, 0x33, 0xb3, 0xcc, 0x64, 0x23, 0x04, 0x3f, 0xb8, 0xaf, 0xb5, 0xd1, + 0x25, 0x88, 0x18, 0x44, 0x35, 0xf1, 0xa3, 0xcc, 0x1c, 0xe7, 0x2f, 0x83, 0x6c, 0xe2, 0x47, 0xe8, + 0x25, 0x88, 0x3f, 0xd2, 0x88, 0x8a, 0x3b, 0x5d, 0xe7, 0x90, 0x69, 0x2d, 0x46, 0x8d, 0x9c, 0x54, + 0xe9, 0x33, 0x0b, 0x73, 0x34, 0xbb, 0x85, 0x1d, 0xb5, 0x69, 0xb5, 0x49, 0xe6, 0x12, 0x5d, 0xd4, + 0x74, 0x5f, 0x47, 0x45, 0x65, 0xab, 0x4d, 0x56, 0xc3, 0xb1, 0x90, 0x3c, 0xb9, 0x1a, 0x8e, 0x4d, + 0xca, 0xe1, 0xd5, 0x70, 0x2c, 0x2c, 0x4f, 0xad, 0x86, 0x63, 0x53, 0x72, 0x64, 0x35, 0x1c, 0x8b, + 0xc8, 0xd1, 0xd5, 0x70, 0x2c, 0x2a, 0xc7, 0x56, 0xc3, 0xb1, 0x98, 0x1c, 0x5f, 0x0d, 0xc7, 0xe2, + 0x32, 0xac, 0x86, 0x63, 0x20, 0x27, 0x56, 0xc3, 0xb1, 0x84, 0x3c, 0xbd, 0x1a, 0x8e, 0x4d, 0xcb, + 0xc9, 0xd5, 0x70, 0x2c, 0x29, 0xa7, 0x56, 0xc3, 0xb1, 0x94, 0x3c, 0xb3, 0x1a, 0x8e, 0xcd, 0xc8, + 0xf2, 0x6a, 0x38, 0x26, 0xcb, 0xe9, 0xd5, 0x70, 0x2c, 0x2d, 0xa3, 0x6c, 0x55, 0x1c, 0x1c, 0x68, + 0xe8, 0x2b, 0x01, 0x3d, 0x8d, 0xbd, 0xe7, 0x63, 0x8d, 0xf2, 0x3f, 0x91, 0x40, 0xae, 0xe3, 0x07, + 0x3d, 0x6c, 0x36, 0xf1, 0x7d, 0xad, 0x5d, 0xde, 0xef, 0x99, 0x07, 0xe8, 0x55, 0x98, 0x69, 0xd2, + 0x1f, 0x2a, 0x4f, 0x53, 0x52, 0x8d, 0x49, 0x4c, 0x63, 0x49, 0x26, 0xae, 0x53, 0x29, 0x55, 0xdc, + 0xcb, 0x00, 0xa2, 0x1e, 0x7d, 0x9f, 0x21, 0x56, 0x25, 0xce, 0xab, 0x50, 0x67, 0x33, 0x00, 0x63, + 0x5b, 0x8f, 0x98, 0x73, 0x0b, 0xc0, 0x28, 0xd6, 0x23, 0xb4, 0x08, 0x73, 0x26, 0x7e, 0xec, 0xa8, + 0x83, 0x95, 0x99, 0x23, 0x53, 0xd2, 0xb4, 0xac, 0xec, 0x6f, 0x90, 0xff, 0xfb, 0x10, 0xcc, 0xb8, + 0x83, 0x76, 0xbd, 0xff, 0x1e, 0xc8, 0xf4, 0xed, 0x1a, 0xba, 0xea, 0x58, 0x1c, 0xc9, 0xe5, 0x81, + 0x77, 0x47, 0xe5, 0x97, 0x82, 0x28, 0xf4, 0xb9, 0xa6, 0x37, 0x2c, 0xd6, 0x1d, 0x27, 0x42, 0x25, + 0x49, 0xfc, 0xb2, 0xec, 0x0e, 0xa4, 0xdc, 0x46, 0x5c, 0x82, 0xca, 0x10, 0x09, 0xf4, 0xf7, 0xd9, + 0x31, 0xfa, 0x73, 0x55, 0xad, 0x88, 0xa6, 0xd9, 0x6f, 0x00, 0x3a, 0xd9, 0xb7, 0x9f, 0x84, 0xa7, + 0x38, 0x09, 0x6f, 0x05, 0x49, 0xf8, 0xcb, 0xe7, 0x9b, 0x9b, 0x6f, 0xd8, 0xfe, 0x64, 0xd6, 0xdf, + 0x84, 0x20, 0xc5, 0x29, 0xcc, 0x63, 0x5f, 0xea, 0xcc, 0xa8, 0xaf, 0x34, 0xcc, 0x56, 0xff, 0x68, + 0x85, 0xce, 0x2f, 0xa4, 0xc8, 0x6e, 0x81, 0x57, 0xf9, 0x15, 0x1a, 0xa3, 0x68, 0x7a, 0xf0, 0x0c, + 0x26, 0x44, 0x63, 0x0d, 0x4d, 0xf7, 0x2a, 0xdd, 0x80, 0x14, 0x8b, 0x33, 0xfb, 0xb5, 0x26, 0x59, + 0xad, 0x24, 0x93, 0x7a, 0xd5, 0x4a, 0x90, 0x24, 0x5d, 0xcd, 0x77, 0x9e, 0x13, 0x66, 0x4a, 0x3d, + 0xe3, 0xe0, 0x6a, 0x9a, 0xb6, 0xf1, 0x87, 0x0e, 0x36, 0x26, 0xbd, 0x0e, 0x56, 0xbb, 0x16, 0xcf, + 0xcd, 0x4c, 0x2a, 0x71, 0x2e, 0xd9, 0xb6, 0x08, 0xda, 0x61, 0xa6, 0xc2, 0x74, 0xa1, 0xea, 0x5c, + 0x39, 0x99, 0x08, 0xeb, 0xe5, 0xf5, 0xf1, 0xd5, 0xa9, 0xcc, 0x90, 0xa0, 0x20, 0xff, 0x67, 0x12, + 0x5c, 0xa6, 0x71, 0x1b, 0x5f, 0x69, 0x65, 0x76, 0x44, 0xed, 0x5a, 0xa7, 0x06, 0x51, 0x16, 0x10, + 0x7a, 0x81, 0xef, 0xca, 0xf1, 0x51, 0x2e, 0x42, 0x6b, 0x3f, 0x37, 0x1f, 0x46, 0x28, 0x70, 0x8d, + 0xe5, 0x1c, 0x1c, 0x5b, 0x33, 0x09, 0x3b, 0xb8, 0xa1, 0xaf, 0xad, 0x83, 0x3b, 0xbb, 0x74, 0x07, + 0x1c, 0x62, 0xd4, 0x37, 0x17, 0x28, 0xdc, 0xe0, 0x65, 0xf9, 0x2c, 0x64, 0x06, 0x87, 0xec, 0x25, + 0x34, 0xff, 0x1f, 0xcc, 0x6f, 0xe2, 0x47, 0xc3, 0x66, 0x53, 0x82, 0x28, 0xf7, 0x74, 0xae, 0xc9, + 0xdf, 0x1c, 0x74, 0x3a, 0xfe, 0x53, 0xfa, 0x02, 0x1b, 0x69, 0x83, 0x35, 0x50, 0xdc, 0x86, 0xf9, + 0x4f, 0xe0, 0xf2, 0x00, 0xba, 0xf7, 0xfa, 0xde, 0x87, 0x08, 0x71, 0x34, 0x47, 0x04, 0x72, 0xa9, + 0x71, 0xd0, 0xeb, 0x8e, 0xe6, 0xf4, 0x88, 0x22, 0xda, 0xe5, 0x6f, 0xc0, 0x2b, 0xc5, 0x9e, 0x63, + 0x51, 0x03, 0x11, 0x71, 0x2f, 0x6e, 0x5a, 0x66, 0xd3, 0x68, 0x1b, 0x9a, 0xef, 0x08, 0x35, 0xff, + 0x2a, 0x5c, 0x1f, 0x55, 0xcd, 0xd3, 0x84, 0xc2, 0x32, 0xbb, 0xbd, 0x0e, 0xa6, 0x35, 0xd7, 0x0d, + 0xe2, 0xa0, 0xf7, 0x61, 0x5a, 0x58, 0x18, 0x35, 0x3c, 0x57, 0x0d, 0x67, 0x18, 0x69, 0xc2, 0xf6, + 0x40, 0x48, 0xfe, 0xcf, 0x25, 0x98, 0xad, 0xd8, 0x56, 0xb7, 0x8b, 0x75, 0x41, 0x61, 0x5c, 0xb7, + 0x2e, 0x73, 0x49, 0x3e, 0xe6, 0xda, 0x84, 0x50, 0xad, 0x22, 0xf6, 0x33, 0x77, 0x9f, 0x77, 0x9b, + 0x54, 0xab, 0xa0, 0x2f, 0x73, 0x05, 0xf7, 0x08, 0xf3, 0xc7, 0xa9, 0x13, 0x09, 0xbe, 0xe0, 0x19, + 0x61, 0x5f, 0xb3, 0x3d, 0x92, 0xff, 0x61, 0x14, 0x2e, 0xf9, 0x5f, 0xda, 0x72, 0xd9, 0x1d, 0xf8, + 0xa7, 0x10, 0x75, 0x93, 0x3c, 0x63, 0xf8, 0xdd, 0x61, 0x10, 0x05, 0xa1, 0x0f, 0x7f, 0xa2, 0xc7, + 0xc5, 0x44, 0x75, 0x48, 0x1b, 0xa6, 0x83, 0xed, 0x36, 0xd6, 0x1e, 0xd2, 0xb0, 0x86, 0xea, 0x4c, + 0x24, 0xd5, 0xc7, 0x0d, 0x0d, 0x64, 0x1f, 0x00, 0x0f, 0x31, 0x3e, 0x85, 0x59, 0x3f, 0xa8, 0x3b, + 0xfe, 0xd1, 0xd9, 0x5c, 0x36, 0xbc, 0x3e, 0xac, 0x9b, 0x76, 0xf6, 0x01, 0xb9, 0x29, 0xa9, 0x0f, + 0xbd, 0x1d, 0x09, 0xcf, 0xd8, 0xdf, 0xb9, 0xb0, 0x46, 0x2a, 0x03, 0xbb, 0x93, 0x40, 0x8c, 0xcd, + 0x48, 0xf5, 0x57, 0x14, 0x63, 0xdf, 0x87, 0x08, 0x4f, 0xea, 0x8a, 0xc3, 0xb3, 0xbb, 0x17, 0x9d, + 0x02, 0x4f, 0x16, 0x2b, 0x02, 0x2d, 0xfb, 0x7d, 0x09, 0xa6, 0xfd, 0xaf, 0x1b, 0x19, 0x10, 0x63, + 0xea, 0x77, 0x5d, 0xe4, 0xe4, 0x0b, 0xdf, 0xb9, 0x73, 0x53, 0xaa, 0xe9, 0x34, 0xb0, 0xd3, 0x6d, + 0xab, 0xdb, 0x3f, 0x3c, 0x9d, 0x54, 0x62, 0x54, 0x40, 0x83, 0xd6, 0xec, 0x37, 0x21, 0xee, 0x29, + 0xdd, 0x97, 0x8b, 0x9b, 0x7c, 0x81, 0xb9, 0xb8, 0x91, 0xfd, 0x57, 0x20, 0x19, 0xd0, 0x18, 0x9a, + 0xf7, 0xc6, 0x10, 0x2e, 0x45, 0xf8, 0x18, 0xce, 0x44, 0xc9, 0xff, 0x38, 0x0a, 0xb3, 0xc3, 0x3c, + 0xf7, 0x47, 0x20, 0xfb, 0xfc, 0x96, 0xda, 0x36, 0x88, 0x23, 0x6c, 0xf3, 0xd6, 0xe8, 0xed, 0xbc, + 0xcf, 0xf9, 0x09, 0x53, 0x4c, 0xd9, 0x41, 0x97, 0xf8, 0x09, 0xa4, 0x74, 0x3e, 0x70, 0x91, 0x6f, + 0x17, 0x37, 0x75, 0x46, 0x6d, 0xc3, 0x87, 0x38, 0x40, 0x81, 0x9e, 0xd4, 0x7d, 0x45, 0x04, 0x35, + 0x21, 0xe9, 0x81, 0x1f, 0x76, 0xc5, 0x7e, 0xee, 0xf9, 0x9d, 0xe1, 0xb4, 0xdb, 0x0b, 0xc5, 0x44, + 0x2d, 0x98, 0x71, 0x3b, 0x71, 0xb7, 0xfe, 0xf1, 0x17, 0xd2, 0x8d, 0xab, 0x98, 0xba, 0x48, 0x05, + 0x7c, 0x47, 0x82, 0x59, 0xb7, 0x27, 0x2f, 0x65, 0x6d, 0xe8, 0xcc, 0x9d, 0x25, 0x4b, 0xf5, 0xe3, + 0xa3, 0x5c, 0x5a, 0x68, 0xc6, 0xcd, 0x9c, 0x3c, 0xb7, 0xdd, 0xa5, 0xf5, 0x01, 0x40, 0x9d, 0xc6, + 0x24, 0xb4, 0x9c, 0x76, 0x3c, 0xd5, 0x8f, 0x49, 0xa8, 0x63, 0x7b, 0xfe, 0x98, 0x84, 0xfe, 0xac, + 0xe9, 0xe8, 0xbb, 0x12, 0xa4, 0xf9, 0x31, 0x59, 0xa7, 0xe7, 0x68, 0xfc, 0xa0, 0xdc, 0xcd, 0x09, + 0x7c, 0x74, 0x7c, 0x94, 0x9b, 0x61, 0xaf, 0x77, 0x43, 0x94, 0xb1, 0x6e, 0x4b, 0x17, 0xed, 0xb6, + 0x8f, 0x22, 0xb6, 0xd0, 0x9e, 0x40, 0x47, 0x6b, 0x90, 0xe2, 0x89, 0x12, 0xf7, 0x26, 0x20, 0x4b, + 0x16, 0x24, 0x4b, 0xd7, 0x9f, 0x1d, 0xe5, 0x16, 0x86, 0xac, 0x13, 0x9e, 0x63, 0xb9, 0xcf, 0xeb, + 0x2a, 0xc9, 0x3d, 0xff, 0x23, 0x5a, 0x87, 0x19, 0x1e, 0xc8, 0xf6, 0xaf, 0xf7, 0xc0, 0xf8, 0x87, + 0xbb, 0x3c, 0x08, 0xf6, 0xa4, 0x22, 0xff, 0x35, 0x0f, 0x73, 0x43, 0x63, 0xb0, 0x5f, 0x44, 0x60, + 0x3e, 0xe8, 0x56, 0xbd, 0x28, 0x49, 0x1d, 0xe4, 0xdb, 0xf7, 0xc6, 0x76, 0xcd, 0xde, 0x45, 0x1f, + 0xe6, 0x1a, 0xdd, 0xa7, 0x41, 0xc6, 0xfd, 0x74, 0x80, 0xbd, 0x2e, 0x80, 0xcf, 0x5e, 0xef, 0x00, + 0xbe, 0x4b, 0x61, 0x1f, 0x7a, 0xcc, 0xc2, 0x33, 0x5e, 0xef, 0x5f, 0x00, 0x9e, 0xb5, 0xf7, 0x2e, + 0x29, 0xb9, 0xdc, 0xf2, 0x77, 0x12, 0x24, 0x03, 0x33, 0xfb, 0x75, 0x92, 0xcb, 0xb6, 0x17, 0x5b, + 0xf1, 0xcb, 0x46, 0xef, 0x9c, 0x7f, 0x5a, 0xc1, 0x90, 0x2b, 0xfb, 0x17, 0x12, 0x24, 0x03, 0x8a, + 0xfc, 0x15, 0xd1, 0xd2, 0x8b, 0x1f, 0xf9, 0x2e, 0xa4, 0x82, 0xaf, 0xc8, 0xd7, 0x87, 0xf4, 0x62, + 0xfa, 0xc8, 0x7f, 0x09, 0x22, 0x5c, 0x82, 0x10, 0xa4, 0x3e, 0x28, 0xd6, 0x1a, 0xb5, 0xcd, 0x65, + 0x75, 0x69, 0x4b, 0x51, 0x97, 0xcb, 0xf2, 0x04, 0x9a, 0x86, 0x58, 0xa5, 0xba, 0x5e, 0xa5, 0x42, + 0x59, 0x42, 0x09, 0x88, 0xb2, 0xa7, 0x6a, 0x45, 0x0e, 0xe5, 0x4b, 0x20, 0x73, 0xec, 0x3d, 0x4c, + 0x69, 0x86, 0xee, 0x4a, 0x50, 0x01, 0x66, 0xd9, 0x0e, 0xa2, 0x43, 0x23, 0x2b, 0xba, 0xbc, 0x55, + 0x5f, 0x2c, 0x9e, 0xf6, 0x8a, 0xe8, 0xea, 0xdd, 0xd4, 0x3a, 0x38, 0xff, 0xd3, 0x30, 0xa4, 0xfb, + 0x20, 0x2e, 0xc9, 0xfe, 0x89, 0xd4, 0xdf, 0x1f, 0x45, 0xce, 0x3c, 0xef, 0x3c, 0xd1, 0x5e, 0x6c, + 0x95, 0xc4, 0xb9, 0xe3, 0x07, 0x74, 0xd1, 0x3c, 0x3b, 0xca, 0xa5, 0x07, 0x07, 0x4b, 0x9e, 0xf3, + 0x40, 0xd2, 0x1d, 0x22, 0xcb, 0xf9, 0x1a, 0xe6, 0x81, 0xda, 0xbf, 0x0f, 0xc6, 0x73, 0xbe, 0x86, + 0x79, 0xb0, 0xa3, 0xd4, 0x94, 0x28, 0x2d, 0xdc, 0xb1, 0x0d, 0xb4, 0x0a, 0x61, 0xab, 0xeb, 0xb8, + 0x1b, 0xf2, 0xb7, 0xcf, 0x35, 0xa5, 0xad, 0xae, 0x98, 0x8f, 0xc2, 0x30, 0xd0, 0x2a, 0x3f, 0xc2, + 0xef, 0x2b, 0x9a, 0x39, 0xe4, 0x31, 0x5d, 0x68, 0x32, 0xf0, 0x22, 0xb2, 0x2d, 0x98, 0xf6, 0x6b, + 0x6c, 0xc8, 0xc9, 0x45, 0x31, 0x98, 0x34, 0xf9, 0xec, 0x58, 0x43, 0x17, 0x1b, 0x56, 0xdf, 0x61, + 0xd6, 0x97, 0x20, 0xee, 0xcd, 0xe3, 0x3c, 0x67, 0x7a, 0xdc, 0xc7, 0x7b, 0x19, 0xc1, 0x29, 0x39, + 0x92, 0xff, 0x61, 0x08, 0xa6, 0x15, 0x4c, 0xac, 0xf6, 0x43, 0xac, 0xd3, 0x08, 0xca, 0xbb, 0xa6, + 0x2b, 0x8d, 0x7f, 0x4d, 0xb7, 0x08, 0xf1, 0x3e, 0x03, 0x9d, 0xe3, 0x0e, 0x60, 0xbf, 0x15, 0xfa, + 0x08, 0x92, 0xbb, 0x56, 0xcf, 0xd4, 0x35, 0xfb, 0x90, 0xc5, 0x55, 0x2c, 0x02, 0x49, 0x8d, 0xbc, + 0x91, 0xe5, 0x1f, 0x75, 0xa1, 0x24, 0x1a, 0xd3, 0xf8, 0x49, 0x99, 0xde, 0xf5, 0x3d, 0xe5, 0xdf, + 0x85, 0x69, 0x7f, 0x29, 0x8a, 0x41, 0x78, 0x73, 0x6b, 0xb3, 0xca, 0xd7, 0x64, 0xa9, 0x58, 0x5e, + 0x5b, 0xaa, 0xad, 0xaf, 0xcb, 0x12, 0x95, 0x57, 0x3f, 0xac, 0x35, 0xe4, 0x10, 0xbf, 0x35, 0x59, + 0x6f, 0x14, 0x95, 0x86, 0x9b, 0x38, 0xcd, 0x63, 0x48, 0xfa, 0xfb, 0xa3, 0x9e, 0x8f, 0x86, 0x9d, + 0x4c, 0x10, 0xd8, 0x79, 0xbf, 0x36, 0xe6, 0x88, 0x5d, 0x0b, 0xb2, 0xfd, 0xa8, 0xf9, 0x7f, 0x0c, + 0x01, 0xea, 0xbf, 0x78, 0xcf, 0x59, 0x7d, 0x08, 0xd0, 0xdc, 0xc7, 0xcd, 0x83, 0xae, 0x65, 0x98, + 0x8e, 0xd8, 0x6b, 0xbe, 0x33, 0x96, 0xed, 0x78, 0xce, 0xaa, 0xec, 0xb5, 0x57, 0x7c, 0x58, 0xe8, + 0xf7, 0x47, 0xa7, 0xe8, 0x27, 0x59, 0x8a, 0x9e, 0xad, 0xfc, 0x5f, 0x6b, 0x9a, 0x3e, 0x5b, 0x04, + 0xe8, 0x8f, 0x18, 0xbd, 0x05, 0x53, 0xe7, 0xc8, 0x6e, 0xf0, 0xba, 0x7e, 0x5b, 0xcf, 0xff, 0x4f, + 0x18, 0x50, 0xd9, 0xc6, 0x9a, 0x83, 0xa9, 0x8b, 0x26, 0xa3, 0x52, 0x1c, 0x25, 0x98, 0xe2, 0x5b, + 0xfa, 0xd0, 0x79, 0xb6, 0xf4, 0x6e, 0xd7, 0xac, 0x29, 0xfa, 0x1a, 0x4c, 0x37, 0xad, 0x76, 0xaf, + 0x63, 0xaa, 0xec, 0x22, 0x90, 0xd8, 0x7f, 0x7c, 0x71, 0xd4, 0x1b, 0x3b, 0x31, 0xb8, 0x42, 0xd9, + 0x6a, 0xd3, 0x67, 0xef, 0xbe, 0x39, 0x03, 0x64, 0x35, 0xd0, 0x55, 0x88, 0x7b, 0x9e, 0x87, 0x99, + 0x43, 0x5c, 0xe9, 0x0b, 0xd0, 0x6d, 0x98, 0xd2, 0x88, 0x6a, 0xed, 0xb1, 0x60, 0xfa, 0xac, 0xa5, + 0xa8, 0x84, 0x35, 0xb2, 0xb5, 0x87, 0x5e, 0x87, 0x74, 0x47, 0x7b, 0xac, 0xee, 0xd9, 0xfc, 0x46, + 0xb1, 0x6a, 0xe8, 0x6d, 0xee, 0x09, 0x25, 0x65, 0xa6, 0xa3, 0x3d, 0x5e, 0x12, 0xf2, 0x9a, 0xde, + 0xc6, 0xe8, 0x2d, 0x48, 0xee, 0x3d, 0xe0, 0x5b, 0x2b, 0xce, 0x4a, 0xfc, 0x56, 0xd5, 0xcc, 0xf1, + 0x51, 0x2e, 0xb1, 0x74, 0x8f, 0x29, 0x86, 0x72, 0x92, 0x92, 0xd8, 0x7b, 0xe0, 0x3d, 0x64, 0xff, + 0x5b, 0x82, 0xa8, 0x98, 0x11, 0xea, 0x02, 0x08, 0xf5, 0x18, 0x3a, 0x7f, 0xa7, 0xc9, 0xd2, 0xbd, + 0xe3, 0xa3, 0x5c, 0xbc, 0xcc, 0xa4, 0xb5, 0x0a, 0x79, 0x76, 0x94, 0x7b, 0xff, 0xa2, 0x8c, 0xe2, + 0x82, 0x28, 0x71, 0xde, 0x49, 0x4d, 0x67, 0x79, 0xe1, 0x7d, 0x8d, 0xa8, 0xfb, 0x06, 0x71, 0xac, + 0x96, 0xad, 0x75, 0xc4, 0xfd, 0x80, 0xe9, 0x7d, 0x8d, 0xac, 0xb8, 0x32, 0x94, 0xa5, 0xb1, 0xd9, + 0x43, 0x7e, 0x8f, 0x8b, 0xdf, 0x16, 0xf1, 0x9e, 0xd1, 0x6d, 0xb8, 0xe4, 0x35, 0x56, 0xa9, 0xa6, + 0x76, 0x7b, 0xcd, 0x03, 0xcc, 0x38, 0x88, 0x3a, 0xf7, 0x59, 0xaf, 0x70, 0x43, 0x7b, 0x5c, 0xe2, + 0x45, 0xf9, 0x4b, 0x30, 0xeb, 0x7b, 0xad, 0x5e, 0x24, 0x8d, 0x41, 0xe6, 0xd7, 0x18, 0x7c, 0x9f, + 0x50, 0xdc, 0x83, 0x99, 0x81, 0x2f, 0x84, 0x84, 0xff, 0xf5, 0x67, 0x1c, 0x83, 0x9f, 0x14, 0x15, + 0xca, 0xfc, 0xd1, 0xdd, 0x1b, 0xa4, 0x9a, 0x81, 0xe7, 0xfc, 0x2c, 0xa4, 0xbd, 0x6e, 0xbc, 0xbe, + 0x5f, 0x82, 0x2b, 0xf5, 0x7b, 0xeb, 0x6c, 0x3c, 0x65, 0xab, 0xd3, 0xe5, 0x2f, 0xd5, 0x4d, 0x42, + 0x5e, 0x85, 0xec, 0xc9, 0x42, 0xaf, 0xe9, 0xbf, 0xce, 0x40, 0x74, 0x5b, 0x3b, 0x6c, 0x5b, 0x9a, + 0x8e, 0x16, 0x20, 0xe1, 0xde, 0xdd, 0x72, 0x87, 0x1a, 0x57, 0xfc, 0xa2, 0xa0, 0x85, 0xca, 0xec, + 0x4c, 0xca, 0x67, 0xa1, 0x06, 0xa4, 0x7a, 0x04, 0xdb, 0xd4, 0x78, 0x54, 0xf6, 0x25, 0x14, 0x67, + 0xaa, 0x52, 0xe9, 0xd9, 0x51, 0xee, 0xee, 0x78, 0xef, 0x1d, 0x37, 0x7b, 0xb6, 0xe1, 0x1c, 0x16, + 0xea, 0xf7, 0xd6, 0x77, 0x04, 0x14, 0x75, 0x33, 0x96, 0x92, 0xec, 0xf9, 0x1f, 0xc5, 0x15, 0x3d, + 0xfa, 0x0e, 0xd5, 0x8e, 0xd1, 0xb4, 0x2d, 0xe2, 0x9e, 0xfc, 0x08, 0xe9, 0x06, 0x13, 0xa2, 0xd7, + 0x60, 0x66, 0xcf, 0x30, 0xd9, 0x31, 0xa7, 0x5b, 0x8f, 0x1f, 0xfa, 0xa4, 0x5c, 0xb1, 0xa8, 0xf8, + 0x10, 0x52, 0xbe, 0xbb, 0x71, 0xd4, 0x7e, 0x23, 0xcc, 0x7e, 0xb7, 0x8e, 0x8f, 0x72, 0xc9, 0xbe, + 0x3f, 0xe0, 0x36, 0xfc, 0x3c, 0x51, 0x51, 0xb2, 0xdf, 0x0d, 0xb5, 0xe0, 0x39, 0x98, 0x62, 0xdf, + 0xc9, 0xf1, 0x3b, 0xcf, 0x0a, 0x7f, 0x40, 0x55, 0x48, 0x8a, 0x2c, 0x0a, 0xff, 0x88, 0x4e, 0x5c, + 0x28, 0x5c, 0xf0, 0x59, 0x8d, 0xfb, 0x99, 0x5d, 0xa1, 0x6a, 0x36, 0x2d, 0x1d, 0xeb, 0x55, 0xfa, + 0xac, 0x88, 0xa4, 0x31, 0x7b, 0x20, 0x68, 0x19, 0x52, 0xcd, 0x36, 0xd6, 0xcc, 0x5e, 0xd7, 0xc5, + 0x41, 0x63, 0xe2, 0x24, 0x45, 0x3b, 0x01, 0xb4, 0x09, 0x68, 0x8f, 0xdd, 0x97, 0xf2, 0x8f, 0x8a, + 0x1d, 0x72, 0x8e, 0x03, 0x26, 0xb3, 0xb6, 0x4a, 0x7f, 0x64, 0xe8, 0x3a, 0x24, 0x4d, 0xcb, 0x6c, + 0x6a, 0x66, 0x13, 0xb7, 0x99, 0x53, 0xe6, 0xe7, 0xa2, 0x41, 0x21, 0x2a, 0x41, 0x84, 0x9f, 0xb5, + 0x8b, 0xed, 0xef, 0xcd, 0x71, 0xbf, 0x0c, 0x58, 0x99, 0x50, 0x44, 0x4b, 0x54, 0x85, 0xa8, 0xcd, + 0xef, 0x7d, 0xb0, 0xf3, 0xf7, 0x33, 0xd3, 0x50, 0xbe, 0x5b, 0x25, 0x2b, 0x13, 0x8a, 0xdb, 0x16, + 0x35, 0xdc, 0xeb, 0xaf, 0x9c, 0x82, 0xc5, 0x0d, 0xc6, 0xc2, 0x98, 0x9b, 0x8b, 0x3e, 0x60, 0x00, + 0x85, 0x4e, 0xd0, 0x60, 0xa7, 0x62, 0xec, 0x64, 0x7e, 0xf4, 0x04, 0x03, 0x37, 0x40, 0xe8, 0x04, + 0x79, 0x4b, 0xb4, 0x49, 0x63, 0x08, 0x37, 0x2c, 0x60, 0x67, 0xf6, 0x89, 0xdb, 0x9f, 0x3b, 0x4f, + 0xe8, 0xbc, 0x32, 0xa1, 0xf8, 0x10, 0xd0, 0x3d, 0x48, 0x34, 0xfb, 0xde, 0x2d, 0x33, 0xc3, 0x00, + 0xdf, 0x38, 0x17, 0xc5, 0xad, 0x50, 0x5a, 0xeb, 0x4b, 0xd1, 0xc7, 0x90, 0x22, 0x81, 0xad, 0x56, + 0xe6, 0x12, 0x43, 0x7d, 0xf3, 0xbc, 0xa9, 0xde, 0x95, 0x09, 0x65, 0x00, 0x09, 0xfd, 0x7f, 0x90, + 0x9d, 0x81, 0xf3, 0x25, 0x76, 0x94, 0x3e, 0xfa, 0x7a, 0xe9, 0x29, 0xa7, 0x68, 0x2b, 0x13, 0xca, + 0x09, 0x34, 0xf4, 0x29, 0xcc, 0x90, 0xe0, 0x67, 0x4c, 0x99, 0xcb, 0xac, 0x83, 0xcf, 0x8f, 0xff, + 0xe1, 0x53, 0x1f, 0x7f, 0x10, 0x8b, 0xc2, 0x9b, 0xc1, 0x63, 0x2a, 0x76, 0xa5, 0x63, 0x34, 0xfc, + 0xf0, 0x63, 0x33, 0x0a, 0x3f, 0x80, 0x85, 0xd6, 0x20, 0xde, 0x71, 0xe9, 0x82, 0x5d, 0x88, 0x18, + 0xbd, 0x3b, 0x19, 0x64, 0xb0, 0x95, 0x09, 0xa5, 0xdf, 0x1e, 0xfd, 0xb6, 0x04, 0x57, 0xb5, 0x11, + 0xe7, 0x59, 0x99, 0x97, 0xce, 0x4c, 0xe1, 0x8f, 0x71, 0x6a, 0xb6, 0x32, 0xa1, 0x8c, 0xec, 0x05, + 0xed, 0x01, 0x22, 0x0f, 0xda, 0x03, 0x84, 0x96, 0xb9, 0x76, 0xf6, 0xb7, 0x1e, 0xa7, 0x51, 0xe4, + 0xca, 0x84, 0x32, 0x04, 0x11, 0x7d, 0x06, 0xa6, 0xbb, 0x5a, 0x8f, 0x60, 0xd5, 0xc6, 0x1a, 0xb1, + 0xcc, 0xcc, 0x55, 0xce, 0x87, 0x4c, 0xa6, 0x30, 0x11, 0xfa, 0x06, 0xe4, 0x6c, 0xec, 0xd8, 0x06, + 0x0b, 0x9a, 0xf0, 0x63, 0xdc, 0xec, 0xb1, 0x30, 0x6b, 0x4f, 0x33, 0xda, 0x3d, 0x1b, 0xab, 0x6d, + 0xab, 0x95, 0x59, 0x60, 0x2e, 0x77, 0xf4, 0x8e, 0x47, 0x20, 0x54, 0x5d, 0x80, 0x25, 0xde, 0x5e, + 0xb9, 0x6a, 0x9f, 0x56, 0xb4, 0x6e, 0xb5, 0x4a, 0x71, 0x88, 0x8a, 0xd3, 0x65, 0xef, 0xea, 0x07, + 0xbf, 0xf4, 0xc1, 0xaf, 0x7b, 0x64, 0xe5, 0x97, 0xf2, 0xdf, 0x4b, 0x40, 0xcc, 0xdb, 0x6e, 0x2c, + 0x02, 0xf2, 0x02, 0xc1, 0xfe, 0x25, 0x78, 0xca, 0xf2, 0xa1, 0x95, 0x09, 0x25, 0xed, 0x96, 0xf5, + 0xef, 0xc1, 0xdf, 0x0d, 0xdc, 0x90, 0x1b, 0xe7, 0x3b, 0x4d, 0x6a, 0x2f, 0xde, 0x15, 0x3a, 0xca, + 0xbe, 0xe2, 0x9e, 0xb4, 0xc7, 0xbe, 0xfc, 0x1c, 0x21, 0xe5, 0x8a, 0x05, 0xfb, 0xde, 0x80, 0x94, + 0xdd, 0x33, 0xd9, 0xa1, 0xb2, 0xc8, 0xde, 0xf0, 0xe8, 0x37, 0x29, 0xa4, 0x22, 0x01, 0x53, 0x1e, + 0x20, 0x84, 0x5b, 0x67, 0x12, 0x82, 0x3b, 0xf7, 0x15, 0xc9, 0x63, 0x84, 0xa5, 0x41, 0x46, 0x78, + 0xfd, 0x6c, 0x46, 0xf0, 0xc1, 0x78, 0x94, 0xb0, 0x33, 0x94, 0x12, 0x16, 0xc7, 0xf4, 0x69, 0x3e, + 0xc4, 0x20, 0x27, 0x94, 0x07, 0x38, 0xe1, 0xd6, 0x99, 0x9c, 0xe0, 0x9f, 0xa3, 0x20, 0x85, 0xad, + 0x21, 0xa4, 0xf0, 0xc6, 0xb9, 0x36, 0x96, 0x2b, 0x52, 0x80, 0x15, 0x94, 0x61, 0xac, 0x50, 0x18, + 0x8f, 0x15, 0x7c, 0x90, 0x01, 0x5a, 0xf8, 0xe4, 0x04, 0x2d, 0xc8, 0x67, 0xfb, 0xd5, 0xa1, 0x29, + 0xbb, 0x15, 0xe9, 0x04, 0x2f, 0x68, 0x43, 0x78, 0x21, 0xcd, 0xe0, 0xdf, 0x3a, 0x07, 0x2f, 0xf8, + 0x3a, 0x38, 0x49, 0x0c, 0x1f, 0xc2, 0xb4, 0xdf, 0x99, 0xb3, 0xab, 0x60, 0xa3, 0x69, 0xe7, 0x94, + 0xcf, 0x61, 0x99, 0x0d, 0xf8, 0x8a, 0xd0, 0xd7, 0x4e, 0x72, 0xc2, 0xec, 0x99, 0xe0, 0xa7, 0x5c, + 0x76, 0x58, 0x91, 0x4e, 0x92, 0xc2, 0xba, 0x9f, 0x14, 0xe6, 0xce, 0x0c, 0x19, 0x4e, 0xec, 0x37, + 0x56, 0x24, 0x3f, 0x2b, 0x7c, 0x57, 0x82, 0xab, 0xa3, 0xdc, 0xba, 0xe0, 0xe3, 0xf7, 0x2e, 0xc8, + 0x0a, 0xbe, 0x4e, 0x47, 0x76, 0x83, 0x5a, 0x43, 0x69, 0x81, 0x73, 0xf5, 0x17, 0xcf, 0x45, 0x0b, + 0xbe, 0x2e, 0x87, 0xf1, 0xc2, 0xab, 0x10, 0x73, 0x6c, 0xad, 0xc9, 0x4e, 0xd4, 0x2e, 0xb1, 0x63, + 0x53, 0x96, 0xcf, 0x6c, 0x50, 0x19, 0xcb, 0x7b, 0xb2, 0x1f, 0x7a, 0x09, 0x20, 0xe6, 0x5e, 0x32, + 0xf2, 0xf9, 0xea, 0xfc, 0x0f, 0x24, 0x98, 0x5c, 0xb5, 0x76, 0xd1, 0xcb, 0xbe, 0x24, 0x7b, 0x52, + 0xa4, 0x62, 0xa6, 0x56, 0xad, 0x5d, 0x91, 0x2d, 0x7f, 0xaf, 0xdf, 0x5a, 0xe4, 0x2c, 0x5e, 0x19, + 0x31, 0x09, 0xef, 0x8c, 0xc2, 0x6b, 0x84, 0xbe, 0x0a, 0xd1, 0x2e, 0xdf, 0xd8, 0x09, 0xd7, 0x9d, + 0x1f, 0xd5, 0x9e, 0xd7, 0x54, 0xdc, 0x26, 0xf9, 0xff, 0x0a, 0xc1, 0x95, 0x53, 0x89, 0x09, 0xcd, + 0x07, 0xd2, 0xec, 0x71, 0x37, 0x59, 0x8e, 0xbe, 0x00, 0xf3, 0x7d, 0x16, 0xe4, 0x17, 0xed, 0x02, + 0x8e, 0x7f, 0xce, 0x2b, 0x65, 0x77, 0xed, 0x84, 0xfb, 0x7f, 0x13, 0xfa, 0x72, 0x15, 0x9b, 0x03, + 0x5b, 0x3a, 0xe4, 0x95, 0x55, 0x4d, 0x97, 0x30, 0x4c, 0x48, 0x18, 0x26, 0x71, 0xe8, 0x5e, 0xc1, + 0x3d, 0xd7, 0x9c, 0x2a, 0x6d, 0x08, 0x25, 0x7e, 0x69, 0xac, 0xed, 0x19, 0xbf, 0xb4, 0x78, 0x6f, + 0xbd, 0x26, 0x70, 0x58, 0x4e, 0x0b, 0xfa, 0x4f, 0x0a, 0xb8, 0x3d, 0xd4, 0x74, 0xf4, 0xb6, 0xbb, + 0x4d, 0x9b, 0x1a, 0x73, 0xcf, 0x23, 0x36, 0x72, 0xaf, 0xc1, 0x8c, 0x63, 0xf7, 0x4c, 0xfe, 0x8d, + 0x31, 0x47, 0x60, 0x59, 0x15, 0x25, 0xe5, 0x89, 0x59, 0xfd, 0xd7, 0x6f, 0xf9, 0xff, 0x27, 0x8c, + 0x0d, 0x4b, 0xc7, 0x28, 0x05, 0xb0, 0xad, 0x11, 0xd2, 0xdd, 0xb7, 0x35, 0x82, 0xe5, 0x09, 0x14, + 0x85, 0xc9, 0xb5, 0x8d, 0xba, 0x2c, 0xbd, 0xfe, 0xa1, 0xff, 0x40, 0xa2, 0xa2, 0x14, 0x6b, 0x9b, + 0xb5, 0xcd, 0x65, 0x75, 0xb3, 0xb8, 0x51, 0xad, 0xcb, 0x13, 0x28, 0x03, 0x73, 0x1f, 0x14, 0x6b, + 0x0d, 0x71, 0x42, 0xa1, 0xd6, 0x36, 0x1b, 0x55, 0xe5, 0x7e, 0x71, 0x5d, 0x96, 0xd0, 0x3c, 0x20, + 0x65, 0xab, 0xbc, 0x56, 0xaf, 0x94, 0xd4, 0xf2, 0xd6, 0xc6, 0x76, 0xb1, 0xdc, 0xa8, 0x6d, 0x6d, + 0xca, 0x21, 0x14, 0x83, 0x70, 0x65, 0x6b, 0xb3, 0x2a, 0xc3, 0xeb, 0xdf, 0x9f, 0x82, 0x30, 0xcb, + 0xa5, 0x5e, 0x87, 0xc4, 0xce, 0x66, 0x7d, 0xbb, 0x5a, 0xae, 0x2d, 0xd5, 0xaa, 0x15, 0x79, 0x22, + 0x3b, 0xfb, 0xe4, 0xe9, 0xc2, 0x0c, 0x2d, 0xda, 0x31, 0x49, 0x17, 0x37, 0x19, 0x77, 0xa3, 0x2c, + 0x44, 0x4a, 0xc5, 0xf2, 0xda, 0xce, 0xb6, 0x2c, 0x65, 0x53, 0x4f, 0x9e, 0x2e, 0x00, 0xad, 0xc0, + 0x79, 0x17, 0x5d, 0xe5, 0xb9, 0xd6, 0x2d, 0xa5, 0x2a, 0x87, 0xb2, 0x33, 0x4f, 0x9e, 0x2e, 0x24, + 0x58, 0x0a, 0x57, 0x70, 0xe7, 0x6b, 0x90, 0xac, 0x97, 0x57, 0xaa, 0x1b, 0x45, 0xb5, 0xbc, 0x52, + 0xdc, 0x5c, 0xae, 0xca, 0x93, 0xd9, 0xb9, 0x27, 0x4f, 0x17, 0xe4, 0x41, 0xff, 0x4b, 0xbb, 0xa8, + 0x6d, 0x6c, 0x6f, 0x29, 0x0d, 0x39, 0xdc, 0xef, 0x82, 0xd3, 0x1e, 0xca, 0x03, 0xf0, 0xd6, 0x4b, + 0xd5, 0x6a, 0x45, 0x9e, 0xca, 0xa2, 0x27, 0x4f, 0x17, 0x52, 0xb4, 0xbc, 0xcf, 0x66, 0xe8, 0x06, + 0x4c, 0x97, 0x95, 0x6a, 0xb1, 0x51, 0x55, 0xeb, 0x8d, 0x62, 0xa3, 0x2e, 0x47, 0xfa, 0x33, 0xf1, + 0x31, 0x14, 0x2a, 0x40, 0xba, 0xb8, 0xd3, 0xd8, 0x52, 0x03, 0x75, 0xa3, 0xd9, 0xcb, 0x4f, 0x9e, + 0x2e, 0xcc, 0xd2, 0xba, 0xd4, 0xef, 0xf8, 0xeb, 0x7f, 0x0e, 0xe4, 0xc0, 0xf8, 0xd5, 0xe5, 0xb2, + 0x1c, 0xcb, 0xce, 0x3f, 0x79, 0xba, 0x80, 0x06, 0xa7, 0xb0, 0x5c, 0xa6, 0x8b, 0xa2, 0xf1, 0xd1, + 0x76, 0xb5, 0x52, 0xad, 0x97, 0xd5, 0xe0, 0xb4, 0xe3, 0xd9, 0xcc, 0x93, 0xa7, 0x0b, 0x73, 0xb4, + 0xcd, 0x89, 0xa9, 0xbf, 0x01, 0x72, 0xbd, 0xa1, 0x54, 0x8b, 0x1b, 0x6a, 0x6d, 0x73, 0xb9, 0x5a, + 0x67, 0x2f, 0x0b, 0xfa, 0x43, 0x1a, 0xe0, 0x12, 0x3a, 0x85, 0xcd, 0xea, 0x07, 0x03, 0xf8, 0x89, + 0x7e, 0xfd, 0x01, 0x7a, 0x40, 0x0b, 0x10, 0xdf, 0xa8, 0x2d, 0x2b, 0x45, 0x86, 0x3b, 0x9d, 0x4d, + 0x3f, 0x79, 0xba, 0x90, 0xa4, 0xf5, 0x3c, 0x67, 0x8f, 0x6a, 0x90, 0x63, 0x4a, 0xa9, 0x6f, 0x17, + 0x37, 0xd5, 0xf2, 0xd6, 0xe6, 0x52, 0x6d, 0x59, 0x55, 0xaa, 0xe5, 0xad, 0xcd, 0x72, 0x6d, 0xbd, + 0xc6, 0xdb, 0x25, 0xb3, 0xd7, 0x9f, 0x3c, 0x5d, 0x58, 0x70, 0x55, 0x74, 0xaa, 0x6b, 0xfe, 0x02, + 0xcc, 0xd5, 0xef, 0xad, 0x73, 0xbd, 0xfa, 0x8d, 0x2f, 0x95, 0xcd, 0x3e, 0x79, 0xba, 0x30, 0xcf, + 0xe6, 0x73, 0xc2, 0x0b, 0x67, 0x63, 0xbf, 0xf3, 0x83, 0x6b, 0x13, 0x7f, 0xfc, 0xc3, 0x6b, 0x13, + 0xa5, 0x9b, 0x3f, 0xfb, 0x8f, 0x6b, 0x13, 0x3f, 0x3b, 0xbe, 0x26, 0xfd, 0xfc, 0xf8, 0x9a, 0xf4, + 0x8b, 0xe3, 0x6b, 0xd2, 0xbf, 0x1f, 0x5f, 0x93, 0x7e, 0xf7, 0x97, 0xd7, 0x26, 0x7e, 0xfe, 0xcb, + 0x6b, 0x13, 0xbf, 0xf8, 0xe5, 0xb5, 0x89, 0x8f, 0x23, 0xdc, 0x8f, 0xed, 0x46, 0x58, 0xde, 0xe9, + 0xad, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0xfa, 0x71, 0xe5, 0xaa, 0xba, 0x48, 0x00, 0x00, } func (this *BackupEncryptionOptions) Equal(that interface{}) bool { @@ -6317,58 +6281,6 @@ func (m *MigrationProgress) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *ExecutionEvent) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *ExecutionEvent) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *ExecutionEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.ExecutionError) > 0 { - i -= len(m.ExecutionError) - copy(dAtA[i:], m.ExecutionError) - i = encodeVarintJobs(dAtA, i, uint64(len(m.ExecutionError))) - i-- - dAtA[i] = 0x2a - } - if m.EventTimeMicros != 0 { - i = encodeVarintJobs(dAtA, i, uint64(m.EventTimeMicros)) - i-- - dAtA[i] = 0x20 - } - if len(m.Status) > 0 { - i -= len(m.Status) - copy(dAtA[i:], m.Status) - i = encodeVarintJobs(dAtA, i, uint64(len(m.Status))) - i-- - dAtA[i] = 0x1a - } - if m.InstanceId != 0 { - i = encodeVarintJobs(dAtA, i, uint64(m.InstanceId)) - i-- - dAtA[i] = 0x10 - } - if m.Type != 0 { - i = encodeVarintJobs(dAtA, i, uint64(m.Type)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - func (m *SQLStatsCompactionDetails) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -6435,19 +6347,10 @@ func (m *Payload) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.Details != nil { - { - size := m.Details.Size() - i -= size - if _, err := m.Details.MarshalTo(dAtA[i:]); err != nil { - return 0, err - } - } - } - if len(m.ExecutionLog) > 0 { - for iNdEx := len(m.ExecutionLog) - 1; iNdEx >= 0; iNdEx-- { + if len(m.RetriableExecutionFailureLog) > 0 { + for iNdEx := len(m.RetriableExecutionFailureLog) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.ExecutionLog[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.RetriableExecutionFailureLog[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -6455,9 +6358,18 @@ func (m *Payload) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x1 + dAtA[i] = 0x2 i-- - dAtA[i] = 0xea + dAtA[i] = 0x82 + } + } + if m.Details != nil { + { + size := m.Details.Size() + i -= size + if _, err := m.Details.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } } } if len(m.PauseReason) > 0 { @@ -7310,6 +7222,70 @@ func (m *Job) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *RetriableExecutionFailure) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RetriableExecutionFailure) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RetriableExecutionFailure) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.TruncatedError) > 0 { + i -= len(m.TruncatedError) + copy(dAtA[i:], m.TruncatedError) + i = encodeVarintJobs(dAtA, i, uint64(len(m.TruncatedError))) + i-- + dAtA[i] = 0x32 + } + if m.Error != nil { + { + size, err := m.Error.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintJobs(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + if m.InstanceID != 0 { + i = encodeVarintJobs(dAtA, i, uint64(m.InstanceID)) + i-- + dAtA[i] = 0x20 + } + if m.ExecutionEndMicros != 0 { + i = encodeVarintJobs(dAtA, i, uint64(m.ExecutionEndMicros)) + i-- + dAtA[i] = 0x18 + } + if m.ExecutionStartMicros != 0 { + i = encodeVarintJobs(dAtA, i, uint64(m.ExecutionStartMicros)) + i-- + dAtA[i] = 0x10 + } + if len(m.Status) > 0 { + i -= len(m.Status) + copy(dAtA[i:], m.Status) + i = encodeVarintJobs(dAtA, i, uint64(len(m.Status))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func encodeVarintJobs(dAtA []byte, offset int, v uint64) int { offset -= sovJobs(v) base := offset @@ -8435,32 +8411,6 @@ func (m *MigrationProgress) Size() (n int) { return n } -func (m *ExecutionEvent) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Type != 0 { - n += 1 + sovJobs(uint64(m.Type)) - } - if m.InstanceId != 0 { - n += 1 + sovJobs(uint64(m.InstanceId)) - } - l = len(m.Status) - if l > 0 { - n += 1 + l + sovJobs(uint64(l)) - } - if m.EventTimeMicros != 0 { - n += 1 + sovJobs(uint64(m.EventTimeMicros)) - } - l = len(m.ExecutionError) - if l > 0 { - n += 1 + l + sovJobs(uint64(l)) - } - return n -} - func (m *SQLStatsCompactionDetails) Size() (n int) { if m == nil { return 0 @@ -8542,8 +8492,8 @@ func (m *Payload) Size() (n int) { if l > 0 { n += 2 + l + sovJobs(uint64(l)) } - if len(m.ExecutionLog) > 0 { - for _, e := range m.ExecutionLog { + if len(m.RetriableExecutionFailureLog) > 0 { + for _, e := range m.RetriableExecutionFailureLog { l = e.Size() n += 2 + l + sovJobs(uint64(l)) } @@ -8929,6 +8879,36 @@ func (m *Job) Size() (n int) { return n } +func (m *RetriableExecutionFailure) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Status) + if l > 0 { + n += 1 + l + sovJobs(uint64(l)) + } + if m.ExecutionStartMicros != 0 { + n += 1 + sovJobs(uint64(m.ExecutionStartMicros)) + } + if m.ExecutionEndMicros != 0 { + n += 1 + sovJobs(uint64(m.ExecutionEndMicros)) + } + if m.InstanceID != 0 { + n += 1 + sovJobs(uint64(m.InstanceID)) + } + if m.Error != nil { + l = m.Error.Size() + n += 1 + l + sovJobs(uint64(l)) + } + l = len(m.TruncatedError) + if l > 0 { + n += 1 + l + sovJobs(uint64(l)) + } + return n +} + func sovJobs(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -16932,7 +16912,7 @@ func (m *MigrationProgress) Unmarshal(dAtA []byte) error { } return nil } -func (m *ExecutionEvent) Unmarshal(dAtA []byte) error { +func (m *SQLStatsCompactionDetails) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -16955,194 +16935,23 @@ func (m *ExecutionEvent) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecutionEvent: wiretype end group for non-group") + return fmt.Errorf("proto: SQLStatsCompactionDetails: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecutionEvent: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SQLStatsCompactionDetails: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) - } - m.Type = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowJobs - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Type |= ExecutionEvent_EventType(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field InstanceId", wireType) + default: + iNdEx = preIndex + skippy, err := skipJobs(dAtA[iNdEx:]) + if err != nil { + return err } - m.InstanceId = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowJobs - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.InstanceId |= int32(b&0x7F) << shift - if b < 0x80 { - break - } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthJobs } - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowJobs - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthJobs - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthJobs - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Status = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field EventTimeMicros", wireType) - } - m.EventTimeMicros = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowJobs - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.EventTimeMicros |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ExecutionError", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowJobs - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthJobs - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthJobs - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ExecutionError = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipJobs(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthJobs - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *SQLStatsCompactionDetails) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowJobs - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: SQLStatsCompactionDetails: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: SQLStatsCompactionDetails: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - default: - iNdEx = preIndex - skippy, err := skipJobs(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthJobs - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF } iNdEx += skippy } @@ -18050,9 +17859,9 @@ func (m *Payload) Unmarshal(dAtA []byte) error { } m.PauseReason = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 29: + case 30: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ExecutionLog", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SqlStatsCompaction", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -18079,14 +17888,15 @@ func (m *Payload) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ExecutionLog = append(m.ExecutionLog, &ExecutionEvent{}) - if err := m.ExecutionLog[len(m.ExecutionLog)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + v := &SQLStatsCompactionDetails{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } + m.Details = &Payload_SqlStatsCompaction{v} iNdEx = postIndex - case 30: + case 32: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SqlStatsCompaction", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RetriableExecutionFailureLog", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -18113,11 +17923,10 @@ func (m *Payload) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - v := &SQLStatsCompactionDetails{} - if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.RetriableExecutionFailureLog = append(m.RetriableExecutionFailureLog, &RetriableExecutionFailure{}) + if err := m.RetriableExecutionFailureLog[len(m.RetriableExecutionFailureLog)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } - m.Details = &Payload_SqlStatsCompaction{v} iNdEx = postIndex default: iNdEx = preIndex @@ -18902,6 +18711,213 @@ func (m *Job) Unmarshal(dAtA []byte) error { } return nil } +func (m *RetriableExecutionFailure) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RetriableExecutionFailure: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RetriableExecutionFailure: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthJobs + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Status = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionStartMicros", wireType) + } + m.ExecutionStartMicros = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ExecutionStartMicros |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionEndMicros", wireType) + } + m.ExecutionEndMicros = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ExecutionEndMicros |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field InstanceID", wireType) + } + m.InstanceID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.InstanceID |= github_com_cockroachdb_cockroach_pkg_base.SQLInstanceID(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthJobs + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Error == nil { + m.Error = &errorspb.EncodedError{} + } + if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TruncatedError", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthJobs + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TruncatedError = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipJobs(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthJobs + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipJobs(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 8e3885a85bef..886d3367a405 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -705,41 +705,6 @@ message MigrationDetails { } message MigrationProgress { - -} - -// ExecutionEvent holds information about a job when it starts or -// ends an execution. It contains information about the event time, -// the node on which the job is executed, the status of the job,and -// any errors occurred during execution. -message ExecutionEvent { - enum EventType { - option (gogoproto.goproto_enum_prefix) = false; - - // JobEventUnspecified is the zero value for EventType and should be used. - UNSPECIFIED = 0 [(gogoproto.enumvalue_customname) = "JobEventUnspecified"]; - - // JobStartEvent indicates that this event was created at the beginning of - // job execution. - START = 1 [(gogoproto.enumvalue_customname) = "JobStartEvent"]; - - // JobEndEvent indicates that this event was created at the end of job - // execution. - END = 2 [(gogoproto.enumvalue_customname) = "JobEndEvent"]; - } - // Type of event, start or end of execution when this event is created. - EventType type = 1; - // ID of the node on which the job is executed. - int32 instance_id = 2; - // Status of the job in this execution. It should be a jobs.Status. - string status = 3; - // Event timestamp when the event is created, represented as the number - // of microseconds elapsed since UnixEpoch. See timeutil.ToUnixMicros - // for more information about conversion to micros. - int64 event_time_micros = 4; - // Errors during execution: it is empty for a JobStartEvent. For a JobEndEvent, - // it consists of errors encountered during the current job execution. - string execution_error = 5; } message SQLStatsCompactionDetails { @@ -798,17 +763,13 @@ message Payload { // PauseReason is used to describe the reason that the job is currently paused // or has been requested to be paused. string pause_reason = 28; + // RetriableExecutionFailureLog stores a history of retriable execution + // failures. These failures may occur in either the RUNNING or REVERTING + // status. A finite number of these entries will be kept, as governed by + // the jobs.execution_errors.max_entries cluster setting. + repeated RetriableExecutionFailure retriable_execution_failure_log = 32; - // An ExecutionEvent is created whenever a job's Resumer starts and ends - // an execution. At the beginning of the execution of a job's Resumer, - // a EventType.START event is created and appended to executionLog in - // job's payload. When the Resumer completes execution, which can be with - // or without an error, an EventType.END event is created and appended to - // job's executionLog. The sequence of events in this log show the lifecycle - // of a job and the errors encountered during job execution. - repeated ExecutionEvent executionLog = 29; - - // NEXT ID: 31. + // NEXT ID: 33. } message Progress { @@ -868,3 +829,23 @@ message Job { Progress progress = 2; Payload payload = 3; } + +// RetriableExecutionFailure is used in Payload.RetriableExecutionFailureLog +// to store a history of executions which failed. +message RetriableExecutionFailure { + // Status is the status of the job when this failure occurred. + string status = 1; + // ExecutionStartMicros is the timestamp at which this execution occurred. + int64 execution_start_micros = 2; + // ExecutionEndMicros is the timestamp at which this execution concluded. + int64 execution_end_micros = 3; + // InstanceID is the instance which coordinated the execution. + int32 instance_id = 4 [(gogoproto.customname) = "InstanceID", (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/base.SQLInstanceID", (gogoproto.nullable) = false]; + // Error stores the structured error which occurred. It might be nil if it + // was too large. In that case, the TruncatedError will be populated. + errorspb.EncodedError error = 5; + // TruncatedError is a fragment of a error message populated in the case + // that the error was too large. While the structure may be lost, at least + // some information will be preserved. + string truncated_error = 6; +} diff --git a/pkg/jobs/jobspb/json_encoding.go b/pkg/jobs/jobspb/json_encoding.go deleted file mode 100644 index bc5f6ef2e8a2..000000000000 --- a/pkg/jobs/jobspb/json_encoding.go +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package jobspb - -import ( - "github.com/cockroachdb/cockroach/pkg/sql/protoreflect" - "github.com/cockroachdb/cockroach/pkg/util/json" -) - -// ExecutionLogToJSON converts an executionLog in a job payload in a JSON object. -// It internally uses protoreflect.MessageToJSON to individually convert an -// ExecutionEvent in executionLog. Default values are omitted from the resulting -// JSON object. -func ExecutionLogToJSON(executionLog []*ExecutionEvent) (json.JSON, error) { - b := json.NewArrayBuilder(len(executionLog)) - for _, event := range executionLog { - eventJSON, err := protoreflect.MessageToJSON(event, false /* emitDefaults */) - if err != nil { - return nil, err - } - b.Add(eventJSON) - } - return b.Build(), nil -} diff --git a/pkg/jobs/jobspb/wrap.go b/pkg/jobs/jobspb/wrap.go index 8cc5121f8e2d..12286c1db670 100644 --- a/pkg/jobs/jobspb/wrap.go +++ b/pkg/jobs/jobspb/wrap.go @@ -15,6 +15,7 @@ import ( "fmt" "strings" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/errors" @@ -60,6 +61,10 @@ func (p *Payload) Type() Type { return DetailsType(p.Details) } +// Import base which is in the generated proto field but won't get picked up +// by bazel if it were not imported in a non-generated file. +var _ base.SQLInstanceID + // AutoStatsName is the name to use for statistics created automatically. // The name is chosen to be something that users are unlikely to choose when // running CREATE STATISTICS manually. diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index f7be4cfc5cc1..1aecec80c39b 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -1136,39 +1136,6 @@ func (r *Registry) createResumer(job *Job, settings *cluster.Settings) (Resumer, return fn(job, settings), nil } -type retryJobError string - -// retryJobErrorSentinel exists so the errors returned from NewRetryJobError can -// be marked with it, allowing more robust detection of retry errors even if -// they are wrapped, etc. This was originally introduced to deal with injected -// retry errors from testing knobs. -var retryJobErrorSentinel = retryJobError("") - -// NewRetryJobError creates a new error that, if returned by a Resumer, -// indicates to the jobs registry that the job should be restarted in the -// background. -func NewRetryJobError(s string) error { - return errors.Mark(retryJobError(s), retryJobErrorSentinel) -} - -func (r retryJobError) Error() string { - return string(r) -} - -// Registry does not retry a job that fails due to a permanent error. -var errJobPermanentSentinel = errors.New("permanent job-error") - -// MarkAsPermanentJobError marks an error as a permanent job error, which indicates -// Registry to not retry the job when it fails due to this error. -func MarkAsPermanentJobError(err error) error { - return errors.Mark(err, errJobPermanentSentinel) -} - -// IsPermanentJobError checks whether the given error is a permanent error. -func IsPermanentJobError(err error) bool { - return errors.Is(err, errJobPermanentSentinel) -} - // stepThroughStateMachine implements the state machine of the job lifecycle. // The job is executed with the ctx, so ctx must only be canceled if the job // should also be canceled. resultsCh is passed to the resumable func and should @@ -1181,7 +1148,18 @@ func (r *Registry) stepThroughStateMachine( jobType := payload.Type() log.Infof(ctx, "%s job %d: stepping through state %s with error: %+v", jobType, job.ID(), status, jobErr) jm := r.metrics.JobMetrics[jobType] - + onExecutionFailed := func(cause error) error { + log.InfofDepth( + ctx, 1, + "job %d: %s execution encountered retriable error: %v", + job.ID(), status, cause, + ) + start := job.getRunStats().LastRun + end := r.clock.Now().GoTime() + return newRetriableExecutionError( + r.nodeID.SQLInstanceID(), status, start, end, cause, + ) + } switch status { case StatusRunning: if jobErr != nil { @@ -1214,9 +1192,9 @@ func (r *Registry) stepThroughStateMachine( return errors.Errorf("job %d: node liveness error: restarting in background", job.ID()) } // TODO(spaskob): enforce a limit on retries. - if errors.Is(err, retryJobErrorSentinel) { + if errors.Is(err, errRetryJobSentinel) { jm.ResumeRetryError.Inc(1) - return errors.Errorf("job %d: %s: restarting in background", job.ID(), err) + return onExecutionFailed(err) } jm.ResumeFailed.Inc(1) if sErr := (*InvalidStatusError)(nil); errors.As(err, &sErr) { @@ -1286,9 +1264,9 @@ func (r *Registry) stepThroughStateMachine( // mark the job as failed because it can be resumed by another node. return errors.Errorf("job %d: node liveness error: restarting in background", job.ID()) } - if errors.Is(err, retryJobErrorSentinel) { + if errors.Is(err, errRetryJobSentinel) { jm.FailOrCancelRetryError.Inc(1) - return errors.Errorf("job %d: %s: restarting in background", job.ID(), err) + return onExecutionFailed(err) } // A non-cancelable job is always retried while reverting unless the error is marked as permanent. if job.Payload().Noncancelable && !IsPermanentJobError(err) { @@ -1390,3 +1368,38 @@ func (r *Registry) RetryMaxDelay() float64 { } return retryMaxDelaySetting.Get(&r.settings.SV).Seconds() } + +// maybeRecordRetriableExeuctionFailure will record a +// RetriableExecutionFailureError into the job payload. +func (r *Registry) maybeRecordExecutionFailure(ctx context.Context, err error, j *Job) { + var efe *retriableExecutionError + if !errors.As(err, &efe) { + return + } + + updateErr := j.Update(ctx, nil, func( + txn *kv.Txn, md JobMetadata, ju *JobUpdater, + ) error { + pl := md.Payload + { // Append the entry to the log + maxSize := int(executionErrorsMaxEntrySize.Get(&r.settings.SV)) + pl.RetriableExecutionFailureLog = append(pl.RetriableExecutionFailureLog, + efe.toRetriableExecutionFailure(ctx, maxSize)) + } + { // Maybe truncate the log. + maxEntries := int(executionErrorsMaxEntriesSetting.Get(&r.settings.SV)) + log := &pl.RetriableExecutionFailureLog + if len(*log) > maxEntries { + *log = (*log)[len(*log)-maxEntries:] + } + } + ju.UpdatePayload(pl) + return nil + }) + if ctx.Err() != nil { + return + } + if updateErr != nil { + log.Warningf(ctx, "failed to record error for job %d: %v: %v", j.ID(), err, err) + } +} diff --git a/pkg/jobs/registry_external_test.go b/pkg/jobs/registry_external_test.go index 05c96767f6f8..4fc2165d2a2c 100644 --- a/pkg/jobs/registry_external_test.go +++ b/pkg/jobs/registry_external_test.go @@ -17,6 +17,7 @@ import ( "fmt" "reflect" "regexp" + "strconv" "strings" "sync/atomic" "testing" @@ -31,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" @@ -40,6 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" @@ -416,3 +419,308 @@ func TestGCDurationControl(t *testing.T) { // Wait for the job to be deleted. tdb.CheckQueryResultsRetry(t, existsQuery, [][]string{{"false"}}) } + +// TestErrorsPopulatedOnRetry confirms that when a job fails with a retriable +// error that that error makes its way to the payload. +func TestErrorsPopulatedOnRetry(t *testing.T) { + defer leaktest.AfterTest(t)() + ls := log.Scope(t) + defer ls.Close(t) + + type event struct { + id jobspb.JobID + resume chan error + } + mkEvent := func(j *jobs.Job) event { + return event{id: j.ID(), resume: make(chan error)} + } + evChan := make(chan event) + jobs.RegisterConstructor(jobspb.TypeImport, func(j *jobs.Job, cs *cluster.Settings) jobs.Resumer { + execFn := func(ctx context.Context) error { + ev := mkEvent(j) + select { + case evChan <- ev: + case <-ctx.Done(): + return ctx.Err() + } + select { + case err := <-ev.resume: + return err + case <-ctx.Done(): + return ctx.Err() + } + } + return jobs.FakeResumer{ + OnResume: execFn, + FailOrCancel: execFn, + } + }) + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + }, + }) + tdb := sqlutils.MakeSQLRunner(sqlDB) + ctx := context.Background() + defer s.Stopper().Stop(ctx) + registry := s.JobRegistry().(*jobs.Registry) + ie := s.InternalExecutor().(sqlutil.InternalExecutor) + mkJob := func(t *testing.T) jobspb.JobID { + id := registry.MakeJobID() + _, err := registry.CreateJobWithTxn(ctx, jobs.Record{ + // Job does not accept an empty Details field, so arbitrarily provide + // ImportDetails. + Details: jobspb.ImportDetails{}, + Progress: jobspb.ImportProgress{}, + }, id, nil /* txn */) + require.NoError(t, err) + return id + } + type parsedError struct { + start, end time.Time + status jobs.Status + error string + instance base.SQLInstanceID + } + var ( + executionErrorRE = regexp.MustCompile( + `(?P\w+) execution from '(?P.*)' to '(?P.*)' on (?P\d+) failed: (?P.*)`, + ) + statusIdx = executionErrorRE.SubexpIndex("status") + startIdx = executionErrorRE.SubexpIndex("Start") + endIdx = executionErrorRE.SubexpIndex("End") + instanceIdx = executionErrorRE.SubexpIndex("instance") + errorIdx = executionErrorRE.SubexpIndex("error") + ) + parseTimestamp := func(t *testing.T, s string) time.Time { + ptc := tree.NewParseTimeContext(timeutil.Now()) + ts, _, err := tree.ParseDTimestamp(ptc, s, time.Microsecond) + require.NoError(t, err) + return ts.Time + } + parseInstanceID := func(t *testing.T, s string) base.SQLInstanceID { + i, err := strconv.ParseInt(s, 10, 32) + require.NoError(t, err) + return base.SQLInstanceID(i) + } + parseExecutionError := func(t *testing.T, s string) (ret parsedError) { + matches := executionErrorRE.FindStringSubmatch(s) + require.NotNil(t, matches) + ret.status = jobs.Status(matches[statusIdx]) + ret.start = parseTimestamp(t, matches[startIdx]) + ret.end = parseTimestamp(t, matches[endIdx]) + ret.instance = parseInstanceID(t, matches[instanceIdx]) + ret.error = matches[errorIdx] + return ret + } + parseExecutionErrors := func(t *testing.T, s [][]string) (ret []parsedError) { + for _, res := range s { + require.Len(t, res, 1) + ret = append(ret, parseExecutionError(t, res[0])) + } + return ret + } + tsEqual := func(t *testing.T, a, b time.Time) { + require.Truef(t, a.Equal(b), "%v != %v", a, b) + } + tsBefore := func(t *testing.T, a, b time.Time) { + require.Truef(t, a.Before(b), "%v >= %v", a, b) + } + executionErrorEqual := func(t *testing.T, a, b parsedError) { + tsEqual(t, a.start, b.start) + tsEqual(t, a.end, b.end) + require.Equal(t, a.instance, b.instance) + require.Equal(t, a.error, b.error) + require.Equal(t, a.status, b.status) + } + waitForEvent := func(t *testing.T, id jobspb.JobID) (ev event, start time.Time) { + ev = <-evChan + require.Equal(t, id, ev.id) + tdb.QueryRow(t, "SELECT last_run FROM crdb_internal.jobs WHERE job_id = $1", id).Scan(&start) + return ev, start + } + checkExecutionError := func( + t *testing.T, execErr parsedError, status jobs.Status, start, afterEnd time.Time, cause string, + ) { + require.Equal(t, base.SQLInstanceID(1), execErr.instance) + require.Equal(t, status, execErr.status) + tsEqual(t, start, execErr.start) + tsBefore(t, execErr.start, execErr.end) + tsBefore(t, execErr.end, afterEnd) + require.Equal(t, cause, execErr.error) + } + getExecErrors := func(t *testing.T, id jobspb.JobID) []parsedError { + return parseExecutionErrors(t, + tdb.QueryStr(t, ` +SELECT unnest(execution_errors) + FROM crdb_internal.jobs + WHERE job_id = $1;`, id), + ) + } + checkLogEntry := func( + t *testing.T, id jobspb.JobID, status jobs.Status, + from, to time.Time, cause string, + ) { + log.Flush() + entries, err := log.FetchEntriesFromFiles( + from.UnixNano(), to.UnixNano(), 2, + regexp.MustCompile(fmt.Sprintf( + "job %d: %s execution encountered retriable error: %s", + id, status, cause, + )), + log.WithFlattenedSensitiveData, + ) + require.NoError(t, err) + require.Len(t, entries, 1) + } + t.Run("retriable error makes it into payload", func(t *testing.T) { + id := mkJob(t) + firstRun, firstStart := waitForEvent(t, id) + const err1 = "boom1" + firstRun.resume <- jobs.MarkAsRetryJobError(errors.New(err1)) + + // Wait for the job to get restarted. + secondRun, secondStart := waitForEvent(t, id) + + // Confirm the previous execution error was properly recorded. + var firstExecErr parsedError + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 1) + firstExecErr = execErrs[0] + checkExecutionError(t, firstExecErr, jobs.StatusRunning, firstStart, secondStart, err1) + checkLogEntry(t, id, jobs.StatusRunning, firstStart, secondStart, err1) + } + const err2 = "boom2" + secondRun.resume <- jobs.MarkAsRetryJobError(errors.New(err2)) + thirdRun, thirdStart := waitForEvent(t, id) + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 2) + executionErrorEqual(t, firstExecErr, execErrs[0]) + secondExecErr := execErrs[1] + checkExecutionError(t, secondExecErr, jobs.StatusRunning, secondStart, thirdStart, err2) + checkLogEntry(t, id, jobs.StatusRunning, secondStart, thirdStart, err2) + } + close(thirdRun.resume) + require.NoError(t, registry.WaitForJobs(ctx, ie, []jobspb.JobID{id})) + }) + t.Run("fail or cancel error", func(t *testing.T) { + id := mkJob(t) + firstRun, firstStart := waitForEvent(t, id) + const err1 = "boom1" + firstRun.resume <- jobs.MarkAsRetryJobError(errors.New(err1)) + + // Wait for the job to get restarted. + secondRun, secondStart := waitForEvent(t, id) + + // Confirm the previous execution error was properly recorded. + var firstExecErr parsedError + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 1) + firstExecErr = execErrs[0] + checkExecutionError(t, firstExecErr, jobs.StatusRunning, firstStart, secondStart, err1) + checkLogEntry(t, id, jobs.StatusRunning, firstStart, secondStart, err1) + } + const err2 = "boom2" + secondRun.resume <- errors.New(err2) + thirdRun, thirdStart := waitForEvent(t, id) // thirdRun is Reverting + // Confirm that no new error was recorded in the log. It will be in + // FinalResumeError. + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 1) + executionErrorEqual(t, firstExecErr, execErrs[0]) + } + const err3 = "boom3" + thirdRun.resume <- jobs.MarkAsRetryJobError(errors.New(err3)) + fourthRun, fourthStart := waitForEvent(t, id) + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 2) + executionErrorEqual(t, firstExecErr, execErrs[0]) + checkExecutionError(t, execErrs[1], jobs.StatusReverting, thirdStart, fourthStart, err3) + checkLogEntry(t, id, jobs.StatusReverting, thirdStart, fourthStart, err3) + } + close(fourthRun.resume) + require.Regexp(t, err2, registry.WaitForJobs(ctx, ie, []jobspb.JobID{id})) + }) + t.Run("truncation", func(t *testing.T) { + id := mkJob(t) + firstRun, firstStart := waitForEvent(t, id) + const maxSize, largeSize = 2 << 10, 8 << 10 + tdb.Exec(t, "SET CLUSTER SETTING "+jobs.ExecutionErrorsMaxEntrySizeKey+" = $1", maxSize) + tdb.Exec(t, "SET CLUSTER SETTING "+jobs.ExecutionErrorsMaxEntriesKey+" = $1", 1) + err1 := strings.Repeat("a", largeSize) + firstRun.resume <- jobs.MarkAsRetryJobError(errors.New(err1)) + + // Wait for the job to get restarted. + secondRun, secondStart := waitForEvent(t, id) + // Confirm the previous execution error was properly recorded. + var firstExecErr parsedError + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 1) + firstExecErr = execErrs[0] + // Ensure we see the truncated error in the table but the full error + // in the logs. + expTruncatedError := "(truncated) " + err1[:maxSize] + checkExecutionError(t, firstExecErr, jobs.StatusRunning, firstStart, secondStart, expTruncatedError) + checkLogEntry(t, id, jobs.StatusRunning, firstStart, secondStart, err1) + } + const err2 = "boom2" + secondRun.resume <- jobs.MarkAsRetryJobError(errors.New(err2)) + thirdRun, thirdStart := waitForEvent(t, id) + var secondExecErr parsedError + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 1) + secondExecErr = execErrs[0] + checkExecutionError(t, secondExecErr, jobs.StatusRunning, secondStart, thirdStart, err2) + checkLogEntry(t, id, jobs.StatusRunning, secondStart, thirdStart, err2) + } + // Fail the job so we can also test the truncation of reverting retry + // errors. + const err3 = "boom3" + thirdRun.resume <- errors.New(err3) // not retriable + fourthRun, fourthStart := waitForEvent(t, id) // first Reverting run + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 1) + executionErrorEqual(t, secondExecErr, execErrs[0]) + } + err4 := strings.Repeat("b", largeSize) + fourthRun.resume <- jobs.MarkAsRetryJobError(errors.New(err4)) + fifthRun, fifthStart := waitForEvent(t, id) + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 1) + // Ensure we see the truncated error in the table but the full error + // in the logs. + expTruncatedError := "(truncated) " + err4[:maxSize] + checkExecutionError(t, execErrs[0], jobs.StatusReverting, fourthStart, fifthStart, expTruncatedError) + checkLogEntry(t, id, jobs.StatusReverting, fourthStart, fifthStart, err4) + } + const err5 = "boom5" + fifthRun.resume <- jobs.MarkAsRetryJobError(errors.New(err5)) + sixthRun, sixthStart := waitForEvent(t, id) + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 1) + checkExecutionError(t, execErrs[0], jobs.StatusReverting, fifthStart, sixthStart, err5) + checkLogEntry(t, id, jobs.StatusReverting, fifthStart, sixthStart, err5) + } + const err6 = "boom5" + tdb.Exec(t, "SET CLUSTER SETTING "+jobs.ExecutionErrorsMaxEntriesKey+" = $1", 0) + sixthRun.resume <- jobs.MarkAsRetryJobError(errors.New(err6)) + seventhRun, seventhStart := waitForEvent(t, id) + { + execErrs := getExecErrors(t, id) + require.Len(t, execErrs, 0) + checkLogEntry(t, id, jobs.StatusReverting, sixthStart, seventhStart, err6) + } + close(seventhRun.resume) + require.Regexp(t, err3, registry.WaitForJobs(ctx, ie, []jobspb.JobID{id})) + }) +} diff --git a/pkg/jobs/registry_test.go b/pkg/jobs/registry_test.go index a5d545fd0f34..61b567115da5 100644 --- a/pkg/jobs/registry_test.go +++ b/pkg/jobs/registry_test.go @@ -558,7 +558,7 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { // can be retried in the next adopt-loop. // - We wait until the resumer completes one execution and the job needs // to be picked up again in the next adopt-loop. - // - Now we validate that resumedJobs counter has increment, which ensures + // - Now we validate that resumedJobs counter has incremented, which ensures // that the job has completed only one cycle in this time. // // If retries do not happen based on exponential-backoff times, our counters @@ -622,7 +622,7 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { expectedResumed := int64(0) runTest(t, jobID, retryCnt, expectedResumed, lastRun, &bti, func(_ int64) { <-bti.resumeCh - bti.errCh <- NewRetryJobError("injecting error to retry running") + bti.errCh <- MarkAsRetryJobError(errors.New("injecting error to retry running")) <-bti.transitionCh }) }) @@ -669,13 +669,13 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { <-bti.resumeCh bti.errCh <- errors.Errorf("injecting error to revert") <-bti.failOrCancelCh - bti.errCh <- NewRetryJobError("injecting error in reverting state to retry") + bti.errCh <- MarkAsRetryJobError(errors.New("injecting error in reverting state to retry")) <-bti.transitionCh expectedResumed := bti.resumed.Count() retryCnt := 1 runTest(t, jobID, retryCnt, expectedResumed, lastRun, &bti, func(_ int64) { <-bti.failOrCancelCh - bti.errCh <- NewRetryJobError("injecting error in reverting state to retry") + bti.errCh <- MarkAsRetryJobError(errors.New("injecting error in reverting state to retry")) <-bti.transitionCh }) }) @@ -692,12 +692,12 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { pauseOrCancelJob(t, ctx, bti.kvDB, bti.registry, jobID, cancel) bti.errCh <- nil <-bti.failOrCancelCh - bti.errCh <- NewRetryJobError("injecting error in reverting state") + bti.errCh <- MarkAsRetryJobError(errors.New("injecting error in reverting state")) expectedResumed := bti.resumed.Count() retryCnt := 1 runTest(t, jobID, retryCnt, expectedResumed, lastRun, &bti, func(retryCnt int64) { <-bti.failOrCancelCh - bti.errCh <- NewRetryJobError("injecting error in reverting state") + bti.errCh <- MarkAsRetryJobError(errors.New("injecting error in reverting state")) waitUntilCount(t, bti.jobMetrics.FailOrCancelRetryError, retryCnt+1) }) }) @@ -719,7 +719,7 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { <-bti.resumeCh bti.errCh <- errors.Errorf("injecting error to revert") <-bti.failOrCancelCh - bti.errCh <- NewRetryJobError("injecting error in reverting state to retry") + bti.errCh <- MarkAsRetryJobError(errors.New("injecting error in reverting state to retry")) <-bti.transitionCh expectedResumed := bti.resumed.Count() retryCnt := 1 @@ -730,7 +730,7 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { // failed regardless of the fact that it is currently pause-requested in the // jobs table. This is because we currently do not check the current status // of a job before marking it as failed. - bti.errCh <- NewRetryJobError("injecting error in reverting state to retry") + bti.errCh <- MarkAsRetryJobError(errors.New("injecting error in reverting state to retry")) <-bti.transitionCh waitUntilStatus(t, bti.tdb, jobID, StatusPaused) require.NoError(t, bti.registry.Unpause(ctx, nil, jobID)) diff --git a/pkg/jobs/update.go b/pkg/jobs/update.go index a83058499aa9..6ff62dc8bcf6 100644 --- a/pkg/jobs/update.go +++ b/pkg/jobs/update.go @@ -141,9 +141,11 @@ func (j *Job) Update(ctx context.Context, txn *kv.Txn, updateFn UpdateFn) error func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateFn UpdateFn) error { var payload *jobspb.Payload var progress *jobspb.Progress + var runStats *RunStats backoffIsActive := j.registry.settings.Version.IsActive(ctx, clusterversion.RetryJobsWithExponentialBackoff) if err := j.runInTxn(ctx, txn, func(ctx context.Context, txn *kv.Txn) error { + payload, progress, runStats = nil, nil, nil var err error var row tree.Datums row, err = j.registry.ex.QueryRowEx( @@ -268,6 +270,7 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF } if backoffIsActive && ju.md.RunStats != nil { + runStats = ju.md.RunStats addSetter("last_run", ju.md.RunStats.LastRun) addSetter("num_runs", ju.md.RunStats.NumRuns) } @@ -289,16 +292,19 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF }); err != nil { return err } - if payload != nil { + func() { j.mu.Lock() - j.mu.payload = *payload - j.mu.Unlock() - } - if progress != nil { - j.mu.Lock() - j.mu.progress = *progress - j.mu.Unlock() - } + defer j.mu.Unlock() + if payload != nil { + j.mu.payload = *payload + } + if progress != nil { + j.mu.progress = *progress + } + if runStats != nil { + j.mu.runStats = runStats + } + }() return nil } diff --git a/pkg/migration/migrations/retry_jobs_with_exponential_backoff_external_test.go b/pkg/migration/migrations/retry_jobs_with_exponential_backoff_external_test.go index 7aa5e93f2b1b..cedc7b38701b 100644 --- a/pkg/migration/migrations/retry_jobs_with_exponential_backoff_external_test.go +++ b/pkg/migration/migrations/retry_jobs_with_exponential_backoff_external_test.go @@ -568,7 +568,7 @@ func validateJobRetries(t *testing.T, tdb *sqlutils.SQLRunner, eventCh chan upda continue } failed.Store(true) - ev.errChan <- jobs.NewRetryJobError("failing job to retry") + ev.errChan <- jobs.MarkAsRetryJobError(errors.New("failing job to retry")) } }() runGcJob(t, tdb) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 0f4f3c364b9d..e1d201f719f4 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -675,7 +675,7 @@ CREATE TABLE crdb_internal.jobs ( last_run TIMESTAMP, next_run TIMESTAMP, num_runs INT, - execution_log JSON + execution_errors STRING[] )`, comment: `decoded job metadata from system.jobs (KV scan)`, generator: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { @@ -744,7 +744,7 @@ CREATE TABLE crdb_internal.jobs ( var jobType, description, statement, username, descriptorIDs, started, runningStatus, finished, modified, fractionCompleted, highWaterTimestamp, errorStr, coordinatorID, - traceID, lastRun, nextRun, numRuns, executionLog = tree.DNull, tree.DNull, tree.DNull, + traceID, lastRun, nextRun, numRuns, executionErrors = tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull @@ -846,12 +846,9 @@ CREATE TABLE crdb_internal.jobs ( if backoffIsEnabled { lastRun, numRuns, nextRun = r[7], r[8], r[9] - if payload != nil && payload.ExecutionLog != nil && len(payload.ExecutionLog) > 0 { - execLogJSON, err := jobspb.ExecutionLogToJSON(payload.ExecutionLog) - if err != nil { - return nil, err - } - executionLog = tree.NewDJSON(execLogJSON) + if payload != nil { + executionErrors = jobs. + FormatRetriableExecutionErrorLogToStringArray(ctx, payload) } } @@ -877,7 +874,7 @@ CREATE TABLE crdb_internal.jobs ( lastRun, nextRun, numRuns, - executionLog, + executionErrors, ) return container, nil } diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index 65408e854021..b48d22bea6c6 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -844,7 +844,7 @@ func TestInternalJobsTableRetryColumns(t *testing.T) { SELECT last_run IS NULL, next_run IS NOT NULL, num_runs = 0, - execution_log IS NULL + execution_errors IS NULL FROM crdb_internal.jobs WHERE job_id = 1`, [][]string{{"true", "true", "true", "true"}}) })) @@ -900,7 +900,7 @@ SELECT last_run IS NULL, SELECT last_run IS NULL, next_run IS NULL, num_runs IS NULL, - execution_log IS NULL + execution_errors IS NULL FROM crdb_internal.jobs WHERE job_id = 1`, [][]string{{"true", "true", "true", "true"}}) }) diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index 8b7dc4471b44..fa948a81efc5 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -557,7 +557,7 @@ func (r *createStatsResumer) Resume(ctx context.Context, execCtx interface{}) er ); err != nil { // Check if this was a context canceled error and restart if it was. if grpcutil.IsContextCanceled(err) { - return jobs.NewRetryJobError("node failure") + return jobs.MarkAsRetryJobError(err) } // We can't re-use the txn from above since it has a fixed timestamp set on diff --git a/pkg/sql/gcjob/gc_job.go b/pkg/sql/gcjob/gc_job.go index 405f6a706b4b..8589234319c0 100644 --- a/pkg/sql/gcjob/gc_job.go +++ b/pkg/sql/gcjob/gc_job.go @@ -89,7 +89,7 @@ func performGC( func (r schemaChangeGCResumer) Resume(ctx context.Context, execCtx interface{}) (err error) { defer func() { if err != nil && !r.isPermanentGCError(err) { - err = errors.Mark(err, jobs.NewRetryJobError("gc")) + err = jobs.MarkAsRetryJobError(err) } }() p := execCtx.(sql.JobExecContext) diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 80391008afce..7d9377814724 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -155,7 +155,7 @@ Channel query ITTTTTTTTTTTRTTIITTIT colnames SELECT * FROM crdb_internal.jobs WHERE false ---- -job_id job_type description statement user_name descriptor_ids status running_status created started finished modified fraction_completed high_water_timestamp error coordinator_id trace_id last_run next_run num_runs execution_log +job_id job_type description statement user_name descriptor_ids status running_status created started finished modified fraction_completed high_water_timestamp error coordinator_id trace_id last_run next_run num_runs execution_errors query IITTITTT colnames SELECT * FROM crdb_internal.schema_changes WHERE table_id < 0 diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant index 3bbcb97c8a53..c59d29340857 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant @@ -172,7 +172,7 @@ Channel query ITTTTTTTTTTTRTTIITTIT colnames SELECT * FROM crdb_internal.jobs WHERE false ---- -job_id job_type description statement user_name descriptor_ids status running_status created started finished modified fraction_completed high_water_timestamp error coordinator_id trace_id last_run next_run num_runs execution_log +job_id job_type description statement user_name descriptor_ids status running_status created started finished modified fraction_completed high_water_timestamp error coordinator_id trace_id last_run next_run num_runs execution_errors query IITTITTT colnames SELECT * FROM crdb_internal.schema_changes WHERE table_id < 0 diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index d9d63bedeadf..f6320b0b6bbc 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -520,7 +520,7 @@ CREATE TABLE crdb_internal.jobs ( last_run TIMESTAMP NULL, next_run TIMESTAMP NULL, num_runs INT8 NULL, - execution_log JSONB NULL + execution_errors STRING[] NULL ) CREATE TABLE crdb_internal.jobs ( job_id INT8 NULL, job_type STRING NULL, @@ -542,7 +542,7 @@ CREATE TABLE crdb_internal.jobs ( last_run TIMESTAMP NULL, next_run TIMESTAMP NULL, num_runs INT8 NULL, - execution_log JSONB NULL + execution_errors STRING[] NULL ) {} {} CREATE TABLE crdb_internal.kv_node_liveness ( node_id INT8 NOT NULL, diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 4b8a1f28c499..d3df17387336 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -2318,7 +2318,7 @@ func (r schemaChangeResumer) OnFailOrCancel(ctx context.Context, execCtx interfa case !IsPermanentSchemaChangeError(rollbackErr): // Check if the error is on a allowlist of errors we should retry on, and // have the job registry retry. - return jobs.NewRetryJobError(rollbackErr.Error()) + return jobs.MarkAsRetryJobError(rollbackErr) default: // All other errors lead to a failed job. // diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index c325a02c86bc..a07755619d47 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -6136,7 +6136,7 @@ func TestMultipleRevert(t *testing.T) { ranCancelCommand = true } // Keep returning a retryable error until the job was actually canceled. - return jobs.NewRetryJobError("retry until cancel") + return jobs.MarkAsRetryJobError(errors.New("retry until cancel")) }, RunBeforeOnFailOrCancel: func(_ jobspb.JobID) error { // Allow the backfill to proceed normally once the job was actually @@ -6152,7 +6152,7 @@ func TestMultipleRevert(t *testing.T) { } shouldRetryAfterReversingMutations = false // After cancelation, the job should get one more retryable error. - return jobs.NewRetryJobError("retry once after cancel") + return jobs.MarkAsRetryJobError(errors.New("retry once after cancel")) }, }, } @@ -6323,7 +6323,7 @@ func TestDropTableWhileSchemaChangeReverting(t *testing.T) { // Return a retry error, so that we can be sure to test the path where // the job is marked as failed by the DROP TABLE instead of running to // completion and ending up in the failed state on its own. - return jobs.NewRetryJobError("injected retry error") + return jobs.MarkAsRetryJobError(errors.New("injected retry error")) }, }, // Decrease the adopt loop interval so that retries happen quickly. diff --git a/pkg/sql/type_change.go b/pkg/sql/type_change.go index 18e240fa33dc..d2f5fc707c80 100644 --- a/pkg/sql/type_change.go +++ b/pkg/sql/type_change.go @@ -1321,7 +1321,7 @@ func (t *typeChangeResumer) OnFailOrCancel(ctx context.Context, execCtx interfac tc.typeID, ) case !IsPermanentSchemaChangeError(rollbackErr): - return jobs.NewRetryJobError(rollbackErr.Error()) + return jobs.MarkAsRetryJobError(rollbackErr) default: return rollbackErr }