From 67ad54f5939385d3ad9c9c0c8eade621020d525c Mon Sep 17 00:00:00 2001 From: David Taylor Date: Tue, 17 Aug 2021 17:34:49 +0000 Subject: [PATCH 1/4] storageccl: remove non-ReturnSST ExportRequest Release note: none. --- pkg/ccl/backupccl/BUILD.bazel | 1 + pkg/ccl/backupccl/backup_processor.go | 9 +- pkg/ccl/storageccl/BUILD.bazel | 2 - pkg/ccl/storageccl/export.go | 136 +++----------------------- 4 files changed, 23 insertions(+), 125 deletions(-) diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index a74f7981f065..f5fb879e3838 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -82,6 +82,7 @@ go_library( "//pkg/sql/roleoption", "//pkg/sql/rowenc", "//pkg/sql/rowexec", + "//pkg/sql/sem/builtins", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/sqlerrors", diff --git a/pkg/ccl/backupccl/backup_processor.go b/pkg/ccl/backupccl/backup_processor.go index c46754380672..956e445e31ef 100644 --- a/pkg/ccl/backupccl/backup_processor.go +++ b/pkg/ccl/backupccl/backup_processor.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/contextutil" @@ -606,7 +607,7 @@ func (s *sstSink) flushFile(ctx context.Context) error { } func (s *sstSink) open(ctx context.Context) error { - s.outName = storageccl.GenerateUniqueSSTName(s.conf.id) + s.outName = generateUniqueSSTName(s.conf.id) if s.ctx == nil { s.ctx, s.cancel = context.WithCancel(ctx) } @@ -714,6 +715,12 @@ func (s *sstSink) write(ctx context.Context, resp returnedSST) error { return nil } +func generateUniqueSSTName(nodeID base.SQLInstanceID) string { + // The data/ prefix, including a /, is intended to group SSTs in most of the + // common file/bucket browse UIs. + return fmt.Sprintf("data/%d.sst", builtins.GenerateUniqueInt(nodeID)) +} + func init() { rowexec.NewBackupDataProcessor = newBackupDataProcessor } diff --git a/pkg/ccl/storageccl/BUILD.bazel b/pkg/ccl/storageccl/BUILD.bazel index b9573eee0d78..6a860c3e30a5 100644 --- a/pkg/ccl/storageccl/BUILD.bazel +++ b/pkg/ccl/storageccl/BUILD.bazel @@ -23,11 +23,9 @@ go_library( "//pkg/roachpb:with-mocks", "//pkg/settings", "//pkg/settings/cluster", - "//pkg/sql/sem/builtins", "//pkg/storage", "//pkg/util/hlc", "//pkg/util/humanizeutil", - "//pkg/util/log", "//pkg/util/retry", "//pkg/util/tracing", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/ccl/storageccl/export.go b/pkg/ccl/storageccl/export.go index 8a9ff72e5665..48557f520308 100644 --- a/pkg/ccl/storageccl/export.go +++ b/pkg/ccl/storageccl/export.go @@ -9,23 +9,17 @@ package storageccl import ( - "bytes" "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/gogo/protobuf/types" @@ -63,8 +57,6 @@ var ExportRequestMaxAllowedFileSizeOverage = settings.RegisterByteSizeSetting( 64<<20, /* 64 MiB */ ).WithPublic() -const maxUploadRetries = 5 - func init() { batcheval.RegisterReadOnlyCommand(roachpb.Export, declareKeysExport, evalExport) } @@ -101,6 +93,14 @@ func evalExport( } evalExportSpan.RecordStructured(&evalExportTrace) + if !args.ReturnSST { + return result.Result{}, errors.New("ReturnSST is required") + } + + if args.Encryption != nil { + return result.Result{}, errors.New("returned SSTs cannot be encrypted") + } + // For MVCC_All backups with no start time, they'll only be capturing the // *revisions* since the gc threshold, so noting that in the reply allows the // BACKUP to correctly note the supported time bounds for RESTORE AS OF SYSTEM @@ -109,47 +109,6 @@ func evalExport( reply.StartTime = cArgs.EvalCtx.GetGCThreshold() } - makeExternalStorage := !args.ReturnSST || args.Storage != roachpb.ExternalStorage{} || - (args.StorageByLocalityKV != nil && len(args.StorageByLocalityKV) > 0) - if makeExternalStorage || log.V(1) { - log.Infof(ctx, "export [%s,%s)", args.Key, args.EndKey) - } else { - // Requests that don't write to export storage are expected to be small. - log.Eventf(ctx, "export [%s,%s)", args.Key, args.EndKey) - } - - if makeExternalStorage { - if _, ok := roachpb.TenantFromContext(ctx); ok { - if args.Storage.Provider == roachpb.ExternalStorageProvider_userfile { - return result.Result{}, errors.Errorf("requests to userfile on behalf of tenants must be made by the tenant's SQL process") - } - } - } - - // To get the store to export to, first try to match the locality of this node - // to the locality KVs in args.StorageByLocalityKV (used for partitioned - // backups). If that map isn't set or there's no match, fall back to - // args.Storage. - var localityKV string - var exportStore cloud.ExternalStorage - if makeExternalStorage { - var storeConf roachpb.ExternalStorage - var err error - foundStoreByLocality := false - if args.StorageByLocalityKV != nil && len(args.StorageByLocalityKV) > 0 { - locality := cArgs.EvalCtx.GetNodeLocality() - localityKV, storeConf, foundStoreByLocality = getMatchingStore(&locality, args.StorageByLocalityKV) - } - if !foundStoreByLocality { - storeConf = args.Storage - } - exportStore, err = cArgs.EvalCtx.GetExternalStorage(ctx, storeConf) - if err != nil { - return result.Result{}, err - } - defer exportStore.Close() - } - var exportAllRevisions bool switch args.MVCCFilter { case roachpb.MVCCFilter_Latest: @@ -181,11 +140,9 @@ func evalExport( // Only use resume timestamp if splitting mid key is enabled. resumeKeyTS := hlc.Timestamp{} if args.SplitMidKey { - if !args.ReturnSST { - return result.Result{}, errors.New("SplitMidKey could only be used with ReturnSST option") - } resumeKeyTS = args.ResumeKeyTS } + var curSizeOfExportedSSTs int64 for start := args.Key; start != nil; { destFile := &storage.MemFile{} @@ -214,57 +171,16 @@ func evalExport( span.EndKey = args.EndKey } exported := roachpb.ExportResponse_File{ - Span: span, - EndKeyTS: resumeTS, - Exported: summary, - LocalityKV: localityKV, - } - - returnSST := args.ReturnSST - if args.ReturnSstBelowSize > 0 && len(data) < int(args.ReturnSstBelowSize) { - returnSST = true - } - - if returnSST { - exported.SST = data - } else { - if args.Encryption != nil { - data, err = EncryptFile(data, args.Encryption.Key) - if err != nil { - return result.Result{}, err - } - } - - exported.Path = GenerateUniqueSSTName(base.SQLInstanceID(cArgs.EvalCtx.NodeID())) - var attemptNum int - if err := retry.WithMaxAttempts(ctx, base.DefaultRetryOptions(), maxUploadRetries, func() error { - attemptNum++ - retryTracingEvent := roachpb.RetryTracingEvent{ - Operation: fmt.Sprintf("%s.ExportRequest.WriteFile", exportStore.Conf().Provider.String()), - AttemptNumber: int32(attemptNum), - } - // We blindly retry any error here because we expect the caller to have - // verified the target is writable before sending ExportRequests for it. - if err := cloud.WriteFile(ctx, exportStore, exported.Path, bytes.NewReader(data)); err != nil { - log.VEventf(ctx, 1, "failed to put file: %+v", err) - retryTracingEvent.RetryError = fmt.Sprintf("failed to put file: %s", tracing.RedactAndTruncateError(err)) - evalExportSpan.RecordStructured(&retryTracingEvent) - return err - } - evalExportSpan.RecordStructured(&retryTracingEvent) - return nil - }); err != nil { - return result.Result{}, err - } + Span: span, + EndKeyTS: resumeTS, + Exported: summary, + SST: data, } reply.Files = append(reply.Files, exported) start = resume resumeKeyTS = resumeTS - // If we are not returning the SSTs to the processor, there is no need to - // paginate the ExportRequest since the reply size will not grow large - // enough to cause an OOM. - if args.ReturnSST && h.TargetBytes > 0 { + if h.TargetBytes > 0 { curSizeOfExportedSSTs += summary.DataSize // There could be a situation where the size of exported SSTs is larger // than the TargetBytes. In such a scenario, we want to report back @@ -307,27 +223,3 @@ func evalExport( return result.Result{}, nil } - -func getMatchingStore( - locality *roachpb.Locality, storageByLocalityKV map[string]*roachpb.ExternalStorage, -) (string, roachpb.ExternalStorage, bool) { - kvs := locality.Tiers - // When matching, more specific KVs in the node locality take precedence - // over less specific ones. - for i := len(kvs) - 1; i >= 0; i-- { - if store, ok := storageByLocalityKV[kvs[i].String()]; ok { - return kvs[i].String(), *store, true - } - } - return "", roachpb.ExternalStorage{}, false -} - -// GenerateUniqueSSTName generates a name for a backup SST that will not collide -// with another name generated by this node or another node. -func GenerateUniqueSSTName(nodeID base.SQLInstanceID) string { - // The data/ prefix, including a /, is intended to group SSTs in most of the - // common file/bucket browse UIs. - // TODO(dt): don't reach out into a SQL builtin here; this code lives in KV. - // Create a unique int differently. - return fmt.Sprintf("data/%d.sst", builtins.GenerateUniqueInt(nodeID)) -} From a452dbccd41913437e03b64981329196a29f581d Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Mon, 23 Aug 2021 15:26:27 +0000 Subject: [PATCH 2/4] roachtest: move roachtest stress CI job instructions to README Release justification: non-production code changes Release note: None --- pkg/cmd/roachtest/README.md | 12 ++++++++++++ pkg/cmd/roachtest/test_runner.go | 12 +----------- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/pkg/cmd/roachtest/README.md b/pkg/cmd/roachtest/README.md index e67545ffb8ec..abf7178582f9 100644 --- a/pkg/cmd/roachtest/README.md +++ b/pkg/cmd/roachtest/README.md @@ -185,6 +185,8 @@ The HTTP endpoint (by default `:8080`) is useful to find the "run" numbers for failing tests (to find the artifacts) and to get a general overview of the progress of the invocation. +### Stressing a roachtest + A solid foundation for building the binaries and stressing a roachtest is provided via the [roachstress.sh] script, which can either be used outright or saved and adjusted. The script can be invoked without parameters from a clean @@ -192,3 +194,13 @@ checkout of the cockroach repository at the revision to be tested. It will prompt for user input on which test to stress. [roachstress.sh]: https://github.com/cockroachdb/cockroach/blob/master/pkg/cmd/roachtest/roachstress.sh + +Another option is to start a [`Cockroach_Nightlies_RoachtestStress`](https://teamcity.cockroachdb.com/buildConfiguration/Cockroach_Nightlies_RoachtestStress) +CI job, which allows running a bunch of tests without having to keep your +laptop online. The CI job is run as follows: + +1. Go to https://teamcity.cockroachdb.com/buildConfiguration/Cockroach_Nightlies_RoachtestStress +2. Click the ellipsis (...) next to the Run button and fill in: + * Changes → Build branch: `` + * Parameters → `env.TESTS`: `^$` + * Parameters → `env.COUNT`: `` \ No newline at end of file diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 7ecf87786915..dbd3e7babee2 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -946,18 +946,8 @@ func (r *testRunner) maybePostGithubIssue( ReproductionCommand: func(renderer *issues.Renderer) { issues.ReproductionAsLink( "roachtest README", - "https://github.com/cockroachdb/cockroach/tree/master/pkg/cmd/roachtest", + "https://github.com/cockroachdb/cockroach/blob/master/pkg/cmd/roachtest/README.md", )(renderer) - issues.ReproductionAsLink( - "CI job to stress roachtests", - "https://teamcity.cockroachdb.com/buildConfiguration/Cockroach_Nightlies_RoachtestStress", - )(renderer) - renderer.P(func() { - renderer.Escaped("For the CI stress job, click the ellipsis (...) next to the Run button and fill in:\n") - renderer.Escaped(fmt.Sprintf("* Changes / Build branch: %s\n", branch)) - renderer.Escaped(fmt.Sprintf("* Parameters / `env.TESTS`: `^%s$`\n", t.Name())) - renderer.Escaped("* Parameters / `env.COUNT`: \n") - }) }, } if err := issues.Post( From 3ccdba8e6839bc301ed56aa30457af91b88d1330 Mon Sep 17 00:00:00 2001 From: Sajjad Rizvi Date: Sat, 14 Aug 2021 12:21:03 -0400 Subject: [PATCH 3/4] sql: add jobs backoff state columns in jobs internal table This commit adds new columns in `crdb_internal.jobs` table, which show the current exponential-backoff state of a job and its execution history. Release justification: This commit adds low-risk updates to new functionality. Jobs subsystem now supports job retries with exponential-backoff. We want to give users more insights about the backoff state of jobs and jobs' lifecycles through additional columns in `crdb_internal.jobs` table. Release note (general change): The functionality to retry failed jobs with exponential-backoff has introduced recently in the system. This commit adds new columns in `crdb_internal.jobs` table, which show the current backoff-state of a job and its execution log. The execution log consists of a sequence of job start and end events and any associated errors that were encountered during the job's each execution. Now users can query internal jobs table to get more insights about jobs through the following columns: (a) `last_run` shows the last execution time of a job, (b) `next_run` shows the next execution time of a job based on exponential-backoff delay, (c) `num_runs` shows the number of times the job has been executed, and (d) `execution_log` provides a set of events that are generated when a job starts and ends its execution. --- pkg/jobs/adopt.go | 44 +- pkg/jobs/jobs_test.go | 38 + pkg/jobs/jobspb/BUILD.bazel | 7 +- pkg/jobs/jobspb/jobs.pb.go | 1103 +++++++++++------ pkg/jobs/jobspb/jobs.proto | 46 +- pkg/jobs/jobspb/json_encoding.go | 32 + pkg/jobs/registry.go | 18 + pkg/sql/crdb_internal.go | 46 +- pkg/sql/crdb_internal_test.go | 102 ++ .../testdata/logic_test/crdb_internal | 4 +- .../testdata/logic_test/crdb_internal_tenant | 4 +- .../testdata/logic_test/create_statements | 12 +- pkg/sql/opt/exec/execbuilder/testdata/explain | 2 +- 13 files changed, 1067 insertions(+), 391 deletions(-) create mode 100644 pkg/jobs/jobspb/json_encoding.go diff --git a/pkg/jobs/adopt.go b/pkg/jobs/adopt.go index dc890868451d..057ead3a5dd0 100644 --- a/pkg/jobs/adopt.go +++ b/pkg/jobs/adopt.go @@ -119,17 +119,19 @@ const ( // canRunArgs are used in canRunClause, which specify whether a job can be // run now or not. - canRunArgs = `(SELECT $3::TIMESTAMP AS ts, $4::FLOAT AS initial_delay, $5::FLOAT AS max_delay) args` - canRunClause = ` -args.ts >= COALESCE(last_run, created) + least( - IF( - args.initial_delay * (power(2, least(62, COALESCE(num_runs, 0))) - 1)::FLOAT >= 0.0, - args.initial_delay * (power(2, least(62, COALESCE(num_runs, 0))) - 1)::FLOAT, - args.max_delay - ), - args.max_delay -)::INTERVAL -` + canRunArgs = `(SELECT $3::TIMESTAMP AS ts, $4::FLOAT AS initial_delay, $5::FLOAT AS max_delay) args` + // NextRunClause calculates the next execution time of a job with exponential backoff delay, calculated + // using last_run and num_runs values. + NextRunClause = ` +COALESCE(last_run, created) + least( + IF( + args.initial_delay * (power(2, least(62, COALESCE(num_runs, 0))) - 1)::FLOAT >= 0.0, + args.initial_delay * (power(2, least(62, COALESCE(num_runs, 0))) - 1)::FLOAT, + args.max_delay + ), + args.max_delay +)::INTERVAL` + canRunClause = `args.ts >= ` + NextRunClause // processQueryBase and processQueryWhereBase select IDs of the jobs that // can be processed among the claimed jobs. processQueryBase = `SELECT id FROM system.jobs` @@ -158,15 +160,7 @@ func getProcessQuery( if r.settings.Version.IsActive(ctx, clusterversion.RetryJobsWithExponentialBackoff) { // Select only those jobs that can be executed right now. query = processQueryWithBackoff - initDelay := retryInitialDelaySetting.Get(&r.settings.SV).Seconds() - maxDelay := retryMaxDelaySetting.Get(&r.settings.SV).Seconds() - if r.knobs.IntervalOverrides.RetryInitialDelay != nil { - initDelay = r.knobs.IntervalOverrides.RetryInitialDelay.Seconds() - } - if r.knobs.IntervalOverrides.RetryMaxDelay != nil { - maxDelay = r.knobs.IntervalOverrides.RetryMaxDelay.Seconds() - } - args = append(args, r.clock.Now().GoTime(), initDelay, maxDelay) + args = append(args, r.clock.Now().GoTime(), r.RetryInitialDelay(), r.RetryMaxDelay()) } return query, args } @@ -256,15 +250,7 @@ func (r *Registry) resumeJob(ctx context.Context, jobID jobspb.JobID, s sqlliven backoffIsActive := r.settings.Version.IsActive(ctx, clusterversion.RetryJobsWithExponentialBackoff) if backoffIsActive { resumeQuery = resumeQueryWithBackoff - initDelay := retryInitialDelaySetting.Get(&r.settings.SV).Seconds() - maxDelay := retryMaxDelaySetting.Get(&r.settings.SV).Seconds() - if r.knobs.IntervalOverrides.RetryInitialDelay != nil { - initDelay = r.knobs.IntervalOverrides.RetryInitialDelay.Seconds() - } - if r.knobs.IntervalOverrides.RetryMaxDelay != nil { - maxDelay = r.knobs.IntervalOverrides.RetryMaxDelay.Seconds() - } - args = append(args, r.clock.Now().GoTime(), initDelay, maxDelay) + args = append(args, r.clock.Now().GoTime(), r.RetryInitialDelay(), r.RetryMaxDelay()) } row, err := r.ex.QueryRowEx( ctx, "get-job-row", nil, diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index baa90f3ced94..871dc74153d7 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -3257,3 +3257,41 @@ 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 b44419671c22..63d842164ddc 100644 --- a/pkg/jobs/jobspb/BUILD.bazel +++ b/pkg/jobs/jobspb/BUILD.bazel @@ -4,13 +4,18 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "jobspb", - srcs = ["wrap.go"], + srcs = [ + "json_encoding.go", + "wrap.go", + ], embed = [":jobspb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/jobs/jobspb", visibility = ["//visibility:public"], deps = [ "//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 3c5f82bca2bd..9e2aabcdf135 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -270,6 +270,39 @@ 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. @@ -2277,6 +2310,55 @@ 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 Payload struct { Description string `protobuf:"bytes,1,opt,name=description,proto3" json:"description,omitempty"` // If empty, the description is assumed to be the statement. @@ -2319,13 +2401,21 @@ 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"` } 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{34} + return fileDescriptor_6c315f3a2536c4ef, []int{35} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2543,7 +2633,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{35} + return fileDescriptor_6c315f3a2536c4ef, []int{36} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2780,7 +2870,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{36} + return fileDescriptor_6c315f3a2536c4ef, []int{37} } func (m *Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2813,6 +2903,7 @@ 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") @@ -2874,6 +2965,7 @@ 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((*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") @@ -2882,351 +2974,361 @@ func init() { func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_6c315f3a2536c4ef) } var fileDescriptor_6c315f3a2536c4ef = []byte{ - // 5499 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x7c, 0x4d, 0x6c, 0x23, 0xc9, - 0x75, 0xbf, 0x9a, 0xa2, 0xf8, 0xf1, 0x24, 0x52, 0xcd, 0x92, 0x46, 0xc3, 0xa5, 0x67, 0x87, 0x32, - 0x77, 0x66, 0x77, 0x66, 0xed, 0xa5, 0xec, 0x59, 0x7f, 0xac, 0xc7, 0xde, 0xd9, 0xe5, 0x97, 0x24, - 0xea, 0x7b, 0x9b, 0xd4, 0xec, 0xd7, 0x7f, 0xdd, 0xff, 0x66, 0x77, 0x49, 0xea, 0x88, 0xec, 0xe6, - 0x74, 0x35, 0x67, 0x46, 0x0e, 0x60, 0x18, 0x76, 0x0c, 0x04, 0x73, 0x4a, 0x80, 0x24, 0x97, 0x64, - 0x80, 0x00, 0xb6, 0x01, 0x1f, 0x62, 0x04, 0x31, 0x02, 0x27, 0xc8, 0x29, 0x40, 0x72, 0xf0, 0x21, - 0x01, 0x0c, 0x04, 0x41, 0x9c, 0x1c, 0x18, 0x47, 0xbe, 0xe4, 0x18, 0x24, 0x40, 0x0e, 0x73, 0x0a, - 0xea, 0xa3, 0x9b, 0xdd, 0x94, 0x44, 0x51, 0x9a, 0xb1, 0x7d, 0xd1, 0x74, 0xbd, 0xaa, 0xfa, 0xbd, - 0xaa, 0x57, 0xaf, 0xde, 0x7b, 0xf5, 0xaa, 0x38, 0xb0, 0xf0, 0x5b, 0x76, 0x8b, 0x2c, 0xd1, 0x3f, - 0xdd, 0x16, 0xfb, 0xa7, 0xd8, 0x75, 0x6c, 0xd7, 0x46, 0x2f, 0xe9, 0xb6, 0x7e, 0xe8, 0xd8, 0x9a, - 0x7e, 0x50, 0x24, 0x0f, 0xda, 0x45, 0x56, 0xc3, 0x5b, 0xe5, 0xae, 0x60, 0xc7, 0xb1, 0x1d, 0xda, - 0x9e, 0x7f, 0xf0, 0x1e, 0xb9, 0xf9, 0x7d, 0x7b, 0xdf, 0x66, 0x9f, 0x4b, 0xf4, 0x4b, 0x50, 0x11, - 0xc3, 0xe8, 0xb6, 0x96, 0x0c, 0xcd, 0xd5, 0x04, 0x2d, 0xeb, 0xd1, 0x4c, 0xfb, 0x8d, 0x3d, 0xdb, - 0xe9, 0x68, 0xae, 0x87, 0xf1, 0x0a, 0x79, 0xd0, 0x5e, 0xd2, 0x35, 0x57, 0x6b, 0xdb, 0xfb, 0x4b, - 0x06, 0x26, 0x7a, 0xb7, 0xb5, 0x44, 0x5c, 0xa7, 0xa7, 0xbb, 0x3d, 0x07, 0x1b, 0xa2, 0x51, 0xfe, - 0x94, 0x46, 0x2e, 0xb6, 0x34, 0xcb, 0xf5, 0xf0, 0x7b, 0xae, 0xd9, 0x5e, 0x3a, 0x68, 0xeb, 0x4b, - 0xae, 0xd9, 0xc1, 0xc4, 0xd5, 0x3a, 0x5d, 0x51, 0xf3, 0x69, 0xda, 0x95, 0xe8, 0x07, 0xb8, 0xa3, - 0xe9, 0x07, 0x9a, 0xb5, 0x8f, 0x9d, 0x25, 0xce, 0x43, 0xef, 0xb6, 0x44, 0x93, 0x1b, 0x7a, 0xbb, - 0x47, 0x5c, 0xec, 0x3c, 0xc4, 0x0e, 0x31, 0x6d, 0x6b, 0x49, 0x14, 0x55, 0x51, 0xe6, 0xad, 0x0a, - 0x3f, 0x88, 0xc0, 0xd5, 0xb2, 0xa6, 0x1f, 0xf6, 0xba, 0x35, 0x4b, 0x77, 0x8e, 0xba, 0xae, 0x69, - 0x5b, 0xdb, 0xec, 0x2f, 0x41, 0x32, 0x4c, 0x1e, 0xe2, 0xa3, 0xac, 0xb4, 0x28, 0xdd, 0x9a, 0x51, - 0xe8, 0x27, 0x7a, 0x1b, 0xa2, 0x1d, 0xdb, 0xc0, 0xd9, 0xc8, 0xa2, 0x74, 0x2b, 0x7d, 0xe7, 0x76, - 0xf1, 0x4c, 0xd9, 0x16, 0x07, 0x68, 0x9b, 0xb6, 0x81, 0x15, 0xd6, 0x0d, 0xb5, 0x20, 0x71, 0xd8, - 0x21, 0xaa, 0x69, 0xed, 0xd9, 0xd9, 0xc9, 0x45, 0xe9, 0xd6, 0xf4, 0x9d, 0xbb, 0x23, 0x20, 0xce, - 0x18, 0x56, 0x71, 0x7d, 0xb3, 0x51, 0xb7, 0xf6, 0xec, 0xf2, 0xf4, 0x71, 0x3f, 0x1f, 0x17, 0x05, - 0x25, 0x7e, 0xd8, 0x21, 0xf4, 0x23, 0xb7, 0x0d, 0x1e, 0x8d, 0x8e, 0xbf, 0xe7, 0x98, 0x6c, 0xfc, - 0x49, 0x85, 0x7e, 0xa2, 0xcf, 0x02, 0xc2, 0x1c, 0x0f, 0x1b, 0x2a, 0x5d, 0x48, 0x95, 0x4e, 0x30, - 0xc2, 0x26, 0x28, 0xfb, 0x35, 0x55, 0xcd, 0xd5, 0xd6, 0xf1, 0xd1, 0xdd, 0xe8, 0x7f, 0xfe, 0x69, - 0x5e, 0xe2, 0x7f, 0x0b, 0xdf, 0x9a, 0x84, 0xf4, 0x60, 0x28, 0x0c, 0x7e, 0x15, 0x62, 0x6c, 0x05, - 0x30, 0xe3, 0x90, 0xbe, 0xf3, 0xb9, 0xb1, 0xc4, 0x41, 0xbb, 0x16, 0x1b, 0xac, 0x9f, 0x22, 0xfa, - 0x23, 0x04, 0x51, 0xa2, 0xb5, 0x5d, 0x31, 0x10, 0xf6, 0x8d, 0xfe, 0x58, 0x82, 0xc5, 0xe1, 0x11, - 0x95, 0x8f, 0xd6, 0x37, 0x1b, 0x9b, 0x1a, 0x5d, 0xc6, 0x75, 0x7c, 0x54, 0xaf, 0x66, 0x27, 0x17, - 0x27, 0x6f, 0x4d, 0xdf, 0xd9, 0x1e, 0x9f, 0x71, 0xed, 0x1c, 0xc4, 0x9a, 0xe5, 0x3a, 0x47, 0xca, - 0xb9, 0x8c, 0x73, 0x0d, 0xb8, 0x39, 0x16, 0x54, 0x50, 0x87, 0x92, 0x5c, 0x87, 0xe6, 0x61, 0xea, - 0xa1, 0xd6, 0xee, 0x61, 0x31, 0x5b, 0x5e, 0xb8, 0x1b, 0x79, 0x4b, 0x2a, 0x5c, 0x85, 0x18, 0x17, - 0x0c, 0x4a, 0x41, 0xb2, 0x54, 0x6b, 0xdc, 0xf9, 0xe2, 0x97, 0x56, 0x2a, 0x9b, 0xf2, 0x84, 0x58, - 0x82, 0x9f, 0x48, 0xb0, 0xd0, 0x70, 0x1d, 0xac, 0x75, 0xea, 0xd6, 0x3e, 0x26, 0x74, 0x4e, 0x55, - 0xec, 0x6a, 0x66, 0x9b, 0xa0, 0x9b, 0x90, 0x26, 0xac, 0x46, 0xd5, 0x0c, 0xc3, 0xc1, 0x84, 0x08, - 0x86, 0x29, 0x4e, 0x2d, 0x71, 0x22, 0xfa, 0x3c, 0x44, 0x49, 0x57, 0xb3, 0x18, 0xe7, 0xe9, 0x3b, - 0x57, 0x03, 0x62, 0x13, 0x1b, 0xb9, 0xd8, 0xe8, 0x6a, 0x56, 0x39, 0xfa, 0xd3, 0x7e, 0x7e, 0x42, - 0x61, 0x4d, 0x51, 0x19, 0x80, 0xb8, 0x9a, 0xe3, 0xaa, 0x74, 0x07, 0x0a, 0xa5, 0x7d, 0x39, 0xd0, - 0x91, 0xee, 0xd0, 0xe2, 0x41, 0x5b, 0x2f, 0x36, 0xbd, 0x1d, 0x2a, 0xba, 0x27, 0x59, 0x37, 0x4a, - 0x2d, 0xfc, 0xcb, 0x24, 0x5c, 0x1d, 0x1a, 0xf8, 0x8e, 0x63, 0xef, 0xb3, 0x21, 0x2d, 0xc3, 0x8c, - 0xde, 0x73, 0xed, 0x87, 0xd8, 0xe1, 0x1c, 0xa4, 0xf1, 0x39, 0x4c, 0x8b, 0x8e, 0x94, 0x8e, 0xbe, - 0x09, 0xa8, 0xab, 0x39, 0xae, 0x49, 0xc1, 0xd5, 0xae, 0x40, 0xcf, 0x46, 0x98, 0x7e, 0xd4, 0x47, - 0xe8, 0xc7, 0x19, 0xe3, 0x2a, 0xee, 0x78, 0x60, 0x1e, 0x85, 0x2d, 0xa7, 0xe0, 0x9c, 0xe9, 0x0e, - 0xd7, 0xe6, 0xf6, 0x21, 0x73, 0xa2, 0x0b, 0x52, 0x00, 0x99, 0x0c, 0x19, 0x1b, 0xaa, 0x6f, 0xc1, - 0x2e, 0x32, 0xc5, 0x8c, 0xd7, 0xdd, 0xaf, 0xc8, 0x3d, 0x91, 0x60, 0xe1, 0xf4, 0xc1, 0x9d, 0xa2, - 0x6b, 0x1f, 0x05, 0x75, 0x6d, 0xfa, 0x4e, 0xf5, 0x45, 0x08, 0x22, 0xa8, 0xb1, 0x3f, 0x8a, 0xc0, - 0x4b, 0x54, 0x65, 0x8d, 0x5e, 0x1b, 0xef, 0x34, 0x1b, 0x95, 0x03, 0xcd, 0xb4, 0x4c, 0x6b, 0x5f, - 0xc1, 0xba, 0xed, 0x18, 0xe8, 0xf7, 0x25, 0xc8, 0x51, 0x2b, 0x8b, 0xf5, 0x90, 0x00, 0x54, 0x87, - 0x55, 0x73, 0xbb, 0x5a, 0x6e, 0xfc, 0x5b, 0x3f, 0xff, 0xe6, 0xbe, 0xe9, 0x1e, 0xf4, 0x5a, 0x45, - 0xdd, 0xee, 0x2c, 0xf9, 0x23, 0x34, 0x5a, 0x83, 0xef, 0xa5, 0xee, 0xe1, 0xfe, 0x12, 0x73, 0x06, - 0xbd, 0x9e, 0x69, 0x14, 0x77, 0x77, 0xeb, 0xd5, 0xe3, 0x7e, 0x3e, 0xbb, 0xe3, 0x81, 0xfb, 0xe2, - 0xe1, 0x9c, 0x95, 0x6c, 0xf7, 0x8c, 0x1a, 0x74, 0x1f, 0x62, 0x9a, 0x4e, 0xa7, 0x23, 0x6c, 0xf8, - 0xbd, 0x51, 0x22, 0x39, 0x6b, 0x66, 0xc5, 0x9d, 0x66, 0xa3, 0xc4, 0x50, 0x14, 0x81, 0x56, 0xb8, - 0x01, 0x49, 0x9f, 0x88, 0x00, 0x62, 0xbb, 0x3b, 0xd5, 0x52, 0xb3, 0x26, 0x4f, 0xa0, 0x69, 0x88, - 0x2b, 0xb5, 0x8d, 0x5a, 0xa9, 0x51, 0x93, 0xa5, 0xc2, 0x3f, 0xc5, 0x21, 0xc5, 0xcd, 0xba, 0xb7, - 0x73, 0xc3, 0xfb, 0x4b, 0xba, 0xcc, 0xfe, 0x42, 0xf7, 0x20, 0x81, 0x2d, 0x2e, 0x60, 0xb1, 0xd0, - 0x63, 0x21, 0xc4, 0xb1, 0xc5, 0xc4, 0x83, 0x5e, 0xe2, 0x7e, 0x82, 0x6e, 0xee, 0x64, 0x39, 0x7e, - 0xdc, 0xcf, 0x4f, 0xee, 0x2a, 0x75, 0xee, 0x30, 0xbe, 0x23, 0xc1, 0x5c, 0xcf, 0x31, 0x89, 0xda, - 0x3a, 0x52, 0xdb, 0xb6, 0xae, 0xb5, 0x4d, 0xf7, 0x48, 0x3d, 0x7c, 0x98, 0x9d, 0x62, 0x1b, 0xeb, - 0xde, 0xb9, 0xde, 0x4b, 0x4c, 0xb3, 0xb8, 0xeb, 0x98, 0xa4, 0x7c, 0xb4, 0x21, 0x10, 0xd6, 0x1f, - 0xf2, 0xdd, 0x34, 0x7f, 0xdc, 0xcf, 0xcb, 0xbb, 0x4a, 0x3d, 0x58, 0x75, 0x5f, 0x91, 0x7b, 0x43, - 0x8d, 0xd1, 0xd7, 0x20, 0x67, 0xe0, 0xae, 0x83, 0x75, 0x8d, 0x2a, 0x52, 0x8b, 0x21, 0xab, 0x1d, - 0xcd, 0x32, 0xf7, 0x30, 0x71, 0xb3, 0x51, 0x66, 0x47, 0xb3, 0x83, 0x16, 0x9c, 0xf5, 0xa6, 0xa8, - 0x47, 0x9a, 0xef, 0xf4, 0xa8, 0x6d, 0xb0, 0xb9, 0x17, 0xcd, 0xc6, 0x98, 0xa0, 0xee, 0x5c, 0xdc, - 0xff, 0x2a, 0x19, 0x7c, 0x22, 0x52, 0x50, 0x60, 0x36, 0xc0, 0x82, 0xf9, 0xf7, 0x24, 0xc3, 0xbf, - 0x3d, 0xb6, 0x6b, 0x52, 0xd2, 0x38, 0xec, 0x5e, 0xcf, 0xd9, 0x3d, 0xf1, 0xdf, 0xc4, 0xee, 0x79, - 0x0b, 0xd2, 0xba, 0xdd, 0x6e, 0x63, 0xa6, 0xe6, 0xea, 0xae, 0x52, 0xcf, 0x26, 0x98, 0xd2, 0x64, - 0x8e, 0xfb, 0xf9, 0x54, 0xc5, 0xaf, 0xa1, 0xea, 0x93, 0xd2, 0x83, 0x45, 0xf4, 0x07, 0x12, 0x5c, - 0x23, 0x62, 0x3f, 0xa9, 0x5d, 0x97, 0xa8, 0xba, 0xd8, 0x51, 0xde, 0x7c, 0x80, 0xc9, 0xeb, 0x0b, - 0x97, 0xd9, 0x8e, 0xe5, 0x97, 0x8f, 0xfb, 0xf9, 0xb3, 0xed, 0x90, 0xf2, 0x92, 0xc7, 0x78, 0xc7, - 0x25, 0xe1, 0xaa, 0x5c, 0x05, 0xae, 0x9c, 0xaa, 0x9a, 0xe7, 0xf9, 0xed, 0x64, 0xd0, 0x0a, 0xca, - 0x90, 0xe6, 0xba, 0xe2, 0x99, 0xc8, 0xc2, 0xdf, 0x2c, 0x40, 0x5a, 0xc1, 0xc4, 0xb5, 0x1d, 0xec, - 0x6d, 0xf4, 0xe0, 0x26, 0x8d, 0x5e, 0x62, 0x93, 0xfe, 0x58, 0x82, 0x39, 0x1a, 0x23, 0x3b, 0x66, - 0xd7, 0xb5, 0x1d, 0xd5, 0xc1, 0x8f, 0x1c, 0xd3, 0xc5, 0x9e, 0x8b, 0x2b, 0x8d, 0x90, 0x5b, 0x78, - 0x20, 0xc5, 0xaa, 0x0f, 0xa2, 0x08, 0x0c, 0xbe, 0x19, 0xef, 0x7d, 0xfb, 0xdf, 0xf3, 0x77, 0xc7, - 0x52, 0xa5, 0x93, 0x61, 0x7b, 0xb1, 0x5e, 0x55, 0x90, 0x71, 0x02, 0x18, 0x5d, 0x83, 0x28, 0xdd, - 0xcc, 0x2c, 0x4e, 0x4b, 0x96, 0x13, 0xc7, 0xfd, 0x7c, 0x94, 0x6e, 0x77, 0x85, 0x51, 0x91, 0x0b, - 0xf3, 0x62, 0x2f, 0xfb, 0xa6, 0x85, 0x6d, 0x9d, 0x38, 0x9b, 0xd2, 0xd7, 0xc6, 0x9f, 0x12, 0x97, - 0xbe, 0xb7, 0x84, 0x2c, 0x38, 0xe6, 0xd2, 0x43, 0xad, 0x13, 0x35, 0x68, 0x07, 0xd2, 0x34, 0xf2, - 0x6d, 0x69, 0x04, 0xab, 0x74, 0xc8, 0x24, 0x2b, 0x33, 0x7e, 0xc3, 0x5b, 0x95, 0x3c, 0x68, 0xd3, - 0x36, 0xc5, 0xaa, 0x68, 0x1c, 0x90, 0x5b, 0xca, 0x08, 0xd0, 0x08, 0x5a, 0x81, 0x69, 0x57, 0x6b, - 0xb5, 0x3d, 0x38, 0x6e, 0x1b, 0x5f, 0x3d, 0x03, 0xae, 0x49, 0x5b, 0x06, 0xb0, 0xc0, 0xf5, 0x08, - 0x04, 0x55, 0x01, 0xdc, 0xa3, 0xae, 0x87, 0x93, 0x66, 0x38, 0x37, 0xcf, 0xc2, 0x39, 0xea, 0x06, - 0x61, 0x92, 0xae, 0x28, 0x13, 0xb4, 0x06, 0x33, 0xfc, 0x64, 0x24, 0x70, 0x66, 0x19, 0xce, 0x6b, - 0x67, 0xe0, 0xb0, 0x88, 0x53, 0x0b, 0x20, 0x4d, 0x13, 0x9f, 0x42, 0x50, 0x09, 0xe2, 0xfc, 0x44, - 0x46, 0xb2, 0x29, 0x06, 0xf3, 0xe9, 0xb3, 0x86, 0xc3, 0x5a, 0x05, 0x44, 0xef, 0xf5, 0x43, 0x4b, - 0x30, 0x4d, 0xa3, 0x34, 0xc7, 0x34, 0xb0, 0x6a, 0xb4, 0x98, 0xdd, 0x4d, 0x96, 0xd3, 0xc7, 0xfd, - 0x3c, 0x6c, 0x0b, 0x72, 0xb5, 0xac, 0x80, 0xd7, 0xa4, 0xda, 0x42, 0x9f, 0x81, 0x4c, 0xd7, 0xc1, - 0x5d, 0xcd, 0xc1, 0xaa, 0x6e, 0x77, 0xba, 0x6d, 0xec, 0x62, 0x83, 0xd9, 0x99, 0x84, 0x22, 0x8b, - 0x8a, 0x8a, 0x47, 0xe7, 0x91, 0xaf, 0xe6, 0xd2, 0x43, 0x15, 0xc1, 0x0e, 0x6d, 0x99, 0x64, 0x2d, - 0x53, 0x8c, 0x5a, 0x17, 0x44, 0x74, 0x04, 0x0b, 0xe4, 0x88, 0xb8, 0xb8, 0xa3, 0x32, 0x71, 0x13, - 0xb5, 0x63, 0xee, 0x3b, 0xd4, 0x57, 0x64, 0x33, 0x6c, 0x5a, 0x95, 0xf1, 0x95, 0xad, 0xc1, 0x70, - 0xd8, 0x32, 0x92, 0x4d, 0x81, 0xc2, 0x8f, 0x0d, 0xf3, 0xe4, 0x94, 0x2a, 0xf4, 0x26, 0x5c, 0x19, - 0xec, 0x0c, 0xa2, 0x76, 0x7b, 0xad, 0xb6, 0x49, 0x0e, 0x30, 0xb7, 0x78, 0x09, 0x65, 0x3e, 0x50, - 0xb9, 0xe3, 0xd5, 0xa1, 0xa3, 0xd0, 0x66, 0xd7, 0xa9, 0x74, 0xb4, 0x7d, 0x9c, 0x9d, 0x5e, 0x94, - 0x6e, 0x4d, 0x95, 0x57, 0x9f, 0xf5, 0xf3, 0xd5, 0xb1, 0x77, 0x2a, 0xc1, 0x9d, 0x25, 0xd7, 0xc1, - 0x38, 0xb0, 0xf1, 0x2b, 0x02, 0x2f, 0xb8, 0x67, 0x3d, 0x1a, 0x52, 0x00, 0x06, 0x9e, 0x28, 0x3b, - 0x73, 0x69, 0x37, 0x19, 0x40, 0x41, 0x16, 0x20, 0x07, 0x3f, 0xd4, 0xda, 0xa6, 0xa1, 0xb9, 0x58, - 0x35, 0x2d, 0x03, 0x3f, 0xc6, 0x24, 0x8b, 0x98, 0xe8, 0xbf, 0x32, 0xbe, 0xe8, 0x15, 0x1f, 0xa3, - 0x4e, 0x21, 0xbc, 0x20, 0xd9, 0x09, 0x93, 0x31, 0x41, 0x7f, 0x2e, 0x01, 0xf2, 0x37, 0x79, 0xc7, - 0x36, 0xcc, 0x3d, 0x13, 0x3b, 0x24, 0x3b, 0xc7, 0x18, 0xbe, 0x7b, 0x01, 0x5b, 0x29, 0x30, 0x36, - 0x3d, 0x88, 0x17, 0x63, 0x2a, 0x33, 0xc6, 0x30, 0x6e, 0xee, 0x7f, 0x24, 0xc8, 0x9c, 0xb0, 0xcc, - 0xa8, 0x09, 0x11, 0x93, 0x07, 0xca, 0xa9, 0x32, 0xf5, 0xd9, 0x91, 0x7a, 0xf5, 0x59, 0xff, 0xb9, - 0x58, 0x47, 0x4c, 0x03, 0xed, 0x43, 0x92, 0x6e, 0x22, 0xcb, 0x55, 0x4d, 0x83, 0x79, 0xb3, 0x54, - 0x79, 0xed, 0xb8, 0x9f, 0x4f, 0xec, 0x30, 0xe2, 0x73, 0xb3, 0x48, 0x70, 0xf0, 0xba, 0x81, 0xf2, - 0x30, 0xed, 0xda, 0x2a, 0x7e, 0x6c, 0x12, 0xd7, 0xb4, 0xf6, 0x59, 0x80, 0x99, 0x50, 0xc0, 0xb5, - 0x6b, 0x82, 0x92, 0xfb, 0x93, 0x08, 0xa0, 0x93, 0xc6, 0x1b, 0xfd, 0xb5, 0x04, 0xd7, 0xbc, 0xa8, - 0xd3, 0x76, 0xcc, 0x7d, 0xd3, 0xd2, 0xda, 0xa1, 0xf0, 0x53, 0x62, 0x0b, 0xf9, 0xd1, 0xf3, 0x78, - 0x08, 0x11, 0x92, 0x6e, 0x0b, 0xf8, 0xe1, 0xd0, 0xf4, 0x1a, 0x8d, 0x90, 0x78, 0x68, 0x7a, 0xa2, - 0xc9, 0x7d, 0x25, 0xdb, 0x3b, 0xa3, 0x73, 0x6e, 0x1d, 0x5e, 0x1e, 0x09, 0x7c, 0x91, 0xc0, 0x22, - 0xf7, 0x6d, 0x09, 0xae, 0x9e, 0xe1, 0xae, 0x83, 0x38, 0x29, 0x8e, 0xf3, 0x5e, 0xf8, 0xb0, 0xf7, - 0xd5, 0xe7, 0x08, 0x09, 0x82, 0x83, 0x58, 0x81, 0x97, 0xce, 0x34, 0x79, 0xe7, 0xcd, 0x26, 0x11, - 0x04, 0xfa, 0x57, 0x09, 0x66, 0x87, 0x76, 0x30, 0xfa, 0x30, 0xa0, 0xe0, 0xf5, 0xe3, 0x7e, 0x3e, - 0xce, 0x98, 0xbc, 0x10, 0x2d, 0x3f, 0x3c, 0xa9, 0xe5, 0x5b, 0x94, 0x03, 0x63, 0xcc, 0x38, 0xbc, - 0x73, 0x69, 0x0e, 0x1c, 0x62, 0xa0, 0xe9, 0xb9, 0xbf, 0x95, 0x40, 0x1e, 0x36, 0x16, 0x68, 0x1b, - 0x64, 0xfc, 0xd8, 0x75, 0x34, 0x35, 0xe0, 0xd4, 0xa5, 0x8b, 0x38, 0xf5, 0x34, 0xeb, 0xde, 0xf4, - 0x3d, 0xfb, 0xc7, 0x90, 0x72, 0xf0, 0x3e, 0x0d, 0xbd, 0x75, 0xdb, 0xda, 0x33, 0xf7, 0xc5, 0x4a, - 0x7f, 0x69, 0xec, 0xc8, 0xa5, 0xa8, 0xb0, 0xee, 0x15, 0xd6, 0x5b, 0x99, 0x71, 0x02, 0xa5, 0xdc, - 0xb7, 0x24, 0x58, 0x38, 0xdd, 0xde, 0x9d, 0xa2, 0x6b, 0x3b, 0x61, 0x5d, 0xbb, 0x7b, 0x79, 0x93, - 0x1a, 0xd0, 0x90, 0xb5, 0x68, 0x42, 0x92, 0x23, 0x85, 0xcf, 0x51, 0x35, 0x61, 0x7d, 0xfc, 0x44, - 0xca, 0xcb, 0x00, 0x07, 0xe6, 0xfe, 0x81, 0xfa, 0x48, 0x73, 0xb1, 0x23, 0x12, 0xb2, 0x49, 0x4a, - 0x79, 0x9f, 0x12, 0x0a, 0x7f, 0x0f, 0x90, 0xaa, 0x77, 0xba, 0xb6, 0xe3, 0x7a, 0xd1, 0xf6, 0x06, - 0xc4, 0xb8, 0xa3, 0x17, 0x02, 0x2f, 0x8e, 0x18, 0x60, 0xa8, 0x27, 0x8f, 0xcf, 0x84, 0x67, 0x11, - 0x18, 0x68, 0x1b, 0xe2, 0x3c, 0x28, 0x22, 0xd9, 0xab, 0x0c, 0x6e, 0x69, 0x6c, 0x38, 0x1e, 0x5e, - 0x79, 0x31, 0x91, 0x40, 0xf1, 0xe3, 0xe2, 0xc8, 0xa9, 0x71, 0xf1, 0xdb, 0x10, 0xe3, 0xd9, 0x74, - 0x91, 0x6f, 0xcb, 0x9f, 0x92, 0xa8, 0xab, 0x6f, 0x2f, 0x9b, 0x6d, 0xbc, 0xcc, 0x9a, 0x79, 0xa3, - 0xe5, 0x9d, 0xd0, 0xab, 0x90, 0x20, 0xc4, 0x55, 0x89, 0xf9, 0x0d, 0x7e, 0xd2, 0x98, 0xe4, 0x99, - 0xe2, 0x46, 0xa3, 0xd9, 0x30, 0xbf, 0x81, 0x95, 0x38, 0x21, 0x2e, 0xfd, 0x40, 0xd7, 0x81, 0x45, - 0x5d, 0x44, 0xa3, 0xb1, 0x14, 0x0b, 0x9b, 0x26, 0x95, 0x00, 0x85, 0xe1, 0x1c, 0x9a, 0x5d, 0x75, - 0xef, 0x90, 0xf0, 0x58, 0x45, 0xe0, 0x1c, 0x9a, 0xdd, 0xe5, 0x75, 0xa2, 0xc4, 0x69, 0xe5, 0xf2, - 0x21, 0x41, 0x39, 0x48, 0x3c, 0xd2, 0xda, 0x6d, 0x76, 0xb2, 0x99, 0x62, 0x28, 0x7e, 0x39, 0xec, - 0x6a, 0x62, 0xbf, 0x5a, 0x57, 0x23, 0xce, 0x12, 0x5d, 0xcd, 0x3d, 0x60, 0xa7, 0xe3, 0xa4, 0x02, - 0x9c, 0xb4, 0xa3, 0xb9, 0x07, 0x28, 0x0b, 0x71, 0x3e, 0x2f, 0x92, 0x4d, 0x2c, 0x4e, 0xde, 0x9a, - 0x51, 0xbc, 0x22, 0x7a, 0x0d, 0x66, 0x79, 0x9a, 0x4d, 0x35, 0x4c, 0x07, 0xeb, 0x6e, 0xfb, 0x88, - 0xc5, 0x59, 0x09, 0x25, 0xcd, 0xc9, 0x55, 0x41, 0x45, 0xb7, 0x41, 0x1e, 0x0e, 0x4c, 0x59, 0x7c, - 0x94, 0x50, 0x66, 0x87, 0xe2, 0x52, 0x1a, 0xc3, 0x8a, 0xb5, 0x0e, 0x04, 0x7c, 0x59, 0x1e, 0xc3, - 0x8a, 0x8a, 0x41, 0xb0, 0x77, 0x1b, 0x64, 0x11, 0x95, 0x0e, 0xda, 0xa6, 0x38, 0x2e, 0xa7, 0x0f, - 0x9a, 0x16, 0x61, 0xae, 0xab, 0x39, 0x04, 0xab, 0xad, 0x9e, 0x65, 0xb4, 0xb1, 0xca, 0xb1, 0xb2, - 0x69, 0xd6, 0x3a, 0xc3, 0xaa, 0xca, 0xac, 0x86, 0xeb, 0xdd, 0x79, 0x49, 0x84, 0x85, 0xdf, 0x44, - 0x12, 0xe1, 0x16, 0xc8, 0x06, 0xde, 0xd3, 0x7a, 0x6d, 0x57, 0x35, 0x2d, 0xa1, 0xa7, 0x2f, 0xd1, - 0xc0, 0x56, 0x49, 0x0b, 0x7a, 0xdd, 0x62, 0x1a, 0x9a, 0xfb, 0x41, 0x04, 0xa6, 0xd8, 0x7e, 0x44, - 0x77, 0x21, 0x4a, 0x57, 0x5d, 0x24, 0xc8, 0xc6, 0x3d, 0x5b, 0xb1, 0x3e, 0x08, 0x41, 0xd4, 0xd2, - 0x3a, 0x38, 0x8b, 0x98, 0x4e, 0xb0, 0x6f, 0x74, 0x15, 0xe2, 0x04, 0x3f, 0x50, 0x1f, 0x6a, 0xed, - 0xec, 0x1c, 0x53, 0xd9, 0x18, 0xc1, 0x0f, 0xee, 0x6b, 0x6d, 0x74, 0x05, 0x62, 0x26, 0x51, 0x2d, - 0xfc, 0x28, 0x3b, 0xcf, 0xfd, 0x97, 0x49, 0xb6, 0xf0, 0x23, 0xf4, 0x29, 0x48, 0x3e, 0xd2, 0x88, - 0x8a, 0x3b, 0x5d, 0xf7, 0x88, 0x49, 0x2d, 0x41, 0x95, 0x9c, 0xd4, 0x68, 0x99, 0x85, 0x39, 0x9a, - 0xb3, 0x8f, 0x5d, 0x55, 0xb7, 0xdb, 0x24, 0x7b, 0x85, 0x6e, 0x6a, 0x7a, 0xae, 0xa3, 0xa4, 0x8a, - 0xdd, 0x26, 0x6b, 0xd1, 0x44, 0x44, 0x9e, 0x5c, 0x8b, 0x26, 0x26, 0xe5, 0xe8, 0x5a, 0x34, 0x11, - 0x95, 0xa7, 0xd6, 0xa2, 0x89, 0x29, 0x39, 0xb6, 0x16, 0x4d, 0xc4, 0xe4, 0xf8, 0x5a, 0x34, 0x11, - 0x97, 0x13, 0x6b, 0xd1, 0x44, 0x42, 0x4e, 0xae, 0x45, 0x13, 0x49, 0x19, 0xd6, 0xa2, 0x09, 0x90, - 0xa7, 0xd7, 0xa2, 0x89, 0x69, 0x79, 0x66, 0x2d, 0x9a, 0x98, 0x91, 0x53, 0x6b, 0xd1, 0x44, 0x4a, - 0x4e, 0xaf, 0x45, 0x13, 0x69, 0x79, 0x76, 0x2d, 0x9a, 0x98, 0x95, 0xe5, 0xb5, 0x68, 0x42, 0x96, - 0x33, 0x6b, 0xd1, 0x44, 0x46, 0x46, 0xb9, 0x9a, 0xb8, 0x38, 0xd0, 0xd0, 0x57, 0x43, 0x72, 0x1a, - 0xfb, 0xcc, 0xc7, 0x3a, 0x15, 0x7e, 0x2c, 0x81, 0xdc, 0xc0, 0x0f, 0x7a, 0xd8, 0xd2, 0xf1, 0x7d, - 0xad, 0x5d, 0x39, 0xe8, 0x59, 0x87, 0xe8, 0x55, 0x98, 0xd5, 0xe9, 0x87, 0xca, 0xd3, 0x94, 0x54, - 0x62, 0x12, 0x93, 0x58, 0x8a, 0x91, 0x1b, 0x94, 0x4a, 0x05, 0xf7, 0x32, 0x80, 0x68, 0x47, 0xd7, - 0x33, 0xc2, 0x9a, 0x24, 0x79, 0x13, 0x6a, 0x6c, 0x86, 0x60, 0x1c, 0xfb, 0x11, 0x33, 0x6e, 0x21, - 0x18, 0xc5, 0x7e, 0x84, 0x96, 0x60, 0xde, 0xc2, 0x8f, 0x5d, 0x75, 0xb8, 0x31, 0x33, 0x64, 0x4a, - 0x86, 0xd6, 0x55, 0x82, 0x1d, 0x0a, 0xff, 0x18, 0x81, 0x59, 0x6f, 0xd0, 0x9e, 0xf5, 0xdf, 0x03, - 0x99, 0xae, 0xae, 0x69, 0xa8, 0xae, 0xcd, 0x91, 0x3c, 0x3f, 0xf0, 0xf6, 0xa8, 0xfc, 0x52, 0x18, - 0x85, 0x96, 0xeb, 0x46, 0xd3, 0x66, 0xec, 0xb8, 0x23, 0x54, 0x52, 0x24, 0x48, 0xcb, 0xed, 0x42, - 0xda, 0xeb, 0xc4, 0x29, 0xa8, 0x02, 0xb1, 0x10, 0xbf, 0xcf, 0x8c, 0xc1, 0xcf, 0x13, 0xb5, 0x22, - 0xba, 0xe6, 0x7e, 0x1b, 0xd0, 0x49, 0xde, 0x41, 0x27, 0x3c, 0xc5, 0x9d, 0xf0, 0x76, 0xd8, 0x09, - 0x7f, 0xe5, 0x62, 0x73, 0x0b, 0x0c, 0x3b, 0x98, 0xcc, 0xfa, 0xbb, 0x08, 0xa4, 0xb9, 0x0b, 0xf3, - 0xbd, 0x2f, 0x35, 0x66, 0xd4, 0x56, 0x9a, 0xd6, 0xfe, 0xe0, 0x6a, 0x85, 0xce, 0x2f, 0xa2, 0xc8, - 0x5e, 0x85, 0xdf, 0xf8, 0x15, 0x1a, 0xa3, 0x68, 0x46, 0xf8, 0x0e, 0x26, 0x42, 0x63, 0x0d, 0xcd, - 0xf0, 0x1b, 0xdd, 0x84, 0x34, 0x8b, 0x33, 0x07, 0xad, 0x26, 0x59, 0xab, 0x14, 0xa3, 0xfa, 0xcd, - 0xca, 0x90, 0x22, 0x5d, 0x2d, 0x70, 0x9f, 0x13, 0x65, 0x42, 0x3d, 0xe7, 0xe2, 0x6a, 0x86, 0xf6, - 0x09, 0x86, 0x0e, 0x0e, 0x26, 0xbd, 0x0e, 0x56, 0xbb, 0x36, 0xcf, 0xcd, 0x4c, 0x2a, 0x49, 0x4e, - 0xd9, 0xb1, 0x09, 0xda, 0x65, 0xaa, 0xc2, 0x64, 0xa1, 0x1a, 0x5c, 0x38, 0xd9, 0x18, 0xe3, 0xf2, - 0xfa, 0xf8, 0xe2, 0x54, 0x66, 0x49, 0x98, 0x50, 0xf8, 0x0b, 0x09, 0xae, 0xd2, 0xb8, 0x8d, 0xef, - 0xb4, 0x0a, 0xbb, 0xa2, 0xf6, 0xb4, 0x53, 0x83, 0x38, 0x0b, 0x08, 0xfd, 0xc0, 0x77, 0xf5, 0xb8, - 0x9f, 0x8f, 0xd1, 0xd6, 0xcf, 0xed, 0x0f, 0x63, 0x14, 0xb8, 0xce, 0x72, 0x0e, 0xae, 0xa3, 0x59, - 0x84, 0x5d, 0xdc, 0xd0, 0x65, 0xeb, 0xe0, 0x4e, 0x8b, 0x9e, 0x80, 0x23, 0xcc, 0xf5, 0xcd, 0x87, - 0x2a, 0x37, 0x79, 0x5d, 0x21, 0x07, 0xd9, 0xe1, 0x21, 0xfb, 0x09, 0xcd, 0xff, 0x07, 0x0b, 0x5b, - 0xf8, 0xd1, 0x69, 0xb3, 0x29, 0x43, 0x9c, 0x5b, 0x3a, 0x4f, 0xe5, 0x6f, 0x0d, 0x1b, 0x9d, 0xe0, - 0x2d, 0x7d, 0x91, 0x8d, 0xb4, 0xc9, 0x3a, 0x28, 0x5e, 0xc7, 0xc2, 0xc7, 0x70, 0x75, 0x08, 0xdd, - 0x5f, 0xbe, 0x77, 0x21, 0x46, 0x5c, 0xcd, 0x15, 0x81, 0x5c, 0x7a, 0x1c, 0xf4, 0x86, 0xab, 0xb9, - 0x3d, 0xa2, 0x88, 0x7e, 0x85, 0x9b, 0xf0, 0x4a, 0xa9, 0xe7, 0xda, 0x54, 0x41, 0x44, 0xdc, 0x8b, - 0x75, 0xdb, 0xd2, 0xcd, 0xb6, 0xa9, 0x05, 0xae, 0x50, 0x0b, 0xaf, 0xc2, 0x8d, 0x51, 0xcd, 0x7c, - 0x49, 0x28, 0x2c, 0xb3, 0xdb, 0xeb, 0x60, 0xda, 0x72, 0xc3, 0x24, 0x2e, 0x7a, 0x17, 0x66, 0x84, - 0x86, 0x51, 0xc5, 0xf3, 0xc4, 0x70, 0x8e, 0x92, 0x4e, 0x3b, 0x3e, 0x08, 0x29, 0xfc, 0xa5, 0x04, - 0x73, 0x55, 0xc7, 0xee, 0x76, 0xb1, 0x21, 0x5c, 0x18, 0x97, 0xad, 0xe7, 0xb9, 0xa4, 0x80, 0xe7, - 0xda, 0x82, 0x48, 0xbd, 0x2a, 0xce, 0x33, 0xf7, 0x9e, 0xf7, 0x98, 0x54, 0xaf, 0xa2, 0xaf, 0x70, - 0x01, 0xf7, 0x08, 0xb3, 0xc7, 0xe9, 0x13, 0x09, 0xbe, 0xf0, 0x1d, 0xe1, 0x40, 0xb2, 0x3d, 0x52, - 0xf8, 0x7e, 0x1c, 0xae, 0x04, 0x17, 0x6d, 0xa5, 0xe2, 0x0d, 0xfc, 0x13, 0x88, 0x7b, 0x49, 0x9e, - 0x31, 0xec, 0xee, 0x69, 0x10, 0x45, 0x21, 0x8f, 0x60, 0xa2, 0xc7, 0xc3, 0x44, 0x0d, 0xc8, 0x98, - 0x96, 0x8b, 0x9d, 0x36, 0xd6, 0x1e, 0xd2, 0xb0, 0x86, 0xca, 0x4c, 0x24, 0xd5, 0xc7, 0x0d, 0x0d, - 0xe4, 0x00, 0x00, 0x0f, 0x31, 0x3e, 0x81, 0xb9, 0x20, 0xa8, 0x37, 0xfe, 0xd1, 0xd9, 0x5c, 0x36, - 0xbc, 0x01, 0xac, 0x97, 0x76, 0x0e, 0x00, 0x79, 0x29, 0xa9, 0x0f, 0xfc, 0x13, 0x09, 0xcf, 0xd8, - 0xdf, 0xbd, 0xb4, 0x44, 0xaa, 0x43, 0xa7, 0x93, 0x50, 0x8c, 0xcd, 0x9c, 0xea, 0xaf, 0x28, 0xc6, - 0xbe, 0x0f, 0x31, 0x9e, 0xd4, 0x15, 0x97, 0x67, 0xf7, 0x2e, 0x3b, 0x05, 0x9e, 0x2c, 0x56, 0x04, - 0x5a, 0xee, 0x8f, 0x24, 0x98, 0x09, 0x2e, 0x37, 0x32, 0x21, 0xc1, 0xc4, 0xef, 0x99, 0xc8, 0xc9, - 0x17, 0x7e, 0x72, 0xe7, 0xaa, 0x54, 0x37, 0x68, 0x60, 0x67, 0x38, 0x76, 0x77, 0x70, 0x79, 0x3a, - 0xa9, 0x24, 0x28, 0x81, 0x06, 0xad, 0xb9, 0x6f, 0x42, 0xd2, 0x17, 0x7a, 0x20, 0x17, 0x37, 0xf9, - 0x02, 0x73, 0x71, 0x23, 0xf9, 0x57, 0x21, 0x15, 0x92, 0x18, 0x5a, 0xf0, 0xc7, 0x10, 0x2d, 0xc7, - 0xf8, 0x18, 0xce, 0x45, 0x29, 0xfc, 0x28, 0x0e, 0x73, 0xa7, 0x59, 0xee, 0x0f, 0x41, 0x0e, 0xd8, - 0x2d, 0xb5, 0x6d, 0x12, 0x57, 0xe8, 0xe6, 0xed, 0xd1, 0xc7, 0xf9, 0x80, 0xf1, 0x13, 0xaa, 0x98, - 0x76, 0xc2, 0x26, 0xf1, 0x63, 0x48, 0x1b, 0x7c, 0xe0, 0x22, 0xdf, 0x2e, 0x5e, 0xea, 0x8c, 0x3a, - 0x86, 0x9f, 0x62, 0x00, 0x05, 0x7a, 0xca, 0x08, 0x54, 0x11, 0xa4, 0x43, 0xca, 0x07, 0x3f, 0xea, - 0x8a, 0xf3, 0xdc, 0xf3, 0x1b, 0xc3, 0x19, 0x8f, 0x0b, 0xc5, 0x44, 0xfb, 0x30, 0xeb, 0x31, 0xf1, - 0x8e, 0xfe, 0xc9, 0x17, 0xc2, 0xc6, 0x13, 0x4c, 0x43, 0xa4, 0x02, 0xbe, 0x23, 0xc1, 0x9c, 0xc7, - 0xc9, 0x4f, 0x59, 0x9b, 0x06, 0x33, 0x67, 0xa9, 0x72, 0xe3, 0xb8, 0x9f, 0xcf, 0x08, 0xc9, 0x78, - 0x99, 0x93, 0xe7, 0xd6, 0xbb, 0x8c, 0x31, 0x04, 0x68, 0xd0, 0x98, 0x84, 0xd6, 0x53, 0xc6, 0x53, - 0x83, 0x98, 0x84, 0x1a, 0xb6, 0xe7, 0x8f, 0x49, 0xe8, 0x67, 0xdd, 0x40, 0xdf, 0x95, 0x20, 0xc3, - 0xaf, 0xc9, 0x3a, 0x3d, 0x57, 0xe3, 0x17, 0xe5, 0x5e, 0x4e, 0xe0, 0xc3, 0xe3, 0x7e, 0x7e, 0x96, - 0x2d, 0xef, 0xa6, 0xa8, 0x63, 0x6c, 0xcb, 0x97, 0x65, 0x3b, 0x40, 0x11, 0x47, 0x68, 0x9f, 0x60, - 0xa0, 0x75, 0x48, 0xf3, 0x44, 0x89, 0xf7, 0x12, 0x90, 0x25, 0x0b, 0x52, 0xe5, 0x1b, 0xcf, 0xfa, - 0xf9, 0xc5, 0x53, 0xf6, 0x09, 0xcf, 0xb1, 0xdc, 0xe7, 0x6d, 0x95, 0xd4, 0x5e, 0xb0, 0x88, 0x36, - 0x60, 0x96, 0x07, 0xb2, 0x83, 0xe7, 0x3d, 0x30, 0xfe, 0xe5, 0x2e, 0x0f, 0x82, 0x7d, 0xaa, 0xc8, - 0x7f, 0x2d, 0xc0, 0xfc, 0xa9, 0x31, 0xd8, 0xcf, 0x63, 0xb0, 0x10, 0x36, 0xab, 0x7e, 0x94, 0xa4, - 0x0e, 0xfb, 0xdb, 0x77, 0xc6, 0x36, 0xcd, 0xfe, 0x43, 0x1f, 0x66, 0x1a, 0xbd, 0xd2, 0xb0, 0xc7, - 0xfd, 0x64, 0xc8, 0x7b, 0x5d, 0x02, 0x9f, 0x2d, 0xef, 0x10, 0xbe, 0xe7, 0xc2, 0x3e, 0xf0, 0x3d, - 0x0b, 0xcf, 0x78, 0xbd, 0x7b, 0x09, 0x78, 0xd6, 0xdf, 0x7f, 0xa4, 0xe4, 0xf9, 0x96, 0x7f, 0x90, - 0x20, 0x15, 0x9a, 0xd9, 0xaf, 0xd3, 0xb9, 0xec, 0xf8, 0xb1, 0x15, 0x7f, 0x6c, 0xf4, 0xd6, 0xc5, - 0xa7, 0x15, 0x0e, 0xb9, 0x72, 0x7f, 0x25, 0x41, 0x2a, 0x24, 0xc8, 0x5f, 0x91, 0x5b, 0x7a, 0xf1, - 0x23, 0x6f, 0x41, 0x3a, 0xbc, 0x44, 0x01, 0x1e, 0xd2, 0x8b, 0xe1, 0x51, 0xf8, 0x32, 0xc4, 0x38, - 0x05, 0x21, 0x48, 0xbf, 0x5f, 0xaa, 0x37, 0xeb, 0x5b, 0x2b, 0xea, 0xf2, 0xb6, 0xa2, 0xae, 0x54, - 0xe4, 0x09, 0x34, 0x03, 0x89, 0x6a, 0x6d, 0xa3, 0x46, 0x89, 0xb2, 0x84, 0xa6, 0x21, 0xce, 0x4a, - 0xb5, 0xaa, 0x1c, 0x29, 0x94, 0x41, 0xe6, 0xd8, 0x7b, 0x98, 0xba, 0x19, 0x7a, 0x2a, 0x41, 0x45, - 0x98, 0x63, 0x27, 0x88, 0x0e, 0x8d, 0xac, 0xe8, 0xf6, 0x56, 0x03, 0xb1, 0x78, 0xc6, 0xaf, 0xa2, - 0xbb, 0x77, 0x4b, 0xeb, 0xe0, 0xc2, 0x4f, 0xa2, 0x90, 0x19, 0x80, 0x78, 0x4e, 0xf6, 0xcf, 0xa4, - 0xc1, 0xf9, 0x28, 0x76, 0xee, 0x7d, 0xe7, 0x89, 0xfe, 0xe2, 0xa8, 0x24, 0xee, 0x1d, 0xdf, 0xa7, - 0x9b, 0xe6, 0x59, 0x3f, 0x9f, 0x19, 0x1e, 0x2c, 0x79, 0xce, 0x0b, 0x49, 0x6f, 0x88, 0x2c, 0xe7, - 0x6b, 0x5a, 0x87, 0xea, 0xe0, 0x3d, 0x18, 0xcf, 0xf9, 0x9a, 0xd6, 0xe1, 0xae, 0x52, 0x57, 0xe2, - 0xb4, 0x72, 0xd7, 0x31, 0xd1, 0x1a, 0x44, 0xed, 0xae, 0xeb, 0x1d, 0xc8, 0xbf, 0x74, 0xa1, 0x29, - 0x6d, 0x77, 0xc5, 0x7c, 0x14, 0x86, 0x81, 0xd6, 0xf8, 0x15, 0xfe, 0x40, 0xd0, 0xcc, 0x20, 0x8f, - 0x69, 0x42, 0x53, 0xa1, 0x85, 0xc8, 0xed, 0xc3, 0x4c, 0x50, 0x62, 0xa7, 0xdc, 0x5c, 0x94, 0xc2, - 0x49, 0x93, 0xcf, 0x8c, 0x35, 0x74, 0x71, 0x60, 0x0d, 0x5c, 0x66, 0x7d, 0x19, 0x92, 0xfe, 0x3c, - 0x2e, 0x72, 0xa7, 0xc7, 0x6d, 0xbc, 0x9f, 0x11, 0x9c, 0x92, 0x63, 0x85, 0xef, 0x47, 0x60, 0x46, - 0xc1, 0xc4, 0x6e, 0x3f, 0xc4, 0x06, 0x8d, 0xa0, 0xfc, 0x67, 0xba, 0xd2, 0xf8, 0xcf, 0x74, 0x4b, - 0x90, 0x1c, 0x78, 0xa0, 0x0b, 0xbc, 0x01, 0x1c, 0xf4, 0x42, 0x1f, 0x42, 0xaa, 0x65, 0xf7, 0x2c, - 0x43, 0x73, 0x8e, 0x58, 0x5c, 0xc5, 0x22, 0x90, 0xf4, 0xc8, 0x17, 0x59, 0xc1, 0x51, 0x17, 0xcb, - 0xa2, 0x33, 0x8d, 0x9f, 0x94, 0x99, 0x56, 0xa0, 0x54, 0x78, 0x1b, 0x66, 0x82, 0xb5, 0x28, 0x01, - 0xd1, 0xad, 0xed, 0xad, 0x1a, 0xdf, 0x93, 0xe5, 0x52, 0x65, 0x7d, 0xb9, 0xbe, 0xb1, 0x21, 0x4b, - 0x94, 0x5e, 0xfb, 0xa0, 0xde, 0x94, 0x23, 0xfc, 0xd5, 0x64, 0xa3, 0x59, 0x52, 0x9a, 0x5e, 0xe2, - 0xb4, 0x80, 0x21, 0x15, 0xe4, 0x47, 0x2d, 0x1f, 0x0d, 0x3b, 0x19, 0x21, 0x74, 0xf2, 0x7e, 0x6d, - 0xcc, 0x11, 0x7b, 0x1a, 0xe4, 0x04, 0x51, 0x0b, 0xff, 0x1c, 0x01, 0x34, 0x58, 0x78, 0xdf, 0x58, - 0x7d, 0x00, 0xa0, 0x1f, 0x60, 0xfd, 0xb0, 0x6b, 0x9b, 0x96, 0x2b, 0xce, 0x9a, 0x6f, 0x8d, 0xa5, - 0x3b, 0xbe, 0xb1, 0xaa, 0xf8, 0xfd, 0x95, 0x00, 0x16, 0xfa, 0xc3, 0xd1, 0x29, 0xfa, 0x49, 0x96, - 0xa2, 0x67, 0x3b, 0xff, 0xd7, 0x9a, 0xa6, 0xcf, 0x95, 0x00, 0x06, 0x23, 0x46, 0x6f, 0xc2, 0xd4, - 0x05, 0xb2, 0x1b, 0xbc, 0x6d, 0x50, 0xd7, 0x0b, 0xff, 0x1d, 0x05, 0x54, 0x71, 0xb0, 0xe6, 0x62, - 0x6a, 0xa2, 0xc9, 0xa8, 0x14, 0x47, 0x19, 0xa6, 0xf8, 0x91, 0x3e, 0x72, 0x91, 0x23, 0xbd, 0xc7, - 0x9a, 0x75, 0x45, 0x5f, 0x87, 0x19, 0xdd, 0x6e, 0xf7, 0x3a, 0x96, 0xca, 0x1e, 0x02, 0x89, 0xf3, - 0xc7, 0x17, 0x47, 0xad, 0xd8, 0x89, 0xc1, 0x15, 0x2b, 0x76, 0x9b, 0x96, 0xfd, 0xf7, 0xe6, 0x0c, - 0x90, 0xb5, 0x40, 0xd7, 0x20, 0xe9, 0x5b, 0x1e, 0xa6, 0x0e, 0x49, 0x65, 0x40, 0x40, 0x77, 0x60, - 0x4a, 0x23, 0xaa, 0xbd, 0xc7, 0x82, 0xe9, 0xf3, 0xb6, 0xa2, 0x12, 0xd5, 0xc8, 0xf6, 0x1e, 0x7a, - 0x1d, 0x32, 0x1d, 0xed, 0xb1, 0xba, 0xe7, 0xf0, 0x17, 0xc5, 0xaa, 0x69, 0xb4, 0xb9, 0x25, 0x94, - 0x94, 0xd9, 0x8e, 0xf6, 0x78, 0x59, 0xd0, 0xeb, 0x46, 0x1b, 0xa3, 0x37, 0x21, 0xb5, 0xf7, 0x80, - 0x1f, 0xad, 0xb8, 0x57, 0xe2, 0xaf, 0xaa, 0x66, 0x8f, 0xfb, 0xf9, 0xe9, 0xe5, 0xf7, 0x98, 0x60, - 0xa8, 0x4f, 0x52, 0xa6, 0xf7, 0x1e, 0xf8, 0x85, 0xdc, 0x7f, 0x49, 0x10, 0x17, 0x33, 0x42, 0x5d, - 0x00, 0x21, 0x1e, 0xd3, 0xe0, 0x6b, 0x9a, 0x2a, 0xbf, 0x77, 0xdc, 0xcf, 0x27, 0x2b, 0x8c, 0x5a, - 0xaf, 0x92, 0x67, 0xfd, 0xfc, 0xbb, 0x97, 0xf5, 0x28, 0x1e, 0x88, 0x92, 0xe4, 0x4c, 0xea, 0x06, - 0xcb, 0x0b, 0x1f, 0x68, 0x44, 0x3d, 0x30, 0x89, 0x6b, 0xef, 0x3b, 0x5a, 0x47, 0xbc, 0x0f, 0x98, - 0x39, 0xd0, 0xc8, 0xaa, 0x47, 0x43, 0x39, 0x1a, 0x9b, 0x3d, 0xe4, 0xef, 0xb8, 0xf8, 0x6b, 0x11, - 0xbf, 0x8c, 0xee, 0xc0, 0x15, 0xbf, 0xb3, 0x4a, 0x25, 0xd5, 0xea, 0xe9, 0x87, 0x98, 0xf9, 0x20, - 0x6a, 0xdc, 0xe7, 0xfc, 0xca, 0x4d, 0xed, 0x71, 0x99, 0x57, 0x15, 0xae, 0xc0, 0x5c, 0x60, 0x59, - 0xfd, 0x48, 0x1a, 0x83, 0xcc, 0x9f, 0x31, 0x04, 0x7e, 0x42, 0xf1, 0x1e, 0xcc, 0x0e, 0xfd, 0x42, - 0x48, 0xd8, 0xdf, 0x60, 0xc6, 0x31, 0xfc, 0x93, 0xa2, 0x62, 0x85, 0x17, 0xbd, 0xb3, 0x41, 0x5a, - 0x0f, 0x95, 0x0b, 0x73, 0x90, 0xf1, 0xd9, 0xf8, 0xbc, 0xff, 0x37, 0x05, 0xf1, 0x1d, 0xed, 0xa8, - 0x6d, 0x6b, 0x06, 0x5a, 0x84, 0x69, 0xef, 0x01, 0x96, 0xc7, 0x2f, 0xa9, 0x04, 0x49, 0x61, 0x35, - 0x93, 0xd9, 0xc5, 0x52, 0x40, 0xcd, 0x4c, 0x48, 0xf7, 0x08, 0x76, 0xa8, 0x06, 0xa8, 0xec, 0xe7, - 0x4c, 0xdc, 0xdd, 0x94, 0xcb, 0xcf, 0xfa, 0xf9, 0x7b, 0xe3, 0x2d, 0x1e, 0xd6, 0x7b, 0x8e, 0xe9, - 0x1e, 0x15, 0x1b, 0xef, 0x6d, 0xec, 0x0a, 0x28, 0x6a, 0x2b, 0x6c, 0x25, 0xd5, 0x0b, 0x16, 0xc5, - 0x3b, 0x3b, 0xba, 0x10, 0x6a, 0xc7, 0xd4, 0x1d, 0x9b, 0x78, 0xd7, 0x37, 0x82, 0xba, 0xc9, 0x88, - 0xe8, 0x35, 0x98, 0xdd, 0x33, 0x2d, 0x76, 0x57, 0xe9, 0xb5, 0xe3, 0x37, 0x37, 0x69, 0x8f, 0x2c, - 0x1a, 0x3e, 0x84, 0x74, 0xe0, 0x81, 0x1b, 0x55, 0xc2, 0x18, 0x53, 0xc2, 0xed, 0xe3, 0x7e, 0x3e, - 0x35, 0xd8, 0xd4, 0x5c, 0x11, 0x9f, 0x27, 0xb4, 0x49, 0x0d, 0xd8, 0x50, 0x35, 0x9c, 0x87, 0x29, - 0xf6, 0x63, 0x37, 0xfe, 0x70, 0x59, 0xe1, 0x05, 0x54, 0x83, 0x94, 0x48, 0x85, 0xf0, 0x5f, 0xc2, - 0x89, 0x57, 0x81, 0x8b, 0x81, 0xa5, 0xf7, 0x7e, 0x2b, 0x57, 0xac, 0x59, 0xba, 0x6d, 0x60, 0xa3, - 0x46, 0xcb, 0x8a, 0xc8, 0xfc, 0xb2, 0x02, 0x41, 0x2b, 0x90, 0xd6, 0xdb, 0x58, 0xb3, 0x7a, 0x5d, - 0x0f, 0x07, 0x8d, 0x89, 0x93, 0x12, 0xfd, 0x04, 0xd0, 0x16, 0xa0, 0x3d, 0xf6, 0xe8, 0x29, 0x38, - 0x2a, 0x76, 0x53, 0x39, 0x0e, 0x98, 0xcc, 0xfa, 0x2a, 0x83, 0x91, 0xa1, 0x1b, 0x90, 0xb2, 0x6c, - 0x4b, 0xd7, 0x2c, 0x1d, 0xb7, 0x99, 0x65, 0xe5, 0x97, 0x9b, 0x61, 0x22, 0x2a, 0x43, 0x8c, 0x5f, - 0x98, 0x8b, 0x33, 0xec, 0xad, 0x71, 0x9f, 0xf7, 0xaf, 0x4e, 0x28, 0xa2, 0x27, 0xaa, 0x41, 0xdc, - 0xe1, 0x8f, 0x37, 0xd8, 0x25, 0xfa, 0xb9, 0xb9, 0xa4, 0xc0, 0xd3, 0x90, 0xd5, 0x09, 0xc5, 0xeb, - 0x8b, 0x9a, 0xde, 0x1b, 0x56, 0xee, 0x47, 0xc5, 0x33, 0xc4, 0xe2, 0x98, 0x27, 0x84, 0x01, 0x60, - 0x08, 0x85, 0x4e, 0xd0, 0x64, 0x57, 0x5b, 0xec, 0x7a, 0x7d, 0xf4, 0x04, 0x43, 0xcf, 0x38, 0xe8, - 0x04, 0x79, 0x4f, 0xb4, 0x45, 0x03, 0x01, 0xcf, 0xb7, 0xb3, 0x8b, 0xf7, 0xe9, 0x3b, 0x9f, 0xbd, - 0x48, 0xfc, 0xbb, 0x3a, 0xa1, 0x04, 0x10, 0xd0, 0x7b, 0x30, 0xad, 0x0f, 0x4c, 0x54, 0x76, 0x96, - 0x01, 0xbe, 0x71, 0x21, 0x3f, 0xb5, 0x4a, 0x7d, 0xd3, 0x80, 0x8a, 0x3e, 0x82, 0x34, 0x09, 0x9d, - 0x97, 0xb2, 0x57, 0x18, 0xea, 0xe7, 0x2e, 0x9a, 0xaf, 0x5d, 0x9d, 0x50, 0x86, 0x90, 0xd0, 0xff, - 0x07, 0xd9, 0x1d, 0xba, 0x24, 0x62, 0xf7, 0xe1, 0xa3, 0xdf, 0x88, 0x9e, 0x71, 0x15, 0xb6, 0x3a, - 0xa1, 0x9c, 0x40, 0x43, 0x9f, 0xc0, 0x2c, 0x09, 0xff, 0x16, 0x29, 0x7b, 0x95, 0x31, 0xf8, 0xfc, - 0xf8, 0xbf, 0x5e, 0x1a, 0xe0, 0x0f, 0x63, 0x51, 0x78, 0x2b, 0x7c, 0xd7, 0xc4, 0xde, 0x65, 0x8c, - 0x86, 0x3f, 0xfd, 0xee, 0x8b, 0xc2, 0x0f, 0x61, 0xa1, 0x75, 0x48, 0x76, 0x3c, 0x9b, 0xcf, 0x5e, - 0x35, 0x8c, 0x3e, 0x62, 0x0c, 0xbb, 0xa1, 0xd5, 0x09, 0x65, 0xd0, 0x1f, 0xfd, 0x8e, 0x04, 0xd7, - 0xb4, 0x11, 0x97, 0x52, 0xd9, 0x4f, 0x9d, 0x9b, 0x87, 0x1f, 0xe3, 0xea, 0x6b, 0x75, 0x42, 0x19, - 0xc9, 0x05, 0x7d, 0x1a, 0x66, 0xba, 0x5a, 0x8f, 0x60, 0xd5, 0xc1, 0x1a, 0xb1, 0xad, 0xec, 0x35, - 0xee, 0xa7, 0x18, 0x4d, 0x61, 0xa4, 0x72, 0x12, 0xe2, 0xe2, 0xf6, 0xd4, 0x7f, 0xda, 0xc0, 0x1f, - 0x35, 0xf0, 0xe7, 0x0c, 0x39, 0xf9, 0x53, 0x85, 0x1f, 0x02, 0x24, 0xfc, 0x70, 0x7a, 0x09, 0x90, - 0x1f, 0xe8, 0x0c, 0x1e, 0x79, 0x53, 0x07, 0x18, 0x59, 0x9d, 0x50, 0x32, 0x5e, 0xdd, 0xe0, 0x9d, - 0xf7, 0xbd, 0xd0, 0x0b, 0xb0, 0x71, 0x7e, 0x87, 0x48, 0x45, 0xe9, 0x3f, 0x11, 0xa3, 0x8e, 0x49, - 0xbc, 0x03, 0xf6, 0x1d, 0x13, 0xcf, 0x93, 0xa7, 0x3d, 0xb2, 0x70, 0x4c, 0x37, 0x21, 0xed, 0xf4, - 0x2c, 0x76, 0x69, 0x2a, 0xb2, 0x13, 0x3c, 0xba, 0x4b, 0x09, 0xaa, 0x48, 0x30, 0x54, 0x86, 0x6c, - 0xe5, 0xed, 0x73, 0x6d, 0xa5, 0x37, 0xf7, 0x55, 0xc9, 0x37, 0x96, 0xcb, 0xc3, 0xc6, 0xf2, 0xf5, - 0xf3, 0x8d, 0x65, 0x00, 0xc6, 0xb7, 0x96, 0xbb, 0xa7, 0x5a, 0xcb, 0xa5, 0x31, 0xb7, 0x7b, 0x00, - 0x31, 0x6c, 0x2e, 0x2b, 0x43, 0xe6, 0xf2, 0xf6, 0xb9, 0xe6, 0x32, 0x38, 0x47, 0x61, 0x2f, 0xb7, - 0x4f, 0xb1, 0x97, 0x6f, 0x5c, 0xe8, 0xe0, 0xb4, 0x2a, 0x85, 0x0c, 0xa6, 0x72, 0x9a, 0xc1, 0x2c, - 0x8e, 0x67, 0x30, 0x03, 0x90, 0x21, 0x8b, 0xf9, 0xf1, 0x09, 0x8b, 0x29, 0x9f, 0x6f, 0x72, 0x4e, - 0x4d, 0x49, 0xad, 0x4a, 0x27, 0x4c, 0xa6, 0x76, 0x8a, 0xc9, 0xcc, 0x30, 0xf8, 0x37, 0x2f, 0x60, - 0x32, 0x03, 0x0c, 0x4e, 0xda, 0xcc, 0x0f, 0x60, 0x26, 0x68, 0xe7, 0xd8, 0x53, 0xa7, 0xd1, 0x16, - 0xf9, 0x8c, 0x9f, 0x7b, 0x32, 0x1d, 0x08, 0x54, 0xa1, 0xaf, 0x9f, 0x34, 0x97, 0x73, 0xe7, 0x82, - 0x9f, 0x71, 0x99, 0xbf, 0x2a, 0x9d, 0xb4, 0x97, 0x1b, 0x41, 0x7b, 0x39, 0x7f, 0xae, 0x37, 0x3d, - 0x11, 0x4f, 0xaf, 0x4a, 0x41, 0x83, 0xf9, 0x5d, 0x09, 0xae, 0x8d, 0xb2, 0x78, 0xc2, 0x55, 0xbd, - 0x73, 0x49, 0x83, 0x19, 0x60, 0x3a, 0x92, 0x0d, 0x7a, 0x15, 0x12, 0xae, 0xa3, 0xe9, 0xec, 0x22, - 0xe7, 0x0a, 0xbb, 0xad, 0x63, 0x69, 0xb4, 0x26, 0xa5, 0xb1, 0x74, 0x1b, 0xfb, 0x30, 0xca, 0x00, - 0x09, 0xef, 0x6d, 0x4b, 0xc0, 0x84, 0x16, 0xbe, 0x27, 0xc1, 0xe4, 0x9a, 0xdd, 0x42, 0x2f, 0x07, - 0x72, 0xbb, 0x29, 0x91, 0x01, 0x98, 0x5a, 0xb3, 0x5b, 0x22, 0x49, 0xfb, 0xce, 0xa0, 0xb7, 0x38, - 0x2a, 0xbf, 0x32, 0x62, 0x62, 0x7e, 0x6a, 0xdc, 0xef, 0x84, 0xbe, 0x06, 0xf1, 0x2e, 0x3f, 0x8a, - 0x08, 0x8b, 0x5a, 0x18, 0xd5, 0x9f, 0xb7, 0x54, 0xbc, 0x2e, 0xaf, 0xdf, 0x0e, 0xfe, 0x8f, 0x00, - 0x9b, 0xb6, 0x81, 0x51, 0x1a, 0x60, 0x47, 0x23, 0xa4, 0x7b, 0xe0, 0x68, 0x04, 0xcb, 0x13, 0x28, - 0x0e, 0x93, 0xeb, 0x9b, 0x0d, 0x59, 0x7a, 0xfd, 0x83, 0x60, 0x62, 0xb6, 0xaa, 0x94, 0xea, 0x5b, - 0xf5, 0xad, 0x15, 0x75, 0xab, 0xb4, 0x59, 0x6b, 0xc8, 0x13, 0x28, 0x0b, 0xf3, 0xef, 0x97, 0xea, - 0x4d, 0x91, 0xa9, 0x55, 0xeb, 0x5b, 0xcd, 0x9a, 0x72, 0xbf, 0xb4, 0x21, 0x4b, 0x68, 0x01, 0x90, - 0xb2, 0x5d, 0x59, 0x6f, 0x54, 0xcb, 0x6a, 0x65, 0x7b, 0x73, 0xa7, 0x54, 0x69, 0xd6, 0xb7, 0xb7, - 0xe4, 0x08, 0x4a, 0x40, 0xb4, 0xba, 0xbd, 0x55, 0x93, 0xe1, 0xf5, 0x5f, 0x44, 0x21, 0xca, 0x72, - 0x4a, 0x37, 0x60, 0x7a, 0x77, 0xab, 0xb1, 0x53, 0xab, 0xd4, 0x97, 0xeb, 0xb5, 0xaa, 0x3c, 0x91, - 0x9b, 0x7b, 0xf2, 0x74, 0x71, 0x96, 0x56, 0xed, 0x5a, 0xa4, 0x8b, 0x75, 0x66, 0xe3, 0x51, 0x0e, - 0x62, 0xe5, 0x52, 0x65, 0x7d, 0x77, 0x47, 0x96, 0x72, 0xe9, 0x27, 0x4f, 0x17, 0x81, 0x36, 0xe0, - 0xf6, 0x19, 0x5d, 0xe3, 0x39, 0xa7, 0x6d, 0xa5, 0x26, 0x47, 0x72, 0xb3, 0x4f, 0x9e, 0x2e, 0x4e, - 0xb3, 0x54, 0x96, 0xb0, 0xb1, 0xaf, 0x41, 0xaa, 0x51, 0x59, 0xad, 0x6d, 0x96, 0xd4, 0xca, 0x6a, - 0x69, 0x6b, 0xa5, 0x26, 0x4f, 0xe6, 0xe6, 0x9f, 0x3c, 0x5d, 0x94, 0x87, 0xf7, 0x29, 0x65, 0x51, - 0xdf, 0xdc, 0xd9, 0x56, 0x9a, 0x72, 0x74, 0xc0, 0x82, 0x9b, 0x47, 0x54, 0x00, 0xe0, 0xbd, 0x97, - 0x6b, 0xb5, 0xaa, 0x3c, 0x95, 0x43, 0x4f, 0x9e, 0x2e, 0xa6, 0x69, 0xfd, 0xc0, 0xea, 0xa1, 0x9b, - 0x30, 0x53, 0x51, 0x6a, 0xa5, 0x66, 0x4d, 0x6d, 0x34, 0x4b, 0xcd, 0x86, 0x1c, 0x1b, 0xcc, 0x24, - 0x60, 0xc9, 0x50, 0x11, 0x32, 0xa5, 0xdd, 0xe6, 0xb6, 0x1a, 0x6a, 0x1b, 0xcf, 0x5d, 0x7d, 0xf2, - 0x74, 0x71, 0x8e, 0xb6, 0xa5, 0xfa, 0x19, 0x6c, 0xff, 0x59, 0x90, 0x43, 0xe3, 0x57, 0x57, 0x2a, - 0x72, 0x22, 0xb7, 0xf0, 0xe4, 0xe9, 0x22, 0x1a, 0x9e, 0xc2, 0x4a, 0x05, 0x7d, 0x01, 0x16, 0x9a, - 0x1f, 0xee, 0xd4, 0xaa, 0xb5, 0x46, 0x45, 0x0d, 0x4f, 0x3b, 0x99, 0xcb, 0x3e, 0x79, 0xba, 0x38, - 0x4f, 0xfb, 0x9c, 0x98, 0xfa, 0x1b, 0x20, 0x37, 0x9a, 0x4a, 0xad, 0xb4, 0xa9, 0xd6, 0xb7, 0x56, - 0x6a, 0x0d, 0xb6, 0x58, 0x30, 0x18, 0xd2, 0x90, 0xcd, 0xa1, 0x53, 0xd8, 0xaa, 0xbd, 0x3f, 0x84, - 0x3f, 0x3d, 0x68, 0x3f, 0x64, 0x46, 0xd0, 0x22, 0x24, 0x37, 0xeb, 0x2b, 0x4a, 0x89, 0xe1, 0xce, - 0xe4, 0x32, 0x4f, 0x9e, 0x2e, 0xa6, 0x68, 0x3b, 0xdf, 0x28, 0xa0, 0x3a, 0xe4, 0x99, 0x50, 0x1a, - 0x3b, 0xa5, 0x2d, 0xb5, 0xb2, 0xbd, 0xb5, 0x5c, 0x5f, 0x51, 0x95, 0x5a, 0x65, 0x7b, 0xab, 0x52, - 0xdf, 0xa8, 0xf3, 0x7e, 0xa9, 0xdc, 0x8d, 0x27, 0x4f, 0x17, 0x17, 0x3d, 0x11, 0x9d, 0xb5, 0x85, - 0x73, 0x89, 0xdf, 0xfd, 0xde, 0xf5, 0x89, 0x1f, 0x7e, 0xff, 0xfa, 0x44, 0xf9, 0xd6, 0x4f, 0xff, - 0xe3, 0xfa, 0xc4, 0x4f, 0x8f, 0xaf, 0x4b, 0x3f, 0x3b, 0xbe, 0x2e, 0xfd, 0xfc, 0xf8, 0xba, 0xf4, - 0x8b, 0xe3, 0xeb, 0xd2, 0xef, 0xfd, 0xf2, 0xfa, 0xc4, 0xcf, 0x7e, 0x79, 0x7d, 0xe2, 0xe7, 0xbf, - 0xbc, 0x3e, 0xf1, 0x51, 0x8c, 0x6f, 0x91, 0x56, 0x8c, 0x1d, 0xc2, 0xdf, 0xfc, 0xbf, 0x00, 0x00, - 0x00, 0xff, 0xff, 0x9b, 0x5e, 0x8a, 0x26, 0x8c, 0x45, 0x00, 0x00, + // 5654 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5c, 0x5d, 0x6c, 0x1b, 0xc9, + 0x7d, 0xd7, 0x52, 0x14, 0x3f, 0xfe, 0x14, 0xa9, 0xe5, 0x48, 0x96, 0x79, 0x8c, 0x6d, 0x2a, 0x3c, + 0xfb, 0xce, 0xbe, 0xe4, 0xa8, 0xc4, 0xce, 0xc7, 0xc5, 0xc9, 0xf9, 0x8e, 0x5f, 0x92, 0x48, 0x7d, + 0xde, 0x92, 0xf2, 0x7d, 0xf5, 0xb2, 0x5d, 0x72, 0x47, 0xd2, 0x56, 0xe4, 0x2e, 0xbd, 0xb3, 0x94, + 0xad, 0x14, 0x08, 0x82, 0xa4, 0x01, 0x0a, 0x3f, 0xb5, 0x40, 0xdb, 0x97, 0xd6, 0x40, 0x81, 0x24, + 0x40, 0x1e, 0x1a, 0x14, 0x0d, 0x8a, 0xb4, 0xe8, 0x53, 0x81, 0xb6, 0x40, 0x1e, 0x5a, 0x20, 0x40, + 0x51, 0x34, 0x2d, 0x0a, 0x35, 0x55, 0x5e, 0xfa, 0x58, 0xb4, 0x6f, 0xf7, 0x54, 0xcc, 0xc7, 0x2e, + 0x77, 0x29, 0x89, 0xa2, 0x64, 0x5f, 0xf2, 0x22, 0xef, 0xfe, 0x67, 0xe6, 0x37, 0x33, 0xff, 0xfd, + 0x7f, 0xcf, 0xd0, 0x30, 0xff, 0x5b, 0x56, 0x8b, 0x2c, 0xd2, 0x3f, 0xbd, 0x16, 0xfb, 0xa7, 0xd0, + 0xb3, 0x2d, 0xc7, 0x42, 0x2f, 0xb5, 0xad, 0xf6, 0xbe, 0x6d, 0x69, 0xed, 0xbd, 0x02, 0x79, 0xd4, + 0x29, 0xb0, 0x16, 0xde, 0x2b, 0x7b, 0x05, 0xdb, 0xb6, 0x65, 0xd3, 0xfe, 0xfc, 0x81, 0x8f, 0xc8, + 0xce, 0xed, 0x5a, 0xbb, 0x16, 0x7b, 0x5c, 0xa4, 0x4f, 0x82, 0x8a, 0x18, 0x46, 0xaf, 0xb5, 0xa8, + 0x6b, 0x8e, 0x26, 0x68, 0x19, 0x97, 0x66, 0x58, 0xaf, 0xef, 0x58, 0x76, 0x57, 0x73, 0x5c, 0x8c, + 0x97, 0xc9, 0xa3, 0xce, 0x62, 0x5b, 0x73, 0xb4, 0x8e, 0xb5, 0xbb, 0xa8, 0x63, 0xd2, 0xee, 0xb5, + 0x16, 0x89, 0x63, 0xf7, 0xdb, 0x4e, 0xdf, 0xc6, 0xba, 0xe8, 0x94, 0x3b, 0xa5, 0x93, 0x83, 0x4d, + 0xcd, 0x74, 0x5c, 0xfc, 0xbe, 0x63, 0x74, 0x16, 0xf7, 0x3a, 0xed, 0x45, 0xc7, 0xe8, 0x62, 0xe2, + 0x68, 0xdd, 0x9e, 0x68, 0xf9, 0x34, 0x1d, 0x4a, 0xda, 0x7b, 0xb8, 0xab, 0xb5, 0xf7, 0x34, 0x73, + 0x17, 0xdb, 0x8b, 0x7c, 0x8e, 0x76, 0xaf, 0x25, 0xba, 0xdc, 0x6c, 0x77, 0xfa, 0xc4, 0xc1, 0xf6, + 0x01, 0xb6, 0x89, 0x61, 0x99, 0x8b, 0xe2, 0x55, 0x15, 0xef, 0xbc, 0x57, 0xfe, 0x07, 0x21, 0xb8, + 0x5a, 0xd2, 0xda, 0xfb, 0xfd, 0x5e, 0xd5, 0x6c, 0xdb, 0x87, 0x3d, 0xc7, 0xb0, 0xcc, 0x4d, 0xf6, + 0x97, 0x20, 0x19, 0x26, 0xf7, 0xf1, 0x61, 0x46, 0x5a, 0x90, 0x6e, 0x4f, 0x2b, 0xf4, 0x11, 0xbd, + 0x09, 0xe1, 0xae, 0xa5, 0xe3, 0x4c, 0x68, 0x41, 0xba, 0x9d, 0xba, 0x7b, 0xa7, 0x70, 0x26, 0x6f, + 0x0b, 0x03, 0xb4, 0x75, 0x4b, 0xc7, 0x0a, 0x1b, 0x86, 0x5a, 0x10, 0xdb, 0xef, 0x12, 0xd5, 0x30, + 0x77, 0xac, 0xcc, 0xe4, 0x82, 0x74, 0x3b, 0x71, 0xf7, 0xfe, 0x08, 0x88, 0x33, 0x96, 0x55, 0x58, + 0x5d, 0x6f, 0xd4, 0xcc, 0x1d, 0xab, 0x94, 0x38, 0x3e, 0xca, 0x45, 0xc5, 0x8b, 0x12, 0xdd, 0xef, + 0x12, 0xfa, 0x90, 0xdd, 0x04, 0x97, 0x46, 0xd7, 0xdf, 0xb7, 0x0d, 0xb6, 0xfe, 0xb8, 0x42, 0x1f, + 0xd1, 0x67, 0x01, 0x61, 0x8e, 0x87, 0x75, 0x95, 0x7e, 0x48, 0x95, 0x6e, 0x30, 0xc4, 0x36, 0x28, + 0x7b, 0x2d, 0x15, 0xcd, 0xd1, 0x56, 0xf1, 0xe1, 0xfd, 0xf0, 0x7f, 0xff, 0x69, 0x4e, 0xe2, 0x7f, + 0xf3, 0xdf, 0x9a, 0x84, 0xd4, 0x60, 0x29, 0x0c, 0x7e, 0x05, 0x22, 0xec, 0x0b, 0x60, 0x36, 0x43, + 0xea, 0xee, 0xe7, 0xc6, 0x62, 0x07, 0x1d, 0x5a, 0x68, 0xb0, 0x71, 0x8a, 0x18, 0x8f, 0x10, 0x84, + 0x89, 0xd6, 0x71, 0xc4, 0x42, 0xd8, 0x33, 0xfa, 0x63, 0x09, 0x16, 0x86, 0x57, 0x54, 0x3a, 0x5c, + 0x5d, 0x6f, 0xac, 0x6b, 0xf4, 0x33, 0xae, 0xe2, 0xc3, 0x5a, 0x25, 0x33, 0xb9, 0x30, 0x79, 0x3b, + 0x71, 0x77, 0x73, 0xfc, 0x89, 0xab, 0xe7, 0x20, 0x56, 0x4d, 0xc7, 0x3e, 0x54, 0xce, 0x9d, 0x38, + 0xdb, 0x80, 0x5b, 0x63, 0x41, 0xf9, 0x65, 0x28, 0xce, 0x65, 0x68, 0x0e, 0xa6, 0x0e, 0xb4, 0x4e, + 0x1f, 0x8b, 0xdd, 0xf2, 0x97, 0xfb, 0xa1, 0x37, 0xa4, 0xfc, 0x55, 0x88, 0x70, 0xc6, 0xa0, 0x24, + 0xc4, 0x8b, 0xd5, 0xc6, 0xdd, 0x2f, 0x7e, 0x69, 0xb9, 0xbc, 0x2e, 0x4f, 0x88, 0x4f, 0xf0, 0x13, + 0x09, 0xe6, 0x1b, 0x8e, 0x8d, 0xb5, 0x6e, 0xcd, 0xdc, 0xc5, 0x84, 0xee, 0xa9, 0x82, 0x1d, 0xcd, + 0xe8, 0x10, 0x74, 0x0b, 0x52, 0x84, 0xb5, 0xa8, 0x9a, 0xae, 0xdb, 0x98, 0x10, 0x31, 0x61, 0x92, + 0x53, 0x8b, 0x9c, 0x88, 0x3e, 0x0f, 0x61, 0xd2, 0xd3, 0x4c, 0x36, 0x73, 0xe2, 0xee, 0x55, 0x1f, + 0xdb, 0x84, 0x22, 0x17, 0x1a, 0x3d, 0xcd, 0x2c, 0x85, 0x7f, 0x7a, 0x94, 0x9b, 0x50, 0x58, 0x57, + 0x54, 0x02, 0x20, 0x8e, 0x66, 0x3b, 0x2a, 0xd5, 0x40, 0x21, 0xb4, 0xd7, 0x7d, 0x03, 0xa9, 0x86, + 0x16, 0xf6, 0x3a, 0xed, 0x42, 0xd3, 0xd5, 0x50, 0x31, 0x3c, 0xce, 0x86, 0x51, 0x6a, 0xfe, 0x5f, + 0x27, 0xe1, 0xea, 0xd0, 0xc2, 0xb7, 0x6c, 0x6b, 0x97, 0x2d, 0x69, 0x09, 0xa6, 0xdb, 0x7d, 0xc7, + 0x3a, 0xc0, 0x36, 0x9f, 0x41, 0x1a, 0x7f, 0x86, 0x84, 0x18, 0x48, 0xe9, 0xe8, 0x9b, 0x80, 0x7a, + 0x9a, 0xed, 0x18, 0x14, 0x5c, 0xed, 0x09, 0xf4, 0x4c, 0x88, 0xc9, 0x47, 0x6d, 0x84, 0x7c, 0x9c, + 0xb1, 0xae, 0xc2, 0x96, 0x0b, 0xe6, 0x52, 0xd8, 0xe7, 0x14, 0x33, 0xa7, 0x7b, 0xc3, 0xad, 0xd9, + 0x5d, 0x48, 0x9f, 0x18, 0x82, 0x14, 0x40, 0x06, 0x43, 0xc6, 0xba, 0xea, 0x59, 0xb0, 0x8b, 0x6c, + 0x31, 0xed, 0x0e, 0xf7, 0x1a, 0xb2, 0x4f, 0x25, 0x98, 0x3f, 0x7d, 0x71, 0xa7, 0xc8, 0xda, 0x07, + 0x7e, 0x59, 0x4b, 0xdc, 0xad, 0xbc, 0x08, 0x46, 0xf8, 0x25, 0xf6, 0x47, 0x21, 0x78, 0x89, 0x8a, + 0xac, 0xde, 0xef, 0xe0, 0xad, 0x66, 0xa3, 0xbc, 0xa7, 0x19, 0xa6, 0x61, 0xee, 0x2a, 0xb8, 0x6d, + 0xd9, 0x3a, 0xfa, 0x7d, 0x09, 0xb2, 0xd4, 0xca, 0xe2, 0x76, 0x80, 0x01, 0xaa, 0xcd, 0x9a, 0xb9, + 0x5d, 0x2d, 0x35, 0xfe, 0xfd, 0x28, 0x77, 0x6f, 0xd7, 0x70, 0xf6, 0xfa, 0xad, 0x42, 0xdb, 0xea, + 0x2e, 0x7a, 0x2b, 0xd4, 0x5b, 0x83, 0xe7, 0xc5, 0xde, 0xfe, 0xee, 0x22, 0x73, 0x06, 0xfd, 0xbe, + 0xa1, 0x17, 0xb6, 0xb7, 0x6b, 0x95, 0xe3, 0xa3, 0x5c, 0x66, 0xcb, 0x05, 0xf7, 0xd8, 0xc3, 0x67, + 0x56, 0x32, 0xbd, 0x33, 0x5a, 0xd0, 0x43, 0x88, 0x68, 0x6d, 0xba, 0x1d, 0x61, 0xc3, 0x1f, 0x8c, + 0x62, 0xc9, 0x59, 0x3b, 0x2b, 0x6c, 0x35, 0x1b, 0x45, 0x86, 0xa2, 0x08, 0xb4, 0xfc, 0x4d, 0x88, + 0x7b, 0x44, 0x04, 0x10, 0xd9, 0xde, 0xaa, 0x14, 0x9b, 0x55, 0x79, 0x02, 0x25, 0x20, 0xaa, 0x54, + 0xd7, 0xaa, 0xc5, 0x46, 0x55, 0x96, 0xf2, 0xff, 0x1c, 0x85, 0x24, 0x37, 0xeb, 0xae, 0xe6, 0x06, + 0xf5, 0x4b, 0xba, 0x8c, 0x7e, 0xa1, 0x07, 0x10, 0xc3, 0x26, 0x67, 0xb0, 0xf8, 0xd0, 0x63, 0x21, + 0x44, 0xb1, 0xc9, 0xd8, 0x83, 0x5e, 0xe2, 0x7e, 0x82, 0x2a, 0x77, 0xbc, 0x14, 0x3d, 0x3e, 0xca, + 0x4d, 0x6e, 0x2b, 0x35, 0xee, 0x30, 0xbe, 0x23, 0xc1, 0x6c, 0xdf, 0x36, 0x88, 0xda, 0x3a, 0x54, + 0x3b, 0x56, 0x5b, 0xeb, 0x18, 0xce, 0xa1, 0xba, 0x7f, 0x90, 0x99, 0x62, 0x8a, 0xf5, 0xe0, 0x5c, + 0xef, 0x25, 0xb6, 0x59, 0xd8, 0xb6, 0x0d, 0x52, 0x3a, 0x5c, 0x13, 0x08, 0xab, 0x07, 0x5c, 0x9b, + 0xe6, 0x8e, 0x8f, 0x72, 0xf2, 0xb6, 0x52, 0xf3, 0x37, 0x3d, 0x54, 0xe4, 0xfe, 0x50, 0x67, 0xf4, + 0x35, 0xc8, 0xea, 0xb8, 0x67, 0xe3, 0xb6, 0x46, 0x05, 0xa9, 0xc5, 0x90, 0xd5, 0xae, 0x66, 0x1a, + 0x3b, 0x98, 0x38, 0x99, 0x30, 0xb3, 0xa3, 0x99, 0x41, 0x0f, 0x3e, 0xf5, 0xba, 0x68, 0x47, 0x9a, + 0xe7, 0xf4, 0xa8, 0x6d, 0xb0, 0xb8, 0x17, 0xcd, 0x44, 0x18, 0xa3, 0xee, 0x5e, 0xdc, 0xff, 0x2a, + 0x69, 0x7c, 0x22, 0x52, 0x50, 0x60, 0xc6, 0x37, 0x05, 0xf3, 0xef, 0x71, 0x86, 0x7f, 0x67, 0x6c, + 0xd7, 0xa4, 0xa4, 0x70, 0xd0, 0xbd, 0x9e, 0xa3, 0x3d, 0xd1, 0x5f, 0x87, 0xf6, 0xbc, 0x01, 0xa9, + 0xb6, 0xd5, 0xe9, 0x60, 0x26, 0xe6, 0xea, 0xb6, 0x52, 0xcb, 0xc4, 0x98, 0xd0, 0xa4, 0x8f, 0x8f, + 0x72, 0xc9, 0xb2, 0xd7, 0x42, 0xc5, 0x27, 0xd9, 0xf6, 0xbf, 0xa2, 0x3f, 0x90, 0xe0, 0x1a, 0x11, + 0xfa, 0xa4, 0xf6, 0x1c, 0xa2, 0xb6, 0x85, 0x46, 0xb9, 0xfb, 0x01, 0xc6, 0xaf, 0x2f, 0x5c, 0x46, + 0x1d, 0x4b, 0xd7, 0x8f, 0x8f, 0x72, 0x67, 0xdb, 0x21, 0xe5, 0x25, 0x77, 0xe2, 0x2d, 0x87, 0x04, + 0x9b, 0xb2, 0x65, 0xb8, 0x72, 0xaa, 0x68, 0x9e, 0xe7, 0xb7, 0xe3, 0x7e, 0x2b, 0x28, 0x43, 0x8a, + 0xcb, 0x8a, 0x6b, 0x22, 0xf3, 0x7f, 0x33, 0x0f, 0x29, 0x05, 0x13, 0xc7, 0xb2, 0xb1, 0xab, 0xe8, + 0x7e, 0x25, 0x0d, 0x5f, 0x42, 0x49, 0x7f, 0x2c, 0xc1, 0x2c, 0x8d, 0x91, 0x6d, 0xa3, 0xe7, 0x58, + 0xb6, 0x6a, 0xe3, 0xc7, 0xb6, 0xe1, 0x60, 0xd7, 0xc5, 0x15, 0x47, 0xf0, 0x2d, 0xb8, 0x90, 0x42, + 0xc5, 0x03, 0x51, 0x04, 0x06, 0x57, 0xc6, 0x07, 0xdf, 0xfe, 0xcf, 0xdc, 0xfd, 0xb1, 0x44, 0xe9, + 0x64, 0xd8, 0x5e, 0xa8, 0x55, 0x14, 0xa4, 0x9f, 0x00, 0x46, 0xd7, 0x20, 0x4c, 0x95, 0x99, 0xc5, + 0x69, 0xf1, 0x52, 0xec, 0xf8, 0x28, 0x17, 0xa6, 0xea, 0xae, 0x30, 0x2a, 0x72, 0x60, 0x4e, 0xe8, + 0xb2, 0x67, 0x5a, 0x98, 0xea, 0x44, 0xd9, 0x96, 0xbe, 0x36, 0xfe, 0x96, 0x38, 0xf7, 0xdd, 0x4f, + 0xc8, 0x82, 0x63, 0xce, 0x3d, 0xd4, 0x3a, 0xd1, 0x82, 0xb6, 0x20, 0x45, 0x23, 0xdf, 0x96, 0x46, + 0xb0, 0x4a, 0x97, 0x4c, 0x32, 0x32, 0x9b, 0x6f, 0x58, 0x55, 0xc9, 0xa3, 0x0e, 0xed, 0x53, 0xa8, + 0x88, 0xce, 0x3e, 0xbe, 0x25, 0x75, 0x1f, 0x8d, 0xa0, 0x65, 0x48, 0x38, 0x5a, 0xab, 0xe3, 0xc2, + 0x71, 0xdb, 0xf8, 0xca, 0x19, 0x70, 0x4d, 0xda, 0xd3, 0x87, 0x05, 0x8e, 0x4b, 0x20, 0xa8, 0x02, + 0xe0, 0x1c, 0xf6, 0x5c, 0x9c, 0x14, 0xc3, 0xb9, 0x75, 0x16, 0xce, 0x61, 0xcf, 0x0f, 0x13, 0x77, + 0xc4, 0x3b, 0x41, 0x75, 0x98, 0xe6, 0x99, 0x91, 0xc0, 0x99, 0x61, 0x38, 0xaf, 0x9e, 0x81, 0xc3, + 0x22, 0x4e, 0xcd, 0x87, 0x94, 0x20, 0x1e, 0x85, 0xa0, 0x22, 0x44, 0x79, 0x46, 0x46, 0x32, 0x49, + 0x06, 0xf3, 0xe9, 0xb3, 0x96, 0xc3, 0x7a, 0xf9, 0x58, 0xef, 0x8e, 0x43, 0x8b, 0x90, 0xa0, 0x51, + 0x9a, 0x6d, 0xe8, 0x58, 0xd5, 0x5b, 0xcc, 0xee, 0xc6, 0x4b, 0xa9, 0xe3, 0xa3, 0x1c, 0x6c, 0x0a, + 0x72, 0xa5, 0xa4, 0x80, 0xdb, 0xa5, 0xd2, 0x42, 0x9f, 0x81, 0x74, 0xcf, 0xc6, 0x3d, 0xcd, 0xc6, + 0x6a, 0xdb, 0xea, 0xf6, 0x3a, 0xd8, 0xc1, 0x3a, 0xb3, 0x33, 0x31, 0x45, 0x16, 0x0d, 0x65, 0x97, + 0xce, 0x23, 0x5f, 0xcd, 0xa1, 0x49, 0x15, 0xc1, 0x36, 0xed, 0x19, 0x67, 0x3d, 0x93, 0x8c, 0x5a, + 0x13, 0x44, 0x74, 0x08, 0xf3, 0xe4, 0x90, 0x38, 0xb8, 0xab, 0x32, 0x76, 0x13, 0xb5, 0x6b, 0xec, + 0xda, 0xd4, 0x57, 0x64, 0xd2, 0x6c, 0x5b, 0xe5, 0xf1, 0x85, 0xad, 0xc1, 0x70, 0xd8, 0x67, 0x24, + 0xeb, 0x02, 0x85, 0xa7, 0x0d, 0x73, 0xe4, 0x94, 0x26, 0x74, 0x0f, 0xae, 0x0c, 0x34, 0x83, 0xa8, + 0xbd, 0x7e, 0xab, 0x63, 0x90, 0x3d, 0xcc, 0x2d, 0x5e, 0x4c, 0x99, 0xf3, 0x35, 0x6e, 0xb9, 0x6d, + 0xe8, 0x30, 0xa0, 0xec, 0x6d, 0xca, 0x1d, 0x6d, 0x17, 0x67, 0x12, 0x0b, 0xd2, 0xed, 0xa9, 0xd2, + 0xca, 0xc7, 0x47, 0xb9, 0xca, 0xd8, 0x9a, 0x4a, 0x70, 0x77, 0xd1, 0xb1, 0x31, 0xf6, 0x29, 0x7e, + 0x59, 0xe0, 0xf9, 0x75, 0xd6, 0xa5, 0x21, 0x05, 0x60, 0xe0, 0x89, 0x32, 0xd3, 0x97, 0x76, 0x93, + 0x3e, 0x14, 0x64, 0x02, 0xb2, 0xf1, 0x81, 0xd6, 0x31, 0x74, 0xcd, 0xc1, 0xaa, 0x61, 0xea, 0xf8, + 0x09, 0x26, 0x19, 0xc4, 0x58, 0xff, 0x95, 0xf1, 0x59, 0xaf, 0x78, 0x18, 0x35, 0x0a, 0xe1, 0x06, + 0xc9, 0x76, 0x90, 0x8c, 0x09, 0xfa, 0x73, 0x09, 0x90, 0xa7, 0xe4, 0x5d, 0x4b, 0x37, 0x76, 0x0c, + 0x6c, 0x93, 0xcc, 0x2c, 0x9b, 0xf0, 0xed, 0x0b, 0xd8, 0x4a, 0x81, 0xb1, 0xee, 0x42, 0xbc, 0x18, + 0x53, 0x99, 0xd6, 0x87, 0x71, 0xb3, 0xff, 0x27, 0x41, 0xfa, 0x84, 0x65, 0x46, 0x4d, 0x08, 0x19, + 0x3c, 0x50, 0x4e, 0x96, 0xa8, 0xcf, 0x0e, 0xd5, 0x2a, 0x1f, 0x1f, 0x3d, 0xd7, 0xd4, 0x21, 0x43, + 0x47, 0xbb, 0x10, 0xa7, 0x4a, 0x64, 0x3a, 0xaa, 0xa1, 0x33, 0x6f, 0x96, 0x2c, 0xd5, 0x8f, 0x8f, + 0x72, 0xb1, 0x2d, 0x46, 0x7c, 0xee, 0x29, 0x62, 0x1c, 0xbc, 0xa6, 0xa3, 0x1c, 0x24, 0x1c, 0x4b, + 0xc5, 0x4f, 0x0c, 0xe2, 0x18, 0xe6, 0x2e, 0x0b, 0x30, 0x63, 0x0a, 0x38, 0x56, 0x55, 0x50, 0xb2, + 0x7f, 0x12, 0x02, 0x74, 0xd2, 0x78, 0xa3, 0xbf, 0x96, 0xe0, 0x9a, 0x1b, 0x75, 0x5a, 0xb6, 0xb1, + 0x6b, 0x98, 0x5a, 0x27, 0x10, 0x7e, 0x4a, 0xec, 0x43, 0x7e, 0xf0, 0x3c, 0x1e, 0x42, 0x84, 0xa4, + 0x9b, 0x02, 0x7e, 0x38, 0x34, 0xbd, 0x46, 0x23, 0x24, 0x1e, 0x9a, 0x9e, 0xe8, 0xf2, 0x50, 0xc9, + 0xf4, 0xcf, 0x18, 0x9c, 0x5d, 0x85, 0xeb, 0x23, 0x81, 0x2f, 0x12, 0x58, 0x64, 0xbf, 0x2d, 0xc1, + 0xd5, 0x33, 0xdc, 0xb5, 0x1f, 0x27, 0xc9, 0x71, 0xde, 0x09, 0x26, 0x7b, 0x5f, 0x7d, 0x8e, 0x90, + 0xc0, 0xbf, 0x88, 0x65, 0x78, 0xe9, 0x4c, 0x93, 0x77, 0xde, 0x6e, 0x62, 0x7e, 0xa0, 0x7f, 0x93, + 0x60, 0x66, 0x48, 0x83, 0xd1, 0xfb, 0x3e, 0x01, 0xaf, 0x1d, 0x1f, 0xe5, 0xa2, 0x6c, 0x92, 0x17, + 0x22, 0xe5, 0xfb, 0x27, 0xa5, 0x7c, 0x83, 0xce, 0xc0, 0x26, 0x66, 0x33, 0xbc, 0x75, 0xe9, 0x19, + 0x38, 0xc4, 0x40, 0xd2, 0xb3, 0x7f, 0x2b, 0x81, 0x3c, 0x6c, 0x2c, 0xd0, 0x26, 0xc8, 0xf8, 0x89, + 0x63, 0x6b, 0xaa, 0xcf, 0xa9, 0x4b, 0x17, 0x71, 0xea, 0x29, 0x36, 0xbc, 0xe9, 0x79, 0xf6, 0x0f, + 0x21, 0x69, 0xe3, 0x5d, 0x1a, 0x7a, 0xb7, 0x2d, 0x73, 0xc7, 0xd8, 0x15, 0x5f, 0xfa, 0x4b, 0x63, + 0x47, 0x2e, 0x05, 0x85, 0x0d, 0x2f, 0xb3, 0xd1, 0xca, 0xb4, 0xed, 0x7b, 0xcb, 0x7e, 0x4b, 0x82, + 0xf9, 0xd3, 0xed, 0xdd, 0x29, 0xb2, 0xb6, 0x15, 0x94, 0xb5, 0xfb, 0x97, 0x37, 0xa9, 0x3e, 0x09, + 0xa9, 0x87, 0x63, 0x92, 0x1c, 0xca, 0x7f, 0x8e, 0x8a, 0x09, 0x1b, 0xe3, 0x15, 0x52, 0xae, 0x03, + 0xec, 0x19, 0xbb, 0x7b, 0xea, 0x63, 0xcd, 0xc1, 0xb6, 0x28, 0xc8, 0xc6, 0x29, 0xe5, 0x5d, 0x4a, + 0xc8, 0xff, 0x3d, 0x40, 0xb2, 0xd6, 0xed, 0x59, 0xb6, 0xe3, 0x46, 0xdb, 0x6b, 0x10, 0xe1, 0x8e, + 0x5e, 0x30, 0xbc, 0x30, 0x62, 0x81, 0x81, 0x91, 0x3c, 0x3e, 0x13, 0x9e, 0x45, 0x60, 0xa0, 0x4d, + 0x88, 0xf2, 0xa0, 0x88, 0x64, 0xae, 0x32, 0xb8, 0xc5, 0xb1, 0xe1, 0x78, 0x78, 0xe5, 0xc6, 0x44, + 0x02, 0xc5, 0x8b, 0x8b, 0x43, 0xa7, 0xc6, 0xc5, 0x6f, 0x42, 0x84, 0x57, 0xd3, 0x45, 0xbd, 0x2d, + 0x77, 0x4a, 0xa1, 0xae, 0xb6, 0xb9, 0x64, 0x74, 0xf0, 0x12, 0xeb, 0xe6, 0xae, 0x96, 0x0f, 0x42, + 0xaf, 0x40, 0x8c, 0x10, 0x47, 0x25, 0xc6, 0x37, 0x78, 0xa6, 0x31, 0xc9, 0x2b, 0xc5, 0x8d, 0x46, + 0xb3, 0x61, 0x7c, 0x03, 0x2b, 0x51, 0x42, 0x1c, 0xfa, 0x80, 0x6e, 0x00, 0x8b, 0xba, 0x88, 0x46, + 0x63, 0x29, 0x16, 0x36, 0x4d, 0x2a, 0x3e, 0x0a, 0xc3, 0xd9, 0x37, 0x7a, 0xea, 0xce, 0x3e, 0xe1, + 0xb1, 0x8a, 0xc0, 0xd9, 0x37, 0x7a, 0x4b, 0xab, 0x44, 0x89, 0xd2, 0xc6, 0xa5, 0x7d, 0x82, 0xb2, + 0x10, 0x7b, 0xac, 0x75, 0x3a, 0x2c, 0xb3, 0x99, 0x62, 0x28, 0xde, 0x7b, 0xd0, 0xd5, 0x44, 0x3e, + 0x59, 0x57, 0x23, 0x72, 0x89, 0x9e, 0xe6, 0xec, 0xb1, 0xec, 0x38, 0xae, 0x00, 0x27, 0x6d, 0x69, + 0xce, 0x1e, 0xca, 0x40, 0x94, 0xef, 0x8b, 0x64, 0x62, 0x0b, 0x93, 0xb7, 0xa7, 0x15, 0xf7, 0x15, + 0xbd, 0x0a, 0x33, 0xbc, 0xcc, 0xa6, 0xea, 0x86, 0x8d, 0xdb, 0x4e, 0xe7, 0x90, 0xc5, 0x59, 0x31, + 0x25, 0xc5, 0xc9, 0x15, 0x41, 0x45, 0x77, 0x40, 0x1e, 0x0e, 0x4c, 0x59, 0x7c, 0x14, 0x53, 0x66, + 0x86, 0xe2, 0x52, 0x1a, 0xc3, 0x8a, 0x6f, 0xed, 0x0b, 0xf8, 0x32, 0x3c, 0x86, 0x15, 0x0d, 0x83, + 0x60, 0xef, 0x0e, 0xc8, 0x22, 0x2a, 0x1d, 0xf4, 0x4d, 0x72, 0x5c, 0x4e, 0x1f, 0x74, 0x2d, 0xc0, + 0x6c, 0x4f, 0xb3, 0x09, 0x56, 0x5b, 0x7d, 0x53, 0xef, 0x60, 0x95, 0x63, 0x65, 0x52, 0xac, 0x77, + 0x9a, 0x35, 0x95, 0x58, 0x0b, 0x97, 0xbb, 0xf3, 0x8a, 0x08, 0xf3, 0xbf, 0x8e, 0x22, 0xc2, 0x6d, + 0x90, 0x75, 0xbc, 0xa3, 0xf5, 0x3b, 0x8e, 0x6a, 0x98, 0x42, 0x4e, 0x5f, 0xa2, 0x81, 0xad, 0x92, + 0x12, 0xf4, 0x9a, 0xc9, 0x24, 0x34, 0xfb, 0x83, 0x10, 0x4c, 0x31, 0x7d, 0x44, 0xf7, 0x21, 0x4c, + 0xbf, 0xba, 0x28, 0x90, 0x8d, 0x9b, 0x5b, 0xb1, 0x31, 0x08, 0x41, 0xd8, 0xd4, 0xba, 0x38, 0x83, + 0x98, 0x4c, 0xb0, 0x67, 0x74, 0x15, 0xa2, 0x04, 0x3f, 0x52, 0x0f, 0xb4, 0x4e, 0x66, 0x96, 0x89, + 0x6c, 0x84, 0xe0, 0x47, 0x0f, 0xb5, 0x0e, 0xba, 0x02, 0x11, 0x83, 0xa8, 0x26, 0x7e, 0x9c, 0x99, + 0xe3, 0xfe, 0xcb, 0x20, 0x1b, 0xf8, 0x31, 0xfa, 0x14, 0xc4, 0x1f, 0x6b, 0x44, 0xc5, 0xdd, 0x9e, + 0x73, 0xc8, 0xb8, 0x16, 0xa3, 0x42, 0x4e, 0xaa, 0xf4, 0x9d, 0x85, 0x39, 0x9a, 0xbd, 0x8b, 0x1d, + 0xb5, 0x6d, 0x75, 0x48, 0xe6, 0x0a, 0x55, 0x6a, 0x9a, 0xd7, 0x51, 0x52, 0xd9, 0xea, 0x90, 0x7a, + 0x38, 0x16, 0x92, 0x27, 0xeb, 0xe1, 0xd8, 0xa4, 0x1c, 0xae, 0x87, 0x63, 0x61, 0x79, 0xaa, 0x1e, + 0x8e, 0x4d, 0xc9, 0x91, 0x7a, 0x38, 0x16, 0x91, 0xa3, 0xf5, 0x70, 0x2c, 0x2a, 0xc7, 0xea, 0xe1, + 0x58, 0x4c, 0x8e, 0xd7, 0xc3, 0xb1, 0xb8, 0x0c, 0xf5, 0x70, 0x0c, 0xe4, 0x44, 0x3d, 0x1c, 0x4b, + 0xc8, 0xd3, 0xf5, 0x70, 0x6c, 0x5a, 0x4e, 0xd6, 0xc3, 0xb1, 0xa4, 0x9c, 0xaa, 0x87, 0x63, 0x29, + 0x79, 0xa6, 0x1e, 0x8e, 0xcd, 0xc8, 0x72, 0x3d, 0x1c, 0x93, 0xe5, 0x74, 0x3d, 0x1c, 0x4b, 0xcb, + 0x28, 0x5b, 0x15, 0x07, 0x07, 0x1a, 0xfa, 0x6a, 0x80, 0x4f, 0x63, 0xe7, 0x7c, 0x6c, 0x50, 0xfe, + 0xc7, 0x12, 0xc8, 0x0d, 0xfc, 0xa8, 0x8f, 0xcd, 0x36, 0x7e, 0xa8, 0x75, 0xca, 0x7b, 0x7d, 0x73, + 0x1f, 0xbd, 0x02, 0x33, 0x6d, 0xfa, 0xa0, 0xf2, 0x32, 0x25, 0xe5, 0x98, 0xc4, 0x38, 0x96, 0x64, + 0xe4, 0x06, 0xa5, 0x52, 0xc6, 0x5d, 0x07, 0x10, 0xfd, 0xe8, 0xf7, 0x0c, 0xb1, 0x2e, 0x71, 0xde, + 0x85, 0x1a, 0x9b, 0x21, 0x18, 0xdb, 0x7a, 0xcc, 0x8c, 0x5b, 0x00, 0x46, 0xb1, 0x1e, 0xa3, 0x45, + 0x98, 0x33, 0xf1, 0x13, 0x47, 0x1d, 0xee, 0xcc, 0x0c, 0x99, 0x92, 0xa6, 0x6d, 0x65, 0xff, 0x80, + 0xfc, 0x3f, 0x85, 0x60, 0xc6, 0x5d, 0xb4, 0x6b, 0xfd, 0x77, 0x40, 0xa6, 0x5f, 0xd7, 0xd0, 0x55, + 0xc7, 0xe2, 0x48, 0xae, 0x1f, 0x78, 0x73, 0x54, 0x7d, 0x29, 0x88, 0x42, 0xdf, 0x6b, 0x7a, 0xd3, + 0x62, 0xd3, 0x71, 0x47, 0xa8, 0x24, 0x89, 0x9f, 0x96, 0xdd, 0x86, 0x94, 0x3b, 0x88, 0x53, 0x50, + 0x19, 0x22, 0x81, 0xf9, 0x3e, 0x33, 0xc6, 0x7c, 0x2e, 0xab, 0x15, 0x31, 0x34, 0xfb, 0xdb, 0x80, + 0x4e, 0xce, 0xed, 0x77, 0xc2, 0x53, 0xdc, 0x09, 0x6f, 0x06, 0x9d, 0xf0, 0x57, 0x2e, 0xb6, 0x37, + 0xdf, 0xb2, 0xfd, 0xc5, 0xac, 0xbf, 0x0b, 0x41, 0x8a, 0xbb, 0x30, 0xcf, 0xfb, 0x52, 0x63, 0x46, + 0x6d, 0xa5, 0x61, 0xee, 0x0e, 0x8e, 0x56, 0xe8, 0xfe, 0x42, 0x8a, 0xec, 0x36, 0x78, 0x9d, 0x5f, + 0xa6, 0x31, 0x8a, 0xa6, 0x07, 0xcf, 0x60, 0x42, 0x34, 0xd6, 0xd0, 0x74, 0xaf, 0xd3, 0x2d, 0x48, + 0xb1, 0x38, 0x73, 0xd0, 0x6b, 0x92, 0xf5, 0x4a, 0x32, 0xaa, 0xd7, 0xad, 0x04, 0x49, 0xd2, 0xd3, + 0x7c, 0xe7, 0x39, 0x61, 0xc6, 0xd4, 0x73, 0x0e, 0xae, 0xa6, 0xe9, 0x18, 0x7f, 0xe8, 0x60, 0x63, + 0xd2, 0xef, 0x62, 0xb5, 0x67, 0xf1, 0xda, 0xcc, 0xa4, 0x12, 0xe7, 0x94, 0x2d, 0x8b, 0xa0, 0x6d, + 0x26, 0x2a, 0x8c, 0x17, 0xaa, 0xce, 0x99, 0x93, 0x89, 0xb0, 0x59, 0x5e, 0x1b, 0x9f, 0x9d, 0xca, + 0x0c, 0x09, 0x12, 0xf2, 0x7f, 0x21, 0xc1, 0x55, 0x1a, 0xb7, 0x71, 0x4d, 0x2b, 0xb3, 0x23, 0x6a, + 0x57, 0x3a, 0x35, 0x88, 0xb2, 0x80, 0xd0, 0x0b, 0x7c, 0x57, 0x8e, 0x8f, 0x72, 0x11, 0xda, 0xfb, + 0xb9, 0xfd, 0x61, 0x84, 0x02, 0xd7, 0x58, 0xcd, 0xc1, 0xb1, 0x35, 0x93, 0xb0, 0x83, 0x1b, 0xfa, + 0xd9, 0xba, 0xb8, 0xdb, 0xa2, 0x19, 0x70, 0x88, 0xb9, 0xbe, 0xb9, 0x40, 0xe3, 0x3a, 0x6f, 0xcb, + 0x67, 0x21, 0x33, 0xbc, 0x64, 0xaf, 0xa0, 0xf9, 0x1b, 0x30, 0xbf, 0x81, 0x1f, 0x9f, 0xb6, 0x9b, + 0x12, 0x44, 0xb9, 0xa5, 0x73, 0x45, 0xfe, 0xf6, 0xb0, 0xd1, 0xf1, 0x9f, 0xd2, 0x17, 0xd8, 0x4a, + 0x9b, 0x6c, 0x80, 0xe2, 0x0e, 0xcc, 0x7f, 0x08, 0x57, 0x87, 0xd0, 0xbd, 0xcf, 0xf7, 0x36, 0x44, + 0x88, 0xa3, 0x39, 0x22, 0x90, 0x4b, 0x8d, 0x83, 0xde, 0x70, 0x34, 0xa7, 0x4f, 0x14, 0x31, 0x2e, + 0x7f, 0x0b, 0x5e, 0x2e, 0xf6, 0x1d, 0x8b, 0x0a, 0x88, 0x88, 0x7b, 0x71, 0xdb, 0x32, 0xdb, 0x46, + 0xc7, 0xd0, 0x7c, 0x47, 0xa8, 0xf9, 0x57, 0xe0, 0xe6, 0xa8, 0x6e, 0x1e, 0x27, 0x14, 0x56, 0xd9, + 0xed, 0x77, 0x31, 0xed, 0xb9, 0x66, 0x10, 0x07, 0xbd, 0x0d, 0xd3, 0x42, 0xc2, 0xa8, 0xe0, 0xb9, + 0x6c, 0x38, 0x47, 0x48, 0x13, 0xb6, 0x07, 0x42, 0xf2, 0x7f, 0x29, 0xc1, 0x6c, 0xc5, 0xb6, 0x7a, + 0x3d, 0xac, 0x0b, 0x17, 0xc6, 0x79, 0xeb, 0x7a, 0x2e, 0xc9, 0xe7, 0xb9, 0x36, 0x20, 0x54, 0xab, + 0x88, 0x7c, 0xe6, 0xc1, 0xf3, 0xa6, 0x49, 0xb5, 0x0a, 0xfa, 0x0a, 0x67, 0x70, 0x9f, 0x30, 0x7b, + 0x9c, 0x3a, 0x51, 0xe0, 0x0b, 0x9e, 0x11, 0x0e, 0x38, 0xdb, 0x27, 0xf9, 0xef, 0x47, 0xe1, 0x8a, + 0xff, 0xa3, 0x2d, 0x97, 0xdd, 0x85, 0x7f, 0x04, 0x51, 0xb7, 0xc8, 0x33, 0x86, 0xdd, 0x3d, 0x0d, + 0xa2, 0x20, 0xf8, 0xe1, 0x2f, 0xf4, 0xb8, 0x98, 0xa8, 0x01, 0x69, 0xc3, 0x74, 0xb0, 0xdd, 0xc1, + 0xda, 0x01, 0x0d, 0x6b, 0x28, 0xcf, 0x44, 0x51, 0x7d, 0xdc, 0xd0, 0x40, 0xf6, 0x01, 0xf0, 0x10, + 0xe3, 0x23, 0x98, 0xf5, 0x83, 0xba, 0xeb, 0x1f, 0x5d, 0xcd, 0x65, 0xcb, 0x1b, 0xc0, 0xba, 0x65, + 0x67, 0x1f, 0x90, 0x5b, 0x92, 0x7a, 0xcf, 0xcb, 0x48, 0x78, 0xc5, 0xfe, 0xfe, 0xa5, 0x39, 0x52, + 0x19, 0xca, 0x4e, 0x02, 0x31, 0x36, 0x73, 0xaa, 0x9f, 0x50, 0x8c, 0xfd, 0x10, 0x22, 0xbc, 0xa8, + 0x2b, 0x0e, 0xcf, 0x1e, 0x5c, 0x76, 0x0b, 0xbc, 0x58, 0xac, 0x08, 0xb4, 0xec, 0x1f, 0x49, 0x30, + 0xed, 0xff, 0xdc, 0xc8, 0x80, 0x18, 0x63, 0xbf, 0x6b, 0x22, 0x27, 0x5f, 0x78, 0xe6, 0xce, 0x45, + 0xa9, 0xa6, 0xd3, 0xc0, 0x4e, 0xb7, 0xad, 0xde, 0xe0, 0xf0, 0x74, 0x52, 0x89, 0x51, 0x02, 0x0d, + 0x5a, 0xb3, 0xdf, 0x84, 0xb8, 0xc7, 0x74, 0x5f, 0x2d, 0x6e, 0xf2, 0x05, 0xd6, 0xe2, 0x46, 0xce, + 0x5f, 0x81, 0x64, 0x80, 0x63, 0x68, 0xde, 0x5b, 0x43, 0xb8, 0x14, 0xe1, 0x6b, 0x38, 0x17, 0x25, + 0xff, 0xa3, 0x28, 0xcc, 0x9e, 0x66, 0xb9, 0xdf, 0x07, 0xd9, 0x67, 0xb7, 0xd4, 0x8e, 0x41, 0x1c, + 0x21, 0x9b, 0x77, 0x46, 0xa7, 0xf3, 0x3e, 0xe3, 0x27, 0x44, 0x31, 0x65, 0x07, 0x4d, 0xe2, 0x87, + 0x90, 0xd2, 0xf9, 0xc2, 0x45, 0xbd, 0x5d, 0xdc, 0xd4, 0x19, 0x95, 0x86, 0x9f, 0x62, 0x00, 0x05, + 0x7a, 0x52, 0xf7, 0x35, 0x11, 0xd4, 0x86, 0xa4, 0x07, 0x7e, 0xd8, 0x13, 0xf9, 0xdc, 0xf3, 0x1b, + 0xc3, 0x69, 0x77, 0x16, 0x8a, 0x89, 0x76, 0x61, 0xc6, 0x9d, 0xc4, 0x4d, 0xfd, 0xe3, 0x2f, 0x64, + 0x1a, 0x97, 0x31, 0x0d, 0x51, 0x0a, 0xf8, 0x8e, 0x04, 0xb3, 0xee, 0x4c, 0x5e, 0xc9, 0xda, 0xd0, + 0x99, 0x39, 0x4b, 0x96, 0x1a, 0xc7, 0x47, 0xb9, 0xb4, 0xe0, 0x8c, 0x5b, 0x39, 0x79, 0x6e, 0xb9, + 0x4b, 0xeb, 0x43, 0x80, 0x3a, 0x8d, 0x49, 0x68, 0x3b, 0x9d, 0x78, 0x6a, 0x10, 0x93, 0x50, 0xc3, + 0xf6, 0xfc, 0x31, 0x09, 0x7d, 0xac, 0xe9, 0xe8, 0xbb, 0x12, 0xa4, 0xf9, 0x31, 0x59, 0xb7, 0xef, + 0x68, 0xfc, 0xa0, 0xdc, 0xad, 0x09, 0xbc, 0x7f, 0x7c, 0x94, 0x9b, 0x61, 0x9f, 0x77, 0x5d, 0xb4, + 0xb1, 0x69, 0x4b, 0x97, 0x9d, 0x76, 0x80, 0x22, 0x52, 0x68, 0x8f, 0xa0, 0xa3, 0x55, 0x48, 0xf1, + 0x42, 0x89, 0x7b, 0x13, 0x90, 0x15, 0x0b, 0x92, 0xa5, 0x9b, 0x1f, 0x1f, 0xe5, 0x16, 0x4e, 0xd1, + 0x13, 0x5e, 0x63, 0x79, 0xc8, 0xfb, 0x2a, 0xc9, 0x1d, 0xff, 0x2b, 0x5a, 0x83, 0x19, 0x1e, 0xc8, + 0x0e, 0xae, 0xf7, 0xc0, 0xf8, 0x87, 0xbb, 0x3c, 0x08, 0xf6, 0xa8, 0xa2, 0xfe, 0x35, 0x0f, 0x73, + 0xa7, 0xc6, 0x60, 0x3f, 0x8f, 0xc0, 0x7c, 0xd0, 0xac, 0x7a, 0x51, 0x92, 0x3a, 0xec, 0x6f, 0xdf, + 0x1a, 0xdb, 0x34, 0x7b, 0x17, 0x7d, 0x98, 0x69, 0x74, 0xdf, 0x86, 0x3d, 0xee, 0x47, 0x43, 0xde, + 0xeb, 0x12, 0xf8, 0xec, 0xf3, 0x0e, 0xe1, 0xbb, 0x2e, 0xec, 0x3d, 0xcf, 0xb3, 0xf0, 0x8a, 0xd7, + 0xdb, 0x97, 0x80, 0x67, 0xe3, 0xbd, 0x4b, 0x4a, 0xae, 0x6f, 0xf9, 0x47, 0x09, 0x92, 0x81, 0x9d, + 0xfd, 0x2a, 0x9d, 0xcb, 0x96, 0x17, 0x5b, 0xf1, 0xcb, 0x46, 0x6f, 0x5c, 0x7c, 0x5b, 0xc1, 0x90, + 0x2b, 0xfb, 0x57, 0x12, 0x24, 0x03, 0x8c, 0xfc, 0x84, 0xdc, 0xd2, 0x8b, 0x5f, 0x79, 0x0b, 0x52, + 0xc1, 0x4f, 0xe4, 0x9b, 0x43, 0x7a, 0x31, 0x73, 0xe4, 0xbf, 0x0c, 0x11, 0x4e, 0x41, 0x08, 0x52, + 0xef, 0x16, 0x6b, 0xcd, 0xda, 0xc6, 0xb2, 0xba, 0xb4, 0xa9, 0xa8, 0xcb, 0x65, 0x79, 0x02, 0x4d, + 0x43, 0xac, 0x52, 0x5d, 0xab, 0x52, 0xa2, 0x2c, 0xa1, 0x04, 0x44, 0xd9, 0x5b, 0xb5, 0x22, 0x87, + 0xf2, 0x25, 0x90, 0x39, 0xf6, 0x0e, 0xa6, 0x6e, 0x86, 0x66, 0x25, 0xa8, 0x00, 0xb3, 0x2c, 0x83, + 0xe8, 0xd2, 0xc8, 0x8a, 0xaa, 0xb7, 0xea, 0x8b, 0xc5, 0xd3, 0x5e, 0x13, 0xd5, 0xde, 0x0d, 0xad, + 0x8b, 0xf3, 0x3f, 0x09, 0x43, 0x7a, 0x00, 0xe2, 0x3a, 0xd9, 0x3f, 0x93, 0x06, 0xf9, 0x51, 0xe4, + 0xdc, 0xf3, 0xce, 0x13, 0xe3, 0x45, 0xaa, 0x24, 0xce, 0x1d, 0xdf, 0xa5, 0x4a, 0xf3, 0xf1, 0x51, + 0x2e, 0x3d, 0xbc, 0x58, 0xf2, 0x9c, 0x07, 0x92, 0xee, 0x12, 0x59, 0xcd, 0xd7, 0x30, 0xf7, 0xd5, + 0xc1, 0x7d, 0x30, 0x5e, 0xf3, 0x35, 0xcc, 0xfd, 0x6d, 0xa5, 0xa6, 0x44, 0x69, 0xe3, 0xb6, 0x6d, + 0xa0, 0x3a, 0x84, 0xad, 0x9e, 0xe3, 0x26, 0xe4, 0x5f, 0xba, 0xd0, 0x96, 0x36, 0x7b, 0x62, 0x3f, + 0x0a, 0xc3, 0x40, 0x75, 0x7e, 0x84, 0x3f, 0x60, 0x34, 0x33, 0xc8, 0x63, 0x9a, 0xd0, 0x64, 0xe0, + 0x43, 0x64, 0x77, 0x61, 0xda, 0xcf, 0xb1, 0x53, 0x4e, 0x2e, 0x8a, 0xc1, 0xa2, 0xc9, 0x67, 0xc6, + 0x5a, 0xba, 0x48, 0x58, 0x7d, 0x87, 0x59, 0x5f, 0x86, 0xb8, 0xb7, 0x8f, 0x8b, 0x9c, 0xe9, 0x71, + 0x1b, 0xef, 0x55, 0x04, 0xa7, 0xe4, 0x48, 0xfe, 0xfb, 0x21, 0x98, 0x56, 0x30, 0xb1, 0x3a, 0x07, + 0x58, 0xa7, 0x11, 0x94, 0x77, 0x4d, 0x57, 0x1a, 0xff, 0x9a, 0x6e, 0x11, 0xe2, 0x03, 0x0f, 0x74, + 0x81, 0x3b, 0x80, 0x83, 0x51, 0xe8, 0x7d, 0x48, 0xb6, 0xac, 0xbe, 0xa9, 0x6b, 0xf6, 0x21, 0x8b, + 0xab, 0x58, 0x04, 0x92, 0x1a, 0x79, 0x23, 0xcb, 0xbf, 0xea, 0x42, 0x49, 0x0c, 0xa6, 0xf1, 0x93, + 0x32, 0xdd, 0xf2, 0xbd, 0xe5, 0xdf, 0x84, 0x69, 0x7f, 0x2b, 0x8a, 0x41, 0x78, 0x63, 0x73, 0xa3, + 0xca, 0x75, 0xb2, 0x54, 0x2c, 0xaf, 0x2e, 0xd5, 0xd6, 0xd6, 0x64, 0x89, 0xd2, 0xab, 0xef, 0xd5, + 0x9a, 0x72, 0x88, 0xdf, 0x9a, 0x6c, 0x34, 0x8b, 0x4a, 0xd3, 0x2d, 0x9c, 0xe6, 0x31, 0x24, 0xfd, + 0xf3, 0x51, 0xcb, 0x47, 0xc3, 0x4e, 0x46, 0x08, 0x64, 0xde, 0xaf, 0x8e, 0xb9, 0x62, 0x57, 0x82, + 0x6c, 0x3f, 0x6a, 0xfe, 0x5f, 0x42, 0x80, 0x06, 0x1f, 0xde, 0x33, 0x56, 0xef, 0x01, 0xb4, 0xf7, + 0x70, 0x7b, 0xbf, 0x67, 0x19, 0xa6, 0x23, 0x72, 0xcd, 0x37, 0xc6, 0x92, 0x1d, 0xcf, 0x58, 0x95, + 0xbd, 0xf1, 0x8a, 0x0f, 0x0b, 0xfd, 0xe1, 0xe8, 0x12, 0xfd, 0x24, 0x2b, 0xd1, 0x33, 0xcd, 0xff, + 0x95, 0x96, 0xe9, 0xb3, 0x45, 0x80, 0xc1, 0x8a, 0xd1, 0x3d, 0x98, 0xba, 0x40, 0x75, 0x83, 0xf7, + 0xf5, 0xcb, 0x7a, 0xfe, 0x7f, 0xc3, 0x80, 0xca, 0x36, 0xd6, 0x1c, 0x4c, 0x4d, 0x34, 0x19, 0x55, + 0xe2, 0x28, 0xc1, 0x14, 0x4f, 0xe9, 0x43, 0x17, 0x49, 0xe9, 0xdd, 0xa9, 0xd9, 0x50, 0xf4, 0x75, + 0x98, 0x6e, 0x5b, 0x9d, 0x7e, 0xd7, 0x54, 0xd9, 0x45, 0x20, 0x91, 0x7f, 0x7c, 0x71, 0xd4, 0x17, + 0x3b, 0xb1, 0xb8, 0x42, 0xd9, 0xea, 0xd0, 0x77, 0xef, 0xbe, 0x39, 0x03, 0x64, 0x3d, 0xd0, 0x35, + 0x88, 0x7b, 0x96, 0x87, 0x89, 0x43, 0x5c, 0x19, 0x10, 0xd0, 0x5d, 0x98, 0xd2, 0x88, 0x6a, 0xed, + 0xb0, 0x60, 0xfa, 0x3c, 0x55, 0x54, 0xc2, 0x1a, 0xd9, 0xdc, 0x41, 0xaf, 0x41, 0xba, 0xab, 0x3d, + 0x51, 0x77, 0x6c, 0x7e, 0xa3, 0x58, 0x35, 0xf4, 0x0e, 0xb7, 0x84, 0x92, 0x32, 0xd3, 0xd5, 0x9e, + 0x2c, 0x09, 0x7a, 0x4d, 0xef, 0x60, 0x74, 0x0f, 0x92, 0x3b, 0x8f, 0x78, 0x6a, 0xc5, 0xbd, 0x12, + 0xbf, 0x55, 0x35, 0x73, 0x7c, 0x94, 0x4b, 0x2c, 0xbd, 0xc3, 0x18, 0x43, 0x7d, 0x92, 0x92, 0xd8, + 0x79, 0xe4, 0xbd, 0x64, 0xff, 0x47, 0x82, 0xa8, 0xd8, 0x11, 0xea, 0x01, 0x08, 0xf6, 0x18, 0x3a, + 0xff, 0xa6, 0xc9, 0xd2, 0x3b, 0xc7, 0x47, 0xb9, 0x78, 0x99, 0x51, 0x6b, 0x15, 0xf2, 0xf1, 0x51, + 0xee, 0xed, 0xcb, 0x7a, 0x14, 0x17, 0x44, 0x89, 0xf3, 0x49, 0x6a, 0x3a, 0xab, 0x0b, 0xef, 0x69, + 0x44, 0xdd, 0x33, 0x88, 0x63, 0xed, 0xda, 0x5a, 0x57, 0xdc, 0x0f, 0x98, 0xde, 0xd3, 0xc8, 0x8a, + 0x4b, 0x43, 0x59, 0x1a, 0x9b, 0x1d, 0xf0, 0x7b, 0x5c, 0xfc, 0xb6, 0x88, 0xf7, 0x8e, 0xee, 0xc2, + 0x15, 0x6f, 0xb0, 0x4a, 0x39, 0xd5, 0xea, 0xb7, 0xf7, 0x31, 0xf3, 0x41, 0xd4, 0xb8, 0xcf, 0x7a, + 0x8d, 0xeb, 0xda, 0x93, 0x12, 0x6f, 0xca, 0x5f, 0x81, 0x59, 0xdf, 0x67, 0xf5, 0x22, 0x69, 0x0c, + 0x32, 0xbf, 0xc6, 0xe0, 0xfb, 0x09, 0xc5, 0x3b, 0x30, 0x33, 0xf4, 0x0b, 0x21, 0x61, 0x7f, 0xfd, + 0x15, 0xc7, 0xe0, 0x4f, 0x8a, 0x0a, 0x65, 0xfe, 0xea, 0xe6, 0x06, 0xa9, 0x76, 0xe0, 0x3d, 0x3f, + 0x0b, 0x69, 0x6f, 0x1a, 0x6f, 0xee, 0xff, 0x08, 0x41, 0xaa, 0xfa, 0x04, 0xb7, 0xfb, 0x94, 0x5a, + 0x3d, 0xa0, 0xd2, 0xb2, 0x0c, 0x61, 0x66, 0x70, 0x79, 0x18, 0x74, 0x6f, 0xd4, 0x95, 0xe1, 0xc0, + 0xc0, 0x02, 0xfb, 0xcb, 0xec, 0x2d, 0x03, 0x40, 0x39, 0x48, 0x18, 0x26, 0x71, 0x34, 0xb3, 0x8d, + 0xdd, 0x4b, 0x0f, 0x53, 0x0a, 0xb8, 0xa4, 0x9a, 0x8e, 0xe6, 0x03, 0xc5, 0xbe, 0xb8, 0x1b, 0x38, + 0x51, 0xd9, 0xc3, 0x07, 0x5e, 0x9c, 0xd3, 0x35, 0xda, 0xb6, 0x45, 0xc4, 0x91, 0xcb, 0x0c, 0x6b, + 0xa0, 0xc2, 0xba, 0xce, 0xc8, 0xe8, 0x55, 0x98, 0xc1, 0xee, 0x32, 0x54, 0xf6, 0xc3, 0x31, 0x26, + 0xe5, 0x71, 0x25, 0xe5, 0x91, 0xab, 0x94, 0x9a, 0x27, 0x10, 0xf7, 0x16, 0x88, 0x6e, 0x43, 0x62, + 0x7b, 0xa3, 0xb1, 0x55, 0x2d, 0xd7, 0x96, 0x6a, 0xd5, 0x8a, 0x3c, 0x91, 0xbd, 0xfa, 0xf4, 0xd9, + 0xc2, 0x6c, 0xdd, 0x6a, 0xb1, 0x2e, 0xdb, 0x26, 0xe9, 0xe1, 0xb6, 0xb1, 0x63, 0x60, 0x1d, 0x5d, + 0x83, 0x29, 0x6e, 0xf8, 0xa5, 0x6c, 0xfa, 0xe9, 0xb3, 0x85, 0x64, 0xdd, 0x6a, 0xb1, 0xc3, 0x1e, + 0xce, 0xab, 0x0c, 0x4c, 0x56, 0x37, 0x2a, 0x72, 0x28, 0x3b, 0xf3, 0xf4, 0xd9, 0x42, 0x82, 0x8e, + 0x37, 0x75, 0xd6, 0x92, 0x0d, 0xff, 0xee, 0xf7, 0x6e, 0x4c, 0xe4, 0xff, 0x21, 0x05, 0xd1, 0x2d, + 0xed, 0xb0, 0x63, 0x69, 0x3a, 0x5a, 0x80, 0x84, 0x7b, 0xbf, 0xcd, 0xfd, 0x9c, 0x71, 0xc5, 0x4f, + 0x0a, 0x6a, 0xb1, 0xcc, 0xce, 0xed, 0x7c, 0x5a, 0x6c, 0x40, 0xaa, 0x4f, 0xb0, 0x4d, 0x15, 0x4c, + 0x65, 0xbf, 0x16, 0xe3, 0xde, 0xbc, 0x54, 0xfa, 0xf8, 0x28, 0xf7, 0x60, 0x3c, 0xdd, 0xc0, 0xed, + 0xbe, 0x6d, 0x38, 0x87, 0x85, 0xc6, 0x3b, 0x6b, 0xdb, 0x02, 0x8a, 0x9a, 0x62, 0x4b, 0x49, 0xf6, + 0xfd, 0xaf, 0xe2, 0x1a, 0x23, 0x95, 0x73, 0x97, 0xfb, 0xe2, 0x74, 0x4c, 0x50, 0x07, 0xbc, 0xdf, + 0x31, 0x4c, 0x76, 0x14, 0x1c, 0xfc, 0x4a, 0x29, 0x97, 0x2c, 0x3a, 0x1e, 0x40, 0xca, 0x77, 0x7f, + 0x90, 0xea, 0x78, 0x84, 0xe9, 0xf8, 0xe6, 0xf1, 0x51, 0x2e, 0x39, 0xb0, 0x99, 0x5c, 0xcf, 0x9f, + 0x27, 0x72, 0x4c, 0x0e, 0xa6, 0xa1, 0x5a, 0x3e, 0x07, 0x53, 0x5c, 0x24, 0x62, 0x3c, 0xee, 0x61, + 0x2f, 0xa8, 0x0a, 0x49, 0x51, 0x69, 0xe2, 0x3f, 0x34, 0x14, 0x97, 0x2e, 0x17, 0x7c, 0x92, 0xee, + 0xfe, 0x14, 0xb1, 0x50, 0x35, 0xdb, 0x96, 0x8e, 0x75, 0x26, 0x42, 0x8a, 0x28, 0xac, 0xb3, 0x17, + 0x82, 0x96, 0x21, 0xd5, 0xee, 0x60, 0xcd, 0xec, 0xf7, 0x5c, 0x1c, 0x34, 0x26, 0x4e, 0x52, 0x8c, + 0x13, 0x40, 0x1b, 0x80, 0x76, 0xd8, 0x9d, 0x32, 0xff, 0xaa, 0xd8, 0x41, 0xf0, 0x38, 0x60, 0x32, + 0x1b, 0xab, 0x0c, 0x56, 0x86, 0x6e, 0x42, 0xd2, 0xb4, 0xcc, 0x36, 0x55, 0xb2, 0x0e, 0x73, 0x5c, + 0xfc, 0xec, 0x38, 0x48, 0x44, 0x25, 0x88, 0xf0, 0xfb, 0x08, 0xa2, 0x44, 0x70, 0x7b, 0xdc, 0x5f, + 0x4f, 0xac, 0x4c, 0x28, 0x62, 0x24, 0xaa, 0x42, 0xd4, 0xe6, 0x77, 0x63, 0xd8, 0x1d, 0x85, 0x73, + 0x4b, 0x75, 0xbe, 0x9b, 0x37, 0x2b, 0x13, 0x8a, 0x3b, 0x16, 0x35, 0xdd, 0x2b, 0xc2, 0x3c, 0x4c, + 0x11, 0xb7, 0x3c, 0x0b, 0x63, 0x26, 0x60, 0x03, 0xc0, 0x00, 0x0a, 0xdd, 0xa0, 0xc1, 0x4e, 0x0e, + 0xd9, 0xed, 0x85, 0xd1, 0x1b, 0x0c, 0xdc, 0x92, 0xa1, 0x1b, 0xe4, 0x23, 0xd1, 0x06, 0x8d, 0xb3, + 0xdc, 0xd0, 0x89, 0xdd, 0x6b, 0x48, 0xdc, 0xfd, 0xec, 0x45, 0xd2, 0x8b, 0x95, 0x09, 0xc5, 0x87, + 0x80, 0xde, 0x81, 0x44, 0x7b, 0xe0, 0x01, 0x32, 0x33, 0x0c, 0xf0, 0xf5, 0x0b, 0x85, 0x01, 0x2b, + 0xd4, 0xf5, 0x0f, 0xa8, 0xe8, 0x03, 0x48, 0x91, 0x40, 0x3a, 0x9a, 0xb9, 0xc2, 0x50, 0x3f, 0x77, + 0xd1, 0x72, 0xf8, 0xca, 0x84, 0x32, 0x84, 0x84, 0x7e, 0x13, 0x64, 0x67, 0xe8, 0x0c, 0x8e, 0x5d, + 0x37, 0x18, 0x7d, 0x05, 0xf7, 0x8c, 0x93, 0xc6, 0x95, 0x09, 0xe5, 0x04, 0x1a, 0xfa, 0x08, 0x66, + 0x48, 0xf0, 0xa7, 0x5e, 0x99, 0xab, 0x6c, 0x82, 0xcf, 0x8f, 0xff, 0xe3, 0xb0, 0x01, 0xfe, 0x30, + 0x16, 0x85, 0x37, 0x83, 0x47, 0x79, 0xec, 0xda, 0xcb, 0x68, 0xf8, 0xd3, 0x8f, 0x16, 0x29, 0xfc, + 0x10, 0x16, 0x5a, 0x85, 0x78, 0xd7, 0x75, 0xa9, 0xec, 0xd2, 0xc8, 0xe8, 0x0c, 0x6e, 0xd8, 0xcb, + 0xaf, 0x4c, 0x28, 0x83, 0xf1, 0xe8, 0x77, 0x24, 0xb8, 0xa6, 0x8d, 0x38, 0xf3, 0xcb, 0x7c, 0xea, + 0xdc, 0x63, 0x8e, 0x31, 0x4e, 0x16, 0x57, 0x26, 0x94, 0x91, 0xb3, 0xa0, 0x4f, 0xc3, 0x74, 0x4f, + 0xeb, 0x13, 0xac, 0xda, 0x58, 0x23, 0x96, 0x99, 0xb9, 0xc6, 0xfd, 0x14, 0xa3, 0x29, 0x8c, 0x84, + 0xd6, 0x61, 0xda, 0x73, 0xae, 0x6b, 0xd6, 0x6e, 0xe6, 0xfa, 0xb9, 0x65, 0xfa, 0x60, 0xa4, 0xa0, + 0x04, 0x86, 0x97, 0xe2, 0x10, 0x15, 0x67, 0xdd, 0xde, 0x45, 0x14, 0x7e, 0x05, 0x85, 0x5f, 0x3e, + 0xc9, 0xca, 0x9f, 0xca, 0xff, 0x10, 0x20, 0xe6, 0x25, 0x3f, 0x8b, 0x80, 0xbc, 0xb0, 0x74, 0x70, + 0x25, 0x9f, 0xfa, 0xd3, 0xd0, 0xca, 0x84, 0x92, 0x76, 0xdb, 0x06, 0xb7, 0xf2, 0x1f, 0x04, 0xee, + 0xeb, 0x8d, 0xf3, 0xab, 0x51, 0xfa, 0x65, 0xbc, 0x0b, 0x7d, 0xd4, 0xcf, 0x89, 0x5b, 0xdb, 0x9e, + 0x9f, 0xe3, 0xa7, 0x1a, 0x29, 0x97, 0x2c, 0xfc, 0xdc, 0x2d, 0x48, 0xd9, 0x7d, 0x93, 0x1d, 0x71, + 0x8b, 0xc0, 0x86, 0xc7, 0xe2, 0x49, 0x41, 0x15, 0xe5, 0xa0, 0xf2, 0x90, 0xe9, 0xbd, 0x73, 0xae, + 0xe9, 0x75, 0xf7, 0xbe, 0x22, 0x79, 0xb6, 0x77, 0x69, 0xd8, 0xf6, 0xbe, 0x76, 0xbe, 0xed, 0xf5, + 0xc1, 0x78, 0xc6, 0x77, 0xfb, 0x54, 0xe3, 0xbb, 0x38, 0xa6, 0xf5, 0xf0, 0x21, 0x06, 0xad, 0x6f, + 0x79, 0xc8, 0xfa, 0xde, 0x39, 0xd7, 0xfa, 0xfa, 0xf7, 0x28, 0xcc, 0xef, 0xe6, 0x29, 0xe6, 0xf7, + 0xf5, 0x0b, 0xa5, 0xb9, 0x2b, 0x52, 0xc0, 0xfe, 0x2a, 0xa7, 0xd9, 0xdf, 0xc2, 0x78, 0xf6, 0xd7, + 0x07, 0x19, 0x30, 0xc0, 0x1f, 0x9e, 0x30, 0xc0, 0xf2, 0xf9, 0x16, 0xec, 0xd4, 0x02, 0xe2, 0x8a, + 0x74, 0xc2, 0x02, 0x6b, 0xa7, 0x58, 0xe0, 0x34, 0x83, 0xbf, 0x77, 0x01, 0x0b, 0xec, 0x9b, 0xe0, + 0xa4, 0x09, 0x7e, 0x0f, 0xa6, 0xfd, 0x66, 0x93, 0x5d, 0x4c, 0x1b, 0x6d, 0xe0, 0xcf, 0xf8, 0x71, + 0x2e, 0x93, 0x01, 0x5f, 0x13, 0xfa, 0xfa, 0x49, 0xeb, 0x3b, 0x7b, 0x2e, 0xf8, 0x19, 0x57, 0x2f, + 0x56, 0xa4, 0x93, 0xe6, 0x77, 0xcd, 0x6f, 0x7e, 0xe7, 0xce, 0x75, 0xce, 0x27, 0xb2, 0x9f, 0x15, + 0xc9, 0x6f, 0x7f, 0xbf, 0x2b, 0xc1, 0xb5, 0x51, 0x06, 0x54, 0x78, 0xbe, 0xb7, 0x2e, 0x69, 0x7f, + 0x7d, 0x93, 0x8e, 0x9c, 0x06, 0xbd, 0x02, 0x31, 0xc7, 0xd6, 0x78, 0xce, 0x74, 0x85, 0x9d, 0xad, + 0xb2, 0xa2, 0x67, 0x93, 0xd2, 0x58, 0x71, 0x94, 0x3d, 0xe8, 0x25, 0x80, 0x98, 0x7b, 0x13, 0xc9, + 0x67, 0x42, 0xf3, 0xdf, 0x93, 0x60, 0xb2, 0x6e, 0xb5, 0xd0, 0x75, 0x5f, 0x25, 0x3e, 0x29, 0xea, + 0x35, 0x53, 0x75, 0xab, 0x25, 0x4a, 0xea, 0x6f, 0x0d, 0x46, 0x8b, 0xc2, 0xc6, 0xcb, 0x23, 0x36, + 0xe6, 0x1d, 0x64, 0x78, 0x83, 0xd0, 0xd7, 0x20, 0xda, 0xe3, 0x99, 0x8d, 0xb0, 0xa8, 0xf9, 0x51, + 0xe3, 0x79, 0x4f, 0xc5, 0x1d, 0xf2, 0xda, 0x1d, 0xff, 0xff, 0xdf, 0xb0, 0x6e, 0xe9, 0x18, 0xa5, + 0x00, 0xb6, 0x34, 0x42, 0x7a, 0x7b, 0xb6, 0x46, 0xb0, 0x3c, 0x81, 0xa2, 0x30, 0xb9, 0xba, 0xde, + 0x90, 0xa5, 0xd7, 0xde, 0xf3, 0x97, 0xd1, 0x2b, 0x4a, 0xb1, 0xb6, 0x51, 0xdb, 0x58, 0x56, 0x37, + 0x8a, 0xeb, 0xd5, 0x86, 0x3c, 0x81, 0x32, 0x30, 0xf7, 0x6e, 0xb1, 0xd6, 0x14, 0x75, 0x75, 0xb5, + 0xb6, 0xd1, 0xac, 0x2a, 0x0f, 0x8b, 0x6b, 0xb2, 0x84, 0xe6, 0x01, 0x29, 0x9b, 0xe5, 0xd5, 0x46, + 0xa5, 0xa4, 0x96, 0x37, 0xd7, 0xb7, 0x8a, 0xe5, 0x66, 0x6d, 0x73, 0x43, 0x0e, 0xa1, 0x18, 0x84, + 0x2b, 0x9b, 0x1b, 0x55, 0x19, 0x5e, 0xfb, 0x45, 0x18, 0xc2, 0x2c, 0x1d, 0xbc, 0x39, 0x9c, 0x0e, + 0xce, 0x3e, 0x7d, 0xb6, 0x30, 0x43, 0x9b, 0xfc, 0xa9, 0x60, 0x16, 0x22, 0xa5, 0x62, 0x79, 0x75, + 0x7b, 0x4b, 0x96, 0xb2, 0xa9, 0xa7, 0xcf, 0x16, 0x80, 0x76, 0xe0, 0xf6, 0x19, 0x5d, 0xe3, 0x15, + 0xc2, 0x4d, 0xa5, 0xea, 0x26, 0x83, 0x2c, 0x11, 0x16, 0x36, 0xf6, 0x55, 0x48, 0x36, 0xca, 0x2b, + 0xd5, 0xf5, 0xa2, 0x5a, 0x5e, 0x29, 0x6e, 0x2c, 0x57, 0xe5, 0xc9, 0xec, 0xdc, 0xd3, 0x67, 0x0b, + 0xf2, 0xb0, 0x9e, 0xd2, 0x29, 0x6a, 0xeb, 0x5b, 0x9b, 0x4a, 0x53, 0x0e, 0x0f, 0xa6, 0xe0, 0xe6, + 0x11, 0xe5, 0x01, 0xf8, 0xe8, 0xa5, 0x6a, 0xb5, 0x22, 0x4f, 0x65, 0xd1, 0xd3, 0x67, 0x0b, 0x29, + 0xda, 0x3e, 0xb0, 0x7a, 0xe8, 0x16, 0x4c, 0x97, 0x95, 0x6a, 0xb1, 0x59, 0x55, 0x1b, 0xcd, 0x62, + 0xb3, 0x21, 0x47, 0x06, 0x3b, 0xf1, 0x59, 0x32, 0x54, 0x80, 0x74, 0x71, 0xbb, 0xb9, 0xa9, 0x06, + 0xfa, 0x46, 0x79, 0x12, 0x4c, 0xfb, 0x52, 0xf9, 0xf4, 0xf7, 0xff, 0x2c, 0xc8, 0x81, 0xf5, 0xab, + 0xcb, 0x65, 0x39, 0x96, 0x9d, 0x7f, 0xfa, 0x6c, 0x01, 0x0d, 0x6f, 0x61, 0xb9, 0x8c, 0xbe, 0x00, + 0xf3, 0xcd, 0xf7, 0xb7, 0xaa, 0x95, 0x6a, 0xa3, 0xac, 0x06, 0xb7, 0x1d, 0xcf, 0x66, 0x9e, 0x3e, + 0x5b, 0x98, 0xa3, 0x63, 0x4e, 0x6c, 0xfd, 0x75, 0x90, 0x1b, 0x4d, 0xa5, 0x5a, 0x5c, 0x57, 0x6b, + 0x1b, 0xcb, 0xd5, 0x06, 0xfb, 0x58, 0x30, 0x58, 0xd2, 0x90, 0xcd, 0xa1, 0x5b, 0xd8, 0xa8, 0xbe, + 0x3b, 0x84, 0x9f, 0x18, 0xf4, 0x1f, 0x32, 0x23, 0x68, 0x01, 0xe2, 0xeb, 0xb5, 0x65, 0xa5, 0xc8, + 0x70, 0xa7, 0x79, 0x2e, 0x4f, 0xfb, 0x79, 0x46, 0x01, 0xd5, 0x20, 0xc7, 0x98, 0xd2, 0xd8, 0x2a, + 0x6e, 0xa8, 0xe5, 0xcd, 0x8d, 0xa5, 0xda, 0xb2, 0xaa, 0x54, 0xcb, 0x9b, 0x1b, 0xe5, 0xda, 0x5a, + 0x8d, 0x8f, 0x4b, 0x66, 0x6f, 0x3e, 0x7d, 0xb6, 0xb0, 0xe0, 0xb2, 0xe8, 0x2c, 0x15, 0xce, 0xc6, + 0x68, 0xf2, 0xff, 0xc3, 0xef, 0xdf, 0x98, 0x28, 0xdd, 0xfe, 0xe9, 0x7f, 0xdd, 0x98, 0xf8, 0xe9, + 0xf1, 0x0d, 0xe9, 0x67, 0xc7, 0x37, 0xa4, 0x9f, 0x1f, 0xdf, 0x90, 0x7e, 0x71, 0x7c, 0x43, 0xfa, + 0xbd, 0x5f, 0xde, 0x98, 0xf8, 0xd9, 0x2f, 0x6f, 0x4c, 0xfc, 0xfc, 0x97, 0x37, 0x26, 0x3e, 0x88, + 0x70, 0x15, 0x69, 0x45, 0x58, 0x4e, 0x7f, 0xef, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0x76, 0x3f, + 0xe3, 0x05, 0x3a, 0x47, 0x00, 0x00, } func (this *BackupEncryptionOptions) Equal(that interface{}) bool { @@ -6115,6 +6217,58 @@ 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 *Payload) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -6135,6 +6289,22 @@ func (m *Payload) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.ExecutionLog) > 0 { + for iNdEx := len(m.ExecutionLog) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.ExecutionLog[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintJobs(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xea + } + } if len(m.PauseReason) > 0 { i -= len(m.PauseReason) copy(dAtA[i:], m.PauseReason) @@ -8073,6 +8243,32 @@ 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 *Payload) Size() (n int) { if m == nil { return 0 @@ -8136,6 +8332,12 @@ 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 { + l = e.Size() + n += 2 + l + sovJobs(uint64(l)) + } + } return n } @@ -16496,6 +16698,177 @@ func (m *MigrationProgress) Unmarshal(dAtA []byte) error { } return nil } +func (m *ExecutionEvent) 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: ExecutionEvent: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecutionEvent: 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) + } + 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 + } + } + 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 *Payload) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -17343,6 +17716,40 @@ func (m *Payload) Unmarshal(dAtA []byte) error { } m.PauseReason = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 29: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecutionLog", 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 + } + m.ExecutionLog = append(m.ExecutionLog, &ExecutionEvent{}) + if err := m.ExecutionLog[len(m.ExecutionLog)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipJobs(dAtA[iNdEx:]) diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index a1d6cf698b3b..65ea0da190a2 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -708,6 +708,40 @@ 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 Payload { string description = 1; // If empty, the description is assumed to be the statement. @@ -757,7 +791,17 @@ 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; - // NEXT ID: 29. + + // 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: 30. } message Progress { diff --git a/pkg/jobs/jobspb/json_encoding.go b/pkg/jobs/jobspb/json_encoding.go new file mode 100644 index 000000000000..bc5f6ef2e8a2 --- /dev/null +++ b/pkg/jobs/jobspb/json_encoding.go @@ -0,0 +1,32 @@ +// 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/registry.go b/pkg/jobs/registry.go index 6dbfc81c28f5..f7be4cfc5cc1 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -1372,3 +1372,21 @@ func (r *Registry) unregister(jobID jobspb.JobID) { delete(r.mu.adoptedJobs, jobID) } } + +// RetryInitialDelay returns the value of retryInitialDelaySetting cluster setting, +// in seconds, which is the initial delay in exponential-backoff delay calculation. +func (r *Registry) RetryInitialDelay() float64 { + if r.knobs.IntervalOverrides.RetryInitialDelay != nil { + return r.knobs.IntervalOverrides.RetryInitialDelay.Seconds() + } + return retryInitialDelaySetting.Get(&r.settings.SV).Seconds() +} + +// RetryMaxDelay returns the value of retryMaxDelaySetting cluster setting, +// in seconds, which is the maximum delay between retries of a job. +func (r *Registry) RetryMaxDelay() float64 { + if r.knobs.IntervalOverrides.RetryMaxDelay != nil { + return r.knobs.IntervalOverrides.RetryMaxDelay.Seconds() + } + return retryMaxDelaySetting.Get(&r.settings.SV).Seconds() +} diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 3e1a10a28655..ced209792473 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -669,7 +670,11 @@ CREATE TABLE crdb_internal.jobs ( high_water_timestamp DECIMAL, error STRING, coordinator_id INT, - trace_id INT + trace_id INT, + last_run TIMESTAMP, + next_run TIMESTAMP, + num_runs INT, + execution_log JSON )`, comment: `decoded job metadata from system.jobs (KV scan)`, generator: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { @@ -686,11 +691,26 @@ CREATE TABLE crdb_internal.jobs ( // Beware: we're querying system.jobs as root; we need to be careful to filter // out results that the current user is not able to see. - query := `SELECT id, status, created, payload, progress, claim_session_id, claim_instance_id FROM system.jobs` + const ( + qSelect = `SELECT id, status, created, payload, progress, claim_session_id, claim_instance_id` + qFrom = ` FROM system.jobs` + queryWithoutBackoff = qSelect + qFrom + backoffArgs = `(SELECT $1::FLOAT AS initial_delay, $2::FLOAT AS max_delay) args` + queryWithBackoff = qSelect + `, last_run, COALESCE(num_runs, 0), ` + jobs.NextRunClause + ` as next_run` + qFrom + ", " + backoffArgs + ) + query := queryWithoutBackoff + var args []interface{} + settings := p.execCfg.Settings + backoffIsEnabled := settings.Version.IsActive(ctx, clusterversion.RetryJobsWithExponentialBackoff) + if backoffIsEnabled { + query = queryWithBackoff + args = append(args, p.execCfg.JobRegistry.RetryInitialDelay(), p.execCfg.JobRegistry.RetryMaxDelay()) + } + it, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryIteratorEx( ctx, "crdb-internal-jobs-table", p.txn, sessiondata.InternalExecutorOverride{User: security.RootUserName()}, - query) + query, args...) if err != nil { return nil, nil, err } @@ -709,7 +729,7 @@ CREATE TABLE crdb_internal.jobs ( } // We'll reuse this container on each loop. - container := make(tree.Datums, 0, 16) + container := make(tree.Datums, 0, 21) return func() (datums tree.Datums, e error) { // Loop while we need to skip a row. for { @@ -723,7 +743,8 @@ CREATE TABLE crdb_internal.jobs ( var jobType, description, statement, username, descriptorIDs, started, runningStatus, finished, modified, fractionCompleted, highWaterTimestamp, errorStr, coordinatorID, - traceID = tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, tree.DNull, + traceID, lastRun, nextRun, numRuns, executionLog = 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 @@ -822,6 +843,17 @@ 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) + } + } + container = container[:0] container = append(container, id, @@ -841,6 +873,10 @@ CREATE TABLE crdb_internal.jobs ( errorStr, coordinatorID, traceID, + lastRun, + nextRun, + numRuns, + executionLog, ) return container, nil } diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index a3aad7e427cc..65408e854021 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -19,12 +19,15 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" @@ -44,6 +47,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/jackc/pgtype" @@ -803,3 +807,101 @@ func TestClusterInflightTracesVirtualTable(t *testing.T) { } }) } + +// TestInternalJobsTableRetryColumns tests values of last_run, next_run, and +// num_runs columns in crdb_internal.jobs table. The test creates a job in +// system.jobs table and retrieves the job's information from crdb_internal.jobs +// table for validation. +func TestInternalJobsTableRetryColumns(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testFn := func(validateFn func(context.Context, *sqlutils.SQLRunner)) func(t *testing.T) { + return func(t *testing.T) { + s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + JobsTestingKnobs: &jobs.TestingKnobs{ + DisableAdoptions: true, + }, + }, + }) + ctx := context.Background() + defer s.Stopper().Stop(ctx) + tdb := sqlutils.MakeSQLRunner(db) + + tdb.Exec(t, + "INSERT INTO system.jobs (id, status, created, payload) values ($1, $2, $3, 'test'::bytes)", + 1, jobs.StatusRunning, timeutil.Now(), + ) + + validateFn(ctx, tdb) + } + } + + t.Run("null values", testFn(func(_ context.Context, tdb *sqlutils.SQLRunner) { + // Values should be NULL if not populated. + tdb.CheckQueryResults(t, ` +SELECT last_run IS NULL, + next_run IS NOT NULL, + num_runs = 0, + execution_log IS NULL + FROM crdb_internal.jobs WHERE job_id = 1`, + [][]string{{"true", "true", "true", "true"}}) + })) + + t.Run("valid backoff params", testFn(func(_ context.Context, tdb *sqlutils.SQLRunner) { + lastRun := timeutil.Unix(1, 0) + tdb.Exec(t, "UPDATE system.jobs SET last_run = $1, num_runs = 1 WHERE id = 1", lastRun) + tdb.Exec(t, "SET CLUSTER SETTING jobs.registry.retry.initial_delay = '1s'") + tdb.Exec(t, "SET CLUSTER SETTING jobs.registry.retry.max_delay = '1s'") + + var validLastRun, validNextRun, validNumRuns bool + tdb.QueryRow(t, + "SELECT last_run = $1, next_run = $2, num_runs = 1 FROM crdb_internal.jobs WHERE job_id = 1", + lastRun, lastRun.Add(time.Second), + ).Scan(&validLastRun, &validNextRun, &validNumRuns) + require.True(t, validLastRun) + require.True(t, validNextRun) + require.True(t, validNumRuns) + })) + + t.Run("without new columns", func(t *testing.T) { + // This test validates the use of new columns in a cluster that does not + // support new system.jobs table. It creates the test cluster with a version + // that does not have exponential-backoff params columns in system.jobs table. + // We expect NULL values for the new columns in the internal jobs table when + // system.jobs does not have corresponding columns. + + clusterArgs := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: 1, + BinaryVersionOverride: clusterversion.ByKey( + clusterversion.RetryJobsWithExponentialBackoff - 1), + }, + JobsTestingKnobs: &jobs.TestingKnobs{ + DisableAdoptions: true, + }, + }, + }, + } + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, clusterArgs) + defer tc.Stopper().Stop(ctx) + sqlDB := tc.ServerConn(0) + tdb := sqlutils.MakeSQLRunner(sqlDB) + tdb.Exec(t, + "INSERT INTO system.jobs (id, status, created, payload) values ($1, $2, $3, 'test'::bytes)", + 1, jobs.StatusRunning, timeutil.Now(), + ) + tdb.CheckQueryResults(t, ` +SELECT last_run IS NULL, + next_run IS NULL, + num_runs IS NULL, + execution_log IS NULL + FROM crdb_internal.jobs WHERE job_id = 1`, + [][]string{{"true", "true", "true", "true"}}) + }) +} diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 55a1e5dbd475..8d1af883777c 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -152,10 +152,10 @@ Channel # The validity of the rows in this table are tested elsewhere; we merely assert the columns. -query ITTTTTTTTTTTRTTII colnames +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 +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 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 71b39403f7f5..4121370f3e53 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant @@ -168,10 +168,10 @@ Channel # The validity of the rows in this table are tested elsewhere; we merely assert the columns. -query ITTTTTTTTTTTRTTII colnames +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 +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 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 c30da7bf6555..d9d63bedeadf 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -516,7 +516,11 @@ CREATE TABLE crdb_internal.jobs ( high_water_timestamp DECIMAL NULL, error STRING NULL, coordinator_id INT8 NULL, - trace_id INT8 NULL + trace_id INT8 NULL, + last_run TIMESTAMP NULL, + next_run TIMESTAMP NULL, + num_runs INT8 NULL, + execution_log JSONB NULL ) CREATE TABLE crdb_internal.jobs ( job_id INT8 NULL, job_type STRING NULL, @@ -534,7 +538,11 @@ CREATE TABLE crdb_internal.jobs ( high_water_timestamp DECIMAL NULL, error STRING NULL, coordinator_id INT8 NULL, - trace_id INT8 NULL + trace_id INT8 NULL, + last_run TIMESTAMP NULL, + next_run TIMESTAMP NULL, + num_runs INT8 NULL, + execution_log JSONB NULL ) {} {} CREATE TABLE crdb_internal.kv_node_liveness ( node_id INT8 NOT NULL, diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index a3c2509aa8ef..890f5651fa8b 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -134,7 +134,7 @@ distribution: local vectorized: true · • sort -│ order: -column19,-started +│ order: -column23,-started │ └── • render │ From 6b89733c29ecd0e9161a7b1d30adb8a897cd7a28 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Tue, 24 Aug 2021 07:53:58 +0000 Subject: [PATCH 4/4] roachtest: increase consistency check timeout, and ignore errors This bumps the consistency check timeout to 5 minutes. There are indications that a recent libpq upgrade unmasked previously ignored context cancellation errors, caused by the timeout here being too low. It also ignores errors during the consistency check, since it is best-effort anyway. Release justification: non-production code changes Release note: None --- pkg/cmd/roachtest/cluster.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index a6e5d0568eca..1203c253c439 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1266,7 +1266,7 @@ func (c *clusterImpl) CheckReplicaDivergenceOnDB( // // We've seen the consistency checks hang indefinitely in some cases. rows, err := db.QueryContext(ctx, ` -SET statement_timeout = '3m'; +SET statement_timeout = '5m'; SELECT t.range_id, t.start_key_pretty, t.status, t.detail FROM crdb_internal.check_consistency(true, '', '') as t @@ -1278,20 +1278,22 @@ WHERE t.status NOT IN ('RANGE_CONSISTENT', 'RANGE_INDETERMINATE')`) l.Printf("consistency check failed with %v; ignoring", err) return nil } + defer rows.Close() var finalErr error for rows.Next() { var rangeID int32 var prettyKey, status, detail string if scanErr := rows.Scan(&rangeID, &prettyKey, &status, &detail); scanErr != nil { - return scanErr + l.Printf("consistency check failed with %v; ignoring", scanErr) + return nil } finalErr = errors.CombineErrors(finalErr, errors.Newf("r%d (%s) is inconsistent: %s %s\n", rangeID, prettyKey, status, detail)) } if err := rows.Err(); err != nil { - finalErr = errors.CombineErrors(finalErr, err) + l.Printf("consistency check failed with %v; ignoring", err) + return nil } - return finalErr } @@ -1330,7 +1332,7 @@ func (c *clusterImpl) FailOnReplicaDivergence(ctx context.Context, t test.Test) defer db.Close() if err := contextutil.RunWithTimeout( - ctx, "consistency check", time.Minute, + ctx, "consistency check", 5*time.Minute, func(ctx context.Context) error { return c.CheckReplicaDivergenceOnDB(ctx, t.L(), db) },