From 9fff8d436f892e754986f053cb8867d1f52f13ee Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 4 Jun 2024 21:04:55 +0100 Subject: [PATCH 1/7] remove queue ttl Signed-off-by: Chris Martin --- .../armada/submit/conversion/conversions.go | 5 +- internal/common/eventutil/eventutil.go | 1 - internal/scheduler/adapters/adapters.go | 55 -- internal/scheduler/adapters/adapters_test.go | 37 -- internal/scheduler/jobdb/comparison.go | 49 -- internal/scheduler/jobdb/job.go | 28 - internal/scheduler/jobdb/jobdb.go | 22 +- internal/scheduler/scheduler.go | 54 -- internal/scheduler/scheduler_test.go | 111 +--- .../schedulerobjects/schedulerobjects.pb.go | 307 +++++------ .../schedulerobjects/schedulerobjects.proto | 3 +- internal/scheduleringester/instructions.go | 1 - pkg/api/api.swagger.go | 7 +- pkg/api/api.swagger.json | 7 +- pkg/api/binoculars/api.swagger.go | 2 +- pkg/api/binoculars/api.swagger.json | 2 +- pkg/api/submit.pb.go | 409 +++++++------- pkg/api/submit.proto | 3 +- pkg/armadaevents/events.pb.go | 520 ++++++++---------- pkg/armadaevents/events.proto | 2 - 20 files changed, 593 insertions(+), 1032 deletions(-) diff --git a/internal/armada/submit/conversion/conversions.go b/internal/armada/submit/conversion/conversions.go index 88b2049d489..b0dc957e6e4 100644 --- a/internal/armada/submit/conversion/conversions.go +++ b/internal/armada/submit/conversion/conversions.go @@ -46,9 +46,8 @@ func SubmitJobFromApiRequest( }, }, }, - Objects: ingressesAndServices, - Scheduler: jobReq.Scheduler, - QueueTtlSeconds: jobReq.QueueTtlSeconds, + Objects: ingressesAndServices, + Scheduler: jobReq.Scheduler, } postProcess(msg, config) return msg diff --git a/internal/common/eventutil/eventutil.go b/internal/common/eventutil/eventutil.go index 4768be2bf9c..c83786f868f 100644 --- a/internal/common/eventutil/eventutil.go +++ b/internal/common/eventutil/eventutil.go @@ -165,7 +165,6 @@ func ApiJobFromLogSubmitJob(ownerId string, groups []string, queueName string, j Created: time, Owner: ownerId, QueueOwnershipUserGroups: groups, - QueueTtlSeconds: e.QueueTtlSeconds, }, nil } diff --git a/internal/scheduler/adapters/adapters.go b/internal/scheduler/adapters/adapters.go index b2d7cd3c316..1115d88869d 100644 --- a/internal/scheduler/adapters/adapters.go +++ b/internal/scheduler/adapters/adapters.go @@ -1,28 +1,16 @@ package adapters import ( - "time" - "github.com/pkg/errors" "github.com/sirupsen/logrus" - "golang.org/x/exp/maps" v1 "k8s.io/api/core/v1" "github.com/armadaproject/armada/internal/common/logging" "github.com/armadaproject/armada/internal/common/types" "github.com/armadaproject/armada/internal/scheduler/schedulerobjects" "github.com/armadaproject/armada/pkg/api" - "github.com/armadaproject/armada/pkg/armadaevents" ) -// PodRequirementsFromPod function creates the schedulerobjects and creates a value for the -// annotation field by supplying it with a cloned value of pod.Annotations -func PodRequirementsFromPod(pod *v1.Pod, priorityByPriorityClassName map[string]types.PriorityClass) *schedulerobjects.PodRequirements { - rv := PodRequirementsFromPodSpec(&pod.Spec, priorityByPriorityClassName) - rv.Annotations = maps.Clone(pod.Annotations) - return rv -} - // PodRequirementsFromPodSpec function returns *schedulerobjects.PodRequirements for podSpec. // An error is logged if the podSpec uses an unknown priority class. // This function may mutate podSpec. @@ -48,49 +36,6 @@ func PodRequirementsFromPodSpec(podSpec *v1.PodSpec, priorityByPriorityClassName } } -// SchedulingInfoFromSubmitJob returns a minimal representation of a job containing only the info needed by the scheduler. -func SchedulingInfoFromSubmitJob(submitJob *armadaevents.SubmitJob, submitTime time.Time, priorityClasses map[string]types.PriorityClass) (*schedulerobjects.JobSchedulingInfo, error) { - // Component common to all jobs. - schedulingInfo := &schedulerobjects.JobSchedulingInfo{ - Lifetime: submitJob.Lifetime, - AtMostOnce: submitJob.AtMostOnce, - Preemptible: submitJob.Preemptible, - ConcurrencySafe: submitJob.ConcurrencySafe, - SubmitTime: submitTime, - Priority: submitJob.Priority, - Version: 0, - QueueTtlSeconds: submitJob.QueueTtlSeconds, - } - - // Scheduling requirements specific to the objects that make up this job. - switch object := submitJob.MainObject.Object.(type) { - case *armadaevents.KubernetesMainObject_PodSpec: - podSpec := object.PodSpec.PodSpec - schedulingInfo.PriorityClassName = podSpec.PriorityClassName - podRequirements := PodRequirementsFromPodSpec(podSpec, priorityClasses) - if submitJob.ObjectMeta != nil { - podRequirements.Annotations = maps.Clone(submitJob.ObjectMeta.Annotations) - } - if submitJob.MainObject.ObjectMeta != nil { - if podRequirements.Annotations == nil { - podRequirements.Annotations = make(map[string]string, len(submitJob.MainObject.ObjectMeta.Annotations)) - } - maps.Copy(podRequirements.Annotations, submitJob.MainObject.ObjectMeta.Annotations) - } - schedulingInfo.ObjectRequirements = append( - schedulingInfo.ObjectRequirements, - &schedulerobjects.ObjectRequirements{ - Requirements: &schedulerobjects.ObjectRequirements_PodRequirements{ - PodRequirements: podRequirements, - }, - }, - ) - default: - return nil, errors.Errorf("unsupported object type %T", object) - } - return schedulingInfo, nil -} - // PriorityFromPodSpec returns the priority in a pod spec. // If priority is set directly, that value is returned. // Otherwise, it returns the value of the key podSpec. diff --git a/internal/scheduler/adapters/adapters_test.go b/internal/scheduler/adapters/adapters_test.go index 9398c22d774..6ab18152df8 100644 --- a/internal/scheduler/adapters/adapters_test.go +++ b/internal/scheduler/adapters/adapters_test.go @@ -10,10 +10,8 @@ import ( "github.com/stretchr/testify/require" v1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/resource" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/utils/pointer" - "github.com/armadaproject/armada/internal/armada/configuration" "github.com/armadaproject/armada/internal/common/types" "github.com/armadaproject/armada/internal/scheduler/schedulerobjects" ) @@ -165,41 +163,6 @@ func TestPodRequirementsFromPodSpecPreemptionPolicy(t *testing.T) { } } -func TestPodRequirementsFromPod(t *testing.T) { - podSpec := &v1.PodSpec{ - Priority: &priority, - Containers: []v1.Container{ - { - Resources: v1.ResourceRequirements{ - Limits: v1.ResourceList{ - v1.ResourceName("cpu"): *resource.NewMilliQuantity(5300, resource.DecimalSI), - v1.ResourceName("memory"): *resource.NewQuantity(5*1024*1024*1024, resource.BinarySI), - }, - Requests: v1.ResourceList{ - v1.ResourceName("cpu"): *resource.NewMilliQuantity(300, resource.DecimalSI), - v1.ResourceName("memory"): *resource.NewQuantity(2*1024*1024*1024, resource.BinarySI), - }, - }, - }, - }, - } - pod := v1.Pod{ - Spec: *podSpec, - ObjectMeta: metav1.ObjectMeta{ - Annotations: map[string]string{ - configuration.GangIdAnnotation: "gang-id", - configuration.GangCardinalityAnnotation: "1", - }, - }, - } - rv := PodRequirementsFromPod(&pod, priorityByPriorityClassName) - rv.Annotations["something"] = "test" - // Ensures that any modification made to the returned value of PodRequirementsFromPod function, "rv", does not - // affect the original pod definition. This assertion checks if the length of "pod.Annotation" is altered - // in view of the modification made to "rv" above. - assert.Len(t, pod.Annotations, 2) -} - func TestPriorityFromPodSpec(t *testing.T) { tests := map[string]struct { podSpec *v1.PodSpec diff --git a/internal/scheduler/jobdb/comparison.go b/internal/scheduler/jobdb/comparison.go index 2c2a07dce7c..f8876edef3c 100644 --- a/internal/scheduler/jobdb/comparison.go +++ b/internal/scheduler/jobdb/comparison.go @@ -1,12 +1,7 @@ package jobdb -import ( - "time" -) - type ( JobPriorityComparer struct{} - JobQueueTtlComparer struct{} JobIdHasher struct{} ) @@ -22,50 +17,6 @@ func (JobIdHasher) Equal(a, b *Job) bool { return a == b } -// Compare jobs by their remaining queue time before expiry -// Invariants: -// - Job.queueTtl must be > 0 -// - Job.created must be < `t` -func (j JobQueueTtlComparer) Compare(a, b *Job) int { - // Jobs with equal id are always considered equal. - // This ensures at most one job with a particular id can exist in the jobDb. - if a.id == b.id { - return 0 - } - - // TODO: Calling time.Now() here doesn't sound right. We should probably sort by earliest expiry time. - timeSeconds := time.Now().UTC().Unix() - aDuration := timeSeconds - (a.submittedTime / 1_000_000_000) - bDuration := timeSeconds - (b.submittedTime / 1_000_000_000) - - aRemaining := max(0, a.QueueTtlSeconds()-aDuration) - bRemaining := max(0, b.QueueTtlSeconds()-bDuration) - - // If jobs have different ttl remaining, they are ordered by remaining queue ttl - the smallest ttl first. - if aRemaining != bRemaining { - if aRemaining < bRemaining { - return -1 - } else { - return 1 - } - } - - // Tie-break by logical creation timestamp. - if a.id < b.id { - return -1 - } else if a.id > b.id { - return 1 - } - panic("We should never get here. Since we check for job id equality at the top of this function.") -} - -func max(x, y int64) int64 { - if x < y { - return y - } - return x -} - func (JobPriorityComparer) Compare(job, other *Job) int { return SchedulingOrderCompare(job, other) } diff --git a/internal/scheduler/jobdb/job.go b/internal/scheduler/jobdb/job.go index 4d72ddc16d9..8f9e8bdb7d9 100644 --- a/internal/scheduler/jobdb/job.go +++ b/internal/scheduler/jobdb/job.go @@ -513,12 +513,6 @@ func (job *Job) EfficientResourceRequirements() internaltypes.ResourceList { return job.resourceRequirements } -// QueueTtlSeconds returns the time in seconds that the job should remain queued -// 0 means that this field is unset -func (job *Job) QueueTtlSeconds() int64 { - return job.jobSchedulingInfo.QueueTtlSeconds -} - // PodRequirements returns the pod requirements of the Job func (job *Job) PodRequirements() *schedulerobjects.PodRequirements { return job.jobSchedulingInfo.GetPodRequirements() @@ -738,28 +732,6 @@ func (job *Job) RunById(id uuid.UUID) *JobRun { return job.runsById[id] } -// HasQueueTtlExpired returns true if the given job has reached its queueTtl expiry. -// Invariants: -// - job.created < `t` -func (job *Job) HasQueueTtlExpired() bool { - ttlSeconds := job.QueueTtlSeconds() - if ttlSeconds > 0 { - timeSeconds := time.Now().UTC().Unix() - - // job.Created is populated from the `Submitted` field in postgres, which is a UnixNano time hence the conversion. - createdSeconds := job.submittedTime / 1_000_000_000 - duration := timeSeconds - createdSeconds - return duration > ttlSeconds - } else { - return false - } -} - -// HasQueueTtlSet returns true if the given job has a queueTtl set. -func (job *Job) HasQueueTtlSet() bool { - return job.QueueTtlSeconds() > 0 -} - // WithJobset returns a copy of the job with the jobSet updated. func (job *Job) WithJobset(jobset string) *Job { j := copyJob(*job) diff --git a/internal/scheduler/jobdb/jobdb.go b/internal/scheduler/jobdb/jobdb.go index 7654ce757b9..1829304eb7f 100644 --- a/internal/scheduler/jobdb/jobdb.go +++ b/internal/scheduler/jobdb/jobdb.go @@ -17,16 +17,12 @@ import ( "github.com/armadaproject/armada/internal/scheduler/schedulerobjects" ) -var ( - emptyList = immutable.NewSortedSet[*Job](JobPriorityComparer{}) - emptyQueuedJobsByTtl = immutable.NewSortedSet[*Job](JobQueueTtlComparer{}) -) +var emptyList = immutable.NewSortedSet[*Job](JobPriorityComparer{}) type JobDb struct { jobsById *immutable.Map[string, *Job] jobsByRunId *immutable.Map[uuid.UUID, string] jobsByQueue map[string]immutable.SortedSet[*Job] - queuedJobsByTtl *immutable.SortedSet[*Job] unvalidatedJobs *immutable.Set[*Job] // Configured priority classes. priorityClasses map[string]types.PriorityClass @@ -88,7 +84,6 @@ func NewJobDbWithSchedulingKeyGenerator( jobsByRunId: immutable.NewMap[uuid.UUID, string](&UUIDHasher{}), jobsByQueue: map[string]immutable.SortedSet[*Job]{}, unvalidatedJobs: &unvalidatedJobs, - queuedJobsByTtl: &emptyQueuedJobsByTtl, priorityClasses: priorityClasses, defaultPriorityClass: defaultPriorityClass, schedulingKeyGenerator: skg, @@ -113,7 +108,6 @@ func (jobDb *JobDb) Clone() *JobDb { jobsById: jobDb.jobsById, jobsByRunId: jobDb.jobsByRunId, jobsByQueue: maps.Clone(jobDb.jobsByQueue), - queuedJobsByTtl: jobDb.queuedJobsByTtl, unvalidatedJobs: jobDb.unvalidatedJobs, priorityClasses: jobDb.priorityClasses, defaultPriorityClass: jobDb.defaultPriorityClass, @@ -210,7 +204,6 @@ func (jobDb *JobDb) ReadTxn() *Txn { jobsById: jobDb.jobsById, jobsByRunId: jobDb.jobsByRunId, jobsByQueue: jobDb.jobsByQueue, - queuedJobsByTtl: jobDb.queuedJobsByTtl, unvalidatedJobs: jobDb.unvalidatedJobs, active: true, jobDb: jobDb, @@ -229,7 +222,6 @@ func (jobDb *JobDb) WriteTxn() *Txn { jobsById: jobDb.jobsById, jobsByRunId: jobDb.jobsByRunId, jobsByQueue: maps.Clone(jobDb.jobsByQueue), - queuedJobsByTtl: jobDb.queuedJobsByTtl, unvalidatedJobs: jobDb.unvalidatedJobs, active: true, jobDb: jobDb, @@ -270,7 +262,6 @@ func (txn *Txn) Commit() { txn.jobDb.jobsById = txn.jobsById txn.jobDb.jobsByRunId = txn.jobsByRunId txn.jobDb.jobsByQueue = txn.jobsByQueue - txn.jobDb.queuedJobsByTtl = txn.queuedJobsByTtl txn.jobDb.unvalidatedJobs = txn.unvalidatedJobs txn.active = false @@ -452,11 +443,6 @@ func (txn *Txn) Upsert(jobs []*Job) error { } newQueue = newQueue.Add(job) txn.jobsByQueue[job.queue] = newQueue - - if job.HasQueueTtlSet() { - queuedJobsByTtl := txn.queuedJobsByTtl.Add(job) - txn.queuedJobsByTtl = &queuedJobsByTtl - } } } }() @@ -556,12 +542,6 @@ func (txn *Txn) delete(jobId string) { newQueue := queue.Delete(job) txn.jobsByQueue[job.queue] = newQueue } - - // We only add these jobs into the collection if it has a queueTtl set, hence only remove if this is set. - if job.HasQueueTtlSet() { - newQueuedJobsByExpiry := txn.queuedJobsByTtl.Delete(job) - txn.queuedJobsByTtl = &newQueuedJobsByExpiry - } newUnvalidatedJobs := txn.unvalidatedJobs.Delete(job) txn.unvalidatedJobs = &newUnvalidatedJobs } diff --git a/internal/scheduler/scheduler.go b/internal/scheduler/scheduler.go index 2bd3554506b..f82cf43257f 100644 --- a/internal/scheduler/scheduler.go +++ b/internal/scheduler/scheduler.go @@ -337,15 +337,6 @@ func (s *Scheduler) cycle(ctx *armadacontext.Context, updateAll bool, leaderToke ctx.Infof("Finished looking for jobs to expire, generating %d events", len(expirationEvents)) events = append(events, expirationEvents...) - // Request cancel for any jobs that exceed queueTtl - ctx.Info("Cancelling queued jobs exceeding their queue ttl") - queueTtlCancelEvents, err := s.cancelQueuedJobsIfExpired(txn) - if err != nil { - return overallSchedulerResult, err - } - ctx.Infof("Finished cancelling queued jobs exceeding their queue ttl, generating %d events", len(queueTtlCancelEvents)) - events = append(events, queueTtlCancelEvents...) - // Schedule jobs. if shouldSchedule { var result *SchedulerResult @@ -966,51 +957,6 @@ func (s *Scheduler) expireJobsIfNecessary(ctx *armadacontext.Context, txn *jobdb return events, nil } -// cancelQueuedJobsIfExpired generates cancel request messages for any queued jobs that exceed their queueTtl. -func (s *Scheduler) cancelQueuedJobsIfExpired(txn *jobdb.Txn) ([]*armadaevents.EventSequence, error) { - jobsToCancel := make([]*jobdb.Job, 0) - events := make([]*armadaevents.EventSequence, 0) - it := txn.QueuedJobsByTtl() - - // `it` is ordered such that the jobs with the least ttl remaining come first, hence we exit early if we find a job that is not expired. - for job, _ := it.Next(); job != nil && job.HasQueueTtlExpired(); job, _ = it.Next() { - if job.InTerminalState() { - continue - } - - job = job.WithCancelRequested(true).WithQueued(false).WithCancelled(true) - jobId, err := armadaevents.ProtoUuidFromUlidString(job.Id()) - if err != nil { - return nil, err - } - - reason := "Expired queue ttl" - cancel := &armadaevents.EventSequence{ - Queue: job.Queue(), - JobSetName: job.Jobset(), - Events: []*armadaevents.EventSequence_Event{ - { - Created: s.now(), - Event: &armadaevents.EventSequence_Event_CancelJob{CancelJob: &armadaevents.CancelJob{JobId: jobId, JobIdStr: job.Id(), Reason: reason}}, - }, - { - Created: s.now(), - Event: &armadaevents.EventSequence_Event_CancelledJob{CancelledJob: &armadaevents.CancelledJob{JobId: jobId, JobIdStr: job.Id(), Reason: reason}}, - }, - }, - } - - jobsToCancel = append(jobsToCancel, job) - events = append(events, cancel) - } - - if err := txn.Upsert(jobsToCancel); err != nil { - return nil, err - } - - return events, nil -} - func (s *Scheduler) submitCheck(ctx *armadacontext.Context, txn *jobdb.Txn) ([]*armadaevents.EventSequence, error) { jobsToCheck := make([]*jobdb.Job, 0) diff --git a/internal/scheduler/scheduler_test.go b/internal/scheduler/scheduler_test.go index 1bd5c03374a..e75acb1ce71 100644 --- a/internal/scheduler/scheduler_test.go +++ b/internal/scheduler/scheduler_test.go @@ -127,10 +127,8 @@ var ( }, }, }, - QueueTtlSeconds: 2, - Version: 1, + Version: 1, } - schedulingInfoWithQueueTtlBytes = protoutil.MustMarshall(schedulingInfoWithQueueTtl) schedulerMetrics = NewSchedulerMetrics(configuration.SchedulerMetricsConfig{ ScheduleCycleTimeHistogramSettings: configuration.HistogramConfig{ @@ -161,21 +159,6 @@ var queuedJob = testfixtures.NewJob( true, ) -var queuedJobWithExpiredTtl = testfixtures.NewJob( - util.NewULID(), - "testJobset", - "testQueue", - 0, - schedulingInfoWithQueueTtl, - true, - 0, - false, - false, - false, - 1, - true, -) - var leasedJob = testfixtures.NewJob( util.NewULID(), "testJobset", @@ -668,82 +651,50 @@ func TestScheduler_TestCycle(t *testing.T) { expectedLeased: []string{leasedJob.Id()}, expectedQueuedVersion: leasedJob.QueuedVersion(), }, - "New job from postgres with expired queue ttl is cancel requested": { - jobUpdates: []database.Job{ - { - JobID: queuedJobWithExpiredTtl.Id(), - JobSet: queuedJobWithExpiredTtl.Jobset(), - Queue: queuedJobWithExpiredTtl.Queue(), - Queued: queuedJobWithExpiredTtl.Queued(), - QueuedVersion: queuedJobWithExpiredTtl.QueuedVersion(), - Serial: 1, - Submitted: queuedJobWithExpiredTtl.Created(), - SchedulingInfo: schedulingInfoWithQueueTtlBytes, - Validated: true, - }, - }, - - // We expect to publish request cancel and cancelled message this cycle. - // The job should also be removed from the queue and set to a terminal state. - expectedJobRequestCancel: []string{queuedJobWithExpiredTtl.Id()}, - expectedJobCancelled: []string{queuedJobWithExpiredTtl.Id()}, - expectedQueuedVersion: queuedJobWithExpiredTtl.QueuedVersion(), - expectedTerminal: []string{queuedJobWithExpiredTtl.Id()}, - }, - "Existing jobDb job with expired queue ttl is cancel requested": { - initialJobs: []*jobdb.Job{queuedJobWithExpiredTtl}, - - // We expect to publish request cancel and cancelled message this cycle. - // The job should also be removed from the queue and set to a terminal state. - expectedJobRequestCancel: []string{queuedJobWithExpiredTtl.Id()}, - expectedJobCancelled: []string{queuedJobWithExpiredTtl.Id()}, - expectedQueuedVersion: queuedJobWithExpiredTtl.QueuedVersion(), - expectedTerminal: []string{queuedJobWithExpiredTtl.Id()}, - }, "New postgres job with cancel requested results in cancel messages": { jobUpdates: []database.Job{ { - JobID: queuedJobWithExpiredTtl.Id(), - JobSet: queuedJobWithExpiredTtl.Jobset(), - Queue: queuedJobWithExpiredTtl.Queue(), - Queued: queuedJobWithExpiredTtl.Queued(), - QueuedVersion: queuedJobWithExpiredTtl.QueuedVersion(), + JobID: queuedJob.Id(), + JobSet: queuedJob.Jobset(), + Queue: queuedJob.Queue(), + Queued: queuedJob.Queued(), + QueuedVersion: queuedJob.QueuedVersion(), Serial: 1, - Submitted: queuedJobWithExpiredTtl.Created(), + Submitted: queuedJob.Created(), CancelRequested: true, Cancelled: false, - SchedulingInfo: schedulingInfoWithQueueTtlBytes, + SchedulingInfo: schedulingInfoBytes, }, }, // We have already got a request cancel from the DB, so only publish a cancelled message. // The job should also be removed from the queue and set to a terminal state.# - expectedJobCancelled: []string{queuedJobWithExpiredTtl.Id()}, - expectedQueuedVersion: queuedJobWithExpiredTtl.QueuedVersion(), - expectedTerminal: []string{queuedJobWithExpiredTtl.Id()}, + expectedJobCancelled: []string{queuedJob.Id()}, + expectedQueuedVersion: queuedJob.QueuedVersion(), + expectedTerminal: []string{queuedJob.Id()}, }, "Postgres job with cancel requested results in cancel messages": { - initialJobs: []*jobdb.Job{queuedJobWithExpiredTtl.WithCancelRequested(true)}, + initialJobs: []*jobdb.Job{queuedJob.WithCancelRequested(true)}, jobUpdates: []database.Job{ { - JobID: queuedJobWithExpiredTtl.Id(), - JobSet: queuedJobWithExpiredTtl.Jobset(), - Queue: queuedJobWithExpiredTtl.Queue(), - Queued: queuedJobWithExpiredTtl.Queued(), - QueuedVersion: queuedJobWithExpiredTtl.QueuedVersion(), + JobID: queuedJob.Id(), + JobSet: queuedJob.Jobset(), + Queue: queuedJob.Queue(), + Queued: queuedJob.Queued(), + QueuedVersion: queuedJob.QueuedVersion(), Serial: 1, - Submitted: queuedJobWithExpiredTtl.Created(), + Submitted: queuedJob.Created(), CancelRequested: true, Cancelled: false, - SchedulingInfo: schedulingInfoWithQueueTtlBytes, + SchedulingInfo: schedulingInfoBytes, }, }, // We have already got a request cancel from the DB/existing job state, so only publish a cancelled message. // The job should also be removed from the queue and set to a terminal state. - expectedJobCancelled: []string{queuedJobWithExpiredTtl.Id()}, - expectedQueuedVersion: queuedJobWithExpiredTtl.QueuedVersion(), - expectedTerminal: []string{queuedJobWithExpiredTtl.Id()}, + expectedJobCancelled: []string{queuedJob.Id()}, + expectedQueuedVersion: queuedJob.QueuedVersion(), + expectedTerminal: []string{queuedJob.Id()}, }, "Queued job reprioritised": { initialJobs: []*jobdb.Job{queuedJob}, @@ -1557,17 +1508,6 @@ var ( Validated: true, Serial: 0, } - queuedJobWithTTLA = &database.Job{ - JobID: queuedJobA.JobID, - JobSet: "testJobSet", - Queue: "testQueue", - Queued: true, - QueuedVersion: 0, - SchedulingInfo: schedulingInfoWithQueueTtlBytes, - SchedulingInfoVersion: int32(schedulingInfoWithQueueTtl.Version), - Validated: true, - Serial: 0, - } queuedJobWithFailFastA = &database.Job{ JobID: queuedJobA.JobID, JobSet: "testJobSet", @@ -2375,13 +2315,6 @@ func TestCycleConsistency(t *testing.T) { }, }, }, - "Queued job with expired ttl results in cancellation": { - firstSchedulerDbUpdate: schedulerDbUpdate{ - jobUpdates: []*database.Job{ - queuedJobWithTTLA, - }, - }, - }, "Queued job is re-prioritised": { firstSchedulerDbUpdate: schedulerDbUpdate{ jobUpdates: []*database.Job{ diff --git a/internal/scheduler/schedulerobjects/schedulerobjects.pb.go b/internal/scheduler/schedulerobjects/schedulerobjects.pb.go index 7c985485f5f..52e9af1b4c6 100644 --- a/internal/scheduler/schedulerobjects/schedulerobjects.pb.go +++ b/internal/scheduler/schedulerobjects/schedulerobjects.pb.go @@ -630,8 +630,6 @@ type JobSchedulingInfo struct { // Kubernetes objects that make up this job and their respective scheduling requirements. ObjectRequirements []*ObjectRequirements `protobuf:"bytes,5,rep,name=object_requirements,json=objectRequirements,proto3" json:"objectRequirements,omitempty"` Version uint32 `protobuf:"varint,9,opt,name=version,proto3" json:"version,omitempty"` - // Queuing TTL for this job in seconds. If this job queues for more than this duration it will be cancelled. Zero indicates an infinite lifetime. - QueueTtlSeconds int64 `protobuf:"varint,10,opt,name=queue_ttl_seconds,json=queueTtlSeconds,proto3" json:"queueTtlSeconds,omitempty"` } func (m *JobSchedulingInfo) Reset() { *m = JobSchedulingInfo{} } @@ -726,13 +724,6 @@ func (m *JobSchedulingInfo) GetVersion() uint32 { return 0 } -func (m *JobSchedulingInfo) GetQueueTtlSeconds() int64 { - if m != nil { - return m.QueueTtlSeconds - } - return 0 -} - // Message capturing the scheduling requirements of a particular Kubernetes object. type ObjectRequirements struct { // Types that are valid to be assigned to Requirements: @@ -1007,142 +998,141 @@ func init() { } var fileDescriptor_97dadc5fbd620721 = []byte{ - // 2160 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x39, 0xcd, 0x6f, 0x1b, 0xc7, - 0xf5, 0x5a, 0x7d, 0x92, 0x8f, 0xb2, 0x44, 0x8d, 0x2c, 0x7b, 0x45, 0xdb, 0x5c, 0x85, 0xf1, 0x2f, - 0x50, 0x7e, 0x71, 0xc8, 0x46, 0x29, 0x50, 0xc3, 0xed, 0x45, 0xb4, 0xd4, 0x9a, 0xaa, 0x4d, 0xc9, - 0x4b, 0xa9, 0x45, 0x0b, 0x34, 0x8b, 0x25, 0x77, 0x44, 0x6f, 0xb4, 0x9c, 0xa1, 0x77, 0x67, 0xdd, - 0x30, 0xe7, 0xf6, 0x50, 0x04, 0x48, 0x83, 0xa2, 0x1f, 0x01, 0x0a, 0xb4, 0xc8, 0xad, 0xe7, 0x1e, - 0xda, 0x43, 0xff, 0x01, 0x1f, 0x73, 0xec, 0x89, 0x29, 0xec, 0x1b, 0xaf, 0xfd, 0x07, 0x8a, 0x9d, - 0xd9, 0xe5, 0x0e, 0x77, 0x49, 0x51, 0x4e, 0xea, 0xea, 0x44, 0xce, 0xfb, 0x9e, 0xf7, 0xde, 0xbc, - 0x79, 0x6f, 0x16, 0xee, 0xd9, 0x84, 0x61, 0x97, 0x98, 0x4e, 0xc5, 0x6b, 0x3d, 0xc1, 0x96, 0xef, - 0x60, 0x37, 0xfe, 0x47, 0x9b, 0x1f, 0xe2, 0x16, 0xf3, 0x52, 0x80, 0x72, 0xd7, 0xa5, 0x8c, 0xa2, - 0x7c, 0x12, 0x5e, 0xd0, 0xda, 0x94, 0xb6, 0x1d, 0x5c, 0xe1, 0xf8, 0xa6, 0x7f, 0x5a, 0x61, 0x76, - 0x07, 0x7b, 0xcc, 0xec, 0x74, 0x05, 0x4b, 0xa1, 0x74, 0x76, 0xd7, 0x2b, 0xdb, 0xb4, 0x62, 0x76, - 0xed, 0x4a, 0x8b, 0xba, 0xb8, 0xf2, 0xec, 0xbd, 0x4a, 0x1b, 0x13, 0xec, 0x9a, 0x0c, 0x5b, 0x21, - 0xcd, 0xb7, 0x63, 0x9a, 0x8e, 0xd9, 0x7a, 0x62, 0x13, 0xec, 0xf6, 0x2a, 0xdd, 0xb3, 0x36, 0x67, - 0x72, 0xb1, 0x47, 0x7d, 0xb7, 0x85, 0x53, 0x5c, 0xef, 0xb6, 0x6d, 0xf6, 0xc4, 0x6f, 0x96, 0x5b, - 0xb4, 0x53, 0x69, 0xd3, 0x36, 0x8d, 0x6d, 0x08, 0x56, 0x7c, 0xc1, 0xff, 0x09, 0xf2, 0xd2, 0x5f, - 0xe6, 0x20, 0xb3, 0xff, 0x11, 0x6e, 0xf9, 0x8c, 0xba, 0x68, 0x0b, 0x66, 0x6d, 0x4b, 0x55, 0xb6, - 0x94, 0xed, 0x6c, 0x35, 0x3f, 0xe8, 0x6b, 0xcb, 0xb6, 0x75, 0x87, 0x76, 0x6c, 0x86, 0x3b, 0x5d, - 0xd6, 0xd3, 0x67, 0x6d, 0x0b, 0xbd, 0x05, 0xf3, 0x5d, 0x4a, 0x1d, 0x75, 0x96, 0xd3, 0xa0, 0x41, - 0x5f, 0x5b, 0x09, 0xd6, 0x12, 0x15, 0xc7, 0xa3, 0x5d, 0x58, 0x20, 0xd4, 0xc2, 0x9e, 0x3a, 0xb7, - 0x35, 0xb7, 0x9d, 0xdb, 0xb9, 0x56, 0x4e, 0xb9, 0xae, 0x4e, 0x2d, 0x5c, 0x5d, 0x1f, 0xf4, 0xb5, - 0x55, 0x4e, 0x28, 0x49, 0x10, 0x9c, 0xe8, 0x03, 0x58, 0xe9, 0xd8, 0xc4, 0xee, 0xf8, 0x9d, 0x03, - 0xda, 0x6c, 0xd8, 0x1f, 0x63, 0x75, 0x7e, 0x4b, 0xd9, 0xce, 0xed, 0x14, 0xd3, 0xb2, 0xf4, 0xd0, - 0x19, 0x0f, 0x6d, 0x8f, 0x55, 0xaf, 0x3d, 0xef, 0x6b, 0x33, 0x81, 0x61, 0xa3, 0xdc, 0x7a, 0x62, - 0x1d, 0xc8, 0x77, 0x4c, 0x8f, 0x9d, 0x74, 0x2d, 0x93, 0xe1, 0x63, 0xbb, 0x83, 0xd5, 0x05, 0x2e, - 0xbf, 0x50, 0x16, 0xc1, 0x2b, 0x47, 0x8e, 0x2b, 0x1f, 0x47, 0xc1, 0xab, 0x16, 0x22, 0xd9, 0xa3, - 0x9c, 0x9f, 0x7d, 0xa5, 0x29, 0x7a, 0x02, 0x86, 0x0e, 0x61, 0xdd, 0x27, 0xa6, 0xe7, 0xd9, 0x6d, - 0x82, 0x2d, 0xe3, 0x43, 0xda, 0x34, 0x5c, 0x9f, 0x78, 0x6a, 0x76, 0x6b, 0x6e, 0x3b, 0x5b, 0xd5, - 0x06, 0x7d, 0xed, 0x46, 0x8c, 0x3e, 0xa0, 0x4d, 0xdd, 0x27, 0xb2, 0x13, 0xd6, 0x52, 0xc8, 0xd2, - 0x5f, 0x37, 0x60, 0x3e, 0xf0, 0xda, 0xc5, 0xc2, 0x44, 0xcc, 0x0e, 0x56, 0x97, 0xe3, 0x30, 0x05, - 0x6b, 0x39, 0x4c, 0xc1, 0x1a, 0xed, 0x40, 0x06, 0x87, 0xc1, 0x57, 0xd7, 0x39, 0xed, 0xb5, 0x41, - 0x5f, 0x43, 0x11, 0x4c, 0xa2, 0x1f, 0xd2, 0xa1, 0x47, 0x90, 0x0d, 0x76, 0x6a, 0x78, 0x18, 0x13, - 0x9e, 0x07, 0xe7, 0xbb, 0xec, 0x6a, 0xe8, 0xb2, 0x4c, 0xc0, 0xd4, 0xc0, 0x98, 0x70, 0x67, 0x0d, - 0x57, 0xe8, 0x10, 0xb2, 0x41, 0xbc, 0x0d, 0xd6, 0xeb, 0x62, 0x75, 0x2e, 0x14, 0x37, 0x36, 0x5b, - 0x8e, 0x7b, 0x5d, 0x2c, 0xec, 0x23, 0xe1, 0x4a, 0xb6, 0x2f, 0x82, 0xa1, 0x5d, 0x58, 0x64, 0xa6, - 0x4d, 0x98, 0xa7, 0x2e, 0xf0, 0xdc, 0xdb, 0x2c, 0x8b, 0x73, 0x54, 0x36, 0xbb, 0x76, 0x39, 0x38, - 0x6b, 0xe5, 0x67, 0xef, 0x95, 0x8f, 0x03, 0x8a, 0xea, 0x4a, 0x68, 0x5b, 0xc8, 0xa0, 0x87, 0xbf, - 0xe8, 0x08, 0x16, 0x1d, 0xb3, 0x89, 0x1d, 0x4f, 0x5d, 0xe4, 0x22, 0x4a, 0xe3, 0x0d, 0x2a, 0x3f, - 0xe4, 0x44, 0xfb, 0x84, 0xb9, 0xbd, 0xea, 0xd5, 0x41, 0x5f, 0xcb, 0x0b, 0x2e, 0xc9, 0xac, 0x50, - 0x0e, 0x32, 0x60, 0x95, 0x51, 0x66, 0x3a, 0x46, 0x74, 0x6e, 0x3d, 0x75, 0xe9, 0xd5, 0xb2, 0x99, - 0xb3, 0x47, 0x28, 0x4f, 0x4f, 0xac, 0xd1, 0xdf, 0x14, 0xb8, 0x6d, 0x3a, 0x0e, 0x6d, 0x99, 0xcc, - 0x6c, 0x3a, 0xd8, 0x68, 0xf6, 0x8c, 0xae, 0x6b, 0x53, 0xd7, 0x66, 0x3d, 0xc3, 0x24, 0xd6, 0x50, - 0xaf, 0x9a, 0xe1, 0x3b, 0xfa, 0xde, 0x84, 0x1d, 0xed, 0xc6, 0x22, 0xaa, 0xbd, 0xa3, 0x50, 0xc0, - 0x2e, 0xb1, 0x22, 0x45, 0x62, 0xaf, 0xdb, 0xa1, 0x51, 0x5b, 0xe6, 0x14, 0x72, 0x7d, 0x2a, 0x05, - 0x72, 0x61, 0xdd, 0x63, 0x26, 0xe3, 0x16, 0x87, 0x87, 0xc4, 0xb0, 0x2d, 0x7e, 0x4c, 0x72, 0x3b, - 0xef, 0x4c, 0x30, 0xb3, 0x11, 0x70, 0x54, 0x7b, 0xe2, 0x64, 0xd4, 0x2c, 0x61, 0xd5, 0xf5, 0xd0, - 0xaa, 0x55, 0x6f, 0x14, 0xab, 0x27, 0x01, 0xc8, 0x87, 0xf5, 0xd0, 0x2e, 0x6c, 0x45, 0x7a, 0x6d, - 0x4b, 0x05, 0xae, 0xf3, 0xce, 0xf9, 0xae, 0xc1, 0x16, 0x17, 0x14, 0x29, 0x55, 0x43, 0xa5, 0x79, - 0x33, 0x81, 0xd6, 0x53, 0x10, 0xc4, 0x00, 0x8d, 0xa8, 0x7d, 0xea, 0x63, 0x1f, 0xab, 0xb9, 0x8b, - 0x6a, 0x7d, 0x1c, 0x90, 0x4f, 0xd6, 0xca, 0xd1, 0x7a, 0x0a, 0x12, 0x6c, 0x16, 0x3f, 0xb3, 0x5b, - 0x2c, 0x2e, 0x42, 0x86, 0x6d, 0x79, 0xea, 0xca, 0xb9, 0x6a, 0xf7, 0x05, 0x47, 0xe4, 0x31, 0x2f, - 0xa1, 0x16, 0x27, 0xd0, 0x7a, 0x0a, 0x82, 0xbe, 0x50, 0xa0, 0x48, 0x28, 0x31, 0x4c, 0xb7, 0x63, - 0x5a, 0xa6, 0x11, 0x6f, 0x3c, 0x3e, 0x01, 0x57, 0xb8, 0x09, 0xdf, 0x99, 0x60, 0x42, 0x9d, 0x92, - 0x5d, 0xce, 0x3b, 0x74, 0xc1, 0x30, 0xdb, 0x85, 0x35, 0x6f, 0x86, 0xd6, 0xdc, 0x20, 0x93, 0x29, - 0xf5, 0xf3, 0x90, 0x68, 0x17, 0xae, 0xf8, 0x24, 0xd4, 0x1e, 0x64, 0xa8, 0xba, 0xba, 0xa5, 0x6c, - 0x67, 0xaa, 0x37, 0x06, 0x7d, 0xed, 0xfa, 0x08, 0x42, 0x3a, 0xd1, 0xa3, 0x1c, 0xe8, 0x13, 0x05, - 0xae, 0x47, 0x3b, 0x32, 0x7c, 0xcf, 0x6c, 0xe3, 0x38, 0xb2, 0x79, 0xbe, 0xbf, 0x6f, 0x4d, 0xd8, - 0x5f, 0x64, 0xc6, 0x49, 0xc0, 0x34, 0x12, 0xdd, 0xd2, 0xa0, 0xaf, 0x15, 0xdd, 0x31, 0x68, 0xc9, - 0x8c, 0xab, 0xe3, 0xf0, 0xc1, 0x9d, 0xe3, 0xe2, 0x2e, 0x75, 0x99, 0x4d, 0xda, 0x46, 0x5c, 0x56, - 0xd7, 0x78, 0x69, 0xe7, 0x77, 0xce, 0x10, 0x5d, 0x4f, 0xd7, 0xd0, 0xb5, 0x14, 0xb2, 0x60, 0x42, - 0x4e, 0x2a, 0x72, 0xe8, 0x4d, 0x98, 0x3b, 0xc3, 0xbd, 0xf0, 0xea, 0x59, 0x1b, 0xf4, 0xb5, 0x2b, - 0x67, 0xb8, 0x27, 0x49, 0x08, 0xb0, 0xe8, 0x6d, 0x58, 0x78, 0x66, 0x3a, 0x3e, 0x0e, 0x9b, 0x04, - 0x7e, 0xc7, 0x73, 0x80, 0x7c, 0xc7, 0x73, 0xc0, 0xbd, 0xd9, 0xbb, 0x4a, 0xe1, 0x8f, 0x0a, 0xfc, - 0xdf, 0x85, 0xca, 0x8e, 0xac, 0x7d, 0x61, 0xa2, 0xf6, 0x9a, 0xac, 0x7d, 0x7a, 0x7d, 0x9d, 0x66, - 0xdd, 0xaf, 0x14, 0xb8, 0x3a, 0xae, 0xda, 0x5c, 0xcc, 0x15, 0x0f, 0x64, 0x63, 0x56, 0x76, 0x6e, - 0xa5, 0x8d, 0x11, 0x42, 0x85, 0x86, 0x69, 0xb6, 0x7c, 0xa2, 0xc0, 0xc6, 0xd8, 0x2a, 0x74, 0x31, - 0x63, 0xfe, 0xcb, 0x9e, 0x49, 0x58, 0x13, 0xe7, 0xef, 0xa5, 0x58, 0x73, 0x06, 0x1b, 0x63, 0x6b, - 0xd6, 0xd7, 0x48, 0xd9, 0xcc, 0x54, 0x65, 0xbf, 0x57, 0x60, 0x6b, 0x5a, 0x79, 0xba, 0x94, 0x6c, - 0xfd, 0xb5, 0x02, 0x9b, 0x13, 0xeb, 0xca, 0x65, 0xc4, 0xa5, 0xf4, 0xa7, 0x79, 0xc8, 0x44, 0xd5, - 0x24, 0x68, 0x5c, 0x6b, 0xa2, 0x71, 0x9d, 0x17, 0x8d, 0x6b, 0x6d, 0xa4, 0x71, 0xad, 0x59, 0x52, - 0xf3, 0x36, 0xfb, 0x75, 0x9b, 0xb7, 0xe3, 0x61, 0xf3, 0x26, 0x66, 0x8f, 0xb7, 0x26, 0x77, 0x93, - 0xaf, 0xd0, 0xc0, 0xfd, 0x42, 0x01, 0xe4, 0x13, 0x0f, 0xb3, 0x1a, 0xb1, 0xf0, 0x47, 0xd8, 0x12, - 0x9c, 0xea, 0x3c, 0x57, 0xb1, 0x73, 0x8e, 0x8a, 0x93, 0x14, 0x93, 0x50, 0xb7, 0x35, 0xe8, 0x6b, - 0x37, 0xd3, 0x12, 0x25, 0xd5, 0x63, 0xf4, 0xfd, 0x2f, 0xea, 0x71, 0x07, 0xae, 0x4f, 0xb0, 0xf9, - 0x75, 0xa8, 0x2b, 0x3d, 0x5f, 0x84, 0x4d, 0x9e, 0xa3, 0xf7, 0x1d, 0xdf, 0x63, 0xd8, 0x1d, 0x49, - 0x5f, 0x54, 0x83, 0xa5, 0x96, 0x8b, 0x83, 0xd3, 0xc5, 0xb5, 0x9e, 0x3f, 0x6a, 0xac, 0x87, 0x19, - 0x11, 0xb1, 0xf0, 0x49, 0x23, 0x5a, 0x04, 0x76, 0x89, 0x6b, 0x59, 0xb2, 0xeb, 0x69, 0xe2, 0x56, - 0x15, 0x14, 0xe8, 0x2e, 0x40, 0x34, 0xee, 0xd4, 0x2c, 0x3e, 0x94, 0x64, 0xab, 0xea, 0xa0, 0xaf, - 0x5d, 0x8d, 0xa1, 0x12, 0x93, 0x44, 0x8b, 0x7e, 0xa7, 0x04, 0x37, 0x70, 0x58, 0x07, 0xe2, 0xab, - 0x2c, 0xcc, 0x93, 0xbd, 0x74, 0x9e, 0x4c, 0xdc, 0xfa, 0xf0, 0x98, 0x49, 0x62, 0x44, 0xe6, 0xdc, - 0x0a, 0xb7, 0x39, 0x56, 0x91, 0xa2, 0x8f, 0x03, 0xa3, 0xbf, 0x2b, 0x70, 0x73, 0x0c, 0xfc, 0xbe, - 0x63, 0x7a, 0x5e, 0xdd, 0xe4, 0xb3, 0x6f, 0x60, 0xe0, 0xa3, 0x6f, 0x68, 0xe0, 0x50, 0x9e, 0xb0, - 0xf4, 0x76, 0x68, 0xe9, 0xb9, 0xaa, 0xf5, 0x73, 0xb1, 0x85, 0x4f, 0x15, 0x50, 0x27, 0xb9, 0xe2, - 0x52, 0x6a, 0xec, 0x1f, 0x14, 0x78, 0x63, 0xea, 0xd6, 0x2f, 0xa5, 0xd6, 0xfe, 0x63, 0x0e, 0x0a, - 0xe3, 0x22, 0xa5, 0xf3, 0xb6, 0x6e, 0xf8, 0x76, 0xa3, 0x4c, 0x79, 0xbb, 0x91, 0xce, 0xdc, 0xec, - 0x37, 0x3c, 0x73, 0x9f, 0x2a, 0x90, 0x97, 0xa2, 0xcb, 0x73, 0x29, 0x2c, 0xcb, 0xd5, 0xf4, 0x66, - 0x27, 0xdb, 0x2e, 0xe7, 0x9a, 0xd4, 0x28, 0x17, 0x07, 0x7d, 0xad, 0x90, 0x94, 0x2f, 0xed, 0x27, - 0xa5, 0xbb, 0xf0, 0xb9, 0x02, 0x1b, 0x63, 0x65, 0x5d, 0x2c, 0x60, 0x3f, 0x1a, 0x0d, 0xd8, 0x3b, - 0xaf, 0x70, 0x5c, 0xa6, 0x46, 0xef, 0x97, 0xb3, 0xb0, 0x2c, 0x87, 0x1b, 0x7d, 0x00, 0xd9, 0x78, - 0x56, 0x52, 0xb8, 0xd3, 0xde, 0x3d, 0x3f, 0x43, 0xca, 0x89, 0x09, 0x69, 0x2d, 0x0c, 0x4e, 0x2c, - 0x47, 0x8f, 0xff, 0x16, 0x7e, 0xab, 0xc0, 0xca, 0xe4, 0x9e, 0x65, 0xb2, 0x13, 0x7e, 0x32, 0xea, - 0x84, 0xb2, 0x74, 0x45, 0x0f, 0xdf, 0x29, 0xcb, 0xdd, 0xb3, 0x36, 0xbf, 0xb3, 0x23, 0x75, 0xe5, - 0xc7, 0xbe, 0x49, 0x98, 0xcd, 0x7a, 0x53, 0xfd, 0xf0, 0xd5, 0x02, 0xac, 0x1d, 0xd0, 0x66, 0x43, - 0x6c, 0xd4, 0x26, 0xed, 0x1a, 0x39, 0xa5, 0x68, 0x07, 0x32, 0x8e, 0x7d, 0x8a, 0x99, 0xdd, 0xc1, - 0xdc, 0xbc, 0x2b, 0xe2, 0x25, 0x28, 0x82, 0xc9, 0x2f, 0x41, 0x11, 0x0c, 0xdd, 0x83, 0x65, 0x93, - 0x19, 0x1d, 0xea, 0x31, 0x83, 0x92, 0x56, 0xd4, 0xdc, 0xf1, 0x42, 0x6e, 0xb2, 0x47, 0xd4, 0x63, - 0x87, 0xa4, 0x25, 0x73, 0x42, 0x0c, 0x45, 0xdf, 0x85, 0x5c, 0xd7, 0xc5, 0x01, 0xdc, 0x0e, 0x06, - 0xc3, 0x39, 0xce, 0xba, 0x39, 0xe8, 0x6b, 0x1b, 0x12, 0x58, 0xe2, 0x95, 0xa9, 0xd1, 0x03, 0xc8, - 0xb7, 0x28, 0x69, 0xf9, 0xae, 0x8b, 0x49, 0xab, 0x67, 0x78, 0xe6, 0xa9, 0x78, 0xbc, 0xcc, 0x54, - 0x6f, 0x0d, 0xfa, 0xda, 0xa6, 0x84, 0x6b, 0x98, 0xa7, 0xb2, 0x94, 0xd5, 0x04, 0x2a, 0x18, 0xe8, - 0x86, 0xcf, 0x38, 0xad, 0xa0, 0xc2, 0x18, 0xfc, 0x5d, 0x6f, 0x31, 0x1e, 0xe8, 0xba, 0xc9, 0xfa, - 0x23, 0x0f, 0x74, 0x29, 0x24, 0x6a, 0x40, 0xce, 0xf3, 0x9b, 0x1d, 0x9b, 0x19, 0xdc, 0x95, 0x4b, - 0x53, 0x0f, 0x78, 0xf4, 0x00, 0x05, 0x82, 0x6d, 0xf8, 0xdc, 0x29, 0xad, 0x83, 0xe0, 0x44, 0x9a, - 0xd4, 0x4c, 0x1c, 0x9c, 0x08, 0x26, 0x07, 0x27, 0x82, 0xa1, 0x9f, 0xc3, 0xba, 0x48, 0x61, 0xc3, - 0xc5, 0x4f, 0x7d, 0xdb, 0xc5, 0x1d, 0x1c, 0xbf, 0xd9, 0xdd, 0x4e, 0xe7, 0xf9, 0x21, 0xff, 0xd5, - 0x25, 0x5a, 0xd1, 0x42, 0xd1, 0x14, 0x5c, 0x6e, 0xa1, 0xd2, 0x58, 0x54, 0x81, 0xa5, 0x67, 0xd8, - 0xf5, 0x6c, 0x4a, 0xd4, 0x2c, 0xb7, 0x75, 0x63, 0xd0, 0xd7, 0xd6, 0x42, 0x90, 0xc4, 0x1b, 0x51, - 0xa1, 0x1a, 0xac, 0xf1, 0xb6, 0xc0, 0x60, 0xcc, 0x31, 0x3c, 0xdc, 0xa2, 0xc4, 0xf2, 0x54, 0xd8, - 0x52, 0xb6, 0xe7, 0x44, 0x38, 0x39, 0xf2, 0x98, 0x39, 0x0d, 0x81, 0x92, 0xc3, 0x99, 0x40, 0xdd, - 0x9b, 0xff, 0xfc, 0x0b, 0x4d, 0x29, 0xfd, 0x46, 0x01, 0x94, 0xde, 0x0e, 0x72, 0x60, 0xb5, 0x4b, - 0x2d, 0x19, 0x14, 0xf6, 0x3c, 0x6f, 0xa4, 0xbd, 0x71, 0x34, 0x4a, 0x28, 0x0c, 0x49, 0x70, 0xc7, - 0x86, 0x3c, 0x98, 0xd1, 0x93, 0xa2, 0xab, 0x2b, 0xb0, 0x2c, 0x3b, 0xbe, 0xf4, 0xef, 0x45, 0x58, - 0x4d, 0x48, 0x45, 0x1e, 0x2c, 0x13, 0x6a, 0xe1, 0x06, 0x76, 0x70, 0x8b, 0x51, 0x37, 0x2c, 0x42, - 0xef, 0x4f, 0x35, 0x87, 0x77, 0xbf, 0x11, 0x97, 0x28, 0x45, 0x85, 0x41, 0x5f, 0xbb, 0x26, 0x0b, - 0x93, 0xdc, 0x34, 0xa2, 0x04, 0x1d, 0x41, 0xc6, 0x3c, 0x3d, 0xb5, 0x49, 0x90, 0x4c, 0xa2, 0xc2, - 0xdc, 0x1c, 0x37, 0x04, 0xec, 0x86, 0x34, 0x22, 0xd5, 0x22, 0x0e, 0x39, 0xd5, 0x22, 0x18, 0x3a, - 0x81, 0x1c, 0xa3, 0x0e, 0x76, 0x4d, 0x66, 0x53, 0x12, 0x8d, 0x05, 0xc5, 0xb1, 0x93, 0xc5, 0x90, - 0x6c, 0x78, 0xb1, 0xc9, 0xac, 0xba, 0xbc, 0x40, 0x14, 0x72, 0x26, 0x21, 0x94, 0x85, 0x62, 0x97, - 0x26, 0x8d, 0x02, 0x49, 0xe7, 0xec, 0xc6, 0x4c, 0xc2, 0x37, 0xbc, 0xac, 0x48, 0xa2, 0xe4, 0xb2, - 0x22, 0x81, 0x47, 0x8e, 0xd9, 0x3c, 0x6f, 0x79, 0xa6, 0x1f, 0xb3, 0x03, 0xc8, 0x47, 0x95, 0x89, - 0x92, 0x23, 0xea, 0xd8, 0xad, 0x1e, 0xff, 0xce, 0x91, 0x15, 0x97, 0x67, 0x12, 0x27, 0x5f, 0x9e, - 0x49, 0x1c, 0xfa, 0x18, 0x86, 0xaf, 0x4e, 0x23, 0x59, 0xba, 0xc8, 0xa3, 0xb4, 0x3d, 0xce, 0xa1, - 0xfa, 0x18, 0xfa, 0xea, 0xcd, 0xd0, 0xb5, 0x63, 0xa5, 0xe9, 0x63, 0xa1, 0x85, 0x36, 0xac, 0xa5, - 0x92, 0xea, 0xb5, 0x8c, 0x3f, 0xa7, 0x90, 0x4f, 0x06, 0xe8, 0x75, 0xe8, 0x39, 0x98, 0xcf, 0x64, - 0xf2, 0xd9, 0xd2, 0x9f, 0x15, 0xd8, 0x3c, 0xf2, 0x1d, 0xcf, 0x74, 0x1b, 0x51, 0xda, 0x1c, 0xd0, - 0xe6, 0x1e, 0x66, 0xa6, 0xed, 0x78, 0x81, 0x48, 0xfe, 0xc8, 0x13, 0x6a, 0xe6, 0x22, 0x39, 0x40, - 0x16, 0x29, 0xde, 0x96, 0xdf, 0x86, 0x85, 0xc7, 0xc9, 0xe9, 0x26, 0xd9, 0x0e, 0x09, 0x0a, 0x74, - 0x07, 0x16, 0x83, 0xfb, 0x15, 0xb3, 0x70, 0xb2, 0xe1, 0x83, 0xaf, 0x80, 0xc8, 0x83, 0xaf, 0x80, - 0xfc, 0xff, 0x21, 0xe4, 0xa4, 0x37, 0x2a, 0x94, 0x83, 0xa5, 0x93, 0xfa, 0x0f, 0xeb, 0x87, 0x3f, - 0xae, 0xe7, 0x67, 0x82, 0xc5, 0xd1, 0x7e, 0x7d, 0xaf, 0x56, 0xff, 0x41, 0x5e, 0x09, 0x16, 0xfa, - 0x49, 0xbd, 0x1e, 0x2c, 0x66, 0xd1, 0x15, 0xc8, 0x36, 0x4e, 0xee, 0xdf, 0xdf, 0xdf, 0xdf, 0xdb, - 0xdf, 0xcb, 0xcf, 0x21, 0x80, 0xc5, 0xef, 0xef, 0xd6, 0x1e, 0xee, 0xef, 0xe5, 0xe7, 0xab, 0x3f, - 0x7b, 0xfe, 0xa2, 0xa8, 0x7c, 0xf9, 0xa2, 0xa8, 0xfc, 0xeb, 0x45, 0x51, 0xf9, 0xec, 0x65, 0x71, - 0xe6, 0xcb, 0x97, 0xc5, 0x99, 0x7f, 0xbe, 0x2c, 0xce, 0xfc, 0xf4, 0xbe, 0xf4, 0xe9, 0x52, 0x3c, - 0x1b, 0x77, 0x5d, 0x1a, 0x9c, 0xa1, 0x70, 0x55, 0xb9, 0xc0, 0x37, 0xda, 0xe6, 0x22, 0xbf, 0xc3, - 0xde, 0xff, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x02, 0xb6, 0x98, 0xb3, 0xd1, 0x1d, 0x00, 0x00, + // 2129 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x39, 0x4f, 0x6f, 0x1b, 0xc7, + 0xf5, 0x5a, 0x89, 0xa2, 0xc8, 0x47, 0x59, 0xa2, 0x46, 0x96, 0xbd, 0xa2, 0x6d, 0xae, 0xc2, 0xf8, + 0x17, 0x28, 0xbf, 0x38, 0x64, 0xa3, 0x14, 0xa8, 0xe1, 0xf6, 0x22, 0x5a, 0x6a, 0x4d, 0xd5, 0xa6, + 0xe4, 0x95, 0xd5, 0xa2, 0x05, 0x9a, 0xc5, 0x92, 0x3b, 0xa2, 0x37, 0x5a, 0xce, 0xd0, 0xbb, 0xb3, + 0x6e, 0x98, 0x73, 0x7b, 0x28, 0x02, 0xa4, 0x41, 0x91, 0xb6, 0x01, 0x0a, 0xb4, 0xc8, 0xad, 0xe7, + 0x1e, 0xda, 0x43, 0xbf, 0x80, 0x8f, 0x39, 0xf6, 0xc4, 0x16, 0xf6, 0x8d, 0xd7, 0x7e, 0x81, 0x62, + 0x67, 0x76, 0xb9, 0xc3, 0x5d, 0x52, 0x94, 0x93, 0xba, 0x3e, 0x91, 0xf3, 0xfe, 0xcf, 0x7b, 0x6f, + 0xde, 0xbc, 0x37, 0x0b, 0x77, 0x6c, 0xc2, 0xb0, 0x4b, 0x4c, 0xa7, 0xe6, 0xb5, 0x1f, 0x63, 0xcb, + 0x77, 0xb0, 0x1b, 0xff, 0xa3, 0xad, 0x0f, 0x71, 0x9b, 0x79, 0x29, 0x40, 0xb5, 0xe7, 0x52, 0x46, + 0x51, 0x31, 0x09, 0x2f, 0x69, 0x1d, 0x4a, 0x3b, 0x0e, 0xae, 0x71, 0x7c, 0xcb, 0x3f, 0xad, 0x31, + 0xbb, 0x8b, 0x3d, 0x66, 0x76, 0x7b, 0x82, 0xa5, 0x54, 0x39, 0xbb, 0xed, 0x55, 0x6d, 0x5a, 0x33, + 0x7b, 0x76, 0xad, 0x4d, 0x5d, 0x5c, 0x7b, 0xfa, 0x5e, 0xad, 0x83, 0x09, 0x76, 0x4d, 0x86, 0xad, + 0x90, 0xe6, 0xdb, 0x31, 0x4d, 0xd7, 0x6c, 0x3f, 0xb6, 0x09, 0x76, 0xfb, 0xb5, 0xde, 0x59, 0x87, + 0x33, 0xb9, 0xd8, 0xa3, 0xbe, 0xdb, 0xc6, 0x29, 0xae, 0x77, 0x3b, 0x36, 0x7b, 0xec, 0xb7, 0xaa, + 0x6d, 0xda, 0xad, 0x75, 0x68, 0x87, 0xc6, 0x36, 0x04, 0x2b, 0xbe, 0xe0, 0xff, 0x04, 0x79, 0xe5, + 0xcf, 0x0b, 0x90, 0xdb, 0xff, 0x08, 0xb7, 0x7d, 0x46, 0x5d, 0xb4, 0x05, 0xf3, 0xb6, 0xa5, 0x2a, + 0x5b, 0xca, 0x76, 0xbe, 0x5e, 0x1c, 0x0e, 0xb4, 0x65, 0xdb, 0xba, 0x45, 0xbb, 0x36, 0xc3, 0xdd, + 0x1e, 0xeb, 0xeb, 0xf3, 0xb6, 0x85, 0xde, 0x82, 0x4c, 0x8f, 0x52, 0x47, 0x9d, 0xe7, 0x34, 0x68, + 0x38, 0xd0, 0x56, 0x82, 0xb5, 0x44, 0xc5, 0xf1, 0x68, 0x17, 0x16, 0x09, 0xb5, 0xb0, 0xa7, 0x2e, + 0x6c, 0x2d, 0x6c, 0x17, 0x76, 0xae, 0x54, 0x53, 0xae, 0x6b, 0x52, 0x0b, 0xd7, 0xd7, 0x87, 0x03, + 0x6d, 0x95, 0x13, 0x4a, 0x12, 0x04, 0x27, 0xfa, 0x00, 0x56, 0xba, 0x36, 0xb1, 0xbb, 0x7e, 0xf7, + 0x80, 0xb6, 0x8e, 0xed, 0x8f, 0xb1, 0x9a, 0xd9, 0x52, 0xb6, 0x0b, 0x3b, 0xe5, 0xb4, 0x2c, 0x3d, + 0x74, 0xc6, 0x7d, 0xdb, 0x63, 0xf5, 0x2b, 0xcf, 0x06, 0xda, 0x5c, 0x60, 0xd8, 0x38, 0xb7, 0x9e, + 0x58, 0x07, 0xf2, 0x1d, 0xd3, 0x63, 0x27, 0x3d, 0xcb, 0x64, 0xf8, 0x91, 0xdd, 0xc5, 0xea, 0x22, + 0x97, 0x5f, 0xaa, 0x8a, 0xe0, 0x55, 0x23, 0xc7, 0x55, 0x1f, 0x45, 0xc1, 0xab, 0x97, 0x22, 0xd9, + 0xe3, 0x9c, 0x9f, 0xfd, 0x53, 0x53, 0xf4, 0x04, 0x0c, 0x1d, 0xc2, 0xba, 0x4f, 0x4c, 0xcf, 0xb3, + 0x3b, 0x04, 0x5b, 0xc6, 0x87, 0xb4, 0x65, 0xb8, 0x3e, 0xf1, 0xd4, 0xfc, 0xd6, 0xc2, 0x76, 0xbe, + 0xae, 0x0d, 0x07, 0xda, 0xb5, 0x18, 0x7d, 0x40, 0x5b, 0xba, 0x4f, 0x64, 0x27, 0xac, 0xa5, 0x90, + 0x95, 0xbf, 0x6c, 0x40, 0x26, 0xf0, 0xda, 0xc5, 0xc2, 0x44, 0xcc, 0x2e, 0x56, 0x97, 0xe3, 0x30, + 0x05, 0x6b, 0x39, 0x4c, 0xc1, 0x1a, 0xed, 0x40, 0x0e, 0x87, 0xc1, 0x57, 0xd7, 0x39, 0xed, 0x95, + 0xe1, 0x40, 0x43, 0x11, 0x4c, 0xa2, 0x1f, 0xd1, 0xa1, 0x07, 0x90, 0x0f, 0x76, 0x6a, 0x78, 0x18, + 0x13, 0x9e, 0x07, 0xe7, 0xbb, 0xec, 0x72, 0xe8, 0xb2, 0x5c, 0xc0, 0x74, 0x8c, 0x31, 0xe1, 0xce, + 0x1a, 0xad, 0xd0, 0x21, 0xe4, 0x83, 0x78, 0x1b, 0xac, 0xdf, 0xc3, 0xea, 0x42, 0x28, 0x6e, 0x62, + 0xb6, 0x3c, 0xea, 0xf7, 0xb0, 0xb0, 0x8f, 0x84, 0x2b, 0xd9, 0xbe, 0x08, 0x86, 0x76, 0x21, 0xcb, + 0x4c, 0x9b, 0x30, 0x4f, 0x5d, 0xe4, 0xb9, 0xb7, 0x59, 0x15, 0xe7, 0xa8, 0x6a, 0xf6, 0xec, 0x6a, + 0x70, 0xd6, 0xaa, 0x4f, 0xdf, 0xab, 0x3e, 0x0a, 0x28, 0xea, 0x2b, 0xa1, 0x6d, 0x21, 0x83, 0x1e, + 0xfe, 0xa2, 0x23, 0xc8, 0x3a, 0x66, 0x0b, 0x3b, 0x9e, 0x9a, 0xe5, 0x22, 0x2a, 0x93, 0x0d, 0xaa, + 0xde, 0xe7, 0x44, 0xfb, 0x84, 0xb9, 0xfd, 0xfa, 0xe5, 0xe1, 0x40, 0x2b, 0x0a, 0x2e, 0xc9, 0xac, + 0x50, 0x0e, 0x32, 0x60, 0x95, 0x51, 0x66, 0x3a, 0x46, 0x74, 0x6e, 0x3d, 0x75, 0xe9, 0xe5, 0xb2, + 0x99, 0xb3, 0x47, 0x28, 0x4f, 0x4f, 0xac, 0xd1, 0x5f, 0x15, 0xb8, 0x69, 0x3a, 0x0e, 0x6d, 0x9b, + 0xcc, 0x6c, 0x39, 0xd8, 0x68, 0xf5, 0x8d, 0x9e, 0x6b, 0x53, 0xd7, 0x66, 0x7d, 0xc3, 0x24, 0xd6, + 0x48, 0xaf, 0x9a, 0xe3, 0x3b, 0xfa, 0xde, 0x94, 0x1d, 0xed, 0xc6, 0x22, 0xea, 0xfd, 0xa3, 0x50, + 0xc0, 0x2e, 0xb1, 0x22, 0x45, 0x62, 0xaf, 0xdb, 0xa1, 0x51, 0x5b, 0xe6, 0x0c, 0x72, 0x7d, 0x26, + 0x05, 0x72, 0x61, 0xdd, 0x63, 0x26, 0xe3, 0x16, 0x87, 0x87, 0xc4, 0xb0, 0x2d, 0x7e, 0x4c, 0x0a, + 0x3b, 0xef, 0x4c, 0x31, 0xf3, 0x38, 0xe0, 0xa8, 0xf7, 0xc5, 0xc9, 0x68, 0x58, 0xc2, 0xaa, 0xab, + 0xa1, 0x55, 0xab, 0xde, 0x38, 0x56, 0x4f, 0x02, 0x90, 0x0f, 0xeb, 0xa1, 0x5d, 0xd8, 0x8a, 0xf4, + 0xda, 0x96, 0x0a, 0x5c, 0xe7, 0xad, 0xf3, 0x5d, 0x83, 0x2d, 0x2e, 0x28, 0x52, 0xaa, 0x86, 0x4a, + 0x8b, 0x66, 0x02, 0xad, 0xa7, 0x20, 0x88, 0x01, 0x1a, 0x53, 0xfb, 0xc4, 0xc7, 0x3e, 0x56, 0x0b, + 0x17, 0xd5, 0xfa, 0x30, 0x20, 0x9f, 0xae, 0x95, 0xa3, 0xf5, 0x14, 0x24, 0xd8, 0x2c, 0x7e, 0x6a, + 0xb7, 0x59, 0x5c, 0x84, 0x0c, 0xdb, 0xf2, 0xd4, 0x95, 0x73, 0xd5, 0xee, 0x0b, 0x8e, 0xc8, 0x63, + 0x5e, 0x42, 0x2d, 0x4e, 0xa0, 0xf5, 0x14, 0x04, 0x7d, 0xa9, 0x40, 0x99, 0x50, 0x62, 0x98, 0x6e, + 0xd7, 0xb4, 0x4c, 0x23, 0xde, 0x78, 0x7c, 0x02, 0x2e, 0x71, 0x13, 0xbe, 0x33, 0xc5, 0x84, 0x26, + 0x25, 0xbb, 0x9c, 0x77, 0xe4, 0x82, 0x51, 0xb6, 0x0b, 0x6b, 0xde, 0x0c, 0xad, 0xb9, 0x46, 0xa6, + 0x53, 0xea, 0xe7, 0x21, 0xd1, 0x2e, 0x5c, 0xf2, 0x49, 0xa8, 0x3d, 0xc8, 0x50, 0x75, 0x75, 0x4b, + 0xd9, 0xce, 0xd5, 0xaf, 0x0d, 0x07, 0xda, 0xd5, 0x31, 0x84, 0x74, 0xa2, 0xc7, 0x39, 0xd0, 0x27, + 0x0a, 0x5c, 0x8d, 0x76, 0x64, 0xf8, 0x9e, 0xd9, 0xc1, 0x71, 0x64, 0x8b, 0x7c, 0x7f, 0xdf, 0x9a, + 0xb2, 0xbf, 0xc8, 0x8c, 0x93, 0x80, 0x69, 0x2c, 0xba, 0x95, 0xe1, 0x40, 0x2b, 0xbb, 0x13, 0xd0, + 0x92, 0x19, 0x97, 0x27, 0xe1, 0x83, 0x3b, 0xc7, 0xc5, 0x3d, 0xea, 0x32, 0x9b, 0x74, 0x8c, 0xb8, + 0xac, 0xae, 0xf1, 0xd2, 0xce, 0xef, 0x9c, 0x11, 0xba, 0x99, 0xae, 0xa1, 0x6b, 0x29, 0x64, 0xc9, + 0x84, 0x82, 0x54, 0xe4, 0xd0, 0x9b, 0xb0, 0x70, 0x86, 0xfb, 0xe1, 0xd5, 0xb3, 0x36, 0x1c, 0x68, + 0x97, 0xce, 0x70, 0x5f, 0x92, 0x10, 0x60, 0xd1, 0xdb, 0xb0, 0xf8, 0xd4, 0x74, 0x7c, 0x1c, 0x36, + 0x09, 0xfc, 0x8e, 0xe7, 0x00, 0xf9, 0x8e, 0xe7, 0x80, 0x3b, 0xf3, 0xb7, 0x95, 0xd2, 0x1f, 0x14, + 0xf8, 0xbf, 0x0b, 0x95, 0x1d, 0x59, 0xfb, 0xe2, 0x54, 0xed, 0x0d, 0x59, 0xfb, 0xec, 0xfa, 0x3a, + 0xcb, 0xba, 0x5f, 0x29, 0x70, 0x79, 0x52, 0xb5, 0xb9, 0x98, 0x2b, 0xee, 0xc9, 0xc6, 0xac, 0xec, + 0xdc, 0x48, 0x1b, 0x23, 0x84, 0x0a, 0x0d, 0xb3, 0x6c, 0xf9, 0x44, 0x81, 0x8d, 0x89, 0x55, 0xe8, + 0x62, 0xc6, 0xfc, 0x97, 0x3d, 0x93, 0xb0, 0x26, 0xce, 0xdf, 0xd7, 0x62, 0xcd, 0x19, 0x6c, 0x4c, + 0xac, 0x59, 0x5f, 0x23, 0x65, 0x73, 0x33, 0x95, 0xfd, 0x4e, 0x81, 0xad, 0x59, 0xe5, 0xe9, 0xb5, + 0x64, 0xeb, 0xaf, 0x15, 0xd8, 0x9c, 0x5a, 0x57, 0x5e, 0x47, 0x5c, 0x2a, 0x7f, 0xcc, 0x40, 0x2e, + 0xaa, 0x26, 0x41, 0xe3, 0xda, 0x10, 0x8d, 0x6b, 0x46, 0x34, 0xae, 0x8d, 0xb1, 0xc6, 0xb5, 0x61, + 0x49, 0xcd, 0xdb, 0xfc, 0xd7, 0x6d, 0xde, 0x1e, 0x8d, 0x9a, 0x37, 0x31, 0x7b, 0xbc, 0x35, 0xbd, + 0x9b, 0x7c, 0x89, 0x06, 0xee, 0x17, 0x0a, 0x20, 0x9f, 0x78, 0x98, 0x35, 0x88, 0x85, 0x3f, 0xc2, + 0x96, 0xe0, 0x54, 0x33, 0x5c, 0xc5, 0xce, 0x39, 0x2a, 0x4e, 0x52, 0x4c, 0x42, 0xdd, 0xd6, 0x70, + 0xa0, 0x5d, 0x4f, 0x4b, 0x94, 0x54, 0x4f, 0xd0, 0xf7, 0xbf, 0xa8, 0xc7, 0x5d, 0xb8, 0x3a, 0xc5, + 0xe6, 0x57, 0xa1, 0xae, 0xf2, 0x2c, 0x0b, 0x9b, 0x3c, 0x47, 0xef, 0x3a, 0xbe, 0xc7, 0xb0, 0x3b, + 0x96, 0xbe, 0xa8, 0x01, 0x4b, 0x6d, 0x17, 0x07, 0xa7, 0x8b, 0x6b, 0x3d, 0x7f, 0xd4, 0x58, 0x0f, + 0x33, 0x22, 0x62, 0xe1, 0x93, 0x46, 0xb4, 0x08, 0xec, 0x12, 0xd7, 0xb2, 0x64, 0xd7, 0x93, 0xc4, + 0xad, 0x2a, 0x28, 0xd0, 0x6d, 0x80, 0x68, 0xdc, 0x69, 0x58, 0x7c, 0x28, 0xc9, 0xd7, 0xd5, 0xe1, + 0x40, 0xbb, 0x1c, 0x43, 0x25, 0x26, 0x89, 0x16, 0xfd, 0x56, 0x09, 0x6e, 0xe0, 0xb0, 0x0e, 0xc4, + 0x57, 0x59, 0x98, 0x27, 0x7b, 0xe9, 0x3c, 0x99, 0xba, 0xf5, 0xd1, 0x31, 0x93, 0xc4, 0x88, 0xcc, + 0xb9, 0x11, 0x6e, 0x73, 0xa2, 0x22, 0x45, 0x9f, 0x04, 0x46, 0x7f, 0x53, 0xe0, 0xfa, 0x04, 0xf8, + 0x5d, 0xc7, 0xf4, 0xbc, 0xa6, 0xc9, 0x67, 0xdf, 0xc0, 0xc0, 0x07, 0xdf, 0xd0, 0xc0, 0x91, 0x3c, + 0x61, 0xe9, 0xcd, 0xd0, 0xd2, 0x73, 0x55, 0xeb, 0xe7, 0x62, 0x4b, 0x9f, 0x2a, 0xa0, 0x4e, 0x73, + 0xc5, 0x6b, 0xa9, 0xb1, 0xbf, 0x57, 0xe0, 0x8d, 0x99, 0x5b, 0x7f, 0x2d, 0xb5, 0xf6, 0xef, 0x0b, + 0x50, 0x9a, 0x14, 0x29, 0x9d, 0xb7, 0x75, 0xa3, 0xb7, 0x1b, 0x65, 0xc6, 0xdb, 0x8d, 0x74, 0xe6, + 0xe6, 0xbf, 0xe1, 0x99, 0xfb, 0x54, 0x81, 0xa2, 0x14, 0x5d, 0x9e, 0x4b, 0x61, 0x59, 0xae, 0xa7, + 0x37, 0x3b, 0xdd, 0x76, 0x39, 0xd7, 0xa4, 0x46, 0xb9, 0x3c, 0x1c, 0x68, 0xa5, 0xa4, 0x7c, 0x69, + 0x3f, 0x29, 0xdd, 0xa5, 0x2f, 0x14, 0xd8, 0x98, 0x28, 0xeb, 0x62, 0x01, 0xfb, 0xd1, 0x78, 0xc0, + 0xde, 0x79, 0x89, 0xe3, 0x32, 0x33, 0x7a, 0xbf, 0x9c, 0x87, 0x65, 0x39, 0xdc, 0xe8, 0x03, 0xc8, + 0xc7, 0xb3, 0x92, 0xc2, 0x9d, 0xf6, 0xee, 0xf9, 0x19, 0x52, 0x4d, 0x4c, 0x48, 0x6b, 0x61, 0x70, + 0x62, 0x39, 0x7a, 0xfc, 0xb7, 0xf4, 0xb9, 0x02, 0x2b, 0xd3, 0x7b, 0x96, 0xe9, 0x4e, 0xf8, 0xc9, + 0xb8, 0x13, 0xaa, 0xd2, 0x15, 0x3d, 0x7a, 0xa7, 0xac, 0xf6, 0xce, 0x3a, 0xfc, 0xce, 0x8e, 0xd4, + 0x55, 0x1f, 0xfa, 0x26, 0x61, 0x36, 0xeb, 0xcf, 0xf4, 0xc3, 0xe7, 0x8b, 0xb0, 0x76, 0x40, 0x5b, + 0xc7, 0x62, 0xa3, 0x36, 0xe9, 0x34, 0xc8, 0x29, 0x45, 0x3b, 0x90, 0x73, 0xec, 0x53, 0xcc, 0xec, + 0x2e, 0xe6, 0xe6, 0x5d, 0x12, 0x2f, 0x41, 0x11, 0x4c, 0x7e, 0x09, 0x8a, 0x60, 0xe8, 0x0e, 0x2c, + 0x9b, 0xcc, 0xe8, 0x52, 0x8f, 0x19, 0x94, 0xb4, 0xa3, 0xe6, 0x8e, 0x17, 0x72, 0x93, 0x3d, 0xa0, + 0x1e, 0x3b, 0x24, 0x6d, 0x99, 0x13, 0x62, 0x28, 0xfa, 0x2e, 0x14, 0x7a, 0x2e, 0x0e, 0xe0, 0x76, + 0x30, 0x18, 0x2e, 0x70, 0xd6, 0xcd, 0xe1, 0x40, 0xdb, 0x90, 0xc0, 0x12, 0xaf, 0x4c, 0x8d, 0xee, + 0x41, 0xb1, 0x4d, 0x49, 0xdb, 0x77, 0x5d, 0x4c, 0xda, 0x7d, 0xc3, 0x33, 0x4f, 0xc5, 0xe3, 0x65, + 0xae, 0x7e, 0x63, 0x38, 0xd0, 0x36, 0x25, 0xdc, 0xb1, 0x79, 0x2a, 0x4b, 0x59, 0x4d, 0xa0, 0x82, + 0x81, 0x6e, 0xf4, 0x8c, 0xd3, 0x0e, 0x2a, 0x8c, 0xc1, 0xdf, 0xf5, 0xb2, 0xf1, 0x40, 0xd7, 0x4b, + 0xd6, 0x1f, 0x79, 0xa0, 0x4b, 0x21, 0xd1, 0x31, 0x14, 0x3c, 0xbf, 0xd5, 0xb5, 0x99, 0xc1, 0x5d, + 0xb9, 0x34, 0xf3, 0x80, 0x47, 0x0f, 0x50, 0x20, 0xd8, 0x46, 0xcf, 0x9d, 0xd2, 0x3a, 0x08, 0x4e, + 0xa4, 0x49, 0xcd, 0xc5, 0xc1, 0x89, 0x60, 0x72, 0x70, 0x22, 0x18, 0xfa, 0x39, 0xac, 0x8b, 0x14, + 0x36, 0x5c, 0xfc, 0xc4, 0xb7, 0x5d, 0xdc, 0xc5, 0xf1, 0x9b, 0xdd, 0xcd, 0x74, 0x9e, 0x1f, 0xf2, + 0x5f, 0x5d, 0xa2, 0x15, 0x2d, 0x14, 0x4d, 0xc1, 0xe5, 0x16, 0x2a, 0x8d, 0x45, 0x35, 0x58, 0x7a, + 0x8a, 0x5d, 0xcf, 0xa6, 0x44, 0xcd, 0x73, 0x5b, 0x37, 0x86, 0x03, 0x6d, 0x2d, 0x04, 0x49, 0xbc, + 0x11, 0xd5, 0x9d, 0xcc, 0x17, 0x5f, 0x6a, 0x4a, 0xe5, 0x37, 0x0a, 0xa0, 0xb4, 0x0d, 0xc8, 0x81, + 0xd5, 0x1e, 0xb5, 0x64, 0x50, 0xd8, 0xa8, 0xbc, 0x91, 0xde, 0xc2, 0xd1, 0x38, 0xa1, 0x48, 0x86, + 0x04, 0x77, 0x6c, 0xc0, 0xbd, 0x39, 0x3d, 0x29, 0xba, 0xbe, 0x02, 0xcb, 0xb2, 0xb7, 0x2a, 0xff, + 0xce, 0xc2, 0x6a, 0x42, 0x2a, 0xf2, 0x60, 0x39, 0x98, 0xfc, 0x8f, 0xb1, 0x83, 0xdb, 0x8c, 0xba, + 0x61, 0xe5, 0x78, 0x7f, 0xa6, 0x39, 0xbc, 0x65, 0x8d, 0xb8, 0x44, 0xfd, 0x28, 0x0d, 0x07, 0xda, + 0x15, 0x59, 0x98, 0xe4, 0x9e, 0x31, 0x25, 0xe8, 0x08, 0x72, 0xe6, 0xe9, 0xa9, 0x4d, 0x82, 0x0c, + 0x10, 0x65, 0xe1, 0xfa, 0xa4, 0xce, 0x7d, 0x37, 0xa4, 0x11, 0xf9, 0x11, 0x71, 0xc8, 0xf9, 0x11, + 0xc1, 0xd0, 0x09, 0x14, 0x18, 0x75, 0xb0, 0x6b, 0x32, 0x9b, 0x92, 0xa8, 0x97, 0x2f, 0x4f, 0x1c, + 0x07, 0x46, 0x64, 0xa3, 0xdb, 0x48, 0x66, 0xd5, 0xe5, 0x05, 0xa2, 0x50, 0x30, 0x09, 0xa1, 0x2c, + 0x14, 0xbb, 0x34, 0xad, 0x7f, 0x4f, 0x3a, 0x67, 0x37, 0x66, 0x12, 0xbe, 0xe1, 0xb5, 0x40, 0x12, + 0x25, 0xd7, 0x02, 0x09, 0x3c, 0x76, 0x36, 0x32, 0xbc, 0x4f, 0x99, 0x7d, 0x36, 0x0e, 0xa0, 0x18, + 0x95, 0x13, 0x4a, 0x8e, 0xa8, 0x63, 0xb7, 0xfb, 0xfc, 0xe3, 0x44, 0x5e, 0xdc, 0x78, 0x49, 0x9c, + 0x7c, 0xe3, 0x25, 0x71, 0xe8, 0x63, 0x18, 0x3d, 0x15, 0x8d, 0x65, 0x69, 0x96, 0x47, 0x69, 0x7b, + 0x92, 0x43, 0xf5, 0x09, 0xf4, 0xf5, 0xeb, 0xa1, 0x6b, 0x27, 0x4a, 0xd3, 0x27, 0x42, 0x4b, 0x1d, + 0x58, 0x4b, 0x25, 0xd5, 0x2b, 0x99, 0x59, 0x4e, 0xa1, 0x98, 0x0c, 0xd0, 0xab, 0xd0, 0x73, 0x90, + 0xc9, 0xe5, 0x8a, 0xf9, 0xca, 0x9f, 0x14, 0xd8, 0x3c, 0xf2, 0x1d, 0xcf, 0x74, 0x8f, 0xa3, 0xb4, + 0x39, 0xa0, 0xad, 0x3d, 0xcc, 0x4c, 0xdb, 0xf1, 0x02, 0x91, 0xfc, 0x65, 0x26, 0xd4, 0xcc, 0x45, + 0x72, 0x80, 0x2c, 0x52, 0x3c, 0x08, 0xbf, 0x0d, 0x8b, 0x0f, 0x93, 0x23, 0x49, 0xb2, 0x87, 0x11, + 0x14, 0xe8, 0x16, 0x64, 0x83, 0x4b, 0x11, 0xb3, 0x70, 0x1c, 0xe1, 0xd3, 0xaa, 0x80, 0xc8, 0xd3, + 0xaa, 0x80, 0xfc, 0xff, 0x21, 0x14, 0xa4, 0x87, 0x25, 0x54, 0x80, 0xa5, 0x93, 0xe6, 0x0f, 0x9b, + 0x87, 0x3f, 0x6e, 0x16, 0xe7, 0x82, 0xc5, 0xd1, 0x7e, 0x73, 0xaf, 0xd1, 0xfc, 0x41, 0x51, 0x09, + 0x16, 0xfa, 0x49, 0xb3, 0x19, 0x2c, 0xe6, 0xd1, 0x25, 0xc8, 0x1f, 0x9f, 0xdc, 0xbd, 0xbb, 0xbf, + 0xbf, 0xb7, 0xbf, 0x57, 0x5c, 0x40, 0x00, 0xd9, 0xef, 0xef, 0x36, 0xee, 0xef, 0xef, 0x15, 0x33, + 0xf5, 0x9f, 0x3d, 0x7b, 0x5e, 0x56, 0xbe, 0x7a, 0x5e, 0x56, 0xfe, 0xf5, 0xbc, 0xac, 0x7c, 0xf6, + 0xa2, 0x3c, 0xf7, 0xd5, 0x8b, 0xf2, 0xdc, 0x3f, 0x5e, 0x94, 0xe7, 0x7e, 0x7a, 0x57, 0xfa, 0xde, + 0x28, 0xde, 0x7a, 0x7b, 0x2e, 0x0d, 0xce, 0x50, 0xb8, 0xaa, 0x5d, 0xe0, 0xc3, 0x6a, 0x2b, 0xcb, + 0x2f, 0x9e, 0xf7, 0xff, 0x13, 0x00, 0x00, 0xff, 0xff, 0xaa, 0x1f, 0xf9, 0x4f, 0x86, 0x1d, 0x00, + 0x00, } func (m *Executor) Marshal() (dAtA []byte, err error) { @@ -1812,11 +1802,6 @@ func (m *JobSchedulingInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.QueueTtlSeconds != 0 { - i = encodeVarintSchedulerobjects(dAtA, i, uint64(m.QueueTtlSeconds)) - i-- - dAtA[i] = 0x50 - } if m.Version != 0 { i = encodeVarintSchedulerobjects(dAtA, i, uint64(m.Version)) i-- @@ -2412,9 +2397,6 @@ func (m *JobSchedulingInfo) Size() (n int) { if m.Version != 0 { n += 1 + sovSchedulerobjects(uint64(m.Version)) } - if m.QueueTtlSeconds != 0 { - n += 1 + sovSchedulerobjects(uint64(m.QueueTtlSeconds)) - } return n } @@ -5481,25 +5463,6 @@ func (m *JobSchedulingInfo) Unmarshal(dAtA []byte) error { break } } - case 10: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field QueueTtlSeconds", wireType) - } - m.QueueTtlSeconds = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowSchedulerobjects - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.QueueTtlSeconds |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skipSchedulerobjects(dAtA[iNdEx:]) diff --git a/internal/scheduler/schedulerobjects/schedulerobjects.proto b/internal/scheduler/schedulerobjects/schedulerobjects.proto index d3b5aeab148..079d87ca371 100644 --- a/internal/scheduler/schedulerobjects/schedulerobjects.proto +++ b/internal/scheduler/schedulerobjects/schedulerobjects.proto @@ -131,8 +131,7 @@ message JobSchedulingInfo { // Kubernetes objects that make up this job and their respective scheduling requirements. repeated ObjectRequirements object_requirements = 5; uint32 version = 9; - // Queuing TTL for this job in seconds. If this job queues for more than this duration it will be cancelled. Zero indicates an infinite lifetime. - int64 queue_ttl_seconds = 10; + // ordinal 10 was previously used for queue_ttl_seconds } // Message capturing the scheduling requirements of a particular Kubernetes object. diff --git a/internal/scheduleringester/instructions.go b/internal/scheduleringester/instructions.go index 93caa00857b..9e0468540d8 100644 --- a/internal/scheduleringester/instructions.go +++ b/internal/scheduleringester/instructions.go @@ -437,7 +437,6 @@ func SchedulingInfoFromSubmitJob(submitJob *armadaevents.SubmitJob, submitTime t SubmitTime: submitTime, Priority: submitJob.Priority, Version: 0, - QueueTtlSeconds: submitJob.QueueTtlSeconds, } // Scheduling requirements specific to the objects that make up this job. diff --git a/pkg/api/api.swagger.go b/pkg/api/api.swagger.go index bb440673ae4..702194044db 100644 --- a/pkg/api/api.swagger.go +++ b/pkg/api/api.swagger.go @@ -1655,11 +1655,6 @@ func SwaggerJsonTemplate() string { " \"type\": \"number\",\n" + " \"format\": \"double\"\n" + " },\n" + - " \"queueTtlSeconds\": {\n" + - " \"description\": \"Queuing TTL for this job in seconds. If this job queues for more than this duration it will be cancelled. Zero indicates an infinite lifetime.\",\n" + - " \"type\": \"string\",\n" + - " \"format\": \"int64\"\n" + - " },\n" + " \"requiredNodeLabels\": {\n" + " \"type\": \"object\",\n" + " \"additionalProperties\": {\n" + @@ -5294,7 +5289,7 @@ func SwaggerJsonTemplate() string { " \"x-go-package\": \"k8s.io/api/core/v1\"\n" + " },\n" + " \"v1Time\": {\n" + - " \"description\": \"Programs using times should typically store and pass them as values,\\nnot pointers. That is, time variables and struct fields should be of\\ntype time.Time, not *time.Time.\\n\\nA Time value can be used by multiple goroutines simultaneously except\\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\\nUnmarshalText are not concurrency-safe.\\n\\nTime instants can be compared using the Before, After, and Equal methods.\\nThe Sub method subtracts two instants, producing a Duration.\\nThe Add method adds a Time and a Duration, producing a Time.\\n\\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\\nAs this time is unlikely to come up in practice, the IsZero method gives\\na simple way of detecting a time that has not been initialized explicitly.\\n\\nEach Time has associated with it a Location, consulted when computing the\\npresentation form of the time, such as in the Format, Hour, and Year methods.\\nThe methods Local, UTC, and In return a Time with a specific location.\\nChanging the location in this way changes only the presentation; it does not\\nchange the instant in time being denoted and therefore does not affect the\\ncomputations described in earlier paragraphs.\\n\\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\\nthe location name. They therefore lose information about Daylight Saving Time.\\n\\nIn addition to the required “wall clock” reading, a Time may contain an optional\\nreading of the current process's monotonic clock, to provide additional precision\\nfor comparison or subtraction.\\nSee the “Monotonic Clocks” section in the package documentation for details.\\n\\nNote that the Go == operator compares not just the time instant but also the\\nLocation and the monotonic clock reading. Therefore, Time values should not\\nbe used as map or database keys without first guaranteeing that the\\nidentical Location has been set for all values, which can be achieved\\nthrough use of the UTC or Local method, and that the monotonic clock reading\\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\\nto t == u, since t.Equal uses the most accurate comparison available and\\ncorrectly handles the case when only one of its arguments has a monotonic\\nclock reading.\",\n" + + " \"description\": \"Programs using times should typically store and pass them as values,\\nnot pointers. That is, time variables and struct fields should be of\\ntype time.Time, not *time.Time.\\n\\nA Time value can be used by multiple goroutines simultaneously except\\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\\nUnmarshalText are not concurrency-safe.\\n\\nTime instants can be compared using the Before, After, and Equal methods.\\nThe Sub method subtracts two instants, producing a Duration.\\nThe Add method adds a Time and a Duration, producing a Time.\\n\\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\\nAs this time is unlikely to come up in practice, the IsZero method gives\\na simple way of detecting a time that has not been initialized explicitly.\\n\\nEach time has an associated Location. The methods Local, UTC, and In return a\\nTime with a specific Location. Changing the Location of a Time value with\\nthese methods does not change the actual instant it represents, only the time\\nzone in which to interpret it.\\n\\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\\nthe location name. They therefore lose information about Daylight Saving Time.\\n\\nIn addition to the required “wall clock” reading, a Time may contain an optional\\nreading of the current process's monotonic clock, to provide additional precision\\nfor comparison or subtraction.\\nSee the “Monotonic Clocks” section in the package documentation for details.\\n\\nNote that the Go == operator compares not just the time instant but also the\\nLocation and the monotonic clock reading. Therefore, Time values should not\\nbe used as map or database keys without first guaranteeing that the\\nidentical Location has been set for all values, which can be achieved\\nthrough use of the UTC or Local method, and that the monotonic clock reading\\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\\nto t == u, since t.Equal uses the most accurate comparison available and\\ncorrectly handles the case when only one of its arguments has a monotonic\\nclock reading.\",\n" + " \"type\": \"string\",\n" + " \"format\": \"date-time\",\n" + " \"title\": \"A Time represents an instant in time with nanosecond precision.\",\n" + diff --git a/pkg/api/api.swagger.json b/pkg/api/api.swagger.json index c549d2b5681..f7b96eb063e 100644 --- a/pkg/api/api.swagger.json +++ b/pkg/api/api.swagger.json @@ -1644,11 +1644,6 @@ "type": "number", "format": "double" }, - "queueTtlSeconds": { - "description": "Queuing TTL for this job in seconds. If this job queues for more than this duration it will be cancelled. Zero indicates an infinite lifetime.", - "type": "string", - "format": "int64" - }, "requiredNodeLabels": { "type": "object", "additionalProperties": { @@ -5283,7 +5278,7 @@ "x-go-package": "k8s.io/api/core/v1" }, "v1Time": { - "description": "Programs using times should typically store and pass them as values,\nnot pointers. That is, time variables and struct fields should be of\ntype time.Time, not *time.Time.\n\nA Time value can be used by multiple goroutines simultaneously except\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\nUnmarshalText are not concurrency-safe.\n\nTime instants can be compared using the Before, After, and Equal methods.\nThe Sub method subtracts two instants, producing a Duration.\nThe Add method adds a Time and a Duration, producing a Time.\n\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\nAs this time is unlikely to come up in practice, the IsZero method gives\na simple way of detecting a time that has not been initialized explicitly.\n\nEach Time has associated with it a Location, consulted when computing the\npresentation form of the time, such as in the Format, Hour, and Year methods.\nThe methods Local, UTC, and In return a Time with a specific location.\nChanging the location in this way changes only the presentation; it does not\nchange the instant in time being denoted and therefore does not affect the\ncomputations described in earlier paragraphs.\n\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\nthe location name. They therefore lose information about Daylight Saving Time.\n\nIn addition to the required “wall clock” reading, a Time may contain an optional\nreading of the current process's monotonic clock, to provide additional precision\nfor comparison or subtraction.\nSee the “Monotonic Clocks” section in the package documentation for details.\n\nNote that the Go == operator compares not just the time instant but also the\nLocation and the monotonic clock reading. Therefore, Time values should not\nbe used as map or database keys without first guaranteeing that the\nidentical Location has been set for all values, which can be achieved\nthrough use of the UTC or Local method, and that the monotonic clock reading\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\nto t == u, since t.Equal uses the most accurate comparison available and\ncorrectly handles the case when only one of its arguments has a monotonic\nclock reading.", + "description": "Programs using times should typically store and pass them as values,\nnot pointers. That is, time variables and struct fields should be of\ntype time.Time, not *time.Time.\n\nA Time value can be used by multiple goroutines simultaneously except\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\nUnmarshalText are not concurrency-safe.\n\nTime instants can be compared using the Before, After, and Equal methods.\nThe Sub method subtracts two instants, producing a Duration.\nThe Add method adds a Time and a Duration, producing a Time.\n\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\nAs this time is unlikely to come up in practice, the IsZero method gives\na simple way of detecting a time that has not been initialized explicitly.\n\nEach time has an associated Location. The methods Local, UTC, and In return a\nTime with a specific Location. Changing the Location of a Time value with\nthese methods does not change the actual instant it represents, only the time\nzone in which to interpret it.\n\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\nthe location name. They therefore lose information about Daylight Saving Time.\n\nIn addition to the required “wall clock” reading, a Time may contain an optional\nreading of the current process's monotonic clock, to provide additional precision\nfor comparison or subtraction.\nSee the “Monotonic Clocks” section in the package documentation for details.\n\nNote that the Go == operator compares not just the time instant but also the\nLocation and the monotonic clock reading. Therefore, Time values should not\nbe used as map or database keys without first guaranteeing that the\nidentical Location has been set for all values, which can be achieved\nthrough use of the UTC or Local method, and that the monotonic clock reading\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\nto t == u, since t.Equal uses the most accurate comparison available and\ncorrectly handles the case when only one of its arguments has a monotonic\nclock reading.", "type": "string", "format": "date-time", "title": "A Time represents an instant in time with nanosecond precision.", diff --git a/pkg/api/binoculars/api.swagger.go b/pkg/api/binoculars/api.swagger.go index 206e611b3c9..4a9905f6feb 100644 --- a/pkg/api/binoculars/api.swagger.go +++ b/pkg/api/binoculars/api.swagger.go @@ -238,7 +238,7 @@ func SwaggerJsonTemplate() string { " \"x-go-package\": \"k8s.io/api/core/v1\"\n" + " },\n" + " \"v1Time\": {\n" + - " \"description\": \"Programs using times should typically store and pass them as values,\\nnot pointers. That is, time variables and struct fields should be of\\ntype time.Time, not *time.Time.\\n\\nA Time value can be used by multiple goroutines simultaneously except\\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\\nUnmarshalText are not concurrency-safe.\\n\\nTime instants can be compared using the Before, After, and Equal methods.\\nThe Sub method subtracts two instants, producing a Duration.\\nThe Add method adds a Time and a Duration, producing a Time.\\n\\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\\nAs this time is unlikely to come up in practice, the IsZero method gives\\na simple way of detecting a time that has not been initialized explicitly.\\n\\nEach Time has associated with it a Location, consulted when computing the\\npresentation form of the time, such as in the Format, Hour, and Year methods.\\nThe methods Local, UTC, and In return a Time with a specific location.\\nChanging the location in this way changes only the presentation; it does not\\nchange the instant in time being denoted and therefore does not affect the\\ncomputations described in earlier paragraphs.\\n\\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\\nthe location name. They therefore lose information about Daylight Saving Time.\\n\\nIn addition to the required “wall clock” reading, a Time may contain an optional\\nreading of the current process's monotonic clock, to provide additional precision\\nfor comparison or subtraction.\\nSee the “Monotonic Clocks” section in the package documentation for details.\\n\\nNote that the Go == operator compares not just the time instant but also the\\nLocation and the monotonic clock reading. Therefore, Time values should not\\nbe used as map or database keys without first guaranteeing that the\\nidentical Location has been set for all values, which can be achieved\\nthrough use of the UTC or Local method, and that the monotonic clock reading\\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\\nto t == u, since t.Equal uses the most accurate comparison available and\\ncorrectly handles the case when only one of its arguments has a monotonic\\nclock reading.\",\n" + + " \"description\": \"Programs using times should typically store and pass them as values,\\nnot pointers. That is, time variables and struct fields should be of\\ntype time.Time, not *time.Time.\\n\\nA Time value can be used by multiple goroutines simultaneously except\\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\\nUnmarshalText are not concurrency-safe.\\n\\nTime instants can be compared using the Before, After, and Equal methods.\\nThe Sub method subtracts two instants, producing a Duration.\\nThe Add method adds a Time and a Duration, producing a Time.\\n\\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\\nAs this time is unlikely to come up in practice, the IsZero method gives\\na simple way of detecting a time that has not been initialized explicitly.\\n\\nEach time has an associated Location. The methods Local, UTC, and In return a\\nTime with a specific Location. Changing the Location of a Time value with\\nthese methods does not change the actual instant it represents, only the time\\nzone in which to interpret it.\\n\\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\\nthe location name. They therefore lose information about Daylight Saving Time.\\n\\nIn addition to the required “wall clock” reading, a Time may contain an optional\\nreading of the current process's monotonic clock, to provide additional precision\\nfor comparison or subtraction.\\nSee the “Monotonic Clocks” section in the package documentation for details.\\n\\nNote that the Go == operator compares not just the time instant but also the\\nLocation and the monotonic clock reading. Therefore, Time values should not\\nbe used as map or database keys without first guaranteeing that the\\nidentical Location has been set for all values, which can be achieved\\nthrough use of the UTC or Local method, and that the monotonic clock reading\\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\\nto t == u, since t.Equal uses the most accurate comparison available and\\ncorrectly handles the case when only one of its arguments has a monotonic\\nclock reading.\",\n" + " \"type\": \"string\",\n" + " \"format\": \"date-time\",\n" + " \"title\": \"A Time represents an instant in time with nanosecond precision.\",\n" + diff --git a/pkg/api/binoculars/api.swagger.json b/pkg/api/binoculars/api.swagger.json index e1ee15d781b..adf9a3af046 100644 --- a/pkg/api/binoculars/api.swagger.json +++ b/pkg/api/binoculars/api.swagger.json @@ -227,7 +227,7 @@ "x-go-package": "k8s.io/api/core/v1" }, "v1Time": { - "description": "Programs using times should typically store and pass them as values,\nnot pointers. That is, time variables and struct fields should be of\ntype time.Time, not *time.Time.\n\nA Time value can be used by multiple goroutines simultaneously except\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\nUnmarshalText are not concurrency-safe.\n\nTime instants can be compared using the Before, After, and Equal methods.\nThe Sub method subtracts two instants, producing a Duration.\nThe Add method adds a Time and a Duration, producing a Time.\n\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\nAs this time is unlikely to come up in practice, the IsZero method gives\na simple way of detecting a time that has not been initialized explicitly.\n\nEach Time has associated with it a Location, consulted when computing the\npresentation form of the time, such as in the Format, Hour, and Year methods.\nThe methods Local, UTC, and In return a Time with a specific location.\nChanging the location in this way changes only the presentation; it does not\nchange the instant in time being denoted and therefore does not affect the\ncomputations described in earlier paragraphs.\n\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\nthe location name. They therefore lose information about Daylight Saving Time.\n\nIn addition to the required “wall clock” reading, a Time may contain an optional\nreading of the current process's monotonic clock, to provide additional precision\nfor comparison or subtraction.\nSee the “Monotonic Clocks” section in the package documentation for details.\n\nNote that the Go == operator compares not just the time instant but also the\nLocation and the monotonic clock reading. Therefore, Time values should not\nbe used as map or database keys without first guaranteeing that the\nidentical Location has been set for all values, which can be achieved\nthrough use of the UTC or Local method, and that the monotonic clock reading\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\nto t == u, since t.Equal uses the most accurate comparison available and\ncorrectly handles the case when only one of its arguments has a monotonic\nclock reading.", + "description": "Programs using times should typically store and pass them as values,\nnot pointers. That is, time variables and struct fields should be of\ntype time.Time, not *time.Time.\n\nA Time value can be used by multiple goroutines simultaneously except\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\nUnmarshalText are not concurrency-safe.\n\nTime instants can be compared using the Before, After, and Equal methods.\nThe Sub method subtracts two instants, producing a Duration.\nThe Add method adds a Time and a Duration, producing a Time.\n\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\nAs this time is unlikely to come up in practice, the IsZero method gives\na simple way of detecting a time that has not been initialized explicitly.\n\nEach time has an associated Location. The methods Local, UTC, and In return a\nTime with a specific Location. Changing the Location of a Time value with\nthese methods does not change the actual instant it represents, only the time\nzone in which to interpret it.\n\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\nthe location name. They therefore lose information about Daylight Saving Time.\n\nIn addition to the required “wall clock” reading, a Time may contain an optional\nreading of the current process's monotonic clock, to provide additional precision\nfor comparison or subtraction.\nSee the “Monotonic Clocks” section in the package documentation for details.\n\nNote that the Go == operator compares not just the time instant but also the\nLocation and the monotonic clock reading. Therefore, Time values should not\nbe used as map or database keys without first guaranteeing that the\nidentical Location has been set for all values, which can be achieved\nthrough use of the UTC or Local method, and that the monotonic clock reading\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\nto t == u, since t.Equal uses the most accurate comparison available and\ncorrectly handles the case when only one of its arguments has a monotonic\nclock reading.", "type": "string", "format": "date-time", "title": "A Time represents an instant in time with nanosecond precision.", diff --git a/pkg/api/submit.pb.go b/pkg/api/submit.pb.go index b21c35e4fd1..8ccd6f8847d 100644 --- a/pkg/api/submit.pb.go +++ b/pkg/api/submit.pb.go @@ -151,8 +151,6 @@ type JobSubmitRequestItem struct { // Indicates which scheduler should manage this job. // If empty, the default scheduler is used. Scheduler string `protobuf:"bytes,11,opt,name=scheduler,proto3" json:"scheduler,omitempty"` - // Queuing TTL for this job in seconds. If this job queues for more than this duration it will be cancelled. Zero indicates an infinite lifetime. - QueueTtlSeconds int64 `protobuf:"varint,12,opt,name=queue_ttl_seconds,json=queueTtlSeconds,proto3" json:"queueTtlSeconds,omitempty"` } func (m *JobSubmitRequestItem) Reset() { *m = JobSubmitRequestItem{} } @@ -266,13 +264,6 @@ func (m *JobSubmitRequestItem) GetScheduler() string { return "" } -func (m *JobSubmitRequestItem) GetQueueTtlSeconds() int64 { - if m != nil { - return m.QueueTtlSeconds - } - return 0 -} - type IngressConfig struct { Type IngressType `protobuf:"varint,1,opt,name=type,proto3,enum=api.IngressType" json:"type,omitempty"` // Deprecated: Do not use. Ports []uint32 `protobuf:"varint,2,rep,packed,name=ports,proto3" json:"ports,omitempty"` @@ -2090,192 +2081,192 @@ func init() { proto.RegisterFile("pkg/api/submit.proto", fileDescriptor_e998bacb var fileDescriptor_e998bacb27df16c1 = []byte{ // 2972 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0xcd, 0x6f, 0x1b, 0xc7, - 0x15, 0xe7, 0x8a, 0x12, 0x25, 0x3e, 0xea, 0x83, 0x1a, 0x7d, 0x78, 0x45, 0x3b, 0xa2, 0xbc, 0x49, - 0x5c, 0x59, 0x48, 0xa8, 0x58, 0x69, 0x50, 0xdb, 0x4d, 0xe1, 0x8a, 0x12, 0x6d, 0x4b, 0xb1, 0x65, - 0x85, 0xb2, 0x92, 0xa6, 0x28, 0xca, 0x2c, 0xb9, 0x23, 0x6a, 0x25, 0x72, 0x97, 0xd9, 0x5d, 0xca, - 0x55, 0x8b, 0x00, 0x45, 0x51, 0xb4, 0x87, 0x5e, 0x52, 0xf4, 0xd6, 0x02, 0xb9, 0xf4, 0x96, 0x1e, - 0x7a, 0x29, 0x8a, 0xfe, 0x09, 0x39, 0x06, 0x28, 0x0a, 0xa4, 0x17, 0xa6, 0x75, 0xfa, 0x01, 0xf0, - 0xd6, 0x7b, 0x0f, 0xc5, 0xbc, 0x99, 0xe5, 0xce, 0xf2, 0x43, 0x94, 0x6c, 0xcb, 0xb9, 0xf4, 0xa6, - 0xf9, 0xcd, 0xfb, 0xde, 0x37, 0x6f, 0xde, 0x1b, 0x0a, 0xa6, 0x6b, 0x87, 0xe5, 0x65, 0xbd, 0x66, - 0x2e, 0xbb, 0xf5, 0x62, 0xd5, 0xf4, 0x32, 0x35, 0xc7, 0xf6, 0x6c, 0x12, 0xd5, 0x6b, 0x66, 0xea, - 0x62, 0xd9, 0xb6, 0xcb, 0x15, 0xba, 0x8c, 0x50, 0xb1, 0xbe, 0xb7, 0x4c, 0xab, 0x35, 0xef, 0x98, - 0x53, 0xa4, 0xd2, 0xed, 0x9b, 0x9e, 0x59, 0xa5, 0xae, 0xa7, 0x57, 0x6b, 0x82, 0x40, 0x3b, 0xbc, - 0xee, 0x66, 0x4c, 0x1b, 0x65, 0x97, 0x6c, 0x87, 0x2e, 0x1f, 0x5d, 0x5b, 0x2e, 0x53, 0x8b, 0x3a, - 0xba, 0x47, 0x0d, 0x41, 0xb3, 0x28, 0xd1, 0x58, 0xd4, 0x7b, 0x64, 0x3b, 0x87, 0xa6, 0x55, 0xee, - 0x46, 0x79, 0x49, 0xa8, 0x63, 0x94, 0xba, 0x65, 0xd9, 0x9e, 0xee, 0x99, 0xb6, 0xe5, 0x8a, 0xdd, - 0x57, 0xcb, 0xa6, 0xb7, 0x5f, 0x2f, 0x66, 0x4a, 0x76, 0x75, 0xb9, 0x6c, 0x97, 0xed, 0xc0, 0x2a, - 0xb6, 0xc2, 0x05, 0xfe, 0x25, 0xc8, 0x5b, 0x3e, 0xef, 0x53, 0xbd, 0xe2, 0xed, 0x73, 0x54, 0x6b, - 0xc6, 0x61, 0x7a, 0xd3, 0x2e, 0xee, 0x60, 0x1c, 0xf2, 0xf4, 0x83, 0x3a, 0x75, 0xbd, 0x0d, 0x8f, - 0x56, 0xc9, 0x0a, 0x8c, 0xd4, 0x1c, 0xd3, 0x76, 0x4c, 0xef, 0x58, 0x55, 0x16, 0x94, 0x45, 0x25, - 0x3b, 0xdb, 0x6c, 0xa4, 0x89, 0x8f, 0xbd, 0x62, 0x57, 0x4d, 0x0f, 0x43, 0x93, 0x6f, 0xd1, 0x91, - 0x37, 0x20, 0x6e, 0xe9, 0x55, 0xea, 0xd6, 0xf4, 0x12, 0x55, 0xa3, 0x0b, 0xca, 0x62, 0x3c, 0x7b, - 0xa1, 0xd9, 0x48, 0x4f, 0xb5, 0x40, 0x89, 0x2b, 0xa0, 0x24, 0xaf, 0x43, 0xbc, 0x54, 0x31, 0xa9, - 0xe5, 0x15, 0x4c, 0x43, 0x1d, 0x41, 0x36, 0xd4, 0xc5, 0xc1, 0x0d, 0x43, 0xd6, 0xe5, 0x63, 0x64, - 0x07, 0x62, 0x15, 0xbd, 0x48, 0x2b, 0xae, 0x3a, 0xb8, 0x10, 0x5d, 0x4c, 0xac, 0xbc, 0x9c, 0xd1, - 0x6b, 0x66, 0xa6, 0x9b, 0x2b, 0x99, 0x7b, 0x48, 0x97, 0xb3, 0x3c, 0xe7, 0x38, 0x3b, 0xdd, 0x6c, - 0xa4, 0x93, 0x9c, 0x51, 0x12, 0x2b, 0x44, 0x91, 0x32, 0x24, 0xa4, 0x38, 0xab, 0x43, 0x28, 0x79, - 0xa9, 0xb7, 0xe4, 0xd5, 0x80, 0x98, 0x8b, 0x9f, 0x6b, 0x36, 0xd2, 0x33, 0x92, 0x08, 0x49, 0x87, - 0x2c, 0x99, 0xfc, 0x5c, 0x81, 0x69, 0x87, 0x7e, 0x50, 0x37, 0x1d, 0x6a, 0x14, 0x2c, 0xdb, 0xa0, - 0x05, 0xe1, 0x4c, 0x0c, 0x55, 0x5e, 0xeb, 0xad, 0x32, 0x2f, 0xb8, 0xb6, 0x6c, 0x83, 0xca, 0x8e, - 0x69, 0xcd, 0x46, 0xfa, 0x92, 0xd3, 0xb1, 0x19, 0x18, 0xa0, 0x2a, 0x79, 0xd2, 0xb9, 0x4f, 0x1e, - 0xc0, 0x48, 0xcd, 0x36, 0x0a, 0x6e, 0x8d, 0x96, 0xd4, 0x81, 0x05, 0x65, 0x31, 0xb1, 0x72, 0x31, - 0xc3, 0x13, 0x14, 0x6d, 0x60, 0x49, 0x9c, 0x39, 0xba, 0x96, 0xd9, 0xb6, 0x8d, 0x9d, 0x1a, 0x2d, - 0xe1, 0xf7, 0x9c, 0xac, 0xf1, 0x45, 0x48, 0xf6, 0xb0, 0x00, 0xc9, 0x36, 0xc4, 0x7d, 0x81, 0xae, - 0x3a, 0x8c, 0xee, 0x9c, 0x28, 0x91, 0xa7, 0x15, 0x5f, 0xb8, 0xa1, 0xb4, 0x12, 0x18, 0x59, 0x83, - 0x61, 0xd3, 0x2a, 0x3b, 0xd4, 0x75, 0xd5, 0x38, 0xca, 0x23, 0x28, 0x68, 0x83, 0x63, 0x6b, 0xb6, - 0xb5, 0x67, 0x96, 0xb3, 0x33, 0xcc, 0x30, 0x41, 0x26, 0x49, 0xf1, 0x39, 0xc9, 0x6d, 0x18, 0x71, - 0xa9, 0x73, 0x64, 0x96, 0xa8, 0xab, 0x82, 0x24, 0x65, 0x87, 0x83, 0x42, 0x0a, 0x1a, 0xe3, 0xd3, - 0xc9, 0xc6, 0xf8, 0x18, 0xcb, 0x71, 0xb7, 0xb4, 0x4f, 0x8d, 0x7a, 0x85, 0x3a, 0x6a, 0x22, 0xc8, - 0xf1, 0x16, 0x28, 0xe7, 0x78, 0x0b, 0x24, 0x1b, 0x30, 0xf9, 0x41, 0x9d, 0xd6, 0x69, 0xc1, 0xf3, - 0x2a, 0x05, 0x97, 0x96, 0x6c, 0xcb, 0x70, 0xd5, 0xd1, 0x05, 0x65, 0x31, 0x9a, 0x7d, 0xa1, 0xd9, - 0x48, 0xcf, 0xe1, 0xe6, 0x43, 0xaf, 0xb2, 0xc3, 0xb7, 0x24, 0x21, 0x13, 0x6d, 0x5b, 0x29, 0x1d, - 0x12, 0xd2, 0x87, 0x27, 0x2f, 0x42, 0xf4, 0x90, 0xf2, 0x33, 0x1a, 0xcf, 0x4e, 0x36, 0x1b, 0xe9, - 0xb1, 0x43, 0x2a, 0x1f, 0x4f, 0xb6, 0x4b, 0xae, 0xc2, 0xd0, 0x91, 0x5e, 0xa9, 0x53, 0xfc, 0xc4, - 0xf1, 0xec, 0x54, 0xb3, 0x91, 0x9e, 0x40, 0x40, 0x22, 0xe4, 0x14, 0x37, 0x07, 0xae, 0x2b, 0xa9, - 0x3d, 0x48, 0xb6, 0xa7, 0xf6, 0xb9, 0xe8, 0xa9, 0xc2, 0x85, 0x1e, 0xf9, 0x7c, 0x1e, 0xea, 0xb4, - 0xff, 0x44, 0x61, 0x2c, 0x94, 0x35, 0xe4, 0x26, 0x0c, 0x7a, 0xc7, 0x35, 0x8a, 0x6a, 0xc6, 0x57, - 0x92, 0x72, 0x5e, 0x3d, 0x3c, 0xae, 0x51, 0x2c, 0x17, 0xe3, 0x8c, 0x22, 0x94, 0xeb, 0xc8, 0xc3, - 0x94, 0xd7, 0x6c, 0xc7, 0x73, 0xd5, 0x81, 0x85, 0xe8, 0xe2, 0x18, 0x57, 0x8e, 0x80, 0xac, 0x1c, - 0x01, 0xf2, 0x7e, 0xb8, 0xae, 0x44, 0x31, 0xff, 0x5e, 0xec, 0xcc, 0xe2, 0x27, 0x2f, 0x28, 0x37, - 0x20, 0xe1, 0x55, 0xdc, 0x02, 0xb5, 0xf4, 0x62, 0x85, 0x1a, 0xea, 0xe0, 0x82, 0xb2, 0x38, 0x92, - 0x55, 0x9b, 0x8d, 0xf4, 0xb4, 0xc7, 0x22, 0x8a, 0xa8, 0xc4, 0x0b, 0x01, 0x8a, 0xe5, 0x97, 0x3a, - 0x5e, 0x81, 0x15, 0x64, 0x75, 0x48, 0x2a, 0xbf, 0xd4, 0xf1, 0xb6, 0xf4, 0x2a, 0x0d, 0x95, 0x5f, - 0x81, 0x91, 0x5b, 0x30, 0x56, 0x77, 0x69, 0xa1, 0x54, 0xa9, 0xbb, 0x1e, 0x75, 0x36, 0xb6, 0xd5, - 0x18, 0x6a, 0x4c, 0x35, 0x1b, 0xe9, 0xd9, 0xba, 0x4b, 0xd7, 0x7c, 0x5c, 0x62, 0x1e, 0x95, 0xf1, - 0xe7, 0x95, 0x62, 0x9a, 0x07, 0x63, 0xa1, 0x23, 0x4e, 0xae, 0x77, 0xf9, 0xe4, 0x82, 0x02, 0x3f, - 0x39, 0xe9, 0xfc, 0xe4, 0x67, 0xfe, 0xe0, 0xda, 0x5f, 0x15, 0x48, 0xb6, 0x97, 0x6f, 0xc6, 0x8f, - 0x67, 0x59, 0x38, 0x88, 0xfc, 0x08, 0xc8, 0xfc, 0x08, 0x90, 0xaf, 0x03, 0x1c, 0xd8, 0xc5, 0x82, - 0x4b, 0xf1, 0x4e, 0x1c, 0x08, 0x3e, 0xca, 0x81, 0x5d, 0xdc, 0xa1, 0x6d, 0x77, 0xa2, 0x8f, 0x11, - 0x03, 0x26, 0x19, 0x97, 0xc3, 0xf5, 0x15, 0x18, 0x81, 0x9f, 0x6c, 0x73, 0x3d, 0x6f, 0x14, 0x5e, - 0x7f, 0x0e, 0xec, 0xa2, 0x84, 0x85, 0xea, 0x4f, 0xdb, 0x96, 0xf6, 0x5b, 0x05, 0x26, 0x37, 0xed, - 0xe2, 0xb6, 0x43, 0x19, 0xc1, 0x73, 0x73, 0xee, 0x55, 0x18, 0x66, 0x5c, 0xa6, 0xc1, 0x5d, 0x8a, - 0xf3, 0xab, 0xfc, 0xc0, 0x2e, 0x6e, 0x84, 0xca, 0x65, 0x8c, 0x23, 0xda, 0x7f, 0xf9, 0x17, 0x58, - 0xd3, 0xad, 0x12, 0xad, 0xf8, 0x46, 0x2e, 0x41, 0x8c, 0xcb, 0x90, 0xad, 0x44, 0x06, 0xd9, 0x4a, - 0x04, 0x9e, 0xd0, 0xca, 0x56, 0x18, 0xa2, 0x7d, 0xc3, 0x20, 0x39, 0x34, 0xd8, 0xdf, 0x21, 0xf2, - 0x0a, 0xc4, 0x1c, 0xaa, 0xbb, 0xb6, 0x25, 0xce, 0x28, 0x52, 0x73, 0x44, 0xa6, 0xe6, 0x88, 0xf6, - 0x4f, 0x05, 0xa6, 0x36, 0xd1, 0xa8, 0x70, 0x04, 0xc2, 0x5e, 0x29, 0x67, 0xf5, 0x6a, 0xa0, 0xaf, - 0x57, 0xb7, 0x20, 0xb6, 0x67, 0x56, 0x3c, 0xea, 0x60, 0x04, 0x12, 0x2b, 0x93, 0xad, 0xc4, 0xa3, - 0xde, 0x6d, 0xdc, 0xe0, 0x96, 0x73, 0x22, 0xd9, 0x72, 0x8e, 0x48, 0x7e, 0x0e, 0x9e, 0xc2, 0xcf, - 0xb7, 0x60, 0x54, 0x96, 0x4d, 0xbe, 0x09, 0x31, 0xd7, 0xd3, 0x3d, 0xea, 0xaa, 0xca, 0x42, 0x74, - 0x71, 0x7c, 0x65, 0xac, 0xa5, 0x9e, 0xa1, 0x5c, 0x18, 0x27, 0x90, 0x85, 0x71, 0x44, 0xfb, 0x62, - 0x02, 0xa2, 0x9b, 0x76, 0x91, 0x2c, 0xc0, 0x40, 0x2b, 0x38, 0xc9, 0x66, 0x23, 0x3d, 0x6a, 0xca, - 0x61, 0x19, 0x30, 0x8d, 0x70, 0xcb, 0x3a, 0x76, 0xca, 0x96, 0xf5, 0xdc, 0x33, 0x2a, 0xd4, 0x7f, - 0x0f, 0x9f, 0xba, 0xff, 0xce, 0xb6, 0x5a, 0x69, 0xde, 0x5e, 0x4d, 0xfb, 0x31, 0x3b, 0x43, 0xe7, - 0xfc, 0x4e, 0xf8, 0x86, 0x83, 0x70, 0xd1, 0x79, 0xf2, 0x7b, 0xed, 0xa8, 0x47, 0x9f, 0x9c, 0x40, - 0x05, 0x0b, 0x2d, 0x05, 0xcf, 0xba, 0x2d, 0xbe, 0x0a, 0x43, 0xf6, 0x23, 0x8b, 0x3a, 0x62, 0x1e, - 0xc1, 0xa8, 0x23, 0x20, 0x47, 0x1d, 0x01, 0x42, 0xe1, 0x22, 0x6f, 0xed, 0x70, 0xe9, 0xee, 0x9b, - 0xb5, 0x42, 0xdd, 0xa5, 0x4e, 0xa1, 0xec, 0xd8, 0xf5, 0x9a, 0xab, 0x4e, 0xe0, 0xd9, 0xbe, 0xd2, - 0x6c, 0xa4, 0x35, 0x24, 0x7b, 0xe0, 0x53, 0xed, 0xba, 0xd4, 0xb9, 0x83, 0x34, 0x92, 0x4c, 0xb5, - 0x17, 0x0d, 0xf9, 0xa9, 0x02, 0x57, 0x4a, 0x76, 0xb5, 0xc6, 0xba, 0x05, 0x6a, 0x14, 0x4e, 0x52, - 0x39, 0xb5, 0xa0, 0x2c, 0x8e, 0x66, 0x5f, 0x6b, 0x36, 0xd2, 0xaf, 0x04, 0x1c, 0x6f, 0xf7, 0x57, - 0xae, 0xf5, 0xa7, 0x0e, 0xcd, 0x85, 0x83, 0xa7, 0x9c, 0x0b, 0xe5, 0x19, 0x63, 0xe8, 0x99, 0xcf, - 0x18, 0xa3, 0xcf, 0x62, 0xc6, 0xf8, 0xb5, 0x02, 0x0b, 0xa2, 0x5b, 0x37, 0xad, 0x72, 0xc1, 0xa1, - 0xae, 0x5d, 0x77, 0x4a, 0xb4, 0x20, 0x52, 0xa3, 0x4a, 0x2d, 0xcf, 0x55, 0x67, 0xd0, 0xf6, 0xc5, - 0x6e, 0x9a, 0xf2, 0x82, 0x21, 0x2f, 0xd1, 0x67, 0xaf, 0x7c, 0xda, 0x48, 0x47, 0x9a, 0x8d, 0xf4, - 0x7c, 0x20, 0xb9, 0x1b, 0x5d, 0xbe, 0xcf, 0x3e, 0xd9, 0x80, 0xe1, 0x92, 0x43, 0x75, 0x8f, 0x1a, - 0xd8, 0x66, 0x25, 0x56, 0x52, 0x19, 0xfe, 0x32, 0x90, 0xf1, 0x47, 0xfe, 0xcc, 0x43, 0xff, 0x21, - 0x22, 0x3b, 0x25, 0x94, 0xfa, 0x2c, 0x1f, 0x7d, 0x91, 0x56, 0xf2, 0xfe, 0x42, 0x9e, 0xa5, 0xc6, - 0x9f, 0xc9, 0x2c, 0x95, 0x7c, 0x8a, 0x59, 0xea, 0x7b, 0x90, 0x38, 0xbc, 0xee, 0x16, 0x7c, 0x83, - 0x26, 0x51, 0xd4, 0x65, 0x39, 0xbc, 0xc1, 0xfb, 0x08, 0x0b, 0xb2, 0xb0, 0x92, 0xf7, 0xb5, 0x87, - 0xd7, 0xdd, 0x8d, 0x0e, 0x13, 0x21, 0x40, 0x59, 0x49, 0x62, 0xd2, 0x85, 0x36, 0x95, 0xf4, 0x4e, - 0x13, 0x61, 0x77, 0x4b, 0xae, 0x58, 0xb7, 0xc9, 0x15, 0x68, 0x78, 0x02, 0x9c, 0x7e, 0xba, 0x09, - 0x70, 0xf6, 0xff, 0x13, 0xe0, 0x73, 0x9d, 0x00, 0xff, 0xa5, 0xc0, 0xec, 0x26, 0xeb, 0x67, 0x45, - 0x6d, 0x32, 0x7f, 0x48, 0xfd, 0xce, 0x48, 0x6a, 0xc7, 0x94, 0x53, 0xb4, 0x63, 0xe7, 0x7e, 0x99, - 0xbf, 0x09, 0xa3, 0x16, 0x7d, 0x54, 0x68, 0x2b, 0xb6, 0x78, 0x6f, 0x5a, 0xf4, 0xd1, 0x76, 0x67, - 0xbd, 0x4d, 0x48, 0xb0, 0xf6, 0xbb, 0x01, 0xb8, 0xd0, 0xe1, 0xa8, 0x5b, 0xb3, 0x2d, 0x97, 0x92, - 0xdf, 0x28, 0xa0, 0x3a, 0xc1, 0x06, 0x7e, 0x62, 0x56, 0xf1, 0xea, 0x15, 0x8f, 0xfb, 0x9e, 0x58, - 0xb9, 0xe1, 0x5f, 0xac, 0xdd, 0x04, 0x64, 0xf2, 0x6d, 0xcc, 0x79, 0xce, 0xcb, 0x6f, 0xdc, 0x97, - 0x9b, 0x8d, 0xf4, 0x65, 0xa7, 0x3b, 0x85, 0x64, 0xed, 0x85, 0x1e, 0x24, 0x29, 0x07, 0x2e, 0x9d, - 0x24, 0xff, 0x5c, 0xd2, 0xc2, 0x82, 0x19, 0x69, 0x34, 0xe2, 0x5e, 0xe2, 0x2b, 0xe8, 0x59, 0x06, - 0x86, 0xab, 0x30, 0x44, 0x1d, 0xc7, 0x76, 0x64, 0x9d, 0x08, 0xc8, 0xa4, 0x08, 0x68, 0x1f, 0xe2, - 0x04, 0x15, 0xd6, 0x47, 0xf6, 0x81, 0xf0, 0xe9, 0x8d, 0xaf, 0xc5, 0xf8, 0xc6, 0xbf, 0x47, 0xaa, - 0x7d, 0x7c, 0x0b, 0x6c, 0xcc, 0xce, 0x37, 0x1b, 0xe9, 0x14, 0x0e, 0x69, 0x01, 0x28, 0x47, 0x3a, - 0xd9, 0xbe, 0xa7, 0xfd, 0x65, 0x04, 0x86, 0xf0, 0x82, 0x27, 0x57, 0x60, 0x10, 0xc7, 0x7e, 0xee, - 0x1d, 0x8e, 0xbe, 0x56, 0x78, 0xe4, 0xc7, 0x7d, 0x92, 0x83, 0x09, 0x3f, 0x11, 0x0b, 0x7b, 0x7a, - 0xc9, 0x13, 0x5e, 0x2a, 0xd9, 0x4b, 0xcd, 0x46, 0x5a, 0xf5, 0xb7, 0x6e, 0xe3, 0x8e, 0xc4, 0x3c, - 0x1e, 0xde, 0x21, 0x37, 0x20, 0x81, 0x7d, 0x0a, 0x6f, 0x5b, 0xc4, 0x1c, 0x87, 0x55, 0x97, 0xc1, - 0xbc, 0xdd, 0x90, 0xab, 0x6e, 0x80, 0xb2, 0xe3, 0x80, 0xdd, 0x8d, 0xcf, 0xcb, 0x47, 0x26, 0x3c, - 0x0e, 0x88, 0x77, 0x30, 0x27, 0x24, 0x98, 0x94, 0x61, 0xa2, 0x75, 0xa5, 0x57, 0xcc, 0xaa, 0xe9, - 0xf9, 0x8f, 0xbb, 0xf3, 0x18, 0x58, 0x0c, 0x46, 0xeb, 0x0e, 0xbf, 0x87, 0x04, 0x3c, 0x9b, 0x59, - 0x70, 0x55, 0x27, 0xb4, 0x11, 0x6a, 0x49, 0xc6, 0xc3, 0x7b, 0xe4, 0xf7, 0x0a, 0x5c, 0x69, 0xd3, - 0x54, 0x28, 0x1e, 0xb7, 0x4e, 0x71, 0xa1, 0x54, 0xd1, 0x5d, 0x97, 0x3f, 0xb5, 0x0c, 0x4b, 0x4f, - 0xbd, 0xdd, 0x0c, 0xc8, 0x1e, 0xfb, 0xa7, 0x79, 0x8d, 0x31, 0x6d, 0xe9, 0x55, 0xca, 0x6d, 0xba, - 0x2a, 0x6e, 0xf8, 0xcb, 0x4e, 0x3f, 0xfa, 0x7c, 0x7f, 0x12, 0xb2, 0x03, 0x89, 0x1a, 0x75, 0xaa, - 0xa6, 0xeb, 0x62, 0xe7, 0xce, 0x1f, 0xa0, 0x67, 0x25, 0xab, 0xb6, 0x83, 0x5d, 0x1e, 0x6f, 0x89, - 0x5c, 0x8e, 0xb7, 0x04, 0xa7, 0xfe, 0xad, 0x40, 0x42, 0xe2, 0x23, 0x79, 0x18, 0x71, 0xeb, 0xc5, - 0x03, 0x5a, 0x6a, 0x55, 0x98, 0xf9, 0xee, 0x1a, 0x32, 0x3b, 0x9c, 0x4c, 0x74, 0x0f, 0x82, 0x27, - 0xd4, 0x3d, 0x08, 0x0c, 0xcf, 0x38, 0x75, 0x8a, 0xfc, 0x39, 0xc6, 0x3f, 0xe3, 0x0c, 0x08, 0x9d, - 0x71, 0x06, 0xa4, 0xde, 0x83, 0x61, 0x21, 0x97, 0x65, 0xfc, 0xa1, 0x69, 0x19, 0x72, 0xc6, 0xb3, - 0xb5, 0x9c, 0xf1, 0x6c, 0xdd, 0x3a, 0x19, 0x03, 0x27, 0x9f, 0x8c, 0x94, 0x09, 0x53, 0x5d, 0xf2, - 0xe6, 0x09, 0xaa, 0x94, 0xd2, 0xf7, 0xae, 0xfc, 0x58, 0x81, 0x2b, 0xa7, 0x4b, 0x91, 0xd3, 0xa9, - 0x7f, 0x4b, 0x56, 0xef, 0x0f, 0x53, 0x21, 0x81, 0x6d, 0xda, 0xfa, 0x95, 0xd1, 0x9f, 0x0d, 0xc1, - 0xc5, 0x13, 0xf8, 0x59, 0x93, 0x3d, 0x57, 0xd5, 0x7f, 0x60, 0x56, 0xeb, 0xd5, 0xa0, 0xc3, 0xde, - 0x73, 0xf4, 0x12, 0x2b, 0xf2, 0x22, 0x2f, 0xbe, 0xd5, 0xcf, 0x8a, 0xcc, 0x7d, 0x2e, 0xc1, 0x47, - 0x6f, 0x0b, 0x7e, 0x7e, 0x36, 0xd2, 0xe2, 0x6c, 0x5c, 0xa8, 0x76, 0xa7, 0xca, 0xf7, 0xda, 0x20, - 0x7f, 0x50, 0xe0, 0x72, 0x4f, 0xe3, 0xf0, 0x0c, 0xdb, 0x76, 0x05, 0x73, 0x2d, 0xb1, 0xb2, 0xf6, - 0xa4, 0x46, 0x66, 0x8f, 0xb7, 0x6d, 0xbb, 0x22, 0x2e, 0x4a, 0x61, 0xea, 0x0b, 0xd5, 0x93, 0x68, - 0xf3, 0x27, 0x6f, 0xb3, 0xeb, 0xf2, 0xa4, 0x80, 0x9c, 0x57, 0x22, 0x6a, 0xfd, 0x1d, 0x3c, 0x9d, - 0xea, 0x07, 0xe1, 0x24, 0x7c, 0xa9, 0x33, 0xb2, 0x18, 0x85, 0xb3, 0x25, 0xe2, 0x1f, 0x07, 0x20, - 0xdd, 0x47, 0x06, 0xf9, 0xf8, 0x14, 0xc9, 0xb8, 0x7a, 0x1a, 0x6b, 0xce, 0x29, 0x21, 0xbf, 0x8a, - 0x2f, 0xab, 0xe5, 0x20, 0x8e, 0x25, 0xf9, 0x9e, 0xe9, 0x7a, 0xe4, 0x3a, 0xc4, 0xb0, 0x15, 0xf5, - 0x4b, 0x36, 0x04, 0x25, 0x9b, 0x37, 0xc7, 0x7c, 0x57, 0x6e, 0x8e, 0x39, 0xa2, 0xed, 0x02, 0xe1, - 0xcf, 0x8e, 0x15, 0xa9, 0x7f, 0x23, 0xb7, 0x60, 0xac, 0xc4, 0x51, 0x6a, 0x48, 0x7d, 0x36, 0xfe, - 0x66, 0xd0, 0xda, 0x08, 0x77, 0xdb, 0xa3, 0x32, 0xae, 0xdd, 0x80, 0x09, 0xd4, 0x7e, 0x87, 0xb6, - 0x9e, 0x9d, 0x4f, 0xd9, 0xc0, 0x68, 0xb7, 0x40, 0xdd, 0xf1, 0x1c, 0xaa, 0x57, 0x4d, 0xab, 0xdc, - 0x2e, 0xe3, 0x45, 0x88, 0x5a, 0xf5, 0x2a, 0x8a, 0x18, 0xe3, 0x81, 0xb4, 0xea, 0x55, 0x39, 0x90, - 0x56, 0xbd, 0xaa, 0xbd, 0x09, 0x04, 0xf9, 0xd6, 0x69, 0x85, 0x7a, 0xf4, 0xac, 0xea, 0x3f, 0x51, - 0x00, 0xf8, 0x3b, 0xe5, 0x86, 0xb5, 0x67, 0x9f, 0xba, 0xed, 0xba, 0x01, 0x09, 0x8c, 0xa8, 0x51, - 0x38, 0xb0, 0xf1, 0xa2, 0x53, 0x16, 0x87, 0x78, 0xbf, 0xc4, 0xe1, 0x4d, 0x3b, 0x74, 0xdb, 0x41, - 0x80, 0x32, 0xd6, 0x0a, 0xd5, 0x5d, 0x9f, 0x35, 0x1a, 0xb0, 0x72, 0xb8, 0x9d, 0x35, 0x40, 0xb5, - 0x47, 0x30, 0x85, 0xae, 0xee, 0xd6, 0x0c, 0xdd, 0x0b, 0xc6, 0x86, 0x37, 0xe4, 0x17, 0xfe, 0x70, - 0x36, 0x9c, 0x34, 0xc7, 0x9c, 0xa1, 0x2d, 0xae, 0x83, 0x9a, 0xd5, 0xbd, 0xd2, 0x7e, 0x37, 0xed, - 0xef, 0xc1, 0xd8, 0x9e, 0x6e, 0x56, 0xfc, 0x97, 0x2f, 0x3f, 0x27, 0xd5, 0xc0, 0x8a, 0x30, 0x03, - 0x4f, 0x2b, 0xce, 0xf2, 0x76, 0x7b, 0x9e, 0x8e, 0xca, 0x78, 0xcb, 0xdf, 0x35, 0x7c, 0x23, 0xf9, - 0xaa, 0xfc, 0x6d, 0xd3, 0xde, 0xdf, 0xdf, 0x30, 0xc3, 0x19, 0xfc, 0x4d, 0x40, 0x3c, 0x67, 0x19, - 0xf7, 0x75, 0xe7, 0x90, 0x3a, 0xda, 0x47, 0x0a, 0xcc, 0x84, 0x4f, 0xc6, 0x7d, 0xea, 0xba, 0x7a, - 0x99, 0x92, 0x6f, 0x9c, 0xcd, 0xff, 0xbb, 0x91, 0xe0, 0x19, 0x3a, 0x4a, 0x2d, 0x43, 0x14, 0xf4, - 0x71, 0x64, 0x6b, 0xe9, 0xe3, 0xe7, 0x8b, 0xca, 0x0d, 0xd7, 0xdd, 0x48, 0x9e, 0xd1, 0x67, 0x87, - 0x61, 0x88, 0x1e, 0x51, 0xcb, 0x5b, 0x4a, 0x41, 0x42, 0xfa, 0x0d, 0x96, 0x24, 0x60, 0x58, 0x2c, - 0x93, 0x91, 0xa5, 0xab, 0x90, 0x90, 0x7e, 0xac, 0x23, 0xa3, 0x30, 0xb2, 0x65, 0x1b, 0x74, 0xdb, - 0x76, 0xbc, 0x64, 0x84, 0xad, 0xee, 0x52, 0xdd, 0xa8, 0x30, 0x52, 0x65, 0xe9, 0x97, 0x0a, 0x8c, - 0xf8, 0x0f, 0xff, 0x04, 0x20, 0xf6, 0xf6, 0x6e, 0x6e, 0x37, 0xb7, 0x9e, 0x8c, 0x30, 0x81, 0xdb, - 0xb9, 0xad, 0xf5, 0x8d, 0xad, 0x3b, 0x49, 0x85, 0x2d, 0xf2, 0xbb, 0x5b, 0x5b, 0x6c, 0x31, 0x40, - 0xc6, 0x20, 0xbe, 0xb3, 0xbb, 0xb6, 0x96, 0xcb, 0xad, 0xe7, 0xd6, 0x93, 0x51, 0xc6, 0x74, 0x7b, - 0x75, 0xe3, 0x5e, 0x6e, 0x3d, 0x39, 0xc8, 0xe8, 0x76, 0xb7, 0xde, 0xda, 0x7a, 0xf0, 0xee, 0x56, - 0x72, 0x88, 0xd3, 0x65, 0xef, 0x6f, 0x3c, 0x7c, 0x98, 0x5b, 0x4f, 0xc6, 0x18, 0xdd, 0xbd, 0xdc, - 0xea, 0x4e, 0x6e, 0x3d, 0x39, 0xcc, 0xb6, 0xb6, 0xf3, 0xb9, 0xdc, 0xfd, 0x6d, 0xb6, 0x35, 0xc2, - 0x96, 0x6b, 0xab, 0x5b, 0x6b, 0xb9, 0x7b, 0x4c, 0x4a, 0x7c, 0xe5, 0x4f, 0x51, 0x18, 0xc5, 0x10, - 0xfa, 0x8f, 0x49, 0xaf, 0x43, 0x82, 0x7f, 0x47, 0x3e, 0x8f, 0x49, 0x41, 0x4e, 0xcd, 0x76, 0x3c, - 0xf2, 0xe5, 0x58, 0xb8, 0xb4, 0x08, 0xb9, 0x05, 0xa3, 0x12, 0x93, 0x4b, 0xc6, 0x03, 0x2e, 0x56, - 0xb8, 0x53, 0x2f, 0xe0, 0xba, 0x57, 0x6a, 0x69, 0x11, 0xa6, 0x95, 0x9f, 0x96, 0x33, 0x6a, 0x95, - 0x98, 0xfa, 0x6b, 0x0d, 0x9f, 0x47, 0x2d, 0x42, 0xbe, 0x0d, 0x09, 0x5e, 0x3d, 0xb9, 0xd6, 0x0b, - 0x01, 0x7f, 0xa8, 0xa8, 0x9e, 0x60, 0x42, 0x06, 0x46, 0xee, 0x50, 0x8f, 0xb3, 0x4f, 0x07, 0xec, - 0x41, 0x2d, 0x4f, 0x49, 0xae, 0x68, 0x11, 0xb2, 0x09, 0x71, 0x9f, 0xde, 0x25, 0xdc, 0xbe, 0x5e, - 0xb7, 0x40, 0x2a, 0xd5, 0x65, 0x5b, 0x1c, 0x05, 0x2d, 0xf2, 0x9a, 0xb2, 0xf2, 0x8b, 0x38, 0xc4, - 0xf8, 0xf4, 0x4d, 0xde, 0x01, 0xe0, 0x7f, 0x61, 0xa5, 0x9d, 0xe9, 0xfa, 0xd3, 0x6a, 0x6a, 0xb6, - 0xfb, 0xc8, 0xae, 0xcd, 0xfd, 0xe4, 0xcf, 0xff, 0xf8, 0xd5, 0xc0, 0x94, 0x36, 0xbe, 0x7c, 0x74, - 0x6d, 0xf9, 0xc0, 0x2e, 0x8a, 0xff, 0x41, 0xbb, 0xa9, 0x2c, 0x91, 0x77, 0x01, 0xf8, 0xb5, 0x19, - 0x96, 0x1b, 0xfa, 0x05, 0x2f, 0xc5, 0xc3, 0xd6, 0x79, 0xbd, 0x76, 0x0a, 0xe6, 0x77, 0x27, 0x13, - 0xfc, 0x7d, 0x18, 0x6d, 0x09, 0xde, 0xa1, 0x1e, 0x51, 0xa5, 0x1f, 0xe5, 0xc2, 0xd2, 0x7b, 0xc5, - 0xfe, 0x12, 0x0a, 0x9f, 0xd5, 0x26, 0x85, 0x70, 0x97, 0x7a, 0x92, 0x7c, 0x0b, 0x92, 0xf2, 0x43, - 0x11, 0x9a, 0x7f, 0xb1, 0xfb, 0x13, 0x12, 0x57, 0x73, 0xe9, 0xa4, 0xf7, 0x25, 0x2d, 0x8d, 0xca, - 0xe6, 0xb4, 0x69, 0xdf, 0x13, 0xe9, 0xad, 0x88, 0x32, 0x7d, 0xef, 0x41, 0x42, 0xfc, 0xfc, 0x8c, - 0xaa, 0x5a, 0xa1, 0x0e, 0xff, 0x26, 0xdd, 0xd3, 0x99, 0x14, 0xca, 0x9f, 0xd6, 0x26, 0x7c, 0xf9, - 0x35, 0xce, 0xc7, 0x44, 0xdf, 0x39, 0xfb, 0x81, 0x9c, 0x46, 0x71, 0xe3, 0x5a, 0x9c, 0x89, 0xc3, - 0x12, 0xc8, 0x04, 0x95, 0x9e, 0xee, 0x90, 0xbe, 0x84, 0x42, 0xe7, 0xb5, 0x39, 0x26, 0xb4, 0xc8, - 0xa8, 0xa8, 0xb1, 0xcc, 0x5f, 0xf1, 0xc5, 0x8d, 0xc0, 0x94, 0x6c, 0x9d, 0xfd, 0x20, 0x5f, 0x44, - 0xc1, 0x33, 0xa9, 0x64, 0xcb, 0xda, 0xe5, 0x1f, 0xb1, 0x66, 0xe3, 0x43, 0x61, 0xf4, 0xd3, 0x9c, - 0x71, 0x61, 0x74, 0x2a, 0x64, 0x74, 0x1d, 0x69, 0x24, 0xa3, 0xbf, 0xf3, 0x94, 0x75, 0x40, 0x45, - 0x2d, 0x64, 0xa9, 0xc3, 0x03, 0x72, 0xfb, 0x4c, 0xf5, 0x41, 0xc8, 0x21, 0x9d, 0x72, 0x8c, 0x67, - 0x54, 0x37, 0x44, 0xa2, 0x11, 0x22, 0xc7, 0x83, 0x07, 0xe2, 0x35, 0x85, 0xdc, 0x84, 0xd8, 0x5d, - 0xfc, 0x5f, 0x4c, 0xd2, 0xc3, 0xd3, 0x14, 0x3f, 0xa7, 0x9c, 0x68, 0x6d, 0x9f, 0x96, 0x0e, 0x5b, - 0xb7, 0xfd, 0xfb, 0x9f, 0xff, 0x7d, 0x3e, 0xf2, 0xe3, 0xc7, 0xf3, 0xca, 0xa7, 0x8f, 0xe7, 0x95, - 0xcf, 0x1e, 0xcf, 0x2b, 0x7f, 0x7b, 0x3c, 0xaf, 0x7c, 0xf4, 0xe5, 0x7c, 0xe4, 0xb3, 0x2f, 0xe7, - 0x23, 0x9f, 0x7f, 0x39, 0x1f, 0xf9, 0xee, 0xd7, 0xa4, 0x7f, 0x0f, 0xd5, 0x9d, 0xaa, 0x6e, 0xe8, - 0x35, 0xc7, 0x3e, 0xa0, 0x25, 0x4f, 0xac, 0x96, 0xc5, 0xff, 0x83, 0x7e, 0x32, 0x30, 0xbd, 0x8a, - 0xc0, 0x36, 0xdf, 0xce, 0x6c, 0xd8, 0x99, 0xd5, 0x9a, 0x59, 0x8c, 0xa1, 0x2d, 0xaf, 0xff, 0x2f, - 0x00, 0x00, 0xff, 0xff, 0x45, 0xd0, 0xfd, 0xb7, 0x2c, 0x2b, 0x00, 0x00, + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0xcb, 0x6f, 0x1b, 0xd7, + 0xd5, 0xe7, 0x88, 0x7a, 0xf1, 0x50, 0x0f, 0xea, 0xea, 0xe1, 0x11, 0xed, 0x88, 0xf2, 0x24, 0xf1, + 0x27, 0x0b, 0x09, 0x15, 0x2b, 0x5f, 0x50, 0xdb, 0x4d, 0xe1, 0x8a, 0x12, 0x6d, 0x4b, 0xb1, 0x65, + 0x85, 0xb2, 0x92, 0xa6, 0x28, 0xca, 0x0c, 0x39, 0x57, 0xd4, 0x48, 0xe4, 0x0c, 0x33, 0x33, 0x94, + 0xab, 0x16, 0x01, 0x8a, 0xa2, 0x68, 0x17, 0xdd, 0xa4, 0xe8, 0xae, 0x05, 0xd2, 0x45, 0x77, 0xe9, + 0xa2, 0x9b, 0xa2, 0xe8, 0x9f, 0x90, 0x65, 0x80, 0xa2, 0x40, 0xba, 0x61, 0x5a, 0xbb, 0x0f, 0x80, + 0xbb, 0xee, 0xbb, 0x28, 0xee, 0xb9, 0x77, 0x38, 0x77, 0xf8, 0x10, 0x45, 0xdb, 0x4a, 0x36, 0xdd, + 0xe9, 0xfe, 0xee, 0x79, 0xcf, 0xb9, 0xe7, 0x9e, 0x73, 0x29, 0x98, 0xa9, 0x1e, 0x95, 0x56, 0xf4, + 0xaa, 0xb9, 0xe2, 0xd6, 0x0a, 0x15, 0xd3, 0x4b, 0x57, 0x1d, 0xdb, 0xb3, 0x49, 0x54, 0xaf, 0x9a, + 0xc9, 0x8b, 0x25, 0xdb, 0x2e, 0x95, 0xe9, 0x0a, 0x42, 0x85, 0xda, 0xfe, 0x0a, 0xad, 0x54, 0xbd, + 0x13, 0x4e, 0x91, 0x4c, 0xb5, 0x6e, 0x7a, 0x66, 0x85, 0xba, 0x9e, 0x5e, 0xa9, 0x0a, 0x02, 0xed, + 0xe8, 0xba, 0x9b, 0x36, 0x6d, 0x94, 0x5d, 0xb4, 0x1d, 0xba, 0x72, 0x7c, 0x6d, 0xa5, 0x44, 0x2d, + 0xea, 0xe8, 0x1e, 0x35, 0x04, 0xcd, 0x92, 0x44, 0x63, 0x51, 0xef, 0x91, 0xed, 0x1c, 0x99, 0x56, + 0xa9, 0x13, 0xe5, 0x25, 0xa1, 0x8e, 0x51, 0xea, 0x96, 0x65, 0x7b, 0xba, 0x67, 0xda, 0x96, 0x2b, + 0x76, 0x5f, 0x2d, 0x99, 0xde, 0x41, 0xad, 0x90, 0x2e, 0xda, 0x95, 0x95, 0x92, 0x5d, 0xb2, 0x03, + 0xab, 0xd8, 0x0a, 0x17, 0xf8, 0x97, 0x20, 0x6f, 0xfa, 0x7c, 0x40, 0xf5, 0xb2, 0x77, 0xc0, 0x51, + 0xed, 0xd7, 0x31, 0x98, 0xd9, 0xb2, 0x0b, 0xbb, 0x18, 0x87, 0x1c, 0xfd, 0xa0, 0x46, 0x5d, 0x6f, + 0xd3, 0xa3, 0x15, 0xb2, 0x0a, 0xa3, 0x55, 0xc7, 0xb4, 0x1d, 0xd3, 0x3b, 0x51, 0x95, 0x45, 0x65, + 0x49, 0xc9, 0xcc, 0x35, 0xea, 0x29, 0xe2, 0x63, 0xaf, 0xd8, 0x15, 0xd3, 0xc3, 0xd0, 0xe4, 0x9a, + 0x74, 0xe4, 0x0d, 0x88, 0x59, 0x7a, 0x85, 0xba, 0x55, 0xbd, 0x48, 0xd5, 0xe8, 0xa2, 0xb2, 0x14, + 0xcb, 0x5c, 0x68, 0xd4, 0x53, 0xd3, 0x4d, 0x50, 0xe2, 0x0a, 0x28, 0xc9, 0xeb, 0x10, 0x2b, 0x96, + 0x4d, 0x6a, 0x79, 0x79, 0xd3, 0x50, 0x47, 0x91, 0x0d, 0x75, 0x71, 0x70, 0xd3, 0x90, 0x75, 0xf9, + 0x18, 0xd9, 0x85, 0xe1, 0xb2, 0x5e, 0xa0, 0x65, 0x57, 0x1d, 0x5c, 0x8c, 0x2e, 0xc5, 0x57, 0x5f, + 0x4e, 0xeb, 0x55, 0x33, 0xdd, 0xc9, 0x95, 0xf4, 0x3d, 0xa4, 0xcb, 0x5a, 0x9e, 0x73, 0x92, 0x99, + 0x69, 0xd4, 0x53, 0x09, 0xce, 0x28, 0x89, 0x15, 0xa2, 0x48, 0x09, 0xe2, 0x52, 0x9c, 0xd5, 0x21, + 0x94, 0xbc, 0xdc, 0x5d, 0xf2, 0x5a, 0x40, 0xcc, 0xc5, 0xcf, 0x37, 0xea, 0xa9, 0x59, 0x49, 0x84, + 0xa4, 0x43, 0x96, 0x4c, 0x7e, 0xaa, 0xc0, 0x8c, 0x43, 0x3f, 0xa8, 0x99, 0x0e, 0x35, 0xf2, 0x96, + 0x6d, 0xd0, 0xbc, 0x70, 0x66, 0x18, 0x55, 0x5e, 0xeb, 0xae, 0x32, 0x27, 0xb8, 0xb6, 0x6d, 0x83, + 0xca, 0x8e, 0x69, 0x8d, 0x7a, 0xea, 0x92, 0xd3, 0xb6, 0x19, 0x18, 0xa0, 0x2a, 0x39, 0xd2, 0xbe, + 0x4f, 0x1e, 0xc0, 0x68, 0xd5, 0x36, 0xf2, 0x6e, 0x95, 0x16, 0xd5, 0x81, 0x45, 0x65, 0x29, 0xbe, + 0x7a, 0x31, 0xcd, 0x13, 0x14, 0x6d, 0x60, 0x49, 0x9c, 0x3e, 0xbe, 0x96, 0xde, 0xb1, 0x8d, 0xdd, + 0x2a, 0x2d, 0xe2, 0xf7, 0x9c, 0xaa, 0xf2, 0x45, 0x48, 0xf6, 0x88, 0x00, 0xc9, 0x0e, 0xc4, 0x7c, + 0x81, 0xae, 0x3a, 0x82, 0xee, 0x9c, 0x2a, 0x91, 0xa7, 0x15, 0x5f, 0xb8, 0xa1, 0xb4, 0x12, 0x18, + 0x59, 0x87, 0x11, 0xd3, 0x2a, 0x39, 0xd4, 0x75, 0xd5, 0x18, 0xca, 0x23, 0x28, 0x68, 0x93, 0x63, + 0xeb, 0xb6, 0xb5, 0x6f, 0x96, 0x32, 0xb3, 0xcc, 0x30, 0x41, 0x26, 0x49, 0xf1, 0x39, 0xc9, 0x6d, + 0x18, 0x75, 0xa9, 0x73, 0x6c, 0x16, 0xa9, 0xab, 0x82, 0x24, 0x65, 0x97, 0x83, 0x42, 0x0a, 0x1a, + 0xe3, 0xd3, 0xc9, 0xc6, 0xf8, 0x18, 0xcb, 0x71, 0xb7, 0x78, 0x40, 0x8d, 0x5a, 0x99, 0x3a, 0x6a, + 0x3c, 0xc8, 0xf1, 0x26, 0x28, 0xe7, 0x78, 0x13, 0x4c, 0xea, 0x10, 0x97, 0xbe, 0x16, 0x79, 0x11, + 0xa2, 0x47, 0x94, 0x1f, 0xac, 0x58, 0x66, 0xaa, 0x51, 0x4f, 0x8d, 0x1f, 0x51, 0xf9, 0x4c, 0xb1, + 0x5d, 0x72, 0x15, 0x86, 0x8e, 0xf5, 0x72, 0x8d, 0xe2, 0x77, 0x89, 0x65, 0xa6, 0x1b, 0xf5, 0xd4, + 0x24, 0x02, 0x12, 0x21, 0xa7, 0xb8, 0x39, 0x70, 0x5d, 0x49, 0xee, 0x43, 0xa2, 0x35, 0x1f, 0xcf, + 0x45, 0x4f, 0x05, 0x2e, 0x74, 0x49, 0xc2, 0xf3, 0x50, 0xa7, 0xfd, 0x3b, 0x0a, 0xe3, 0xa1, 0x4f, + 0x4d, 0x6e, 0xc2, 0xa0, 0x77, 0x52, 0xa5, 0xa8, 0x66, 0x62, 0x35, 0x21, 0x27, 0xc3, 0xc3, 0x93, + 0x2a, 0xc5, 0x33, 0x3e, 0xc1, 0x28, 0x42, 0x09, 0x8a, 0x3c, 0x4c, 0x79, 0xd5, 0x76, 0x3c, 0x57, + 0x1d, 0x58, 0x8c, 0x2e, 0x8d, 0x73, 0xe5, 0x08, 0xc8, 0xca, 0x11, 0x20, 0xef, 0x87, 0x8b, 0x41, + 0x14, 0x93, 0xe6, 0xc5, 0xf6, 0xd4, 0x7b, 0xfa, 0x2a, 0x70, 0x03, 0xe2, 0x5e, 0xd9, 0xcd, 0x53, + 0x4b, 0x2f, 0x94, 0xa9, 0xa1, 0x0e, 0x2e, 0x2a, 0x4b, 0xa3, 0x19, 0xb5, 0x51, 0x4f, 0xcd, 0x78, + 0x2c, 0xa2, 0x88, 0x4a, 0xbc, 0x10, 0xa0, 0x58, 0x33, 0xa9, 0xe3, 0xe5, 0x59, 0x15, 0x55, 0x87, + 0xa4, 0x9a, 0x49, 0x1d, 0x6f, 0x5b, 0xaf, 0xd0, 0x50, 0xcd, 0x14, 0x18, 0xb9, 0x05, 0xe3, 0x35, + 0x97, 0xe6, 0x8b, 0xe5, 0x9a, 0xeb, 0x51, 0x67, 0x73, 0x47, 0x1d, 0x46, 0x8d, 0xc9, 0x46, 0x3d, + 0x35, 0x57, 0x73, 0xe9, 0xba, 0x8f, 0x4b, 0xcc, 0x63, 0x32, 0xfe, 0x65, 0xa5, 0x98, 0xe6, 0xc1, + 0x78, 0xe8, 0x5c, 0x92, 0xeb, 0x1d, 0x3e, 0xb9, 0xa0, 0xc0, 0x4f, 0x4e, 0xda, 0x3f, 0x79, 0xdf, + 0x1f, 0x5c, 0xfb, 0x8b, 0x02, 0x89, 0xd6, 0x9a, 0xcb, 0xf8, 0x3f, 0xa8, 0xd1, 0x1a, 0x15, 0x0e, + 0x22, 0x3f, 0x02, 0x32, 0x3f, 0x02, 0xe4, 0xff, 0x01, 0x0e, 0xed, 0x42, 0xde, 0xa5, 0x78, 0x91, + 0x0d, 0x04, 0x1f, 0xe5, 0xd0, 0x2e, 0xec, 0xd2, 0x96, 0x8b, 0xcc, 0xc7, 0x88, 0x01, 0x53, 0x8c, + 0xcb, 0xe1, 0xfa, 0xf2, 0x8c, 0xc0, 0x4f, 0xb6, 0xf9, 0xae, 0xd7, 0x40, 0xe6, 0x85, 0x46, 0x3d, + 0x35, 0x7f, 0x68, 0x17, 0x24, 0x4c, 0xf6, 0x68, 0xb2, 0x65, 0x4b, 0xfb, 0x8d, 0x02, 0x53, 0x5b, + 0x76, 0x61, 0xc7, 0xa1, 0x8c, 0xe0, 0x4b, 0x73, 0xee, 0x55, 0x18, 0x61, 0x5c, 0xa6, 0xc1, 0x5d, + 0x8a, 0xf1, 0xfb, 0xf7, 0xd0, 0x2e, 0x6c, 0x1a, 0xa1, 0xfb, 0x97, 0x23, 0xda, 0x7f, 0xf8, 0x17, + 0x58, 0xd7, 0xad, 0x22, 0x2d, 0xfb, 0x46, 0x2e, 0xc3, 0x30, 0x97, 0x21, 0x5b, 0x89, 0x0c, 0xb2, + 0x95, 0x08, 0x3c, 0xa5, 0x95, 0xcd, 0x30, 0x44, 0x7b, 0x86, 0x41, 0x72, 0x68, 0xb0, 0xb7, 0x43, + 0xe4, 0x15, 0x18, 0x76, 0xa8, 0xee, 0xda, 0x96, 0x38, 0xa3, 0x48, 0xcd, 0x11, 0x99, 0x9a, 0x23, + 0xda, 0x3f, 0x14, 0x98, 0xde, 0x42, 0xa3, 0xc2, 0x11, 0x08, 0x7b, 0xa5, 0xf4, 0xeb, 0xd5, 0x40, + 0x4f, 0xaf, 0x6e, 0xc1, 0xf0, 0xbe, 0x59, 0xf6, 0xa8, 0x83, 0x11, 0x88, 0xaf, 0x4e, 0x35, 0x13, + 0x8f, 0x7a, 0xb7, 0x71, 0x83, 0x5b, 0xce, 0x89, 0x64, 0xcb, 0x39, 0x22, 0xf9, 0x39, 0x78, 0x06, + 0x3f, 0xdf, 0x82, 0x31, 0x59, 0x36, 0xf9, 0x3a, 0x0c, 0xbb, 0x9e, 0xee, 0x51, 0x57, 0x55, 0x16, + 0xa3, 0x4b, 0x13, 0xab, 0xe3, 0x4d, 0xf5, 0x0c, 0xe5, 0xc2, 0x38, 0x81, 0x2c, 0x8c, 0x23, 0xda, + 0x17, 0x93, 0x10, 0xdd, 0xb2, 0x0b, 0x64, 0x11, 0x06, 0x9a, 0xc1, 0x49, 0x34, 0xea, 0xa9, 0x31, + 0x53, 0x0e, 0xcb, 0x80, 0x69, 0x84, 0xfb, 0xcc, 0xf1, 0x33, 0xf6, 0x99, 0xe7, 0x9e, 0x51, 0xa1, + 0xa6, 0x79, 0xe4, 0xcc, 0x4d, 0x73, 0xa6, 0xd9, 0xff, 0xf2, 0x9e, 0x68, 0xc6, 0x8f, 0x59, 0x1f, + 0xed, 0xee, 0x3b, 0xe1, 0x1b, 0x0e, 0xc2, 0x45, 0xe7, 0xe9, 0xef, 0xb5, 0xe3, 0x2e, 0xcd, 0x6d, + 0x1c, 0x15, 0x2c, 0x36, 0x15, 0x3c, 0xef, 0x5e, 0xf6, 0x2a, 0x0c, 0xd9, 0x8f, 0x2c, 0xea, 0x88, + 0x21, 0x02, 0xa3, 0x8e, 0x80, 0x1c, 0x75, 0x04, 0x08, 0x85, 0x8b, 0x18, 0xfe, 0x3c, 0x2e, 0xdd, + 0x03, 0xb3, 0x9a, 0xaf, 0xb9, 0xd4, 0xc9, 0x97, 0x1c, 0xbb, 0x56, 0x75, 0xd5, 0x49, 0x3c, 0xdb, + 0x57, 0x1a, 0xf5, 0x94, 0x86, 0x64, 0x0f, 0x7c, 0xaa, 0x3d, 0x97, 0x3a, 0x77, 0x90, 0x46, 0x92, + 0xa9, 0x76, 0xa3, 0x21, 0x3f, 0x56, 0xe0, 0x4a, 0xd1, 0xae, 0x54, 0x59, 0xb7, 0x40, 0x8d, 0xfc, + 0x69, 0x2a, 0xa7, 0x17, 0x95, 0xa5, 0xb1, 0xcc, 0x6b, 0x8d, 0x7a, 0xea, 0x95, 0x80, 0xe3, 0xed, + 0xde, 0xca, 0xb5, 0xde, 0xd4, 0xa1, 0x61, 0x6e, 0xf0, 0x8c, 0xc3, 0x9c, 0x3c, 0x18, 0x0c, 0x3d, + 0xf7, 0xc1, 0x60, 0xec, 0x79, 0x0c, 0x06, 0xbf, 0x54, 0x60, 0x51, 0xb4, 0xd8, 0xa6, 0x55, 0xca, + 0x3b, 0xd4, 0xb5, 0x6b, 0x4e, 0x91, 0xe6, 0x45, 0x6a, 0x54, 0xa8, 0xe5, 0xb9, 0xea, 0x2c, 0xda, + 0xbe, 0xd4, 0x49, 0x53, 0x4e, 0x30, 0xe4, 0x24, 0xfa, 0xcc, 0x95, 0x4f, 0xeb, 0xa9, 0x48, 0xa3, + 0x9e, 0x5a, 0x08, 0x24, 0x77, 0xa2, 0xcb, 0xf5, 0xd8, 0x27, 0x9b, 0x30, 0x52, 0x74, 0x28, 0x9b, + 0xe6, 0xb1, 0xcd, 0x8a, 0xaf, 0x26, 0xd3, 0x7c, 0x9c, 0x4f, 0xfb, 0x73, 0x7a, 0xfa, 0xa1, 0xff, + 0x7a, 0x90, 0x99, 0x16, 0x4a, 0x7d, 0x96, 0x8f, 0xbe, 0x48, 0x29, 0x39, 0x7f, 0x21, 0x0f, 0x40, + 0x13, 0xcf, 0x65, 0x00, 0x4a, 0x3c, 0xc3, 0x00, 0xf4, 0x1d, 0x88, 0x1f, 0x5d, 0x77, 0xf3, 0xbe, + 0x41, 0x53, 0x28, 0xea, 0xb2, 0x1c, 0xde, 0xe0, 0x51, 0x83, 0x05, 0x59, 0x58, 0xc9, 0xfb, 0xda, + 0xa3, 0xeb, 0xee, 0x66, 0x9b, 0x89, 0x10, 0xa0, 0xac, 0x24, 0x31, 0xe9, 0x42, 0x9b, 0x4a, 0xba, + 0xa7, 0x89, 0xb0, 0xbb, 0x29, 0x57, 0xac, 0x5b, 0xe4, 0x0a, 0x34, 0x3c, 0xb6, 0xcd, 0x9c, 0x75, + 0x6c, 0x23, 0x9b, 0x30, 0xc5, 0xcf, 0xac, 0xe7, 0x95, 0xf3, 0x2e, 0x2d, 0xda, 0x96, 0xe1, 0xaa, + 0x73, 0x8b, 0xca, 0x52, 0x94, 0x77, 0x60, 0xb8, 0xf9, 0xd0, 0x2b, 0xef, 0xf2, 0x2d, 0xb9, 0x03, + 0x6b, 0xd9, 0xfa, 0xdf, 0x04, 0xf8, 0xd4, 0xd3, 0xc0, 0x3f, 0x15, 0x98, 0xdb, 0x62, 0xfd, 0xac, + 0xa8, 0x4d, 0xe6, 0xf7, 0xa9, 0xdf, 0x19, 0x49, 0xed, 0x98, 0x72, 0x86, 0x76, 0xec, 0xdc, 0x2f, + 0xf3, 0x37, 0x61, 0xcc, 0xa2, 0x8f, 0xf2, 0x2d, 0xc5, 0x16, 0xef, 0x4d, 0x8b, 0x3e, 0xda, 0x69, + 0xaf, 0xb7, 0x71, 0x09, 0xd6, 0x7e, 0x3b, 0x00, 0x17, 0xda, 0x1c, 0x75, 0xab, 0xb6, 0xe5, 0x52, + 0xf2, 0x2b, 0x05, 0x54, 0x27, 0xd8, 0xc0, 0x4f, 0xcc, 0x2a, 0x5e, 0xad, 0xec, 0x71, 0xdf, 0xe3, + 0xab, 0x37, 0xfc, 0x8b, 0xb5, 0x93, 0x80, 0x74, 0xae, 0x85, 0x39, 0xc7, 0x79, 0xf9, 0x8d, 0xfb, + 0x72, 0xa3, 0x9e, 0xba, 0xec, 0x74, 0xa6, 0x90, 0xac, 0xbd, 0xd0, 0x85, 0x24, 0xe9, 0xc0, 0xa5, + 0xd3, 0xe4, 0x9f, 0x4b, 0x5a, 0x58, 0x30, 0x2b, 0x8d, 0x46, 0xdc, 0x4b, 0x7c, 0xba, 0xec, 0x67, + 0x60, 0xb8, 0x0a, 0x43, 0xd4, 0x71, 0x6c, 0x47, 0xd6, 0x89, 0x80, 0x4c, 0x8a, 0x80, 0xf6, 0x21, + 0x4e, 0x50, 0x61, 0x7d, 0xe4, 0x00, 0x08, 0x9f, 0xde, 0xf8, 0x5a, 0x8c, 0x6f, 0xfc, 0x7b, 0x24, + 0x5b, 0xc7, 0xb7, 0xc0, 0xc6, 0xcc, 0x42, 0xa3, 0x9e, 0x4a, 0xe2, 0x90, 0x16, 0x80, 0x72, 0xa4, + 0x13, 0xad, 0x7b, 0xda, 0x9f, 0x47, 0x61, 0x08, 0x2f, 0x78, 0x72, 0x05, 0x06, 0x71, 0xec, 0xe7, + 0xde, 0xe1, 0xe8, 0x6b, 0x85, 0x47, 0x7e, 0xdc, 0x27, 0x59, 0x98, 0xf4, 0x13, 0x31, 0xbf, 0xaf, + 0x17, 0x3d, 0xe1, 0xa5, 0x92, 0xb9, 0xd4, 0xa8, 0xa7, 0x54, 0x7f, 0xeb, 0x36, 0xee, 0x48, 0xcc, + 0x13, 0xe1, 0x1d, 0x72, 0x03, 0xe2, 0xd8, 0xa7, 0xf0, 0xb6, 0x45, 0xcc, 0x71, 0x58, 0x75, 0x19, + 0xcc, 0xdb, 0x0d, 0xb9, 0xea, 0x06, 0x28, 0x3b, 0x0e, 0xd8, 0xdd, 0xf8, 0xbc, 0x7c, 0x64, 0xc2, + 0xe3, 0x80, 0x78, 0x1b, 0x73, 0x5c, 0x82, 0x49, 0x09, 0x26, 0x9b, 0x57, 0x7a, 0xd9, 0xac, 0x98, + 0x9e, 0xff, 0x22, 0xbb, 0x80, 0x81, 0xc5, 0x60, 0x34, 0xef, 0xf0, 0x7b, 0x48, 0xc0, 0xb3, 0x99, + 0x05, 0x57, 0x75, 0x42, 0x1b, 0xa1, 0x96, 0x64, 0x22, 0xbc, 0x47, 0x7e, 0xa7, 0xc0, 0x95, 0x16, + 0x4d, 0xf9, 0xc2, 0x49, 0xf3, 0x14, 0xe7, 0x8b, 0x65, 0xdd, 0x75, 0xf9, 0x53, 0xcb, 0x88, 0xf4, + 0x3e, 0xdb, 0xc9, 0x80, 0xcc, 0x89, 0x7f, 0x9a, 0xd7, 0x19, 0xd3, 0xb6, 0x5e, 0xa1, 0xdc, 0xa6, + 0xab, 0xe2, 0x86, 0xbf, 0xec, 0xf4, 0xa2, 0xcf, 0xf5, 0x26, 0x21, 0xbb, 0x10, 0xaf, 0x52, 0xa7, + 0x62, 0xba, 0x2e, 0x76, 0xee, 0xfc, 0xd5, 0x78, 0x4e, 0xb2, 0x6a, 0x27, 0xd8, 0xe5, 0xf1, 0x96, + 0xc8, 0xe5, 0x78, 0x4b, 0x70, 0xf2, 0x5f, 0x0a, 0xc4, 0x25, 0x3e, 0x92, 0x83, 0x51, 0xb7, 0x56, + 0x38, 0xa4, 0xc5, 0x66, 0x85, 0x59, 0xe8, 0xac, 0x21, 0xbd, 0xcb, 0xc9, 0x44, 0xf7, 0x20, 0x78, + 0x42, 0xdd, 0x83, 0xc0, 0xf0, 0x8c, 0x53, 0xa7, 0xc0, 0x9f, 0x63, 0xfc, 0x33, 0xce, 0x80, 0xd0, + 0x19, 0x67, 0x40, 0xf2, 0x3d, 0x18, 0x11, 0x72, 0x59, 0xc6, 0x1f, 0x99, 0x96, 0x21, 0x67, 0x3c, + 0x5b, 0xcb, 0x19, 0xcf, 0xd6, 0xcd, 0x93, 0x31, 0x70, 0xfa, 0xc9, 0x48, 0x9a, 0x30, 0xdd, 0x21, + 0x6f, 0x9e, 0xa2, 0x4a, 0x29, 0x3d, 0xef, 0xca, 0x8f, 0x15, 0xb8, 0x72, 0xb6, 0x14, 0x39, 0x9b, + 0xfa, 0xb7, 0x64, 0xf5, 0xfe, 0x30, 0x15, 0x12, 0xd8, 0xa2, 0xad, 0x57, 0x19, 0xfd, 0xc9, 0x10, + 0x5c, 0x3c, 0x85, 0x9f, 0x35, 0xd9, 0xf3, 0x15, 0xfd, 0x7b, 0x66, 0xa5, 0x56, 0x09, 0x3a, 0xec, + 0x7d, 0x47, 0x2f, 0xb2, 0x22, 0x2f, 0xf2, 0xe2, 0x1b, 0xbd, 0xac, 0x48, 0xdf, 0xe7, 0x12, 0x7c, + 0xf4, 0xb6, 0xe0, 0xe7, 0x67, 0x23, 0x25, 0xce, 0xc6, 0x85, 0x4a, 0x67, 0xaa, 0x5c, 0xb7, 0x0d, + 0xf2, 0x7b, 0x05, 0x2e, 0x77, 0x35, 0x0e, 0xcf, 0xb0, 0x6d, 0x97, 0x31, 0xd7, 0xe2, 0xab, 0xeb, + 0x4f, 0x6b, 0x64, 0xe6, 0x64, 0xc7, 0xb6, 0xcb, 0xe2, 0xa2, 0x14, 0xa6, 0xbe, 0x50, 0x39, 0x8d, + 0x36, 0x77, 0xfa, 0x36, 0xbb, 0x2e, 0x4f, 0x0b, 0xc8, 0x79, 0x25, 0xa2, 0xd6, 0xdb, 0xc1, 0xb3, + 0xa9, 0x7e, 0x10, 0x4e, 0xc2, 0x97, 0xda, 0x23, 0x8b, 0x51, 0xe8, 0x2f, 0x11, 0xff, 0x30, 0x00, + 0xa9, 0x1e, 0x32, 0xc8, 0xc7, 0x67, 0x48, 0xc6, 0xb5, 0xb3, 0x58, 0x73, 0x4e, 0x09, 0xf9, 0x55, + 0x7c, 0x59, 0x2d, 0x0b, 0x31, 0x2c, 0xc9, 0xf7, 0x4c, 0xd7, 0x23, 0xd7, 0x61, 0x18, 0x5b, 0x51, + 0xbf, 0x64, 0x43, 0x50, 0xb2, 0x79, 0x73, 0xcc, 0x77, 0xe5, 0xe6, 0x98, 0x23, 0xda, 0x1e, 0x10, + 0xfe, 0xec, 0x58, 0x96, 0xfa, 0x37, 0x72, 0x0b, 0xc6, 0x8b, 0x1c, 0xa5, 0x86, 0xd4, 0x67, 0xe3, + 0x6f, 0x06, 0xcd, 0x8d, 0x70, 0xb7, 0x3d, 0x26, 0xe3, 0xda, 0x0d, 0x98, 0x44, 0xed, 0x77, 0x68, + 0xf3, 0xd9, 0xf9, 0x8c, 0x0d, 0x8c, 0x76, 0x0b, 0xd4, 0x5d, 0xcf, 0xa1, 0x7a, 0xc5, 0xb4, 0x4a, + 0xad, 0x32, 0x5e, 0x84, 0xa8, 0x55, 0xab, 0xa0, 0x88, 0x71, 0x1e, 0x48, 0xab, 0x56, 0x91, 0x03, + 0x69, 0xd5, 0x2a, 0xda, 0x9b, 0x40, 0x90, 0x6f, 0x83, 0x96, 0xa9, 0x47, 0xfb, 0x55, 0xff, 0x89, + 0x02, 0xc0, 0xdf, 0x29, 0x37, 0xad, 0x7d, 0xfb, 0xcc, 0x6d, 0xd7, 0x0d, 0x88, 0x63, 0x44, 0x8d, + 0xfc, 0xa1, 0x8d, 0x17, 0x9d, 0xb2, 0x34, 0xc4, 0xfb, 0x25, 0x0e, 0x6f, 0xd9, 0xa1, 0xdb, 0x0e, + 0x02, 0x94, 0xb1, 0x96, 0xa9, 0xee, 0xfa, 0xac, 0xd1, 0x80, 0x95, 0xc3, 0xad, 0xac, 0x01, 0xaa, + 0x3d, 0x82, 0x69, 0x74, 0x75, 0xaf, 0x6a, 0xe8, 0x5e, 0x30, 0x36, 0xbc, 0x21, 0xbf, 0xf0, 0x87, + 0xb3, 0xe1, 0xb4, 0x39, 0xa6, 0x8f, 0xb6, 0xb8, 0x06, 0x6a, 0x46, 0xf7, 0x8a, 0x07, 0x9d, 0xb4, + 0xbf, 0x07, 0xe3, 0xfb, 0xba, 0x59, 0xf6, 0x5f, 0xbe, 0xfc, 0x9c, 0x54, 0x03, 0x2b, 0xc2, 0x0c, + 0x3c, 0xad, 0x38, 0xcb, 0xdb, 0xad, 0x79, 0x3a, 0x26, 0xe3, 0x4d, 0x7f, 0xd7, 0xf1, 0x8d, 0xe4, + 0xab, 0xf2, 0xb7, 0x45, 0x7b, 0x6f, 0x7f, 0xc3, 0x0c, 0x7d, 0xf8, 0x1b, 0x87, 0x58, 0xd6, 0x32, + 0xee, 0xeb, 0xce, 0x11, 0x75, 0xb4, 0x8f, 0x14, 0x98, 0x0d, 0x9f, 0x8c, 0xfb, 0xd4, 0x75, 0xf5, + 0x12, 0x25, 0x5f, 0xeb, 0xcf, 0xff, 0xbb, 0x91, 0xe0, 0x19, 0x3a, 0x4a, 0x2d, 0x43, 0x14, 0xf4, + 0x09, 0x64, 0x6b, 0xea, 0xe3, 0xe7, 0x8b, 0xca, 0x0d, 0xd7, 0xdd, 0x48, 0x8e, 0xd1, 0x67, 0x46, + 0x60, 0x88, 0x1e, 0x53, 0xcb, 0x5b, 0x4e, 0x42, 0x5c, 0xfa, 0x0d, 0x96, 0xc4, 0x61, 0x44, 0x2c, + 0x13, 0x91, 0xe5, 0xab, 0x10, 0x97, 0x7e, 0xac, 0x23, 0x63, 0x30, 0xba, 0x6d, 0x1b, 0x74, 0xc7, + 0x76, 0xbc, 0x44, 0x84, 0xad, 0xee, 0x52, 0xdd, 0x28, 0x33, 0x52, 0x65, 0xf9, 0xe7, 0x0a, 0x8c, + 0xfa, 0x0f, 0xff, 0x04, 0x60, 0xf8, 0xed, 0xbd, 0xec, 0x5e, 0x76, 0x23, 0x11, 0x61, 0x02, 0x77, + 0xb2, 0xdb, 0x1b, 0x9b, 0xdb, 0x77, 0x12, 0x0a, 0x5b, 0xe4, 0xf6, 0xb6, 0xb7, 0xd9, 0x62, 0x80, + 0x8c, 0x43, 0x6c, 0x77, 0x6f, 0x7d, 0x3d, 0x9b, 0xdd, 0xc8, 0x6e, 0x24, 0xa2, 0x8c, 0xe9, 0xf6, + 0xda, 0xe6, 0xbd, 0xec, 0x46, 0x62, 0x90, 0xd1, 0xed, 0x6d, 0xbf, 0xb5, 0xfd, 0xe0, 0xdd, 0xed, + 0xc4, 0x10, 0xa7, 0xcb, 0xdc, 0xdf, 0x7c, 0xf8, 0x30, 0xbb, 0x91, 0x18, 0x66, 0x74, 0xf7, 0xb2, + 0x6b, 0xbb, 0xd9, 0x8d, 0xc4, 0x08, 0xdb, 0xda, 0xc9, 0x65, 0xb3, 0xf7, 0x77, 0xd8, 0xd6, 0x28, + 0x5b, 0xae, 0xaf, 0x6d, 0xaf, 0x67, 0xef, 0x31, 0x29, 0xb1, 0xd5, 0x3f, 0x46, 0x61, 0x0c, 0x43, + 0xe8, 0x3f, 0x26, 0xbd, 0x0e, 0x71, 0xfe, 0x1d, 0xf9, 0x3c, 0x26, 0x05, 0x39, 0x39, 0xd7, 0xf6, + 0xc8, 0x97, 0x65, 0xe1, 0xd2, 0x22, 0xe4, 0x16, 0x8c, 0x49, 0x4c, 0x2e, 0x99, 0x08, 0xb8, 0x58, + 0xe1, 0x4e, 0xbe, 0x80, 0xeb, 0x6e, 0xa9, 0xa5, 0x45, 0x98, 0x56, 0x7e, 0x5a, 0xfa, 0xd4, 0x2a, + 0x31, 0xf5, 0xd6, 0x1a, 0x3e, 0x8f, 0x5a, 0x84, 0x7c, 0x13, 0xe2, 0xbc, 0x7a, 0x72, 0xad, 0x17, + 0x02, 0xfe, 0x50, 0x51, 0x3d, 0xc5, 0x84, 0x34, 0x8c, 0xde, 0xa1, 0x1e, 0x67, 0x9f, 0x09, 0xd8, + 0x83, 0x5a, 0x9e, 0x94, 0x5c, 0xd1, 0x22, 0x64, 0x0b, 0x62, 0x3e, 0xbd, 0x4b, 0xb8, 0x7d, 0xdd, + 0x6e, 0x81, 0x64, 0xb2, 0xc3, 0xb6, 0x38, 0x0a, 0x5a, 0xe4, 0x35, 0x65, 0xf5, 0x67, 0x31, 0x18, + 0xe6, 0xd3, 0x37, 0x79, 0x07, 0x80, 0xff, 0x85, 0x95, 0x76, 0xb6, 0xe3, 0x4f, 0xab, 0xc9, 0xb9, + 0xce, 0x23, 0xbb, 0x36, 0xff, 0xa3, 0x3f, 0xfd, 0xfd, 0x17, 0x03, 0xd3, 0xda, 0xc4, 0xca, 0xf1, + 0xb5, 0x95, 0x43, 0xbb, 0x20, 0xfe, 0x71, 0xec, 0xa6, 0xb2, 0x4c, 0xde, 0x05, 0xe0, 0xd7, 0x66, + 0x58, 0x6e, 0xe8, 0x17, 0xbc, 0x24, 0x0f, 0x5b, 0xfb, 0xf5, 0xda, 0x2e, 0x98, 0xdf, 0x9d, 0x4c, + 0xf0, 0x77, 0x61, 0xac, 0x29, 0x78, 0x97, 0x7a, 0x44, 0x95, 0x7e, 0x94, 0x0b, 0x4b, 0xef, 0x16, + 0xfb, 0x4b, 0x28, 0x7c, 0x4e, 0x9b, 0x12, 0xc2, 0x5d, 0xea, 0x49, 0xf2, 0x2d, 0x48, 0xc8, 0x0f, + 0x45, 0x68, 0xfe, 0xc5, 0xce, 0x4f, 0x48, 0x5c, 0xcd, 0xa5, 0xd3, 0xde, 0x97, 0xb4, 0x14, 0x2a, + 0x9b, 0xd7, 0x66, 0x7c, 0x4f, 0xa4, 0xb7, 0x22, 0xca, 0xf4, 0xbd, 0x07, 0x71, 0xf1, 0xf3, 0x33, + 0xaa, 0x6a, 0x86, 0x3a, 0xfc, 0x9b, 0x74, 0x57, 0x67, 0x92, 0x28, 0x7f, 0x46, 0x9b, 0xf4, 0xe5, + 0x57, 0x39, 0x1f, 0x13, 0x7d, 0xa7, 0xff, 0x03, 0x39, 0x83, 0xe2, 0x26, 0xb4, 0x18, 0x13, 0x87, + 0x25, 0x90, 0x09, 0x2a, 0x3e, 0xdb, 0x21, 0x7d, 0x09, 0x85, 0x2e, 0x68, 0xf3, 0x4c, 0x68, 0x81, + 0x51, 0x51, 0x63, 0x85, 0xbf, 0xe2, 0x8b, 0x1b, 0x81, 0x29, 0xd9, 0xee, 0xff, 0x20, 0x5f, 0x44, + 0xc1, 0xb3, 0xc9, 0x44, 0xd3, 0xda, 0x95, 0x1f, 0xb0, 0x66, 0xe3, 0x43, 0x61, 0xf4, 0xb3, 0x9c, + 0x71, 0x61, 0x74, 0x32, 0x64, 0x74, 0x0d, 0x69, 0x24, 0xa3, 0xbf, 0xf5, 0x8c, 0x75, 0x40, 0x45, + 0x2d, 0x64, 0xb9, 0xcd, 0x03, 0x72, 0xbb, 0xaf, 0xfa, 0x20, 0xe4, 0x90, 0x76, 0x39, 0xc6, 0x73, + 0xaa, 0x1b, 0x22, 0xd1, 0x08, 0x91, 0xe3, 0xc1, 0x03, 0xf1, 0x9a, 0x42, 0x6e, 0xc2, 0xf0, 0x5d, + 0xfc, 0x07, 0x4a, 0xd2, 0xc5, 0xd3, 0x24, 0x3f, 0xa7, 0x9c, 0x68, 0xfd, 0x80, 0x16, 0x8f, 0x9a, + 0xb7, 0xfd, 0xfb, 0x9f, 0xff, 0x6d, 0x21, 0xf2, 0xc3, 0xc7, 0x0b, 0xca, 0xa7, 0x8f, 0x17, 0x94, + 0xcf, 0x1e, 0x2f, 0x28, 0x7f, 0x7d, 0xbc, 0xa0, 0x7c, 0xf4, 0x64, 0x21, 0xf2, 0xd9, 0x93, 0x85, + 0xc8, 0xe7, 0x4f, 0x16, 0x22, 0xdf, 0xfe, 0x3f, 0xe9, 0x7f, 0x3a, 0x75, 0xa7, 0xa2, 0x1b, 0x7a, + 0xd5, 0xb1, 0x0f, 0x69, 0xd1, 0x13, 0xab, 0x15, 0xf1, 0x4f, 0x9c, 0x9f, 0x0c, 0xcc, 0xac, 0x21, + 0xb0, 0xc3, 0xb7, 0xd3, 0x9b, 0x76, 0x7a, 0xad, 0x6a, 0x16, 0x86, 0xd1, 0x96, 0xd7, 0xff, 0x1b, + 0x00, 0x00, 0xff, 0xff, 0x52, 0xad, 0xfa, 0xb1, 0xe1, 0x2a, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -3154,11 +3145,6 @@ func (m *JobSubmitRequestItem) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.QueueTtlSeconds != 0 { - i = encodeVarintSubmit(dAtA, i, uint64(m.QueueTtlSeconds)) - i-- - dAtA[i] = 0x60 - } if len(m.Scheduler) > 0 { i -= len(m.Scheduler) copy(dAtA[i:], m.Scheduler) @@ -4959,9 +4945,6 @@ func (m *JobSubmitRequestItem) Size() (n int) { if l > 0 { n += 1 + l + sovSubmit(uint64(l)) } - if m.QueueTtlSeconds != 0 { - n += 1 + sovSubmit(uint64(m.QueueTtlSeconds)) - } return n } @@ -5725,7 +5708,6 @@ func (this *JobSubmitRequestItem) String() string { `Ingress:` + repeatedStringForIngress + `,`, `Services:` + repeatedStringForServices + `,`, `Scheduler:` + fmt.Sprintf("%v", this.Scheduler) + `,`, - `QueueTtlSeconds:` + fmt.Sprintf("%v", this.QueueTtlSeconds) + `,`, `}`, }, "") return s @@ -6917,25 +6899,6 @@ func (m *JobSubmitRequestItem) Unmarshal(dAtA []byte) error { } m.Scheduler = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 12: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field QueueTtlSeconds", wireType) - } - m.QueueTtlSeconds = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowSubmit - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.QueueTtlSeconds |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skipSubmit(dAtA[iNdEx:]) diff --git a/pkg/api/submit.proto b/pkg/api/submit.proto index 9be1b251e06..aec67ce0279 100644 --- a/pkg/api/submit.proto +++ b/pkg/api/submit.proto @@ -29,8 +29,7 @@ message JobSubmitRequestItem { // Indicates which scheduler should manage this job. // If empty, the default scheduler is used. string scheduler = 11; - // Queuing TTL for this job in seconds. If this job queues for more than this duration it will be cancelled. Zero indicates an infinite lifetime. - int64 queue_ttl_seconds = 12; + // Ordinal 12 was previously used for queue_ttl_seconds } message IngressConfig { diff --git a/pkg/armadaevents/events.pb.go b/pkg/armadaevents/events.pb.go index 497fbd2a0ef..d098b1800de 100644 --- a/pkg/armadaevents/events.pb.go +++ b/pkg/armadaevents/events.pb.go @@ -736,8 +736,6 @@ type SubmitJob struct { Scheduler string `protobuf:"bytes,11,opt,name=scheduler,proto3" json:"scheduler,omitempty"` // Indicates whether job is a duplicate IsDuplicate bool `protobuf:"varint,12,opt,name=isDuplicate,proto3" json:"isDuplicate,omitempty"` - // Queuing TTL for this job in seconds. If this job queues for more than this duration it will be cancelled. Zero indicates an infinite lifetime. - QueueTtlSeconds int64 `protobuf:"varint,13,opt,name=queue_ttl_seconds,json=queueTtlSeconds,proto3" json:"queueTtlSeconds,omitempty"` // This is a string representation of the job_id. Eventually we will retire the job_id field and just use strings JobIdStr string `protobuf:"bytes,14,opt,name=job_id_str,json=jobIdStr,proto3" json:"jobIdStr,omitempty"` } @@ -859,13 +857,6 @@ func (m *SubmitJob) GetIsDuplicate() bool { return false } -func (m *SubmitJob) GetQueueTtlSeconds() int64 { - if m != nil { - return m.QueueTtlSeconds - } - return 0 -} - func (m *SubmitJob) GetJobIdStr() string { if m != nil { return m.JobIdStr @@ -3968,249 +3959,247 @@ func init() { func init() { proto.RegisterFile("pkg/armadaevents/events.proto", fileDescriptor_6aab92ca59e015f8) } var fileDescriptor_6aab92ca59e015f8 = []byte{ - // 3863 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x3c, 0x4b, 0x6f, 0x1b, 0x57, - 0x77, 0x1e, 0x92, 0xe2, 0xe3, 0x50, 0x0f, 0xea, 0xea, 0x61, 0x5a, 0xb6, 0x45, 0x99, 0x4e, 0x1b, - 0x3b, 0x48, 0xa8, 0x44, 0x71, 0x82, 0x3c, 0x8a, 0x04, 0xa2, 0xad, 0xd8, 0x56, 0x2c, 0x5b, 0xa1, - 0xac, 0xd4, 0x0d, 0x52, 0x30, 0x43, 0xce, 0x15, 0x35, 0x16, 0x39, 0xc3, 0xcc, 0x43, 0x91, 0x80, - 0x2c, 0x92, 0x22, 0x68, 0x77, 0xa9, 0x17, 0x5d, 0x74, 0x97, 0x76, 0xd9, 0x16, 0x41, 0x81, 0x02, - 0xdd, 0xf4, 0xb1, 0xeb, 0x22, 0x05, 0x8a, 0x22, 0xab, 0xa2, 0x2b, 0xb6, 0x4d, 0xda, 0x2e, 0x08, - 0x7c, 0xdf, 0x3f, 0xf8, 0x80, 0x0f, 0xf7, 0x35, 0x73, 0xef, 0x70, 0x68, 0xd3, 0x92, 0x15, 0x27, - 0x48, 0x56, 0xf6, 0x9c, 0xe7, 0xbd, 0xf7, 0x9c, 0x7b, 0xee, 0x39, 0xe7, 0x5e, 0x0a, 0xce, 0x77, - 0xf7, 0x5a, 0xcb, 0xba, 0xd3, 0xd1, 0x0d, 0x1d, 0xef, 0x63, 0xcb, 0x73, 0x97, 0xd9, 0x3f, 0x95, - 0xae, 0x63, 0x7b, 0x36, 0x1a, 0x97, 0x51, 0x0b, 0xe5, 0xbd, 0xd7, 0xdc, 0x8a, 0x69, 0x2f, 0xeb, - 0x5d, 0x73, 0xb9, 0x69, 0x3b, 0x78, 0x79, 0xff, 0xa5, 0xe5, 0x16, 0xb6, 0xb0, 0xa3, 0x7b, 0xd8, - 0x60, 0x1c, 0x0b, 0x97, 0x24, 0x1a, 0x0b, 0x7b, 0x9f, 0xd8, 0xce, 0x9e, 0x69, 0xb5, 0xe2, 0x28, - 0x4b, 0x2d, 0xdb, 0x6e, 0xb5, 0xf1, 0x32, 0xfd, 0x6a, 0xf8, 0x3b, 0xcb, 0x9e, 0xd9, 0xc1, 0xae, - 0xa7, 0x77, 0xba, 0x9c, 0xe0, 0x4a, 0x28, 0xaa, 0xa3, 0x37, 0x77, 0x4d, 0x0b, 0x3b, 0x87, 0xcb, - 0x74, 0xbc, 0x5d, 0x73, 0xd9, 0xc1, 0xae, 0xed, 0x3b, 0x4d, 0x3c, 0x20, 0xf6, 0x85, 0x96, 0xe9, - 0xed, 0xfa, 0x8d, 0x4a, 0xd3, 0xee, 0x2c, 0xb7, 0xec, 0x96, 0x1d, 0xca, 0x27, 0x5f, 0xf4, 0x83, - 0xfe, 0x8f, 0x93, 0xbf, 0x61, 0x5a, 0x1e, 0x76, 0x2c, 0xbd, 0xbd, 0xec, 0x36, 0x77, 0xb1, 0xe1, - 0xb7, 0xb1, 0x13, 0xfe, 0xcf, 0x6e, 0xdc, 0xc7, 0x4d, 0xcf, 0x1d, 0x00, 0x30, 0xde, 0xf2, 0xff, - 0xcc, 0xc1, 0xc4, 0x1a, 0x59, 0x9a, 0x2d, 0xfc, 0xb1, 0x8f, 0xad, 0x26, 0x46, 0x97, 0x61, 0xec, - 0x63, 0x1f, 0xfb, 0xb8, 0xa8, 0x2d, 0x69, 0x97, 0x72, 0xd5, 0x99, 0x7e, 0xaf, 0x34, 0x45, 0x01, - 0xcf, 0xdb, 0x1d, 0xd3, 0xc3, 0x9d, 0xae, 0x77, 0x58, 0x63, 0x14, 0xe8, 0x0d, 0x18, 0xbf, 0x6f, - 0x37, 0xea, 0x2e, 0xf6, 0xea, 0x96, 0xde, 0xc1, 0xc5, 0x04, 0xe5, 0x28, 0xf6, 0x7b, 0xa5, 0xd9, - 0xfb, 0x76, 0x63, 0x0b, 0x7b, 0xb7, 0xf5, 0x8e, 0xcc, 0x06, 0x21, 0x14, 0xbd, 0x00, 0x19, 0xdf, - 0xc5, 0x4e, 0xdd, 0x34, 0x8a, 0x49, 0xca, 0x36, 0xdb, 0xef, 0x95, 0x0a, 0x04, 0x74, 0xd3, 0x90, - 0x58, 0xd2, 0x0c, 0x82, 0x9e, 0x87, 0x74, 0xcb, 0xb1, 0xfd, 0xae, 0x5b, 0x4c, 0x2d, 0x25, 0x05, - 0x35, 0x83, 0xc8, 0xd4, 0x0c, 0x82, 0xee, 0x40, 0x9a, 0xd9, 0xbb, 0x38, 0xb6, 0x94, 0xbc, 0x94, - 0x5f, 0xb9, 0x50, 0x91, 0x9d, 0xa0, 0xa2, 0x4c, 0x98, 0x7d, 0x31, 0x81, 0x0c, 0x2f, 0x0b, 0xe4, - 0x6e, 0xf3, 0x0f, 0x33, 0x30, 0x46, 0xe9, 0xd0, 0x1d, 0xc8, 0x34, 0x1d, 0x4c, 0x8c, 0x55, 0x44, - 0x4b, 0xda, 0xa5, 0xfc, 0xca, 0x42, 0x85, 0x39, 0x41, 0x45, 0x18, 0xa9, 0x72, 0x57, 0x38, 0x41, - 0xf5, 0x4c, 0xbf, 0x57, 0x9a, 0xe6, 0xe4, 0xa1, 0xd4, 0x07, 0xff, 0x55, 0xd2, 0x6a, 0x42, 0x0a, - 0xda, 0x84, 0x9c, 0xeb, 0x37, 0x3a, 0xa6, 0xb7, 0x6e, 0x37, 0xe8, 0x9a, 0xe7, 0x57, 0x4e, 0xab, - 0xc3, 0xdd, 0x12, 0xe8, 0xea, 0xe9, 0x7e, 0xaf, 0x34, 0x13, 0x50, 0x87, 0x12, 0x6f, 0x9c, 0xaa, - 0x85, 0x42, 0xd0, 0x2e, 0x4c, 0x39, 0xb8, 0xeb, 0x98, 0xb6, 0x63, 0x7a, 0xa6, 0x8b, 0x89, 0xdc, - 0x04, 0x95, 0x7b, 0x5e, 0x95, 0x5b, 0x53, 0x89, 0xaa, 0xe7, 0xfb, 0xbd, 0xd2, 0x99, 0x08, 0xa7, - 0xa2, 0x23, 0x2a, 0x16, 0x79, 0x80, 0x22, 0xa0, 0x2d, 0xec, 0x51, 0x7b, 0xe6, 0x57, 0x96, 0x1e, - 0xaa, 0x6c, 0x0b, 0x7b, 0xd5, 0xa5, 0x7e, 0xaf, 0x74, 0x6e, 0x90, 0x5f, 0x51, 0x19, 0x23, 0x1f, - 0xb5, 0xa1, 0x20, 0x43, 0x0d, 0x32, 0xc1, 0x14, 0xd5, 0xb9, 0x38, 0x5c, 0x27, 0xa1, 0xaa, 0x2e, - 0xf6, 0x7b, 0xa5, 0x85, 0x28, 0xaf, 0xa2, 0x6f, 0x40, 0x32, 0xb1, 0x4f, 0x53, 0xb7, 0x9a, 0xb8, - 0x4d, 0xd4, 0x8c, 0xc5, 0xd9, 0xe7, 0xaa, 0x40, 0x33, 0xfb, 0x04, 0xd4, 0xaa, 0x7d, 0x02, 0x30, - 0xfa, 0x10, 0xc6, 0x83, 0x0f, 0xb2, 0x5e, 0x69, 0xee, 0x47, 0xf1, 0x42, 0xc9, 0x4a, 0x2d, 0xf4, - 0x7b, 0xa5, 0x79, 0x99, 0x47, 0x11, 0xad, 0x48, 0x0b, 0xa5, 0xb7, 0xd9, 0xca, 0x64, 0x86, 0x4b, - 0x67, 0x14, 0xb2, 0xf4, 0xf6, 0xe0, 0x8a, 0x28, 0xd2, 0x88, 0x74, 0xb2, 0x89, 0xfd, 0x66, 0x13, - 0x63, 0x03, 0x1b, 0xc5, 0x6c, 0x9c, 0xf4, 0x75, 0x89, 0x82, 0x49, 0x97, 0x79, 0x54, 0xe9, 0x32, - 0x86, 0xac, 0xf5, 0x7d, 0xbb, 0xb1, 0xe6, 0x38, 0xb6, 0xe3, 0x16, 0x73, 0x71, 0x6b, 0xbd, 0x2e, - 0xd0, 0x6c, 0xad, 0x03, 0x6a, 0x75, 0xad, 0x03, 0x30, 0x1f, 0x6f, 0xcd, 0xb7, 0x6e, 0x61, 0xdd, - 0xc5, 0x46, 0x11, 0x86, 0x8c, 0x37, 0xa0, 0x08, 0xc6, 0x1b, 0x40, 0x06, 0xc6, 0x1b, 0x60, 0x90, - 0x01, 0x93, 0xec, 0x7b, 0xd5, 0x75, 0xcd, 0x96, 0x85, 0x8d, 0x62, 0x9e, 0xca, 0x3f, 0x17, 0x27, - 0x5f, 0xd0, 0x54, 0xcf, 0xf5, 0x7b, 0xa5, 0xa2, 0xca, 0xa7, 0xe8, 0x88, 0xc8, 0x44, 0x1f, 0xc1, - 0x04, 0x83, 0xd4, 0x7c, 0xcb, 0x32, 0xad, 0x56, 0x71, 0x9c, 0x2a, 0x39, 0x1b, 0xa7, 0x84, 0x93, - 0x54, 0xcf, 0xf6, 0x7b, 0xa5, 0xd3, 0x0a, 0x97, 0xa2, 0x42, 0x15, 0x48, 0x22, 0x06, 0x03, 0x84, - 0x86, 0x9d, 0x88, 0x8b, 0x18, 0xeb, 0x2a, 0x11, 0x8b, 0x18, 0x11, 0x4e, 0x35, 0x62, 0x44, 0x90, - 0xa1, 0x3d, 0xb8, 0x91, 0x27, 0x87, 0xdb, 0x83, 0xdb, 0x59, 0xb2, 0x47, 0x8c, 0xa9, 0x15, 0x69, - 0xe8, 0x33, 0x0d, 0xe6, 0x5c, 0x4f, 0xb7, 0x0c, 0xbd, 0x6d, 0x5b, 0xf8, 0xa6, 0xd5, 0x72, 0xb0, - 0xeb, 0xde, 0xb4, 0x76, 0xec, 0x62, 0x81, 0xea, 0xb9, 0x18, 0x09, 0xac, 0x71, 0xa4, 0xd5, 0x8b, - 0xfd, 0x5e, 0xa9, 0x14, 0x2b, 0x45, 0xd1, 0x1c, 0xaf, 0x08, 0x1d, 0xc0, 0x8c, 0x38, 0xd7, 0xb7, - 0x3d, 0xb3, 0x6d, 0xba, 0xba, 0x67, 0xda, 0x56, 0x71, 0x9a, 0xea, 0xbf, 0x10, 0x8d, 0x4f, 0x03, - 0x84, 0xd5, 0x0b, 0xfd, 0x5e, 0xe9, 0x7c, 0x8c, 0x04, 0x45, 0x77, 0x9c, 0x8a, 0xd0, 0x88, 0x9b, - 0x0e, 0x26, 0x84, 0xd8, 0x28, 0xce, 0x0c, 0x37, 0x62, 0x40, 0x24, 0x1b, 0x31, 0x00, 0xc6, 0x19, - 0x31, 0x40, 0x12, 0x4d, 0x5d, 0xdd, 0xf1, 0x4c, 0xa2, 0x76, 0x43, 0x77, 0xf6, 0xb0, 0x53, 0x9c, - 0x8d, 0xd3, 0xb4, 0xa9, 0x12, 0x31, 0x4d, 0x11, 0x4e, 0x55, 0x53, 0x04, 0x89, 0x1e, 0x68, 0xa0, - 0x0e, 0xcd, 0xb4, 0xad, 0x1a, 0x39, 0xb8, 0x5d, 0x32, 0xbd, 0x39, 0xaa, 0xf4, 0xd9, 0x87, 0x4c, - 0x4f, 0x26, 0xaf, 0x3e, 0xdb, 0xef, 0x95, 0x2e, 0x0e, 0x95, 0xa6, 0x0c, 0x64, 0xb8, 0x52, 0x74, - 0x0f, 0xf2, 0x04, 0x89, 0x69, 0x0a, 0x64, 0x14, 0xe7, 0xe9, 0x18, 0xce, 0x0c, 0x8e, 0x81, 0x13, - 0xd0, 0x1c, 0x60, 0x4e, 0xe2, 0x50, 0xf4, 0xc8, 0xa2, 0xd0, 0x17, 0x1a, 0x10, 0x47, 0x8f, 0x9b, - 0xe9, 0x69, 0xaa, 0xe5, 0x99, 0x01, 0x2d, 0x71, 0xd3, 0x7c, 0xa6, 0xdf, 0x2b, 0x2d, 0xc5, 0xcb, - 0x51, 0x74, 0x0f, 0xd1, 0x15, 0xfa, 0x51, 0x70, 0x48, 0x14, 0x8b, 0xc3, 0xfd, 0x28, 0x20, 0x92, - 0xfd, 0x28, 0x00, 0xc6, 0xf9, 0x51, 0x80, 0xe4, 0xc1, 0xe0, 0x7d, 0xbd, 0x6d, 0x1a, 0x34, 0xa1, - 0x3a, 0x33, 0x24, 0x18, 0x04, 0x14, 0x41, 0x30, 0x08, 0x20, 0x03, 0xc1, 0x20, 0xa4, 0xcd, 0xc0, - 0x18, 0x15, 0x51, 0xfe, 0x8f, 0x0c, 0xcc, 0xc4, 0x6c, 0x35, 0xf4, 0x16, 0xa4, 0x1d, 0xdf, 0x22, - 0x19, 0x28, 0x4b, 0xbb, 0x90, 0xaa, 0x78, 0xdb, 0x37, 0x0d, 0x96, 0xfe, 0x3a, 0xbe, 0xa5, 0x24, - 0xa5, 0x63, 0x14, 0x40, 0xf8, 0x49, 0xfa, 0x6b, 0x1a, 0x3c, 0xbd, 0x1a, 0xca, 0x7f, 0xdf, 0x6e, - 0xa8, 0xfc, 0x14, 0x80, 0x30, 0x4c, 0x88, 0x7d, 0x5c, 0x37, 0x49, 0x90, 0x4a, 0xc6, 0x59, 0xf9, - 0x5d, 0xbf, 0x81, 0x1d, 0x0b, 0x7b, 0xd8, 0x15, 0x73, 0xa0, 0x51, 0x8a, 0xae, 0x84, 0x23, 0x41, - 0x24, 0xf9, 0xe3, 0x32, 0x1c, 0xfd, 0x99, 0x06, 0xc5, 0x8e, 0x7e, 0x50, 0x17, 0x40, 0xb7, 0xbe, - 0x63, 0x3b, 0xf5, 0x2e, 0x76, 0x4c, 0xdb, 0xa0, 0xd9, 0x74, 0x7e, 0xe5, 0xf7, 0x1e, 0x19, 0x97, - 0x2a, 0x1b, 0xfa, 0x81, 0x00, 0xbb, 0xef, 0xd8, 0xce, 0x26, 0x65, 0x5f, 0xb3, 0x3c, 0xe7, 0xb0, - 0x7a, 0xfe, 0x9b, 0x5e, 0xe9, 0x14, 0xf1, 0xf2, 0x4e, 0x1c, 0x4d, 0x2d, 0x1e, 0x8c, 0xfe, 0x54, - 0x83, 0x79, 0xcf, 0xf6, 0xf4, 0x76, 0xbd, 0xe9, 0x77, 0xfc, 0xb6, 0xee, 0x99, 0xfb, 0xb8, 0xee, - 0xbb, 0x7a, 0x0b, 0xf3, 0xa4, 0xfd, 0xcd, 0x47, 0x0f, 0xea, 0x2e, 0xe1, 0xbf, 0x1a, 0xb0, 0x6f, - 0x13, 0x6e, 0x36, 0xa6, 0x73, 0x7c, 0x4c, 0xb3, 0x5e, 0x0c, 0x49, 0x2d, 0x16, 0x8a, 0xae, 0x00, - 0x30, 0x7b, 0xd6, 0x5d, 0xcf, 0xa1, 0x59, 0x59, 0xae, 0x3a, 0xdf, 0xef, 0x95, 0x10, 0x35, 0xd7, - 0x96, 0x27, 0xc5, 0xaa, 0x5a, 0x56, 0xc0, 0x08, 0x17, 0xf3, 0x22, 0xca, 0x95, 0x09, 0xb9, 0xa8, - 0x93, 0x44, 0xb8, 0x04, 0x6c, 0xe1, 0x2f, 0x34, 0x58, 0x18, 0xbe, 0xa4, 0xe8, 0x22, 0x24, 0xf7, - 0xf0, 0x21, 0x2f, 0xc1, 0xa6, 0xfb, 0xbd, 0xd2, 0xc4, 0x1e, 0x3e, 0x94, 0x04, 0x11, 0x2c, 0xfa, - 0x03, 0x18, 0xdb, 0xd7, 0xdb, 0x3e, 0xe6, 0xee, 0x57, 0xa9, 0xb0, 0x62, 0xb3, 0x22, 0x17, 0x9b, - 0x95, 0xee, 0x5e, 0x8b, 0x00, 0x2a, 0xc2, 0xfa, 0x95, 0xf7, 0x7c, 0xdd, 0xf2, 0x4c, 0xef, 0x90, - 0xb9, 0x26, 0x15, 0x20, 0xbb, 0x26, 0x05, 0xbc, 0x91, 0x78, 0x4d, 0x5b, 0xf8, 0x4a, 0x83, 0x33, - 0x43, 0x17, 0xf8, 0xc7, 0x30, 0xc2, 0x72, 0x1d, 0x52, 0x64, 0x93, 0x91, 0xe2, 0x70, 0xd7, 0x6c, - 0xed, 0xbe, 0x7a, 0x85, 0x0e, 0x27, 0xcd, 0x6a, 0x39, 0x06, 0x91, 0x6b, 0x39, 0x06, 0x21, 0x05, - 0x6e, 0xdb, 0xfe, 0xe4, 0xd5, 0x2b, 0x74, 0x50, 0x69, 0xa6, 0x84, 0x02, 0x64, 0x25, 0x14, 0x50, - 0xfe, 0xfb, 0x0c, 0xe4, 0x82, 0xea, 0x4b, 0xda, 0xef, 0xda, 0x91, 0xf6, 0xfb, 0x0d, 0x28, 0x18, - 0xd8, 0xf0, 0xbb, 0x6d, 0xb3, 0x49, 0xdd, 0x57, 0x44, 0x8e, 0x1c, 0x0b, 0x9d, 0x0a, 0x4e, 0xe1, - 0x9f, 0x8a, 0xa0, 0xd0, 0x0a, 0x64, 0x79, 0x95, 0x72, 0x48, 0x83, 0xc6, 0x04, 0xf3, 0x38, 0x01, - 0x93, 0x3d, 0x4e, 0xc0, 0x50, 0x0d, 0x80, 0x95, 0xfe, 0x1b, 0xd8, 0xd3, 0x79, 0xbd, 0x54, 0x54, - 0x67, 0x70, 0x27, 0xc0, 0xb3, 0x22, 0x3e, 0xa4, 0x97, 0x8b, 0xf8, 0x10, 0x8a, 0x3e, 0x04, 0xe8, - 0xe8, 0xa6, 0xc5, 0xf8, 0x78, 0x71, 0x54, 0x1e, 0x16, 0xbe, 0x36, 0x02, 0x4a, 0x26, 0x3d, 0xe4, - 0x94, 0xa5, 0x87, 0x50, 0x52, 0x6a, 0xf3, 0x66, 0x45, 0x31, 0x4d, 0x23, 0xc2, 0xe2, 0x30, 0xd1, - 0x5c, 0xec, 0x1c, 0x29, 0xb7, 0x39, 0x8b, 0x24, 0x53, 0x48, 0x21, 0xcb, 0xd6, 0x36, 0x77, 0xb0, - 0x67, 0x76, 0x30, 0xdd, 0xa8, 0x7c, 0xd9, 0x04, 0x4c, 0x5e, 0x36, 0x01, 0x43, 0xaf, 0x01, 0xe8, - 0xde, 0x86, 0xed, 0x7a, 0x77, 0xac, 0x26, 0xa6, 0xe5, 0x4e, 0x96, 0x0d, 0x3f, 0x84, 0xca, 0xc3, - 0x0f, 0xa1, 0xe8, 0x4d, 0xc8, 0x77, 0xf9, 0xf1, 0xda, 0x68, 0x63, 0x5a, 0xce, 0x64, 0x59, 0x36, - 0x20, 0x81, 0x25, 0x5e, 0x99, 0x1a, 0x5d, 0x87, 0xa9, 0xa6, 0x6d, 0x35, 0x7d, 0xc7, 0xc1, 0x56, - 0xf3, 0x70, 0x4b, 0xdf, 0xc1, 0xb4, 0x74, 0xc9, 0x32, 0x57, 0x89, 0xa0, 0x64, 0x57, 0x89, 0xa0, - 0xd0, 0x2b, 0x90, 0x0b, 0x5a, 0x3f, 0xb4, 0x3a, 0xc9, 0xf1, 0x2e, 0x82, 0x00, 0x4a, 0xcc, 0x21, - 0x25, 0x19, 0xbc, 0xe9, 0x5e, 0xe3, 0x4e, 0x87, 0x69, 0xc5, 0xc1, 0x07, 0x2f, 0x81, 0xe5, 0xc1, - 0x4b, 0x60, 0x74, 0x13, 0xa6, 0x69, 0x4a, 0x53, 0xf7, 0xbc, 0x76, 0xdd, 0xc5, 0x4d, 0xdb, 0x32, - 0x5c, 0x5a, 0x50, 0x24, 0xd9, 0xf0, 0x29, 0xf2, 0xae, 0xd7, 0xde, 0x62, 0x28, 0x79, 0xf8, 0x11, - 0x54, 0x24, 0x26, 0x4f, 0x8e, 0x16, 0x93, 0xcb, 0xff, 0xa6, 0xc1, 0x6c, 0x9c, 0xe3, 0x45, 0x36, - 0x81, 0xf6, 0x44, 0x36, 0xc1, 0xfb, 0x90, 0xed, 0xda, 0x46, 0xdd, 0xed, 0xe2, 0x26, 0x8f, 0x73, - 0x91, 0x2d, 0xb0, 0x69, 0x1b, 0x5b, 0x5d, 0xdc, 0xfc, 0x7d, 0xd3, 0xdb, 0x5d, 0xdd, 0xb7, 0x4d, - 0xe3, 0x96, 0xe9, 0x72, 0x5f, 0xed, 0x32, 0x8c, 0x92, 0xc4, 0x64, 0x38, 0xb0, 0x9a, 0x85, 0x34, - 0xd3, 0x52, 0xfe, 0xf7, 0x24, 0x14, 0xa2, 0xce, 0xfe, 0x53, 0x9a, 0x0a, 0xba, 0x07, 0x19, 0x93, - 0xd5, 0x48, 0x3c, 0xc7, 0xf9, 0x1d, 0xe9, 0x24, 0xa8, 0x84, 0x3d, 0xd6, 0xca, 0xfe, 0x4b, 0x15, - 0x5e, 0x4c, 0xd1, 0x25, 0xa0, 0x92, 0x39, 0xa7, 0x2a, 0x99, 0x03, 0x51, 0x0d, 0x32, 0x2e, 0x76, - 0xf6, 0xcd, 0x26, 0xe6, 0x21, 0xad, 0x24, 0x4b, 0x6e, 0xda, 0x0e, 0x26, 0x32, 0xb7, 0x18, 0x49, - 0x28, 0x93, 0xf3, 0xa8, 0x32, 0x39, 0x10, 0xbd, 0x0f, 0xb9, 0xa6, 0x6d, 0xed, 0x98, 0xad, 0x0d, - 0xbd, 0xcb, 0x83, 0xda, 0xf9, 0x38, 0xa9, 0x57, 0x05, 0x11, 0xef, 0xfb, 0x88, 0xcf, 0x48, 0xdf, - 0x27, 0xa0, 0x0a, 0x0d, 0xfa, 0xeb, 0x14, 0x40, 0x68, 0x1c, 0xf4, 0x3a, 0xe4, 0xf1, 0x01, 0x6e, - 0xfa, 0x9e, 0xed, 0x88, 0xd3, 0x85, 0xb7, 0x51, 0x05, 0x58, 0x39, 0x0e, 0x20, 0x84, 0x92, 0xed, - 0x6d, 0xe9, 0x1d, 0xec, 0x76, 0xf5, 0xa6, 0xe8, 0xbf, 0xd2, 0xc1, 0x04, 0x40, 0x79, 0x7b, 0x07, - 0x40, 0xf4, 0xbb, 0x90, 0xa2, 0x1d, 0x5b, 0xd6, 0x7a, 0x45, 0xfd, 0x5e, 0x69, 0xd2, 0x52, 0x7b, - 0xb5, 0x14, 0x8f, 0xde, 0x86, 0x89, 0xbd, 0xc0, 0xf1, 0xc8, 0xd8, 0x52, 0x94, 0x81, 0x26, 0x9f, - 0x21, 0x42, 0x19, 0xdd, 0xb8, 0x0c, 0x47, 0x3b, 0x90, 0xd7, 0x2d, 0xcb, 0xf6, 0xe8, 0xc9, 0x25, - 0xda, 0xb1, 0x97, 0x87, 0xb9, 0x69, 0x65, 0x35, 0xa4, 0x65, 0x79, 0x1c, 0x0d, 0x39, 0x92, 0x04, - 0x39, 0xe4, 0x48, 0x60, 0x54, 0x83, 0x74, 0x5b, 0x6f, 0xe0, 0xb6, 0x38, 0x2a, 0x9e, 0x19, 0xaa, - 0xe2, 0x16, 0x25, 0x63, 0xd2, 0x69, 0xa2, 0xc0, 0xf8, 0xe4, 0x44, 0x81, 0x41, 0x16, 0x76, 0xa0, - 0x10, 0x1d, 0xcf, 0x68, 0x69, 0xcf, 0x65, 0x39, 0xed, 0xc9, 0x3d, 0x32, 0xd1, 0xd2, 0x21, 0x2f, - 0x0d, 0xea, 0x24, 0x54, 0x94, 0xff, 0x4a, 0x83, 0xd9, 0xb8, 0xbd, 0x8b, 0x36, 0xa4, 0x1d, 0xaf, - 0xf1, 0xb6, 0x52, 0x8c, 0xab, 0x73, 0xde, 0x21, 0x5b, 0x3d, 0xdc, 0xe8, 0x55, 0x98, 0xb4, 0x6c, - 0x03, 0xd7, 0x75, 0xa2, 0xa0, 0x6d, 0xba, 0x5e, 0x31, 0x41, 0xdb, 0xf5, 0xb4, 0x1d, 0x45, 0x30, - 0xab, 0x02, 0x21, 0x71, 0x4f, 0x28, 0x88, 0xf2, 0x3f, 0x6a, 0x30, 0x15, 0xe9, 0x16, 0x1f, 0x3b, - 0xf5, 0x92, 0x13, 0xa6, 0xc4, 0x88, 0x09, 0x93, 0x7a, 0xf4, 0x24, 0x47, 0x3c, 0x7a, 0xfe, 0x35, - 0x01, 0x79, 0xa9, 0xfc, 0x3f, 0xf6, 0xc8, 0xef, 0xc3, 0x14, 0x3f, 0x95, 0x4d, 0xab, 0xc5, 0xca, - 0xc4, 0x04, 0xef, 0x65, 0x0d, 0x5c, 0xe9, 0xac, 0xdb, 0x8d, 0xad, 0x80, 0x96, 0x56, 0x89, 0xb4, - 0xd5, 0xe8, 0x2a, 0x30, 0x49, 0xc5, 0xa4, 0x8a, 0x41, 0xf7, 0x60, 0xde, 0xef, 0x92, 0xe2, 0xb9, - 0xee, 0xf2, 0xcb, 0x91, 0xba, 0xe5, 0x77, 0x1a, 0x98, 0xcd, 0x7e, 0xac, 0x5a, 0xee, 0xf7, 0x4a, - 0x8b, 0x8c, 0x42, 0xdc, 0x9e, 0xdc, 0xa6, 0x78, 0x49, 0xe6, 0x6c, 0x1c, 0x3e, 0xb2, 0x96, 0xa9, - 0x11, 0xd7, 0xf2, 0x06, 0xa0, 0xc1, 0x6b, 0x03, 0xc5, 0x96, 0xda, 0x68, 0xb6, 0x2c, 0xff, 0x93, - 0x06, 0x85, 0xe8, 0x6d, 0xc0, 0x4f, 0xc8, 0xa9, 0xfe, 0x56, 0x83, 0x5c, 0x70, 0x21, 0x70, 0xec, - 0x71, 0x3f, 0x0f, 0x69, 0x07, 0xeb, 0xae, 0x6d, 0xf1, 0xe0, 0x41, 0xa3, 0x20, 0x83, 0xc8, 0x51, - 0x90, 0x41, 0x8e, 0x38, 0xe2, 0xbb, 0x30, 0xce, 0xcc, 0xf5, 0x8e, 0xd9, 0xf6, 0xb0, 0x83, 0xae, - 0x41, 0xda, 0xf5, 0x74, 0x0f, 0xbb, 0x45, 0x6d, 0x29, 0x79, 0x69, 0x72, 0x65, 0x7e, 0xf0, 0xc6, - 0x80, 0xa0, 0xd9, 0x58, 0x18, 0xa5, 0x3c, 0x16, 0x06, 0x29, 0xff, 0x91, 0x06, 0xe3, 0xf2, 0xc5, - 0xc8, 0x93, 0x11, 0xfb, 0x78, 0x0b, 0x52, 0xfe, 0xbb, 0x60, 0x10, 0xed, 0x27, 0xe3, 0x47, 0x3f, - 0x84, 0x3d, 0x7e, 0xa5, 0x31, 0x83, 0x04, 0x8d, 0xf8, 0xe3, 0x0e, 0xba, 0x15, 0x36, 0xaf, 0x48, - 0xec, 0x70, 0x69, 0xa0, 0x1f, 0xb5, 0x79, 0x45, 0x8f, 0x03, 0x85, 0x5d, 0x3e, 0x0e, 0x14, 0xc4, - 0x11, 0xe7, 0xfb, 0x59, 0x9a, 0xce, 0x37, 0xbc, 0xaa, 0x79, 0xda, 0xcd, 0xbe, 0x48, 0x8e, 0x97, - 0x7c, 0x8c, 0x1c, 0xef, 0x05, 0xc8, 0xd0, 0x43, 0x35, 0x48, 0xbf, 0xa8, 0x83, 0x10, 0x90, 0x7a, - 0x55, 0xce, 0x20, 0x0f, 0x89, 0xe2, 0x63, 0xc7, 0x8c, 0xe2, 0x75, 0x38, 0xb3, 0xab, 0xbb, 0x75, - 0x71, 0xee, 0x18, 0x75, 0xdd, 0xab, 0x07, 0x11, 0x30, 0x4d, 0x4b, 0x44, 0xda, 0x7c, 0xde, 0xd5, - 0xdd, 0x2d, 0x41, 0xb3, 0xea, 0x6d, 0x0e, 0xc6, 0xc3, 0xf9, 0x78, 0x0a, 0xb4, 0x0d, 0x73, 0xf1, - 0xc2, 0x33, 0x74, 0xe4, 0xf4, 0x6e, 0xc4, 0x7d, 0xa8, 0xe4, 0x99, 0x18, 0x34, 0xfa, 0x5c, 0x83, - 0x22, 0xc9, 0x72, 0x1c, 0xfc, 0xb1, 0x6f, 0x3a, 0xb8, 0x43, 0x2c, 0x56, 0xb7, 0xf7, 0xb1, 0xd3, - 0xd6, 0x0f, 0xf9, 0x35, 0xe3, 0x85, 0xc1, 0xd3, 0x74, 0xd3, 0x36, 0x6a, 0x12, 0x03, 0x9b, 0x5a, - 0x57, 0x05, 0xde, 0x61, 0x42, 0xe4, 0xa9, 0xc5, 0x53, 0x44, 0xdc, 0x18, 0x8e, 0xd4, 0x5c, 0xcc, - 0x8f, 0xd6, 0x5c, 0x5c, 0x4f, 0x65, 0xb3, 0x85, 0x5c, 0xf9, 0xff, 0x12, 0x30, 0xa9, 0xde, 0x3c, - 0x3e, 0xf5, 0x4d, 0x30, 0x10, 0x34, 0x92, 0x3f, 0x48, 0xd0, 0x48, 0x1d, 0x69, 0xb5, 0xc7, 0x46, - 0x5b, 0xed, 0xf2, 0xff, 0x26, 0x60, 0x42, 0xb9, 0x7c, 0xfd, 0x65, 0x99, 0x4f, 0x62, 0x99, 0xff, - 0x3c, 0x01, 0xf3, 0xf1, 0x43, 0x3e, 0x91, 0x56, 0xc8, 0x0d, 0x20, 0x45, 0xcd, 0xcd, 0x30, 0xdf, - 0x9e, 0x1b, 0xe8, 0x84, 0xd0, 0xe5, 0x12, 0x15, 0xd1, 0xc0, 0xfd, 0xb0, 0x60, 0x47, 0xf7, 0x20, - 0x6f, 0x4a, 0x37, 0xd1, 0xc9, 0xb8, 0x0b, 0x43, 0xf9, 0xfe, 0x99, 0x75, 0xd9, 0x86, 0xdc, 0x3a, - 0xcb, 0xa2, 0xaa, 0x69, 0x48, 0x91, 0x82, 0xa0, 0xbc, 0x0f, 0x19, 0x3e, 0x1c, 0xf4, 0x32, 0xe4, - 0xe8, 0x59, 0x41, 0xab, 0x7b, 0x2d, 0x5c, 0x5a, 0x02, 0x8c, 0xbc, 0xc6, 0xca, 0x0a, 0x18, 0x7a, - 0x15, 0x80, 0x84, 0x47, 0x7e, 0x4a, 0x24, 0x68, 0xac, 0xa5, 0x5d, 0x84, 0xae, 0x6d, 0x0c, 0x1c, - 0x0d, 0xb9, 0x00, 0x58, 0xfe, 0x3a, 0x01, 0x79, 0xf9, 0xee, 0xfb, 0x48, 0xca, 0x3f, 0x05, 0xd1, - 0xe1, 0xa9, 0xeb, 0x86, 0x41, 0xfe, 0xc5, 0x22, 0x99, 0x58, 0x1e, 0xba, 0x48, 0xe2, 0xff, 0xab, - 0x82, 0x83, 0xd5, 0xf3, 0xf4, 0x7d, 0x8f, 0x19, 0x41, 0x49, 0x5a, 0x0b, 0x51, 0xdc, 0xc2, 0x1e, - 0xcc, 0xc5, 0x8a, 0x92, 0xab, 0xf0, 0xb1, 0x27, 0x55, 0x85, 0x7f, 0x9d, 0x86, 0xb9, 0xd8, 0x37, - 0x07, 0x4f, 0x3d, 0x62, 0xa8, 0x3b, 0x28, 0xf9, 0x44, 0x76, 0xd0, 0x1f, 0x6b, 0x71, 0x96, 0x65, - 0x17, 0x8e, 0xaf, 0x8f, 0xf0, 0x10, 0xe3, 0x49, 0xd9, 0x58, 0x75, 0xcb, 0xb1, 0x23, 0xed, 0x89, - 0xf4, 0xa8, 0x7b, 0x02, 0xbd, 0xc8, 0x1a, 0x2a, 0x54, 0x17, 0xbb, 0x0c, 0x14, 0x11, 0x22, 0xa2, - 0x2a, 0xc3, 0x41, 0xe8, 0x6d, 0x98, 0x10, 0x1c, 0xac, 0x8d, 0x97, 0x0d, 0x7b, 0x6c, 0x9c, 0x26, - 0xda, 0xc9, 0x1b, 0x97, 0xe1, 0x91, 0x28, 0x9c, 0x3b, 0x52, 0x14, 0x86, 0x11, 0xef, 0x2d, 0x7f, - 0xd0, 0xfd, 0xf2, 0xff, 0x09, 0x98, 0x8a, 0x3c, 0x39, 0xfa, 0xe5, 0x6c, 0x3d, 0x89, 0xb3, 0xf5, - 0x5f, 0x34, 0xc8, 0x05, 0x2f, 0xeb, 0x8e, 0x5d, 0x1a, 0xae, 0x42, 0x1a, 0xb3, 0xd7, 0x5d, 0x2c, - 0x8c, 0xcf, 0x44, 0x5e, 0xdf, 0x12, 0x1c, 0x7f, 0x6f, 0x1b, 0x79, 0xd0, 0x55, 0xe3, 0x8c, 0x47, - 0x2c, 0xfa, 0xfe, 0x39, 0x21, 0x8a, 0xbe, 0x70, 0x26, 0x4f, 0xd5, 0x59, 0xc2, 0x95, 0x48, 0x3e, - 0x99, 0x95, 0x38, 0x59, 0x37, 0xf8, 0x4b, 0x80, 0x31, 0x3a, 0x26, 0xb4, 0x02, 0x59, 0x0f, 0x3b, - 0x1d, 0xd3, 0xd2, 0xdb, 0x74, 0xe9, 0xb2, 0x8c, 0x5b, 0xc0, 0x64, 0x6e, 0x01, 0x43, 0xbb, 0x30, - 0x15, 0xb6, 0xfe, 0xa9, 0x98, 0xf8, 0x67, 0xc7, 0xef, 0xaa, 0x44, 0xec, 0x4a, 0x30, 0xc2, 0xa9, - 0xbe, 0x1b, 0x8a, 0x20, 0x91, 0x01, 0x93, 0x4d, 0xdb, 0xf2, 0x74, 0xd3, 0xc2, 0x0e, 0x53, 0x94, - 0x8c, 0x7b, 0x76, 0x79, 0x55, 0xa1, 0x61, 0xbd, 0x50, 0x95, 0x4f, 0x7d, 0x76, 0xa9, 0xe2, 0xd0, - 0x47, 0x30, 0x21, 0x8a, 0x70, 0xa6, 0x24, 0x15, 0xf7, 0xec, 0x72, 0x4d, 0x26, 0x61, 0x1b, 0x5c, - 0xe1, 0x52, 0x9f, 0x5d, 0x2a, 0x28, 0xd4, 0x86, 0x42, 0xd7, 0x36, 0xb6, 0x2d, 0x5e, 0x7a, 0xea, - 0x8d, 0x36, 0xe6, 0xf7, 0x4d, 0x8b, 0x03, 0xc9, 0xa6, 0x42, 0xc5, 0x0e, 0xc1, 0x28, 0xaf, 0xfa, - 0x90, 0x39, 0x8a, 0x45, 0x1f, 0xc2, 0x78, 0x1b, 0xeb, 0x2e, 0x5e, 0x3b, 0xe8, 0x9a, 0x0e, 0x36, - 0xe2, 0x9f, 0x1d, 0xdf, 0x92, 0x28, 0xd8, 0x11, 0x24, 0xf3, 0xa8, 0xaf, 0xad, 0x64, 0x0c, 0xb1, - 0x7e, 0x47, 0x3f, 0xa8, 0xf9, 0x96, 0xbb, 0x76, 0xc0, 0x9f, 0x90, 0x66, 0xe2, 0xac, 0xbf, 0xa1, - 0x12, 0x31, 0xeb, 0x47, 0x38, 0x55, 0xeb, 0x47, 0x90, 0xe8, 0x16, 0x3d, 0x61, 0x99, 0x49, 0xd8, - 0xf3, 0xe3, 0xf9, 0x81, 0xd5, 0x62, 0xd6, 0x60, 0xed, 0x58, 0xfe, 0xa5, 0x08, 0x0d, 0x24, 0x70, - 0x1b, 0xd0, 0x69, 0xd7, 0xb0, 0xe7, 0x3b, 0x16, 0x36, 0x78, 0x4b, 0x60, 0xd0, 0x06, 0x0a, 0x55, - 0x60, 0x03, 0x05, 0x3a, 0x60, 0x03, 0x05, 0x4b, 0x7c, 0xaa, 0x6b, 0x1b, 0x77, 0xd9, 0x96, 0xf1, - 0x82, 0xf7, 0xc8, 0x67, 0x07, 0x54, 0x85, 0x24, 0xcc, 0xa7, 0x14, 0x2e, 0xd5, 0xa7, 0x14, 0x14, - 0xfa, 0x14, 0x66, 0x23, 0xcf, 0x35, 0xd9, 0x4a, 0xe5, 0xe3, 0xae, 0x73, 0xd7, 0x63, 0x28, 0x59, - 0x7f, 0x28, 0x4e, 0x86, 0xa2, 0x36, 0x56, 0x0b, 0xd1, 0xde, 0xd2, 0xad, 0xd6, 0xba, 0xdd, 0x50, - 0xbd, 0x7a, 0x3c, 0x4e, 0xfb, 0xf5, 0x18, 0x4a, 0xa6, 0x3d, 0x4e, 0x86, 0xaa, 0x3d, 0x8e, 0x22, - 0x78, 0x9a, 0x49, 0x52, 0xd0, 0xe0, 0x09, 0x73, 0xdc, 0xd3, 0x4c, 0x46, 0x20, 0x3d, 0xcd, 0x64, - 0x80, 0x98, 0xa7, 0x99, 0x9c, 0x32, 0x2b, 0x5a, 0xb4, 0xe5, 0xaf, 0x34, 0x98, 0x8a, 0x44, 0x30, - 0xf4, 0x16, 0x04, 0x2f, 0xee, 0xee, 0x1e, 0x76, 0x45, 0xe9, 0xa3, 0xbc, 0xd0, 0x23, 0xf0, 0xb8, - 0x17, 0x7a, 0x04, 0x8e, 0x6e, 0x01, 0x04, 0x67, 0xff, 0xc3, 0x8e, 0x1a, 0x9a, 0x77, 0x87, 0x94, - 0x72, 0xde, 0x1d, 0x42, 0xcb, 0x0f, 0x52, 0x90, 0x15, 0x5b, 0xe0, 0x44, 0x4a, 0xe3, 0x65, 0xc8, - 0x74, 0xb0, 0x4b, 0x5f, 0xea, 0x25, 0xc2, 0x0c, 0x97, 0x83, 0xe4, 0x0c, 0x97, 0x83, 0xd4, 0x04, - 0x3c, 0x79, 0xa4, 0x04, 0x3c, 0x35, 0x72, 0x02, 0x8e, 0xe9, 0xcb, 0x19, 0x29, 0x90, 0x8b, 0x5b, - 0xe7, 0x87, 0x9f, 0x0e, 0xe2, 0x5d, 0x8d, 0xcc, 0x18, 0x79, 0x57, 0x23, 0xa3, 0xd0, 0x1e, 0x4c, - 0x4b, 0x37, 0xe3, 0xbc, 0x7f, 0x4f, 0x42, 0xea, 0xe4, 0xf0, 0x67, 0x4a, 0x35, 0x4a, 0xc5, 0x02, - 0xc7, 0x5e, 0x04, 0x2a, 0x57, 0x30, 0x51, 0x1c, 0x71, 0x30, 0x03, 0x37, 0xfc, 0xd6, 0x06, 0x5f, - 0xf6, 0x4c, 0xe8, 0x60, 0x32, 0x5c, 0x76, 0x30, 0x19, 0x4e, 0x5b, 0x81, 0xea, 0x7c, 0x4f, 0xc4, - 0x31, 0x5e, 0x86, 0x1c, 0x3e, 0x30, 0xbd, 0x7a, 0xd3, 0x36, 0x30, 0x6f, 0x23, 0x50, 0x3b, 0x13, - 0xe0, 0x55, 0xdb, 0x50, 0xec, 0x2c, 0x60, 0xb2, 0x37, 0x25, 0x47, 0xf2, 0xa6, 0xf0, 0xba, 0x24, - 0x35, 0xc2, 0x75, 0x49, 0xac, 0x9d, 0x72, 0x27, 0x63, 0xa7, 0xf2, 0xb7, 0x09, 0x28, 0x44, 0xcf, - 0x89, 0x1f, 0xc7, 0x16, 0x54, 0x77, 0x53, 0x72, 0xe4, 0xdd, 0xf4, 0x36, 0x4c, 0x90, 0x44, 0x52, - 0xf7, 0x3c, 0xfe, 0x93, 0x82, 0x14, 0xcd, 0x06, 0x59, 0x6c, 0xf3, 0xad, 0x55, 0x01, 0x57, 0x62, - 0x9b, 0x04, 0x1f, 0x70, 0xdd, 0xb1, 0xc7, 0x74, 0xdd, 0xcf, 0x13, 0x30, 0xa1, 0x9c, 0x87, 0x3f, - 0xbf, 0x90, 0x56, 0x9e, 0x82, 0x09, 0x25, 0xcd, 0x2c, 0x7f, 0xc1, 0xfc, 0x4c, 0x3d, 0xfd, 0x7e, - 0x7e, 0xeb, 0x32, 0x09, 0xe3, 0x72, 0xbe, 0x5a, 0xae, 0xc2, 0x54, 0x24, 0xbd, 0x94, 0x27, 0xa0, - 0x8d, 0x32, 0x81, 0xf2, 0x3c, 0xcc, 0xc6, 0x65, 0x45, 0xe5, 0xeb, 0x30, 0x1b, 0x97, 0xaf, 0x3c, - 0xbe, 0x82, 0xb7, 0xf8, 0xfb, 0x10, 0x96, 0x59, 0x3c, 0x3e, 0xff, 0xdf, 0xa4, 0x44, 0x53, 0x24, - 0xfc, 0x41, 0xce, 0x07, 0x50, 0xe8, 0x8a, 0x8f, 0xfa, 0x23, 0x6b, 0x77, 0x5a, 0x08, 0x05, 0xf4, - 0xeb, 0x91, 0xd2, 0x75, 0x52, 0xc5, 0xa8, 0xb2, 0x79, 0x35, 0x9d, 0x18, 0x51, 0x76, 0x2d, 0x52, - 0x56, 0x4f, 0xaa, 0x18, 0xf4, 0x87, 0x30, 0x2d, 0x1e, 0xbd, 0xee, 0x63, 0x31, 0xf0, 0xe4, 0x50, - 0xe1, 0xec, 0xf7, 0x43, 0x01, 0x43, 0x74, 0xe4, 0x53, 0x11, 0x54, 0x44, 0x3c, 0x1f, 0x7b, 0x6a, - 0x54, 0xf1, 0xd1, 0xc1, 0x4f, 0x45, 0x50, 0x68, 0x13, 0x66, 0xa3, 0xab, 0x2e, 0x95, 0xdb, 0xa5, - 0x7e, 0xaf, 0x74, 0x56, 0x5d, 0x4b, 0xb5, 0xee, 0x9e, 0x1e, 0x40, 0xaa, 0x12, 0xa5, 0x02, 0x3e, - 0x1d, 0x23, 0xb1, 0x36, 0x58, 0xc9, 0x4f, 0x0f, 0x20, 0xcb, 0x5f, 0x6a, 0x30, 0x15, 0xf9, 0x19, - 0x16, 0xba, 0x06, 0x59, 0xfa, 0x3b, 0xe9, 0x87, 0x7b, 0x09, 0xf5, 0x43, 0x4a, 0xa7, 0xac, 0x42, - 0x86, 0x83, 0xd0, 0x2b, 0x90, 0x0b, 0x7e, 0xad, 0xc5, 0x9f, 0xbf, 0xb0, 0x3d, 0x2b, 0x80, 0xca, - 0x9e, 0x15, 0xc0, 0xf2, 0x9f, 0x24, 0xe0, 0xcc, 0xd0, 0x9f, 0x68, 0x3d, 0xf5, 0x86, 0xcd, 0x91, - 0xfa, 0x4e, 0x91, 0x6e, 0x4b, 0x6a, 0xc4, 0x6e, 0xcb, 0x97, 0x1a, 0xcc, 0xc7, 0xff, 0x84, 0xeb, - 0xd8, 0x1d, 0x38, 0x75, 0x1a, 0x89, 0xd1, 0x5f, 0x19, 0x8d, 0xcb, 0xbf, 0xb6, 0x3a, 0xf6, 0x30, - 0x2e, 0xc3, 0x58, 0xd7, 0xb6, 0xdb, 0x2e, 0x7f, 0x04, 0x48, 0x49, 0x29, 0x40, 0x26, 0xa5, 0x80, - 0x23, 0x36, 0xfc, 0x7e, 0xa3, 0x89, 0x58, 0x18, 0xfe, 0xa8, 0xec, 0x67, 0xe4, 0x42, 0xcf, 0xbd, - 0x08, 0x59, 0xf1, 0xc4, 0x09, 0x01, 0xa4, 0xdf, 0xdb, 0x5e, 0xdb, 0x5e, 0xbb, 0x56, 0x38, 0x85, - 0xf2, 0x90, 0xd9, 0x5c, 0xbb, 0x7d, 0xed, 0xe6, 0xed, 0xeb, 0x05, 0x8d, 0x7c, 0xd4, 0xb6, 0x6f, - 0xdf, 0x26, 0x1f, 0x89, 0xe7, 0x6e, 0xc9, 0x2f, 0xc9, 0x79, 0x75, 0x31, 0x0e, 0xd9, 0xd5, 0x6e, - 0x97, 0x1e, 0x73, 0x8c, 0x77, 0x6d, 0xdf, 0x24, 0x67, 0x53, 0x41, 0x43, 0x19, 0x48, 0xde, 0xb9, - 0xb3, 0x51, 0x48, 0xa0, 0x59, 0x28, 0x5c, 0xc3, 0xba, 0xd1, 0x36, 0x2d, 0x2c, 0xce, 0xd6, 0x42, - 0xb2, 0x7a, 0xff, 0x9b, 0xef, 0x16, 0xb5, 0x6f, 0xbf, 0x5b, 0xd4, 0xfe, 0xfb, 0xbb, 0x45, 0xed, - 0xc1, 0xf7, 0x8b, 0xa7, 0xbe, 0xfd, 0x7e, 0xf1, 0xd4, 0x7f, 0x7e, 0xbf, 0x78, 0xea, 0x83, 0x17, - 0xa5, 0x3f, 0x61, 0xc1, 0xd6, 0xaf, 0xeb, 0xd8, 0xe4, 0xc8, 0xe3, 0x5f, 0xcb, 0xd1, 0x3f, 0xda, - 0xf1, 0xd7, 0x89, 0xf3, 0xab, 0xf4, 0x73, 0x93, 0xd1, 0x55, 0x6e, 0xda, 0x15, 0x06, 0xa0, 0x7f, - 0x77, 0xc1, 0x6d, 0xa4, 0xe9, 0xdf, 0x57, 0x78, 0xf9, 0xb7, 0x01, 0x00, 0x00, 0xff, 0xff, 0x21, - 0x3a, 0x92, 0x47, 0xef, 0x43, 0x00, 0x00, + // 3831 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x3c, 0x4b, 0x6f, 0x1b, 0x47, + 0x9a, 0x6e, 0x92, 0xe2, 0xe3, 0xa3, 0x1e, 0x74, 0xe9, 0x61, 0x5a, 0xb6, 0x45, 0x99, 0xce, 0x6e, + 0xec, 0x20, 0xa1, 0x12, 0xc5, 0x09, 0xf2, 0x58, 0x24, 0x10, 0x6d, 0xc5, 0xb6, 0x62, 0xd9, 0x0a, + 0x65, 0x65, 0xbd, 0x41, 0x16, 0x4c, 0x93, 0x5d, 0xa2, 0xda, 0x22, 0xbb, 0x99, 0x7e, 0x28, 0x12, + 0x90, 0x43, 0xb2, 0x08, 0x76, 0x6f, 0x59, 0x1f, 0xf6, 0xb0, 0xb7, 0xec, 0x1e, 0x77, 0x17, 0xc1, + 0x02, 0x7b, 0xdc, 0xc7, 0x6d, 0x0f, 0x59, 0x60, 0x31, 0xc8, 0x69, 0x30, 0x27, 0xce, 0x4c, 0x32, + 0x33, 0x07, 0x02, 0x33, 0xff, 0x60, 0x80, 0x41, 0xbd, 0xba, 0xab, 0x9a, 0x4d, 0x9b, 0x96, 0xac, + 0x38, 0x41, 0x72, 0xb2, 0xfb, 0x7b, 0x56, 0xd5, 0xf7, 0xd5, 0x57, 0xdf, 0xf7, 0x55, 0x51, 0x70, + 0xae, 0xbb, 0xdb, 0x5a, 0xd2, 0x9d, 0x8e, 0x6e, 0xe8, 0x78, 0x0f, 0x5b, 0x9e, 0xbb, 0xc4, 0xfe, + 0xa9, 0x74, 0x1d, 0xdb, 0xb3, 0xd1, 0xb8, 0x8c, 0x9a, 0x2f, 0xef, 0xbe, 0xe2, 0x56, 0x4c, 0x7b, + 0x49, 0xef, 0x9a, 0x4b, 0x4d, 0xdb, 0xc1, 0x4b, 0x7b, 0x2f, 0x2c, 0xb5, 0xb0, 0x85, 0x1d, 0xdd, + 0xc3, 0x06, 0xe3, 0x98, 0xbf, 0x28, 0xd1, 0x58, 0xd8, 0xfb, 0xc8, 0x76, 0x76, 0x4d, 0xab, 0x15, + 0x47, 0x59, 0x6a, 0xd9, 0x76, 0xab, 0x8d, 0x97, 0xe8, 0x57, 0xc3, 0xdf, 0x5e, 0xf2, 0xcc, 0x0e, + 0x76, 0x3d, 0xbd, 0xd3, 0xe5, 0x04, 0x97, 0x43, 0x51, 0x1d, 0xbd, 0xb9, 0x63, 0x5a, 0xd8, 0x39, + 0x58, 0xa2, 0xe3, 0xed, 0x9a, 0x4b, 0x0e, 0x76, 0x6d, 0xdf, 0x69, 0xe2, 0x01, 0xb1, 0xcf, 0xb5, + 0x4c, 0x6f, 0xc7, 0x6f, 0x54, 0x9a, 0x76, 0x67, 0xa9, 0x65, 0xb7, 0xec, 0x50, 0x3e, 0xf9, 0xa2, + 0x1f, 0xf4, 0x7f, 0x9c, 0xfc, 0x35, 0xd3, 0xf2, 0xb0, 0x63, 0xe9, 0xed, 0x25, 0xb7, 0xb9, 0x83, + 0x0d, 0xbf, 0x8d, 0x9d, 0xf0, 0x7f, 0x76, 0xe3, 0x1e, 0x6e, 0x7a, 0xee, 0x00, 0x80, 0xf1, 0x96, + 0x7f, 0x3d, 0x0b, 0x13, 0xab, 0x64, 0x69, 0x36, 0xf1, 0x87, 0x3e, 0xb6, 0x9a, 0x18, 0x5d, 0x82, + 0xb1, 0x0f, 0x7d, 0xec, 0xe3, 0xa2, 0xb6, 0xa8, 0x5d, 0xcc, 0x55, 0xa7, 0xfb, 0xbd, 0xd2, 0x14, + 0x05, 0x3c, 0x6b, 0x77, 0x4c, 0x0f, 0x77, 0xba, 0xde, 0x41, 0x8d, 0x51, 0xa0, 0xd7, 0x60, 0xfc, + 0x9e, 0xdd, 0xa8, 0xbb, 0xd8, 0xab, 0x5b, 0x7a, 0x07, 0x17, 0x13, 0x94, 0xa3, 0xd8, 0xef, 0x95, + 0x66, 0xee, 0xd9, 0x8d, 0x4d, 0xec, 0xdd, 0xd2, 0x3b, 0x32, 0x1b, 0x84, 0x50, 0xf4, 0x1c, 0x64, + 0x7c, 0x17, 0x3b, 0x75, 0xd3, 0x28, 0x26, 0x29, 0xdb, 0x4c, 0xbf, 0x57, 0x2a, 0x10, 0xd0, 0x0d, + 0x43, 0x62, 0x49, 0x33, 0x08, 0x7a, 0x16, 0xd2, 0x2d, 0xc7, 0xf6, 0xbb, 0x6e, 0x31, 0xb5, 0x98, + 0x14, 0xd4, 0x0c, 0x22, 0x53, 0x33, 0x08, 0xba, 0x0d, 0x69, 0x66, 0xef, 0xe2, 0xd8, 0x62, 0xf2, + 0x62, 0x7e, 0xf9, 0x7c, 0x45, 0x76, 0x82, 0x8a, 0x32, 0x61, 0xf6, 0xc5, 0x04, 0x32, 0xbc, 0x2c, + 0x90, 0xbb, 0xcd, 0x7f, 0x4e, 0xc3, 0x18, 0xa5, 0x43, 0xb7, 0x21, 0xd3, 0x74, 0x30, 0x31, 0x56, + 0x11, 0x2d, 0x6a, 0x17, 0xf3, 0xcb, 0xf3, 0x15, 0xe6, 0x04, 0x15, 0x61, 0xa4, 0xca, 0x1d, 0xe1, + 0x04, 0xd5, 0xd3, 0xfd, 0x5e, 0xe9, 0x24, 0x27, 0x0f, 0xa5, 0xde, 0xff, 0x65, 0x49, 0xab, 0x09, + 0x29, 0x68, 0x03, 0x72, 0xae, 0xdf, 0xe8, 0x98, 0xde, 0x9a, 0xdd, 0xa0, 0x6b, 0x9e, 0x5f, 0x3e, + 0xa5, 0x0e, 0x77, 0x53, 0xa0, 0xab, 0xa7, 0xfa, 0xbd, 0xd2, 0x74, 0x40, 0x1d, 0x4a, 0xbc, 0x7e, + 0xa2, 0x16, 0x0a, 0x41, 0x3b, 0x30, 0xe5, 0xe0, 0xae, 0x63, 0xda, 0x8e, 0xe9, 0x99, 0x2e, 0x26, + 0x72, 0x13, 0x54, 0xee, 0x39, 0x55, 0x6e, 0x4d, 0x25, 0xaa, 0x9e, 0xeb, 0xf7, 0x4a, 0xa7, 0x23, + 0x9c, 0x8a, 0x8e, 0xa8, 0x58, 0xe4, 0x01, 0x8a, 0x80, 0x36, 0xb1, 0x47, 0xed, 0x99, 0x5f, 0x5e, + 0x7c, 0xa0, 0xb2, 0x4d, 0xec, 0x55, 0x17, 0xfb, 0xbd, 0xd2, 0xd9, 0x41, 0x7e, 0x45, 0x65, 0x8c, + 0x7c, 0xd4, 0x86, 0x82, 0x0c, 0x35, 0xc8, 0x04, 0x53, 0x54, 0xe7, 0xc2, 0x70, 0x9d, 0x84, 0xaa, + 0xba, 0xd0, 0xef, 0x95, 0xe6, 0xa3, 0xbc, 0x8a, 0xbe, 0x01, 0xc9, 0xc4, 0x3e, 0x4d, 0xdd, 0x6a, + 0xe2, 0x36, 0x51, 0x33, 0x16, 0x67, 0x9f, 0x2b, 0x02, 0xcd, 0xec, 0x13, 0x50, 0xab, 0xf6, 0x09, + 0xc0, 0xe8, 0x7d, 0x18, 0x0f, 0x3e, 0xc8, 0x7a, 0xa5, 0xb9, 0x1f, 0xc5, 0x0b, 0x25, 0x2b, 0x35, + 0xdf, 0xef, 0x95, 0xe6, 0x64, 0x1e, 0x45, 0xb4, 0x22, 0x2d, 0x94, 0xde, 0x66, 0x2b, 0x93, 0x19, + 0x2e, 0x9d, 0x51, 0xc8, 0xd2, 0xdb, 0x83, 0x2b, 0xa2, 0x48, 0x23, 0xd2, 0xc9, 0x26, 0xf6, 0x9b, + 0x4d, 0x8c, 0x0d, 0x6c, 0x14, 0xb3, 0x71, 0xd2, 0xd7, 0x24, 0x0a, 0x26, 0x5d, 0xe6, 0x51, 0xa5, + 0xcb, 0x18, 0xb2, 0xd6, 0xf7, 0xec, 0xc6, 0xaa, 0xe3, 0xd8, 0x8e, 0x5b, 0xcc, 0xc5, 0xad, 0xf5, + 0x9a, 0x40, 0xb3, 0xb5, 0x0e, 0xa8, 0xd5, 0xb5, 0x0e, 0xc0, 0x7c, 0xbc, 0x35, 0xdf, 0xba, 0x89, + 0x75, 0x17, 0x1b, 0x45, 0x18, 0x32, 0xde, 0x80, 0x22, 0x18, 0x6f, 0x00, 0x19, 0x18, 0x6f, 0x80, + 0x41, 0x06, 0x4c, 0xb2, 0xef, 0x15, 0xd7, 0x35, 0x5b, 0x16, 0x36, 0x8a, 0x79, 0x2a, 0xff, 0x6c, + 0x9c, 0x7c, 0x41, 0x53, 0x3d, 0xdb, 0xef, 0x95, 0x8a, 0x2a, 0x9f, 0xa2, 0x23, 0x22, 0x13, 0x7d, + 0x00, 0x13, 0x0c, 0x52, 0xf3, 0x2d, 0xcb, 0xb4, 0x5a, 0xc5, 0x71, 0xaa, 0xe4, 0x4c, 0x9c, 0x12, + 0x4e, 0x52, 0x3d, 0xd3, 0xef, 0x95, 0x4e, 0x29, 0x5c, 0x8a, 0x0a, 0x55, 0x20, 0x89, 0x18, 0x0c, + 0x10, 0x1a, 0x76, 0x22, 0x2e, 0x62, 0xac, 0xa9, 0x44, 0x2c, 0x62, 0x44, 0x38, 0xd5, 0x88, 0x11, + 0x41, 0x86, 0xf6, 0xe0, 0x46, 0x9e, 0x1c, 0x6e, 0x0f, 0x6e, 0x67, 0xc9, 0x1e, 0x31, 0xa6, 0x56, + 0xa4, 0xa1, 0x4f, 0x34, 0x98, 0x75, 0x3d, 0xdd, 0x32, 0xf4, 0xb6, 0x6d, 0xe1, 0x1b, 0x56, 0xcb, + 0xc1, 0xae, 0x7b, 0xc3, 0xda, 0xb6, 0x8b, 0x05, 0xaa, 0xe7, 0x42, 0x24, 0xb0, 0xc6, 0x91, 0x56, + 0x2f, 0xf4, 0x7b, 0xa5, 0x52, 0xac, 0x14, 0x45, 0x73, 0xbc, 0x22, 0xb4, 0x0f, 0xd3, 0xe2, 0x5c, + 0xdf, 0xf2, 0xcc, 0xb6, 0xe9, 0xea, 0x9e, 0x69, 0x5b, 0xc5, 0x93, 0x54, 0xff, 0xf9, 0x68, 0x7c, + 0x1a, 0x20, 0xac, 0x9e, 0xef, 0xf7, 0x4a, 0xe7, 0x62, 0x24, 0x28, 0xba, 0xe3, 0x54, 0x84, 0x46, + 0xdc, 0x70, 0x30, 0x21, 0xc4, 0x46, 0x71, 0x7a, 0xb8, 0x11, 0x03, 0x22, 0xd9, 0x88, 0x01, 0x30, + 0xce, 0x88, 0x01, 0x92, 0x68, 0xea, 0xea, 0x8e, 0x67, 0x12, 0xb5, 0xeb, 0xba, 0xb3, 0x8b, 0x9d, + 0xe2, 0x4c, 0x9c, 0xa6, 0x0d, 0x95, 0x88, 0x69, 0x8a, 0x70, 0xaa, 0x9a, 0x22, 0x48, 0x74, 0x5f, + 0x03, 0x75, 0x68, 0xa6, 0x6d, 0xd5, 0xc8, 0xc1, 0xed, 0x92, 0xe9, 0xcd, 0x52, 0xa5, 0x4f, 0x3f, + 0x60, 0x7a, 0x32, 0x79, 0xf5, 0xe9, 0x7e, 0xaf, 0x74, 0x61, 0xa8, 0x34, 0x65, 0x20, 0xc3, 0x95, + 0xa2, 0xbb, 0x90, 0x27, 0x48, 0x4c, 0x53, 0x20, 0xa3, 0x38, 0x47, 0xc7, 0x70, 0x7a, 0x70, 0x0c, + 0x9c, 0x80, 0xe6, 0x00, 0xb3, 0x12, 0x87, 0xa2, 0x47, 0x16, 0x85, 0x3e, 0xd3, 0x80, 0x38, 0x7a, + 0xdc, 0x4c, 0x4f, 0x51, 0x2d, 0x4f, 0x0d, 0x68, 0x89, 0x9b, 0xe6, 0x53, 0xfd, 0x5e, 0x69, 0x31, + 0x5e, 0x8e, 0xa2, 0x7b, 0x88, 0xae, 0xd0, 0x8f, 0x82, 0x43, 0xa2, 0x58, 0x1c, 0xee, 0x47, 0x01, + 0x91, 0xec, 0x47, 0x01, 0x30, 0xce, 0x8f, 0x02, 0x24, 0x0f, 0x06, 0xef, 0xea, 0x6d, 0xd3, 0xa0, + 0x09, 0xd5, 0xe9, 0x21, 0xc1, 0x20, 0xa0, 0x08, 0x82, 0x41, 0x00, 0x19, 0x08, 0x06, 0x21, 0x6d, + 0x06, 0xc6, 0xa8, 0x88, 0xf2, 0xcf, 0x33, 0x30, 0x1d, 0xb3, 0xd5, 0xd0, 0x1b, 0x90, 0x76, 0x7c, + 0x8b, 0x64, 0xa0, 0x2c, 0xed, 0x42, 0xaa, 0xe2, 0x2d, 0xdf, 0x34, 0x58, 0xfa, 0xeb, 0xf8, 0x96, + 0x92, 0x94, 0x8e, 0x51, 0x00, 0xe1, 0x27, 0xe9, 0xaf, 0x69, 0xf0, 0xf4, 0x6a, 0x28, 0xff, 0x3d, + 0xbb, 0xa1, 0xf2, 0x53, 0x00, 0xc2, 0x30, 0x21, 0xf6, 0x71, 0xdd, 0x24, 0x41, 0x2a, 0x19, 0x67, + 0xe5, 0xb7, 0xfd, 0x06, 0x76, 0x2c, 0xec, 0x61, 0x57, 0xcc, 0x81, 0x46, 0x29, 0xba, 0x12, 0x8e, + 0x04, 0x91, 0xe4, 0x8f, 0xcb, 0x70, 0xf4, 0x0f, 0x1a, 0x14, 0x3b, 0xfa, 0x7e, 0x5d, 0x00, 0xdd, + 0xfa, 0xb6, 0xed, 0xd4, 0xbb, 0xd8, 0x31, 0x6d, 0x83, 0x66, 0xd3, 0xf9, 0xe5, 0xbf, 0x78, 0x68, + 0x5c, 0xaa, 0xac, 0xeb, 0xfb, 0x02, 0xec, 0xbe, 0x65, 0x3b, 0x1b, 0x94, 0x7d, 0xd5, 0xf2, 0x9c, + 0x83, 0xea, 0xb9, 0xaf, 0x7a, 0xa5, 0x13, 0xc4, 0xcb, 0x3b, 0x71, 0x34, 0xb5, 0x78, 0x30, 0xfa, + 0x7b, 0x0d, 0xe6, 0x3c, 0xdb, 0xd3, 0xdb, 0xf5, 0xa6, 0xdf, 0xf1, 0xdb, 0xba, 0x67, 0xee, 0xe1, + 0xba, 0xef, 0xea, 0x2d, 0xcc, 0x93, 0xf6, 0xd7, 0x1f, 0x3e, 0xa8, 0x3b, 0x84, 0xff, 0x4a, 0xc0, + 0xbe, 0x45, 0xb8, 0xd9, 0x98, 0xce, 0xf2, 0x31, 0xcd, 0x78, 0x31, 0x24, 0xb5, 0x58, 0x28, 0xba, + 0x0c, 0xc0, 0xec, 0x59, 0x77, 0x3d, 0x87, 0x66, 0x65, 0xb9, 0xea, 0x5c, 0xbf, 0x57, 0x42, 0xd4, + 0x5c, 0x9b, 0x9e, 0x14, 0xab, 0x6a, 0x59, 0x01, 0x23, 0x5c, 0xcc, 0x8b, 0x28, 0x57, 0x26, 0xe4, + 0xa2, 0x4e, 0x12, 0xe1, 0x12, 0xb0, 0xf9, 0x7f, 0xd2, 0x60, 0x7e, 0xf8, 0x92, 0xa2, 0x0b, 0x90, + 0xdc, 0xc5, 0x07, 0xbc, 0x04, 0x3b, 0xd9, 0xef, 0x95, 0x26, 0x76, 0xf1, 0x81, 0x24, 0x88, 0x60, + 0xd1, 0x5f, 0xc1, 0xd8, 0x9e, 0xde, 0xf6, 0x31, 0x77, 0xbf, 0x4a, 0x85, 0x15, 0x9b, 0x15, 0xb9, + 0xd8, 0xac, 0x74, 0x77, 0x5b, 0x04, 0x50, 0x11, 0xd6, 0xaf, 0xbc, 0xe3, 0xeb, 0x96, 0x67, 0x7a, + 0x07, 0xcc, 0x35, 0xa9, 0x00, 0xd9, 0x35, 0x29, 0xe0, 0xb5, 0xc4, 0x2b, 0xda, 0xfc, 0x17, 0x1a, + 0x9c, 0x1e, 0xba, 0xc0, 0xdf, 0x87, 0x11, 0x96, 0xeb, 0x90, 0x22, 0x9b, 0x8c, 0x14, 0x87, 0x3b, + 0x66, 0x6b, 0xe7, 0xe5, 0xcb, 0x74, 0x38, 0x69, 0x56, 0xcb, 0x31, 0x88, 0x5c, 0xcb, 0x31, 0x08, + 0x29, 0x70, 0xdb, 0xf6, 0x47, 0x2f, 0x5f, 0xa6, 0x83, 0x4a, 0x33, 0x25, 0x14, 0x20, 0x2b, 0xa1, + 0x80, 0x72, 0x3f, 0x0d, 0xb9, 0xa0, 0xfa, 0x92, 0xf6, 0xbb, 0x76, 0xa8, 0xfd, 0x7e, 0x1d, 0x0a, + 0x06, 0x36, 0xfc, 0x6e, 0xdb, 0x6c, 0x52, 0xf7, 0x15, 0x91, 0x23, 0xc7, 0x42, 0xa7, 0x82, 0x53, + 0xf8, 0xa7, 0x22, 0x28, 0xb4, 0x0c, 0x59, 0x5e, 0xa5, 0x1c, 0xd0, 0xa0, 0x31, 0xc1, 0x3c, 0x4e, + 0xc0, 0x64, 0x8f, 0x13, 0x30, 0x54, 0x03, 0x60, 0xa5, 0xff, 0x3a, 0xf6, 0x74, 0x5e, 0x2f, 0x15, + 0xd5, 0x19, 0xdc, 0x0e, 0xf0, 0xac, 0x88, 0x0f, 0xe9, 0xe5, 0x22, 0x3e, 0x84, 0xa2, 0xf7, 0x01, + 0x3a, 0xba, 0x69, 0x31, 0x3e, 0x5e, 0x1c, 0x95, 0x87, 0x85, 0xaf, 0xf5, 0x80, 0x92, 0x49, 0x0f, + 0x39, 0x65, 0xe9, 0x21, 0x94, 0x94, 0xda, 0xbc, 0x59, 0x51, 0x4c, 0xd3, 0x88, 0xb0, 0x30, 0x4c, + 0x34, 0x17, 0x3b, 0x4b, 0xca, 0x6d, 0xce, 0x22, 0xc9, 0x14, 0x52, 0xc8, 0xb2, 0xb5, 0xcd, 0x6d, + 0xec, 0x99, 0x1d, 0x4c, 0x37, 0x2a, 0x5f, 0x36, 0x01, 0x93, 0x97, 0x4d, 0xc0, 0xd0, 0x2b, 0x00, + 0xba, 0xb7, 0x6e, 0xbb, 0xde, 0x6d, 0xab, 0x89, 0x69, 0xb9, 0x93, 0x65, 0xc3, 0x0f, 0xa1, 0xf2, + 0xf0, 0x43, 0x28, 0x7a, 0x1d, 0xf2, 0x5d, 0x7e, 0xbc, 0x36, 0xda, 0x98, 0x96, 0x33, 0x59, 0x96, + 0x0d, 0x48, 0x60, 0x89, 0x57, 0xa6, 0x46, 0xd7, 0x60, 0xaa, 0x69, 0x5b, 0x4d, 0xdf, 0x71, 0xb0, + 0xd5, 0x3c, 0xd8, 0xd4, 0xb7, 0x31, 0x2d, 0x5d, 0xb2, 0xcc, 0x55, 0x22, 0x28, 0xd9, 0x55, 0x22, + 0x28, 0xf4, 0x12, 0xe4, 0x82, 0xd6, 0x0f, 0xad, 0x4e, 0x72, 0xbc, 0x8b, 0x20, 0x80, 0x12, 0x73, + 0x48, 0x49, 0x06, 0x6f, 0xba, 0x57, 0xb9, 0xd3, 0x61, 0x5a, 0x71, 0xf0, 0xc1, 0x4b, 0x60, 0x79, + 0xf0, 0x12, 0x38, 0x12, 0x48, 0x27, 0x47, 0x0b, 0xa4, 0xe5, 0xff, 0xd7, 0x60, 0x26, 0xce, 0x5b, + 0x22, 0x9e, 0xab, 0x3d, 0x16, 0xcf, 0x7d, 0x17, 0xb2, 0x5d, 0xdb, 0xa8, 0xbb, 0x5d, 0xdc, 0xe4, + 0xc1, 0x29, 0xe2, 0xb7, 0x1b, 0xb6, 0xb1, 0xd9, 0xc5, 0xcd, 0xbf, 0x34, 0xbd, 0x9d, 0x95, 0x3d, + 0xdb, 0x34, 0x6e, 0x9a, 0x2e, 0x77, 0xb0, 0x2e, 0xc3, 0x28, 0x99, 0x47, 0x86, 0x03, 0xab, 0x59, + 0x48, 0x33, 0x2d, 0xe5, 0x9f, 0x25, 0xa1, 0x10, 0xf5, 0xd0, 0x1f, 0xd2, 0x54, 0xd0, 0x5d, 0xc8, + 0x98, 0xac, 0xb0, 0xe1, 0x89, 0xc9, 0x9f, 0x49, 0xe1, 0xbb, 0x12, 0x36, 0x46, 0x2b, 0x7b, 0x2f, + 0x54, 0x78, 0x05, 0x44, 0x97, 0x80, 0x4a, 0xe6, 0x9c, 0xaa, 0x64, 0x0e, 0x44, 0x35, 0xc8, 0xb8, + 0xd8, 0xd9, 0x33, 0x9b, 0x98, 0xc7, 0xa1, 0x92, 0x2c, 0xb9, 0x69, 0x3b, 0x98, 0xc8, 0xdc, 0x64, + 0x24, 0xa1, 0x4c, 0xce, 0xa3, 0xca, 0xe4, 0x40, 0xf4, 0x2e, 0xe4, 0x9a, 0xb6, 0xb5, 0x6d, 0xb6, + 0xd6, 0xf5, 0x2e, 0x8f, 0x44, 0xe7, 0xe2, 0xa4, 0x5e, 0x11, 0x44, 0xbc, 0x59, 0x23, 0x3e, 0x23, + 0xcd, 0x9a, 0x80, 0x2a, 0x34, 0xe8, 0x1f, 0x52, 0x00, 0xa1, 0x71, 0xd0, 0xab, 0x90, 0xc7, 0xfb, + 0xb8, 0xe9, 0x7b, 0xb6, 0x23, 0x8e, 0x04, 0xde, 0xfb, 0x14, 0x60, 0x25, 0x86, 0x43, 0x08, 0x25, + 0x7b, 0xd2, 0xd2, 0x3b, 0xd8, 0xed, 0xea, 0x4d, 0xd1, 0x34, 0xa5, 0x83, 0x09, 0x80, 0xf2, 0x9e, + 0x0c, 0x80, 0xe8, 0xcf, 0x21, 0x45, 0xdb, 0xac, 0xac, 0x5f, 0x8a, 0xfa, 0xbd, 0xd2, 0xa4, 0xa5, + 0x36, 0x58, 0x29, 0x1e, 0xbd, 0x09, 0x13, 0xbb, 0x81, 0xe3, 0x91, 0xb1, 0xa5, 0x28, 0x03, 0xcd, + 0x18, 0x43, 0x84, 0x32, 0xba, 0x71, 0x19, 0x8e, 0xb6, 0x21, 0xaf, 0x5b, 0x96, 0xed, 0xd1, 0xe3, + 0x46, 0xf4, 0x50, 0x2f, 0x0d, 0x73, 0xd3, 0xca, 0x4a, 0x48, 0xcb, 0x92, 0x2f, 0x1a, 0x27, 0x24, + 0x09, 0x72, 0x9c, 0x90, 0xc0, 0xa8, 0x06, 0xe9, 0xb6, 0xde, 0xc0, 0x6d, 0x11, 0xdf, 0x9f, 0x1a, + 0xaa, 0xe2, 0x26, 0x25, 0x63, 0xd2, 0xe9, 0xe9, 0xce, 0xf8, 0xe4, 0xd3, 0x9d, 0x41, 0xe6, 0xb7, + 0xa1, 0x10, 0x1d, 0xcf, 0x68, 0xb9, 0xca, 0x25, 0x39, 0x57, 0xc9, 0x3d, 0x34, 0x3b, 0xd2, 0x21, + 0x2f, 0x0d, 0xea, 0x38, 0x54, 0x94, 0xff, 0x45, 0x83, 0x99, 0xb8, 0xbd, 0x8b, 0xd6, 0xa5, 0x1d, + 0xaf, 0xf1, 0x5e, 0x50, 0x8c, 0xab, 0x73, 0xde, 0x21, 0x5b, 0x3d, 0xdc, 0xe8, 0x55, 0x98, 0xb4, + 0x6c, 0x03, 0xd7, 0x75, 0xa2, 0xa0, 0x6d, 0xba, 0x5e, 0x31, 0x41, 0x7b, 0xec, 0xb4, 0x87, 0x44, + 0x30, 0x2b, 0x02, 0x21, 0x71, 0x4f, 0x28, 0x88, 0xf2, 0x7f, 0x69, 0x30, 0x15, 0x69, 0xf1, 0x1e, + 0x39, 0x5f, 0x92, 0xb3, 0x9c, 0xc4, 0x88, 0x59, 0x8e, 0x7a, 0xf4, 0x24, 0x47, 0x3c, 0x7a, 0xfe, + 0x2f, 0x01, 0x79, 0xa9, 0x66, 0x3f, 0xf2, 0xc8, 0xef, 0xc1, 0x14, 0x3f, 0x4a, 0x4d, 0xab, 0xc5, + 0x6a, 0xbb, 0x04, 0x6f, 0x40, 0x0d, 0xdc, 0xc3, 0xac, 0xd9, 0x8d, 0xcd, 0x80, 0x96, 0x96, 0x76, + 0xb4, 0x3f, 0xe8, 0x2a, 0x30, 0x49, 0xc5, 0xa4, 0x8a, 0x41, 0x77, 0x61, 0xce, 0xef, 0x92, 0x8a, + 0xb7, 0xee, 0xf2, 0x1b, 0x8d, 0xba, 0xe5, 0x77, 0x1a, 0x98, 0xcd, 0x7e, 0xac, 0x5a, 0xee, 0xf7, + 0x4a, 0x0b, 0x8c, 0x42, 0x5c, 0x79, 0xdc, 0xa2, 0x78, 0x49, 0xe6, 0x4c, 0x1c, 0x3e, 0xb2, 0x96, + 0xa9, 0x11, 0xd7, 0xf2, 0x3a, 0xa0, 0xc1, 0x5e, 0xbf, 0x62, 0x4b, 0x6d, 0x34, 0x5b, 0x96, 0xff, + 0x5b, 0x83, 0x42, 0xb4, 0x85, 0xff, 0x03, 0x72, 0xaa, 0x7f, 0xd7, 0x20, 0x17, 0x74, 0xf1, 0x8f, + 0x3c, 0xee, 0x67, 0x21, 0xed, 0x60, 0xdd, 0xb5, 0x2d, 0x1e, 0x3c, 0x68, 0x14, 0x64, 0x10, 0x39, + 0x0a, 0x32, 0xc8, 0x21, 0x47, 0x7c, 0x07, 0xc6, 0x99, 0xb9, 0xde, 0x32, 0xdb, 0x1e, 0x76, 0xd0, + 0x55, 0x48, 0xbb, 0x9e, 0xee, 0x61, 0xb7, 0xa8, 0x2d, 0x26, 0x2f, 0x4e, 0x2e, 0xcf, 0x0d, 0xb6, + 0xf9, 0x09, 0x9a, 0x8d, 0x85, 0x51, 0xca, 0x63, 0x61, 0x90, 0xf2, 0xdf, 0x68, 0x30, 0x2e, 0xdf, + 0x66, 0x3c, 0x1e, 0xb1, 0x8f, 0xb6, 0x20, 0xe5, 0xff, 0x08, 0x06, 0xd1, 0x7e, 0x3c, 0x7e, 0xf4, + 0x5d, 0xd8, 0xe3, 0xf7, 0x1a, 0x33, 0x48, 0xd0, 0x3d, 0x3f, 0xea, 0xa0, 0x5b, 0x61, 0xc7, 0x89, + 0xc4, 0x0e, 0x97, 0x06, 0xfa, 0x51, 0x3b, 0x4e, 0xf4, 0x38, 0x50, 0xd8, 0xe5, 0xe3, 0x40, 0x41, + 0x1c, 0x72, 0xbe, 0x9f, 0xa4, 0xe9, 0x7c, 0xc3, 0xfb, 0x95, 0x27, 0xdd, 0xa1, 0x8b, 0xe4, 0x78, + 0xc9, 0x47, 0xc8, 0xf1, 0x9e, 0x83, 0x0c, 0x3d, 0x54, 0x83, 0xf4, 0x8b, 0x3a, 0x08, 0x01, 0xa9, + 0xf7, 0xdb, 0x0c, 0xf2, 0x80, 0x28, 0x3e, 0x76, 0xc4, 0x28, 0x5e, 0x87, 0xd3, 0x3b, 0xba, 0x5b, + 0x17, 0xe7, 0x8e, 0x51, 0xd7, 0xbd, 0x7a, 0x10, 0x01, 0xd3, 0xb4, 0xae, 0xa3, 0x1d, 0xe3, 0x1d, + 0xdd, 0xdd, 0x14, 0x34, 0x2b, 0xde, 0xc6, 0x60, 0x3c, 0x9c, 0x8b, 0xa7, 0x40, 0x5b, 0x30, 0x1b, + 0x2f, 0x3c, 0x43, 0x47, 0x4e, 0x2f, 0x34, 0xdc, 0x07, 0x4a, 0x9e, 0x8e, 0x41, 0xa3, 0x4f, 0x35, + 0x28, 0x92, 0x2c, 0xc7, 0xc1, 0x1f, 0xfa, 0xa6, 0x83, 0x3b, 0xc4, 0x62, 0x75, 0x7b, 0x0f, 0x3b, + 0x6d, 0xfd, 0x80, 0xdf, 0x0d, 0x9e, 0x1f, 0x3c, 0x4d, 0x37, 0x6c, 0xa3, 0x26, 0x31, 0xb0, 0xa9, + 0x75, 0x55, 0xe0, 0x6d, 0x26, 0x44, 0x9e, 0x5a, 0x3c, 0x45, 0xc4, 0x8d, 0xe1, 0x50, 0x1d, 0xc1, + 0xfc, 0x68, 0x1d, 0xc1, 0xb5, 0x54, 0x36, 0x5b, 0xc8, 0x95, 0x7f, 0x9b, 0x80, 0x49, 0xf5, 0xba, + 0xf0, 0x89, 0x6f, 0x82, 0x81, 0xa0, 0x91, 0xfc, 0x4e, 0x82, 0x46, 0xea, 0x50, 0xab, 0x3d, 0x36, + 0xda, 0x6a, 0x97, 0x7f, 0x93, 0x80, 0x09, 0xe5, 0xc6, 0xf4, 0xa7, 0x65, 0x3e, 0x8e, 0x65, 0xfe, + 0xc7, 0x04, 0xcc, 0xc5, 0x0f, 0xf9, 0x58, 0x5a, 0x21, 0xd7, 0x81, 0x14, 0x35, 0x37, 0xc2, 0x7c, + 0x7b, 0x76, 0xa0, 0x13, 0x42, 0x97, 0x4b, 0x54, 0x44, 0x03, 0x97, 0xba, 0x82, 0x1d, 0xdd, 0x85, + 0xbc, 0x29, 0x5d, 0x1f, 0x27, 0xe3, 0x6e, 0xf9, 0xe4, 0x4b, 0x63, 0xd6, 0x1a, 0x1b, 0x72, 0x55, + 0x2c, 0x8b, 0xaa, 0xa6, 0x21, 0x45, 0x0a, 0x82, 0xf2, 0x1e, 0x64, 0xf8, 0x70, 0xd0, 0x8b, 0x90, + 0xa3, 0x67, 0x05, 0xad, 0xee, 0xb5, 0x70, 0x69, 0x09, 0x30, 0xf2, 0x84, 0x2a, 0x2b, 0x60, 0xe8, + 0x65, 0x00, 0x12, 0x1e, 0xf9, 0x29, 0x91, 0xa0, 0xb1, 0x96, 0x76, 0x11, 0xba, 0xb6, 0x31, 0x70, + 0x34, 0xe4, 0x02, 0x60, 0xf9, 0xcb, 0x04, 0xe4, 0xe5, 0x0b, 0xeb, 0x43, 0x29, 0xff, 0x18, 0x44, + 0x87, 0xa7, 0xae, 0x1b, 0x06, 0xf9, 0x17, 0x8b, 0x64, 0x62, 0x69, 0xe8, 0x22, 0x89, 0xff, 0xaf, + 0x08, 0x0e, 0x56, 0xcf, 0xd3, 0x47, 0x39, 0x66, 0x04, 0x25, 0x69, 0x2d, 0x44, 0x71, 0xf3, 0xbb, + 0x30, 0x1b, 0x2b, 0x4a, 0xae, 0xc2, 0xc7, 0x1e, 0x57, 0x15, 0xfe, 0x65, 0x1a, 0x66, 0x63, 0x1f, + 0x0a, 0x3c, 0xf1, 0x88, 0xa1, 0xee, 0xa0, 0xe4, 0x63, 0xd9, 0x41, 0x7f, 0xab, 0xc5, 0x59, 0x96, + 0xdd, 0x12, 0xbe, 0x3a, 0xc2, 0xeb, 0x89, 0xc7, 0x65, 0x63, 0xd5, 0x2d, 0xc7, 0x0e, 0xb5, 0x27, + 0xd2, 0xa3, 0xee, 0x09, 0xf4, 0x3c, 0x6b, 0xa8, 0x50, 0x5d, 0xec, 0x06, 0x4f, 0x44, 0x88, 0x88, + 0xaa, 0x0c, 0x07, 0xa1, 0x37, 0x61, 0x42, 0x70, 0xb0, 0x36, 0x5e, 0x36, 0xec, 0xb1, 0x71, 0x9a, + 0x68, 0x27, 0x6f, 0x5c, 0x86, 0x47, 0xa2, 0x70, 0xee, 0x50, 0x51, 0x18, 0x46, 0xbc, 0x6c, 0xfc, + 0x4e, 0xf7, 0xcb, 0xef, 0x12, 0x30, 0x15, 0x79, 0x27, 0xf4, 0xd3, 0xd9, 0x7a, 0x1c, 0x67, 0xeb, + 0xff, 0x6a, 0x90, 0x0b, 0x9e, 0xc3, 0x1d, 0xb9, 0x34, 0x5c, 0x81, 0x34, 0x66, 0x4f, 0xb2, 0x58, + 0x18, 0x9f, 0x8e, 0x3c, 0x99, 0x25, 0x38, 0xfe, 0x48, 0x36, 0xf2, 0x0a, 0xab, 0xc6, 0x19, 0x0f, + 0x59, 0xf4, 0xfd, 0x4f, 0x42, 0x14, 0x7d, 0xe1, 0x4c, 0x9e, 0xa8, 0xb3, 0x84, 0x2b, 0x91, 0x7c, + 0x3c, 0x2b, 0x71, 0xbc, 0x6e, 0xf0, 0xcf, 0x00, 0x63, 0x74, 0x4c, 0x68, 0x19, 0xb2, 0x1e, 0x76, + 0x3a, 0xa6, 0xa5, 0xb7, 0xe9, 0xd2, 0x65, 0x19, 0xb7, 0x80, 0xc9, 0xdc, 0x02, 0x86, 0x76, 0x60, + 0x2a, 0x6c, 0xfd, 0x53, 0x31, 0xf1, 0x6f, 0x85, 0xdf, 0x56, 0x89, 0xd8, 0x35, 0x64, 0x84, 0x53, + 0x7d, 0xec, 0x13, 0x41, 0x22, 0x03, 0x26, 0x9b, 0xb6, 0xe5, 0xe9, 0xa6, 0x85, 0x1d, 0xa6, 0x28, + 0x19, 0xf7, 0x56, 0xf2, 0x8a, 0x42, 0xc3, 0x7a, 0xa1, 0x2a, 0x9f, 0xfa, 0x56, 0x52, 0xc5, 0xa1, + 0x0f, 0x60, 0x42, 0x14, 0xe1, 0x4c, 0x49, 0x2a, 0xee, 0xad, 0xe4, 0xaa, 0x4c, 0xc2, 0x36, 0xb8, + 0xc2, 0xa5, 0xbe, 0x95, 0x54, 0x50, 0xa8, 0x0d, 0x85, 0xae, 0x6d, 0x6c, 0x59, 0xbc, 0xf4, 0xd4, + 0x1b, 0x6d, 0xcc, 0xef, 0x9b, 0x16, 0x06, 0x92, 0x4d, 0x85, 0x8a, 0x1d, 0x82, 0x51, 0x5e, 0xf5, + 0xf5, 0x71, 0x14, 0x8b, 0xde, 0x87, 0xf1, 0x36, 0xd6, 0x5d, 0xbc, 0xba, 0xdf, 0x35, 0x1d, 0x6c, + 0xc4, 0xbf, 0x15, 0xbe, 0x29, 0x51, 0xb0, 0x23, 0x48, 0xe6, 0x51, 0x9f, 0x48, 0xc9, 0x18, 0x62, + 0xfd, 0x8e, 0xbe, 0x5f, 0xf3, 0x2d, 0x77, 0x75, 0x9f, 0xbf, 0xfb, 0xcc, 0xc4, 0x59, 0x7f, 0x5d, + 0x25, 0x62, 0xd6, 0x8f, 0x70, 0xaa, 0xd6, 0x8f, 0x20, 0xd1, 0x4d, 0x7a, 0xc2, 0x32, 0x93, 0xb0, + 0x37, 0xc3, 0x73, 0x03, 0xab, 0xc5, 0xac, 0xc1, 0xda, 0xb1, 0xfc, 0x4b, 0x11, 0x1a, 0x48, 0xe0, + 0x36, 0xa0, 0xd3, 0xae, 0x61, 0xcf, 0x77, 0x2c, 0x6c, 0xf0, 0x96, 0xc0, 0xa0, 0x0d, 0x14, 0xaa, + 0xc0, 0x06, 0x0a, 0x74, 0xc0, 0x06, 0x0a, 0x96, 0xf8, 0x54, 0xd7, 0x36, 0xee, 0xb0, 0x2d, 0xe3, + 0x05, 0x8f, 0x88, 0xcf, 0x0c, 0xa8, 0x0a, 0x49, 0x98, 0x4f, 0x29, 0x5c, 0xaa, 0x4f, 0x29, 0x28, + 0xf4, 0x31, 0xcc, 0x44, 0xde, 0x58, 0xb2, 0x95, 0xca, 0xc7, 0x5d, 0xe7, 0xae, 0xc5, 0x50, 0xb2, + 0xfe, 0x50, 0x9c, 0x0c, 0x45, 0x6d, 0xac, 0x16, 0xa2, 0xbd, 0xa5, 0x5b, 0xad, 0x35, 0xbb, 0xa1, + 0x7a, 0xf5, 0x78, 0x9c, 0xf6, 0x6b, 0x31, 0x94, 0x4c, 0x7b, 0x9c, 0x0c, 0x55, 0x7b, 0x1c, 0x45, + 0xf0, 0x9e, 0x92, 0xa4, 0xa0, 0xc1, 0xbb, 0xe3, 0xb8, 0xf7, 0x94, 0x8c, 0x40, 0x7a, 0x4f, 0xc9, + 0x00, 0x31, 0xef, 0x29, 0x39, 0x65, 0x56, 0xb4, 0x68, 0xcb, 0x5f, 0x68, 0x30, 0x15, 0x89, 0x60, + 0xe8, 0x0d, 0x08, 0x9e, 0xc9, 0xdd, 0x39, 0xe8, 0x8a, 0xd2, 0x47, 0x79, 0x56, 0x47, 0xe0, 0x71, + 0xcf, 0xea, 0x08, 0x1c, 0xdd, 0x04, 0x08, 0xce, 0xfe, 0x07, 0x1d, 0x35, 0x34, 0xef, 0x0e, 0x29, + 0xe5, 0xbc, 0x3b, 0x84, 0x96, 0xef, 0xa7, 0x20, 0x2b, 0xb6, 0xc0, 0xb1, 0x94, 0xc6, 0x4b, 0x90, + 0xe9, 0x60, 0x97, 0x3e, 0xaf, 0x4b, 0x84, 0x19, 0x2e, 0x07, 0xc9, 0x19, 0x2e, 0x07, 0xa9, 0x09, + 0x78, 0xf2, 0x50, 0x09, 0x78, 0x6a, 0xe4, 0x04, 0x1c, 0xd3, 0xe7, 0x2e, 0x52, 0x20, 0x17, 0xb7, + 0xce, 0x0f, 0x3e, 0x1d, 0xc4, 0x63, 0x18, 0x99, 0x31, 0xf2, 0x18, 0x46, 0x46, 0xa1, 0x5d, 0x38, + 0x29, 0xdd, 0x8c, 0xf3, 0xfe, 0x3d, 0x09, 0xa9, 0x93, 0xc3, 0xdf, 0x16, 0xd5, 0x28, 0x15, 0x0b, + 0x1c, 0xbb, 0x11, 0xa8, 0x5c, 0xc1, 0x44, 0x71, 0xc4, 0xc1, 0x0c, 0xdc, 0xf0, 0x5b, 0xeb, 0x7c, + 0xd9, 0x33, 0xa1, 0x83, 0xc9, 0x70, 0xd9, 0xc1, 0x64, 0x38, 0x6d, 0x05, 0xaa, 0xf3, 0x3d, 0x16, + 0xc7, 0x78, 0x11, 0x72, 0x78, 0xdf, 0xf4, 0xea, 0x4d, 0xdb, 0xc0, 0xbc, 0x8d, 0x40, 0xed, 0x4c, + 0x80, 0x57, 0x6c, 0x43, 0xb1, 0xb3, 0x80, 0xc9, 0xde, 0x94, 0x1c, 0xc9, 0x9b, 0xc2, 0xeb, 0x92, + 0xd4, 0x08, 0xd7, 0x25, 0xb1, 0x76, 0xca, 0x1d, 0x8f, 0x9d, 0xca, 0x5f, 0x27, 0xa0, 0x10, 0x3d, + 0x27, 0xbe, 0x1f, 0x5b, 0x50, 0xdd, 0x4d, 0xc9, 0x91, 0x77, 0xd3, 0x9b, 0x30, 0x41, 0x12, 0x49, + 0xdd, 0xf3, 0xf8, 0xef, 0x00, 0x52, 0x34, 0x1b, 0x64, 0xb1, 0xcd, 0xb7, 0x56, 0x04, 0x5c, 0x89, + 0x6d, 0x12, 0x7c, 0xc0, 0x75, 0xc7, 0x1e, 0xd1, 0x75, 0x3f, 0x4d, 0xc0, 0x84, 0x72, 0x1e, 0xfe, + 0xf8, 0x42, 0x5a, 0x79, 0x0a, 0x26, 0x94, 0x34, 0xb3, 0xfc, 0x19, 0xf3, 0x33, 0xf5, 0xf4, 0xfb, + 0xf1, 0xad, 0xcb, 0x24, 0x8c, 0xcb, 0xf9, 0x6a, 0xb9, 0x0a, 0x53, 0x91, 0xf4, 0x52, 0x9e, 0x80, + 0x36, 0xca, 0x04, 0xca, 0x73, 0x30, 0x13, 0x97, 0x15, 0x95, 0xaf, 0xc1, 0x4c, 0x5c, 0xbe, 0xf2, + 0xe8, 0x0a, 0xde, 0xe0, 0xef, 0x43, 0x58, 0x66, 0xf1, 0xe8, 0xfc, 0xff, 0x96, 0x12, 0x4d, 0x91, + 0xf0, 0x57, 0x34, 0xef, 0x41, 0xa1, 0x2b, 0x3e, 0xea, 0x0f, 0xad, 0xdd, 0x69, 0x21, 0x14, 0xd0, + 0xaf, 0x45, 0x4a, 0xd7, 0x49, 0x15, 0xa3, 0xca, 0xe6, 0xd5, 0x74, 0x62, 0x44, 0xd9, 0xb5, 0x48, + 0x59, 0x3d, 0xa9, 0x62, 0xd0, 0x5f, 0xc3, 0x49, 0xf1, 0x52, 0x75, 0x0f, 0x8b, 0x81, 0x27, 0x87, + 0x0a, 0x67, 0x3f, 0xfa, 0x09, 0x18, 0xa2, 0x23, 0x9f, 0x8a, 0xa0, 0x22, 0xe2, 0xf9, 0xd8, 0x53, + 0xa3, 0x8a, 0x8f, 0x0e, 0x7e, 0x2a, 0x82, 0x42, 0x1b, 0x30, 0x13, 0x5d, 0x75, 0xa9, 0xdc, 0x2e, + 0xf5, 0x7b, 0xa5, 0x33, 0xea, 0x5a, 0xaa, 0x75, 0xf7, 0xc9, 0x01, 0xa4, 0x2a, 0x51, 0x2a, 0xe0, + 0xd3, 0x31, 0x12, 0x6b, 0x83, 0x95, 0xfc, 0xc9, 0x01, 0x64, 0xf9, 0x73, 0x0d, 0xa6, 0x22, 0xbf, + 0x9d, 0x42, 0x57, 0x21, 0x4b, 0x7f, 0xdc, 0xfc, 0x60, 0x2f, 0xa1, 0x7e, 0x48, 0xe9, 0x94, 0x55, + 0xc8, 0x70, 0x10, 0x7a, 0x09, 0x72, 0xc1, 0x4f, 0xac, 0xf8, 0xf3, 0x17, 0xb6, 0x67, 0x05, 0x50, + 0xd9, 0xb3, 0x02, 0x58, 0xfe, 0xbb, 0x04, 0x9c, 0x1e, 0xfa, 0xbb, 0xaa, 0x27, 0xde, 0xb0, 0x39, + 0x54, 0xdf, 0x29, 0xd2, 0x6d, 0x49, 0x8d, 0xd8, 0x6d, 0xf9, 0x5c, 0x83, 0xb9, 0xf8, 0xdf, 0x5d, + 0x1d, 0xb9, 0x03, 0xa7, 0x4e, 0x23, 0x31, 0xfa, 0x2b, 0xa3, 0x71, 0xf9, 0x27, 0x52, 0x47, 0x1e, + 0xc6, 0x25, 0x18, 0xeb, 0xda, 0x76, 0xdb, 0xe5, 0x8f, 0x00, 0x29, 0x29, 0x05, 0xc8, 0xa4, 0x14, + 0x70, 0xc8, 0x86, 0xdf, 0x1f, 0x35, 0x11, 0x0b, 0xc3, 0x5f, 0x82, 0xfd, 0x88, 0x5c, 0xe8, 0x99, + 0xe7, 0x21, 0x2b, 0x9e, 0x38, 0x21, 0x80, 0xf4, 0x3b, 0x5b, 0xab, 0x5b, 0xab, 0x57, 0x0b, 0x27, + 0x50, 0x1e, 0x32, 0x1b, 0xab, 0xb7, 0xae, 0xde, 0xb8, 0x75, 0xad, 0xa0, 0x91, 0x8f, 0xda, 0xd6, + 0xad, 0x5b, 0xe4, 0x23, 0xf1, 0xcc, 0x4d, 0xf9, 0x25, 0x39, 0xaf, 0x2e, 0xc6, 0x21, 0xbb, 0xd2, + 0xed, 0xd2, 0x63, 0x8e, 0xf1, 0xae, 0xee, 0x99, 0xe4, 0x6c, 0x2a, 0x68, 0x28, 0x03, 0xc9, 0xdb, + 0xb7, 0xd7, 0x0b, 0x09, 0x34, 0x03, 0x85, 0xab, 0x58, 0x37, 0xda, 0xa6, 0x85, 0xc5, 0xd9, 0x5a, + 0x48, 0x56, 0xef, 0x7d, 0xf5, 0xcd, 0x82, 0xf6, 0xf5, 0x37, 0x0b, 0xda, 0xaf, 0xbe, 0x59, 0xd0, + 0xee, 0x7f, 0xbb, 0x70, 0xe2, 0xeb, 0x6f, 0x17, 0x4e, 0xfc, 0xe2, 0xdb, 0x85, 0x13, 0xef, 0x3d, + 0x2f, 0xfd, 0xdd, 0x09, 0xb6, 0x7e, 0x5d, 0xc7, 0x26, 0x47, 0x1e, 0xff, 0x5a, 0x8a, 0xfe, 0xa5, + 0x8d, 0x7f, 0x4d, 0x9c, 0x5b, 0xa1, 0x9f, 0x1b, 0x8c, 0xae, 0x72, 0xc3, 0xae, 0x30, 0x00, 0xfd, + 0x63, 0x09, 0x6e, 0x23, 0x4d, 0xff, 0x28, 0xc2, 0x8b, 0x7f, 0x0a, 0x00, 0x00, 0xff, 0xff, 0x9f, + 0x35, 0xa0, 0x61, 0xa4, 0x43, 0x00, 0x00, } func (m *EventSequence) Marshal() (dAtA []byte, err error) { @@ -5008,11 +4997,6 @@ func (m *SubmitJob) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x72 } - if m.QueueTtlSeconds != 0 { - i = encodeVarintEvents(dAtA, i, uint64(m.QueueTtlSeconds)) - i-- - dAtA[i] = 0x68 - } if m.IsDuplicate { i-- if m.IsDuplicate { @@ -8221,9 +8205,6 @@ func (m *SubmitJob) Size() (n int) { if m.IsDuplicate { n += 2 } - if m.QueueTtlSeconds != 0 { - n += 1 + sovEvents(uint64(m.QueueTtlSeconds)) - } l = len(m.JobIdStr) if l > 0 { n += 1 + l + sovEvents(uint64(l)) @@ -11414,25 +11395,6 @@ func (m *SubmitJob) Unmarshal(dAtA []byte) error { } } m.IsDuplicate = bool(v != 0) - case 13: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field QueueTtlSeconds", wireType) - } - m.QueueTtlSeconds = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEvents - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.QueueTtlSeconds |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } case 14: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field JobIdStr", wireType) diff --git a/pkg/armadaevents/events.proto b/pkg/armadaevents/events.proto index b136baa8719..ab67ecfb217 100644 --- a/pkg/armadaevents/events.proto +++ b/pkg/armadaevents/events.proto @@ -169,8 +169,6 @@ message SubmitJob { string scheduler = 11; // Indicates whether job is a duplicate bool isDuplicate = 12; - // Queuing TTL for this job in seconds. If this job queues for more than this duration it will be cancelled. Zero indicates an infinite lifetime. - int64 queue_ttl_seconds = 13; // This is a string representation of the job_id. Eventually we will retire the job_id field and just use strings string job_id_str = 14; } From b1cd2c64777335a08442d27e1f5c38f358f6bf15 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 4 Jun 2024 21:07:52 +0100 Subject: [PATCH 2/7] lint Signed-off-by: Chris Martin --- internal/scheduler/scheduler_test.go | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/internal/scheduler/scheduler_test.go b/internal/scheduler/scheduler_test.go index e75acb1ce71..a2be45ae1ce 100644 --- a/internal/scheduler/scheduler_test.go +++ b/internal/scheduler/scheduler_test.go @@ -116,19 +116,6 @@ var ( Version: 2, } schedulingInfoWithUpdatedPriorityBytes = protoutil.MustMarshall(schedulingInfoWithUpdatedPriority) - schedulingInfoWithQueueTtl = &schedulerobjects.JobSchedulingInfo{ - AtMostOnce: true, - ObjectRequirements: []*schedulerobjects.ObjectRequirements{ - { - Requirements: &schedulerobjects.ObjectRequirements_PodRequirements{ - PodRequirements: &schedulerobjects.PodRequirements{ - Priority: int32(10), - }, - }, - }, - }, - Version: 1, - } schedulerMetrics = NewSchedulerMetrics(configuration.SchedulerMetricsConfig{ ScheduleCycleTimeHistogramSettings: configuration.HistogramConfig{ From a19e16a839ffc1c2a8010f7331e8646d8477cb2c Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 4 Jun 2024 21:21:01 +0100 Subject: [PATCH 3/7] fix swagger Signed-off-by: Chris Martin --- pkg/api/api.swagger.go | 2 +- pkg/api/api.swagger.json | 2 +- pkg/api/binoculars/api.swagger.go | 2 +- pkg/api/binoculars/api.swagger.json | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/api/api.swagger.go b/pkg/api/api.swagger.go index 702194044db..c3e8f925ffe 100644 --- a/pkg/api/api.swagger.go +++ b/pkg/api/api.swagger.go @@ -5289,7 +5289,7 @@ func SwaggerJsonTemplate() string { " \"x-go-package\": \"k8s.io/api/core/v1\"\n" + " },\n" + " \"v1Time\": {\n" + - " \"description\": \"Programs using times should typically store and pass them as values,\\nnot pointers. That is, time variables and struct fields should be of\\ntype time.Time, not *time.Time.\\n\\nA Time value can be used by multiple goroutines simultaneously except\\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\\nUnmarshalText are not concurrency-safe.\\n\\nTime instants can be compared using the Before, After, and Equal methods.\\nThe Sub method subtracts two instants, producing a Duration.\\nThe Add method adds a Time and a Duration, producing a Time.\\n\\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\\nAs this time is unlikely to come up in practice, the IsZero method gives\\na simple way of detecting a time that has not been initialized explicitly.\\n\\nEach time has an associated Location. The methods Local, UTC, and In return a\\nTime with a specific Location. Changing the Location of a Time value with\\nthese methods does not change the actual instant it represents, only the time\\nzone in which to interpret it.\\n\\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\\nthe location name. They therefore lose information about Daylight Saving Time.\\n\\nIn addition to the required “wall clock” reading, a Time may contain an optional\\nreading of the current process's monotonic clock, to provide additional precision\\nfor comparison or subtraction.\\nSee the “Monotonic Clocks” section in the package documentation for details.\\n\\nNote that the Go == operator compares not just the time instant but also the\\nLocation and the monotonic clock reading. Therefore, Time values should not\\nbe used as map or database keys without first guaranteeing that the\\nidentical Location has been set for all values, which can be achieved\\nthrough use of the UTC or Local method, and that the monotonic clock reading\\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\\nto t == u, since t.Equal uses the most accurate comparison available and\\ncorrectly handles the case when only one of its arguments has a monotonic\\nclock reading.\",\n" + + " \"description\": \"Programs using times should typically store and pass them as values,\\nnot pointers. That is, time variables and struct fields should be of\\ntype time.Time, not *time.Time.\\n\\nA Time value can be used by multiple goroutines simultaneously except\\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\\nUnmarshalText are not concurrency-safe.\\n\\nTime instants can be compared using the Before, After, and Equal methods.\\nThe Sub method subtracts two instants, producing a Duration.\\nThe Add method adds a Time and a Duration, producing a Time.\\n\\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\\nAs this time is unlikely to come up in practice, the IsZero method gives\\na simple way of detecting a time that has not been initialized explicitly.\\n\\nEach Time has associated with it a Location, consulted when computing the\\npresentation form of the time, such as in the Format, Hour, and Year methods.\\nThe methods Local, UTC, and In return a Time with a specific location.\\nChanging the location in this way changes only the presentation; it does not\\nchange the instant in time being denoted and therefore does not affect the\\ncomputations described in earlier paragraphs.\\n\\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\\nthe location name. They therefore lose information about Daylight Saving Time.\\n\\nIn addition to the required “wall clock” reading, a Time may contain an optional\\nreading of the current process's monotonic clock, to provide additional precision\\nfor comparison or subtraction.\\nSee the “Monotonic Clocks” section in the package documentation for details.\\n\\nNote that the Go == operator compares not just the time instant but also the\\nLocation and the monotonic clock reading. Therefore, Time values should not\\nbe used as map or database keys without first guaranteeing that the\\nidentical Location has been set for all values, which can be achieved\\nthrough use of the UTC or Local method, and that the monotonic clock reading\\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\\nto t == u, since t.Equal uses the most accurate comparison available and\\ncorrectly handles the case when only one of its arguments has a monotonic\\nclock reading.\",\n" + " \"type\": \"string\",\n" + " \"format\": \"date-time\",\n" + " \"title\": \"A Time represents an instant in time with nanosecond precision.\",\n" + diff --git a/pkg/api/api.swagger.json b/pkg/api/api.swagger.json index f7b96eb063e..a463a289fce 100644 --- a/pkg/api/api.swagger.json +++ b/pkg/api/api.swagger.json @@ -5278,7 +5278,7 @@ "x-go-package": "k8s.io/api/core/v1" }, "v1Time": { - "description": "Programs using times should typically store and pass them as values,\nnot pointers. That is, time variables and struct fields should be of\ntype time.Time, not *time.Time.\n\nA Time value can be used by multiple goroutines simultaneously except\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\nUnmarshalText are not concurrency-safe.\n\nTime instants can be compared using the Before, After, and Equal methods.\nThe Sub method subtracts two instants, producing a Duration.\nThe Add method adds a Time and a Duration, producing a Time.\n\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\nAs this time is unlikely to come up in practice, the IsZero method gives\na simple way of detecting a time that has not been initialized explicitly.\n\nEach time has an associated Location. The methods Local, UTC, and In return a\nTime with a specific Location. Changing the Location of a Time value with\nthese methods does not change the actual instant it represents, only the time\nzone in which to interpret it.\n\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\nthe location name. They therefore lose information about Daylight Saving Time.\n\nIn addition to the required “wall clock” reading, a Time may contain an optional\nreading of the current process's monotonic clock, to provide additional precision\nfor comparison or subtraction.\nSee the “Monotonic Clocks” section in the package documentation for details.\n\nNote that the Go == operator compares not just the time instant but also the\nLocation and the monotonic clock reading. Therefore, Time values should not\nbe used as map or database keys without first guaranteeing that the\nidentical Location has been set for all values, which can be achieved\nthrough use of the UTC or Local method, and that the monotonic clock reading\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\nto t == u, since t.Equal uses the most accurate comparison available and\ncorrectly handles the case when only one of its arguments has a monotonic\nclock reading.", + "description": "Programs using times should typically store and pass them as values,\nnot pointers. That is, time variables and struct fields should be of\ntype time.Time, not *time.Time.\n\nA Time value can be used by multiple goroutines simultaneously except\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\nUnmarshalText are not concurrency-safe.\n\nTime instants can be compared using the Before, After, and Equal methods.\nThe Sub method subtracts two instants, producing a Duration.\nThe Add method adds a Time and a Duration, producing a Time.\n\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\nAs this time is unlikely to come up in practice, the IsZero method gives\na simple way of detecting a time that has not been initialized explicitly.\n\nEach Time has associated with it a Location, consulted when computing the\npresentation form of the time, such as in the Format, Hour, and Year methods.\nThe methods Local, UTC, and In return a Time with a specific location.\nChanging the location in this way changes only the presentation; it does not\nchange the instant in time being denoted and therefore does not affect the\ncomputations described in earlier paragraphs.\n\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\nthe location name. They therefore lose information about Daylight Saving Time.\n\nIn addition to the required “wall clock” reading, a Time may contain an optional\nreading of the current process's monotonic clock, to provide additional precision\nfor comparison or subtraction.\nSee the “Monotonic Clocks” section in the package documentation for details.\n\nNote that the Go == operator compares not just the time instant but also the\nLocation and the monotonic clock reading. Therefore, Time values should not\nbe used as map or database keys without first guaranteeing that the\nidentical Location has been set for all values, which can be achieved\nthrough use of the UTC or Local method, and that the monotonic clock reading\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\nto t == u, since t.Equal uses the most accurate comparison available and\ncorrectly handles the case when only one of its arguments has a monotonic\nclock reading.", "type": "string", "format": "date-time", "title": "A Time represents an instant in time with nanosecond precision.", diff --git a/pkg/api/binoculars/api.swagger.go b/pkg/api/binoculars/api.swagger.go index 4a9905f6feb..206e611b3c9 100644 --- a/pkg/api/binoculars/api.swagger.go +++ b/pkg/api/binoculars/api.swagger.go @@ -238,7 +238,7 @@ func SwaggerJsonTemplate() string { " \"x-go-package\": \"k8s.io/api/core/v1\"\n" + " },\n" + " \"v1Time\": {\n" + - " \"description\": \"Programs using times should typically store and pass them as values,\\nnot pointers. That is, time variables and struct fields should be of\\ntype time.Time, not *time.Time.\\n\\nA Time value can be used by multiple goroutines simultaneously except\\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\\nUnmarshalText are not concurrency-safe.\\n\\nTime instants can be compared using the Before, After, and Equal methods.\\nThe Sub method subtracts two instants, producing a Duration.\\nThe Add method adds a Time and a Duration, producing a Time.\\n\\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\\nAs this time is unlikely to come up in practice, the IsZero method gives\\na simple way of detecting a time that has not been initialized explicitly.\\n\\nEach time has an associated Location. The methods Local, UTC, and In return a\\nTime with a specific Location. Changing the Location of a Time value with\\nthese methods does not change the actual instant it represents, only the time\\nzone in which to interpret it.\\n\\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\\nthe location name. They therefore lose information about Daylight Saving Time.\\n\\nIn addition to the required “wall clock” reading, a Time may contain an optional\\nreading of the current process's monotonic clock, to provide additional precision\\nfor comparison or subtraction.\\nSee the “Monotonic Clocks” section in the package documentation for details.\\n\\nNote that the Go == operator compares not just the time instant but also the\\nLocation and the monotonic clock reading. Therefore, Time values should not\\nbe used as map or database keys without first guaranteeing that the\\nidentical Location has been set for all values, which can be achieved\\nthrough use of the UTC or Local method, and that the monotonic clock reading\\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\\nto t == u, since t.Equal uses the most accurate comparison available and\\ncorrectly handles the case when only one of its arguments has a monotonic\\nclock reading.\",\n" + + " \"description\": \"Programs using times should typically store and pass them as values,\\nnot pointers. That is, time variables and struct fields should be of\\ntype time.Time, not *time.Time.\\n\\nA Time value can be used by multiple goroutines simultaneously except\\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\\nUnmarshalText are not concurrency-safe.\\n\\nTime instants can be compared using the Before, After, and Equal methods.\\nThe Sub method subtracts two instants, producing a Duration.\\nThe Add method adds a Time and a Duration, producing a Time.\\n\\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\\nAs this time is unlikely to come up in practice, the IsZero method gives\\na simple way of detecting a time that has not been initialized explicitly.\\n\\nEach Time has associated with it a Location, consulted when computing the\\npresentation form of the time, such as in the Format, Hour, and Year methods.\\nThe methods Local, UTC, and In return a Time with a specific location.\\nChanging the location in this way changes only the presentation; it does not\\nchange the instant in time being denoted and therefore does not affect the\\ncomputations described in earlier paragraphs.\\n\\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\\nthe location name. They therefore lose information about Daylight Saving Time.\\n\\nIn addition to the required “wall clock” reading, a Time may contain an optional\\nreading of the current process's monotonic clock, to provide additional precision\\nfor comparison or subtraction.\\nSee the “Monotonic Clocks” section in the package documentation for details.\\n\\nNote that the Go == operator compares not just the time instant but also the\\nLocation and the monotonic clock reading. Therefore, Time values should not\\nbe used as map or database keys without first guaranteeing that the\\nidentical Location has been set for all values, which can be achieved\\nthrough use of the UTC or Local method, and that the monotonic clock reading\\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\\nto t == u, since t.Equal uses the most accurate comparison available and\\ncorrectly handles the case when only one of its arguments has a monotonic\\nclock reading.\",\n" + " \"type\": \"string\",\n" + " \"format\": \"date-time\",\n" + " \"title\": \"A Time represents an instant in time with nanosecond precision.\",\n" + diff --git a/pkg/api/binoculars/api.swagger.json b/pkg/api/binoculars/api.swagger.json index adf9a3af046..e1ee15d781b 100644 --- a/pkg/api/binoculars/api.swagger.json +++ b/pkg/api/binoculars/api.swagger.json @@ -227,7 +227,7 @@ "x-go-package": "k8s.io/api/core/v1" }, "v1Time": { - "description": "Programs using times should typically store and pass them as values,\nnot pointers. That is, time variables and struct fields should be of\ntype time.Time, not *time.Time.\n\nA Time value can be used by multiple goroutines simultaneously except\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\nUnmarshalText are not concurrency-safe.\n\nTime instants can be compared using the Before, After, and Equal methods.\nThe Sub method subtracts two instants, producing a Duration.\nThe Add method adds a Time and a Duration, producing a Time.\n\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\nAs this time is unlikely to come up in practice, the IsZero method gives\na simple way of detecting a time that has not been initialized explicitly.\n\nEach time has an associated Location. The methods Local, UTC, and In return a\nTime with a specific Location. Changing the Location of a Time value with\nthese methods does not change the actual instant it represents, only the time\nzone in which to interpret it.\n\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\nthe location name. They therefore lose information about Daylight Saving Time.\n\nIn addition to the required “wall clock” reading, a Time may contain an optional\nreading of the current process's monotonic clock, to provide additional precision\nfor comparison or subtraction.\nSee the “Monotonic Clocks” section in the package documentation for details.\n\nNote that the Go == operator compares not just the time instant but also the\nLocation and the monotonic clock reading. Therefore, Time values should not\nbe used as map or database keys without first guaranteeing that the\nidentical Location has been set for all values, which can be achieved\nthrough use of the UTC or Local method, and that the monotonic clock reading\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\nto t == u, since t.Equal uses the most accurate comparison available and\ncorrectly handles the case when only one of its arguments has a monotonic\nclock reading.", + "description": "Programs using times should typically store and pass them as values,\nnot pointers. That is, time variables and struct fields should be of\ntype time.Time, not *time.Time.\n\nA Time value can be used by multiple goroutines simultaneously except\nthat the methods GobDecode, UnmarshalBinary, UnmarshalJSON and\nUnmarshalText are not concurrency-safe.\n\nTime instants can be compared using the Before, After, and Equal methods.\nThe Sub method subtracts two instants, producing a Duration.\nThe Add method adds a Time and a Duration, producing a Time.\n\nThe zero value of type Time is January 1, year 1, 00:00:00.000000000 UTC.\nAs this time is unlikely to come up in practice, the IsZero method gives\na simple way of detecting a time that has not been initialized explicitly.\n\nEach Time has associated with it a Location, consulted when computing the\npresentation form of the time, such as in the Format, Hour, and Year methods.\nThe methods Local, UTC, and In return a Time with a specific location.\nChanging the location in this way changes only the presentation; it does not\nchange the instant in time being denoted and therefore does not affect the\ncomputations described in earlier paragraphs.\n\nRepresentations of a Time value saved by the GobEncode, MarshalBinary,\nMarshalJSON, and MarshalText methods store the Time.Location's offset, but not\nthe location name. They therefore lose information about Daylight Saving Time.\n\nIn addition to the required “wall clock” reading, a Time may contain an optional\nreading of the current process's monotonic clock, to provide additional precision\nfor comparison or subtraction.\nSee the “Monotonic Clocks” section in the package documentation for details.\n\nNote that the Go == operator compares not just the time instant but also the\nLocation and the monotonic clock reading. Therefore, Time values should not\nbe used as map or database keys without first guaranteeing that the\nidentical Location has been set for all values, which can be achieved\nthrough use of the UTC or Local method, and that the monotonic clock reading\nhas been stripped by setting t = t.Round(0). In general, prefer t.Equal(u)\nto t == u, since t.Equal uses the most accurate comparison available and\ncorrectly handles the case when only one of its arguments has a monotonic\nclock reading.", "type": "string", "format": "date-time", "title": "A Time represents an instant in time with nanosecond precision.", From 5752128ad3e0f746e338de190dade733fbfd5a5b Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 4 Jun 2024 21:26:26 +0100 Subject: [PATCH 4/7] fix test Signed-off-by: Chris Martin --- internal/scheduler/jobdb/jobdb.go | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/internal/scheduler/jobdb/jobdb.go b/internal/scheduler/jobdb/jobdb.go index 1829304eb7f..582d1260924 100644 --- a/internal/scheduler/jobdb/jobdb.go +++ b/internal/scheduler/jobdb/jobdb.go @@ -241,9 +241,6 @@ type Txn struct { jobsByRunId *immutable.Map[uuid.UUID, string] // Queued jobs for each queue. Stored in the order in which they should be scheduled. jobsByQueue map[string]immutable.SortedSet[*Job] - // Queued jobs for each queue ordered by remaining time-to-live. - // TODO: The ordering is wrong. Since we call time.Now() in the compare function. - queuedJobsByTtl *immutable.SortedSet[*Job] // Jobs that require submit checking unvalidatedJobs *immutable.Set[*Job] // The jobDb from which this transaction was created. @@ -379,9 +376,6 @@ func (txn *Txn) Upsert(jobs []*Job) error { if ok { txn.jobsByQueue[existingJob.queue] = existingQueue.Delete(existingJob) } - newQueuedJobsByTtl := txn.queuedJobsByTtl.Delete(existingJob) - txn.queuedJobsByTtl = &newQueuedJobsByTtl - if !existingJob.Validated() { newUnvalidatedJobs := txn.unvalidatedJobs.Delete(existingJob) txn.unvalidatedJobs = &newUnvalidatedJobs @@ -495,12 +489,7 @@ func (txn *Txn) QueuedJobs(queue string) *immutable.SortedSetIterator[*Job] { } } -// QueuedJobsByTtl returns an iterator for jobs ordered by queue ttl time - the closest to expiry first -func (txn *Txn) QueuedJobsByTtl() *immutable.SortedSetIterator[*Job] { - return txn.queuedJobsByTtl.Iterator() -} - -// UnvalidatedJobs returns an iterator for jobs ordered by queue ttl time - the closest to expiry first +// UnvalidatedJobs returns an iterator for jobs that have not yet been validated func (txn *Txn) UnvalidatedJobs() *immutable.SetIterator[*Job] { return txn.unvalidatedJobs.Iterator() } From e16815bac7d56c50615d2488fc8870fbe64e0dbb Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 4 Jun 2024 21:44:58 +0100 Subject: [PATCH 5/7] fix test Signed-off-by: Chris Martin --- internal/scheduler/scheduler_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/internal/scheduler/scheduler_test.go b/internal/scheduler/scheduler_test.go index a2be45ae1ce..8b27edf8415 100644 --- a/internal/scheduler/scheduler_test.go +++ b/internal/scheduler/scheduler_test.go @@ -674,6 +674,7 @@ func TestScheduler_TestCycle(t *testing.T) { CancelRequested: true, Cancelled: false, SchedulingInfo: schedulingInfoBytes, + Priority: int64(queuedJob.Priority()), }, }, From c9f0870642324f24aaa7478a462e3c1006bcd231 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 6 Jun 2024 14:33:13 +0100 Subject: [PATCH 6/7] add comment Signed-off-by: Chris Martin --- pkg/armadaevents/events.proto | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/armadaevents/events.proto b/pkg/armadaevents/events.proto index ab67ecfb217..a3741729757 100644 --- a/pkg/armadaevents/events.proto +++ b/pkg/armadaevents/events.proto @@ -169,6 +169,7 @@ message SubmitJob { string scheduler = 11; // Indicates whether job is a duplicate bool isDuplicate = 12; + // Ordinal 13 was originally used for queue_ttl_seconds // This is a string representation of the job_id. Eventually we will retire the job_id field and just use strings string job_id_str = 14; } From 20895e84f5044a89ebffa68f3e89741449f2fc47 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 6 Jun 2024 14:36:43 +0100 Subject: [PATCH 7/7] generate proto Signed-off-by: Chris Martin --- pkg/armadaevents/events.pb.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/armadaevents/events.pb.go b/pkg/armadaevents/events.pb.go index d098b1800de..6c8b802d274 100644 --- a/pkg/armadaevents/events.pb.go +++ b/pkg/armadaevents/events.pb.go @@ -736,6 +736,7 @@ type SubmitJob struct { Scheduler string `protobuf:"bytes,11,opt,name=scheduler,proto3" json:"scheduler,omitempty"` // Indicates whether job is a duplicate IsDuplicate bool `protobuf:"varint,12,opt,name=isDuplicate,proto3" json:"isDuplicate,omitempty"` + // Ordinal 13 was originally used for queue_ttl_seconds // This is a string representation of the job_id. Eventually we will retire the job_id field and just use strings JobIdStr string `protobuf:"bytes,14,opt,name=job_id_str,json=jobIdStr,proto3" json:"jobIdStr,omitempty"` }