From 287daaa34f4a95811825d02ede66cc1a65de81fa Mon Sep 17 00:00:00 2001 From: Jason Parraga Date: Thu, 22 May 2025 18:20:36 +0000 Subject: [PATCH 1/3] Add support for native preemption retries Signed-off-by: Jason Parraga --- internal/common/preemption/utils.go | 39 +++++ .../scheduler/configuration/configuration.go | 7 + internal/scheduler/database/job_repository.go | 6 +- .../scheduler/database/job_repository_test.go | 17 ++ internal/scheduler/jobdb/job.go | 56 +++++++ internal/scheduler/jobdb/job_run.go | 2 +- internal/scheduler/jobdb/job_test.go | 146 ++++++++++++++++++ internal/scheduler/scheduler.go | 43 ++++-- internal/scheduler/scheduler_test.go | 146 +++++++++++++++++- .../preempting_queue_scheduler_test.go | 18 ++- .../scheduler/scheduling/scheduling_algo.go | 10 +- .../scheduling/scheduling_algo_test.go | 54 ++++++- .../scheduler/testfixtures/testfixtures.go | 11 ++ internal/server/configuration/constants.go | 3 + internal/server/submit/submit_test.go | 24 +++ .../submit/validation/submit_request.go | 76 +++++++-- 16 files changed, 619 insertions(+), 39 deletions(-) create mode 100644 internal/common/preemption/utils.go diff --git a/internal/common/preemption/utils.go b/internal/common/preemption/utils.go new file mode 100644 index 00000000000..e974e8f42ef --- /dev/null +++ b/internal/common/preemption/utils.go @@ -0,0 +1,39 @@ +package preemption + +import ( + "strconv" + + "github.com/armadaproject/armada/internal/server/configuration" +) + +// AreRetriesEnabled determines whether preemption retries are enabled at the job level. Also returns whether the +// annotation was set. +func AreRetriesEnabled(annotations map[string]string) (enabled bool, annotationSet bool) { + preemptionRetryEnabledStr, exists := annotations[configuration.PreemptionRetryEnabledAnnotation] + if !exists { + return false, false + } + + preemptionRetryEnabled, err := strconv.ParseBool(preemptionRetryEnabledStr) + if err != nil { + return false, true + } + return preemptionRetryEnabled, true +} + +// GetMaxRetryCount gets the max preemption retry count at a job level. Also returns whether the annotation was set. +func GetMaxRetryCount(annotations map[string]string) (maxRetryCount uint, annotationSet bool) { + var preemptionRetryCountMax uint = 0 + preemptionRetryCountMaxStr, exists := annotations[configuration.PreemptionRetryCountMaxAnnotation] + + if !exists { + return preemptionRetryCountMax, false + } + maybePreemptionRetryCountMax, err := strconv.Atoi(preemptionRetryCountMaxStr) + if err != nil { + return preemptionRetryCountMax, true + } else { + preemptionRetryCountMax = uint(maybePreemptionRetryCountMax) + return preemptionRetryCountMax, true + } +} diff --git a/internal/scheduler/configuration/configuration.go b/internal/scheduler/configuration/configuration.go index 7a5e4910e73..b8b26ba94e6 100644 --- a/internal/scheduler/configuration/configuration.go +++ b/internal/scheduler/configuration/configuration.go @@ -248,6 +248,8 @@ type SchedulingConfig struct { Pools []PoolConfig ExperimentalIndicativePricing ExperimentalIndicativePricing ExperimentalIndicativeShare ExperimentalIndicativeShare + // Default preemption retries settings so you don't have to annotate all jobs with retries. + DefaultPreemptionRetry PreemptionRetryConfig } const ( @@ -354,3 +356,8 @@ type PriorityOverrideConfig struct { ServiceUrl string ForceNoTls bool } + +type PreemptionRetryConfig struct { + Enabled bool + DefaultMaxRetryCount *uint +} diff --git a/internal/scheduler/database/job_repository.go b/internal/scheduler/database/job_repository.go index 543e1240287..27f4be4460a 100644 --- a/internal/scheduler/database/job_repository.go +++ b/internal/scheduler/database/job_repository.go @@ -54,7 +54,7 @@ type JobRepository interface { CountReceivedPartitions(ctx *armadacontext.Context, groupId uuid.UUID) (uint32, error) // FindInactiveRuns returns a slice containing all dbRuns that the scheduler does not currently consider active - // Runs are inactive if they don't exist or if they have succeeded, failed or been cancelled + // Runs are inactive if they don't exist or if they have succeeded, failed, preempted or been cancelled FindInactiveRuns(ctx *armadacontext.Context, runIds []string) ([]string, error) // FetchJobRunLeases fetches new job runs for a given executor. A maximum of maxResults rows will be returned, while run @@ -293,7 +293,7 @@ func (r *PostgresJobRepository) FetchJobUpdates(ctx *armadacontext.Context, jobS } // FindInactiveRuns returns a slice containing all dbRuns that the scheduler does not currently consider active -// Runs are inactive if they don't exist or if they have succeeded, failed or been cancelled +// Runs are inactive if they don't exist or if they have succeeded, failed, preempted or been cancelled func (r *PostgresJobRepository) FindInactiveRuns(ctx *armadacontext.Context, runIds []string) ([]string, error) { var inactiveRuns []string err := pgx.BeginTxFunc(ctx, r.db, pgx.TxOptions{ @@ -313,6 +313,7 @@ func (r *PostgresJobRepository) FindInactiveRuns(ctx *armadacontext.Context, run WHERE runs.run_id IS NULL OR runs.succeeded = true OR runs.failed = true + OR runs.preempted = true OR runs.cancelled = true;` rows, err := tx.Query(ctx, fmt.Sprintf(query, tmpTable)) @@ -361,6 +362,7 @@ func (r *PostgresJobRepository) FetchJobRunLeases(ctx *armadacontext.Context, ex AND jr.succeeded = false AND jr.failed = false AND jr.cancelled = false + AND jr.preempted = false ORDER BY jr.serial LIMIT %d; ` diff --git a/internal/scheduler/database/job_repository_test.go b/internal/scheduler/database/job_repository_test.go index 8cae6e1bdda..96337f87da1 100644 --- a/internal/scheduler/database/job_repository_test.go +++ b/internal/scheduler/database/job_repository_test.go @@ -553,6 +553,15 @@ func TestFindInactiveRuns(t *testing.T) { }, expectedInactive: []string{runIds[1]}, }, + "run preempted": { + runsToCheck: runIds, + dbRuns: []Run{ + {RunID: runIds[0]}, + {RunID: runIds[1], Preempted: true}, + {RunID: runIds[2]}, + }, + expectedInactive: []string{runIds[1]}, + }, "run missing": { runsToCheck: runIds, dbRuns: []Run{ @@ -654,6 +663,14 @@ func TestFetchJobRunLeases(t *testing.T) { Pool: "test-pool", Succeeded: true, // should be ignored as terminal }, + { + RunID: uuid.NewString(), + JobID: dbJobs[0].JobID, + JobSet: "test-jobset", + Executor: executorName, + Pool: "test-pool", + Preempted: true, // should be ignored as terminal + }, } expectedLeases := make([]*JobRunLease, 4) for i := range expectedLeases { diff --git a/internal/scheduler/jobdb/job.go b/internal/scheduler/jobdb/job.go index ac2438654c7..551940f1643 100644 --- a/internal/scheduler/jobdb/job.go +++ b/internal/scheduler/jobdb/job.go @@ -4,6 +4,9 @@ import ( "fmt" "time" + "github.com/armadaproject/armada/internal/common/preemption" + "github.com/armadaproject/armada/internal/scheduler/configuration" + "github.com/hashicorp/go-multierror" "github.com/pkg/errors" "golang.org/x/exp/maps" @@ -734,6 +737,59 @@ func (job *Job) NumAttempts() uint { return attempts } +// IsEligibleForPreemptionRetry determines whether the job is eligible for preemption retries. It checks whether the +// scheduler or the job has opted in for preemption retries. It then checks whether the job has exhausted the number +// of retries. +func (job *Job) IsEligibleForPreemptionRetry(defaultPreemptionRetryConfig configuration.PreemptionRetryConfig) bool { + enabled := false + + // Check for platform default first + if defaultPreemptionRetryConfig.Enabled { + enabled = true + } + + // Check if job explicitly enabled/disabled retries + jobRetryEnabled, exists := preemption.AreRetriesEnabled(job.Annotations()) + if exists { + enabled = jobRetryEnabled + } + + if !enabled { + return false + } + + maxRetryCount := job.MaxPreemptionRetryCount(defaultPreemptionRetryConfig) + + return job.NumPreemptedRuns() <= maxRetryCount +} + +func (job *Job) NumPreemptedRuns() uint { + preemptCount := uint(0) + for _, run := range job.runsById { + if run.preempted { + preemptCount++ + } + } + return preemptCount +} + +func (job *Job) MaxPreemptionRetryCount(defaultPreemptionRetryConfig configuration.PreemptionRetryConfig) uint { + var maxRetryCount uint = 0 + + // Check for platform default first + if defaultPreemptionRetryConfig.DefaultMaxRetryCount != nil { + platformDefaultMaxRetryCount := *defaultPreemptionRetryConfig.DefaultMaxRetryCount + maxRetryCount = platformDefaultMaxRetryCount + } + + // Allow jobs to set a custom max retry count + jobMaxRetryCount, exists := preemption.GetMaxRetryCount(job.Annotations()) + if exists { + maxRetryCount = jobMaxRetryCount + } + return maxRetryCount +} + // AllRuns returns all runs associated with job. func (job *Job) AllRuns() []*JobRun { return maps.Values(job.runsById) diff --git a/internal/scheduler/jobdb/job_run.go b/internal/scheduler/jobdb/job_run.go index dc0b257361b..412ca46a9c0 100644 --- a/internal/scheduler/jobdb/job_run.go +++ b/internal/scheduler/jobdb/job_run.go @@ -501,7 +501,7 @@ func (run *JobRun) WithoutTerminal() *JobRun { // InTerminalState returns true if the JobRun is in a terminal state func (run *JobRun) InTerminalState() bool { - return run.succeeded || run.failed || run.cancelled || run.returned + return run.succeeded || run.failed || run.cancelled || run.returned || run.preempted } func (run *JobRun) DeepCopy() *JobRun { diff --git a/internal/scheduler/jobdb/job_test.go b/internal/scheduler/jobdb/job_test.go index e7f905d3a57..0c220fdd693 100644 --- a/internal/scheduler/jobdb/job_test.go +++ b/internal/scheduler/jobdb/job_test.go @@ -10,6 +10,9 @@ import ( "github.com/armadaproject/armada/internal/common/types" "github.com/armadaproject/armada/internal/scheduler/internaltypes" + + configuration2 "github.com/armadaproject/armada/internal/scheduler/configuration" + "github.com/armadaproject/armada/internal/server/configuration" ) var jobSchedulingInfo = &internaltypes.JobSchedulingInfo{ @@ -26,6 +29,35 @@ var jobSchedulingInfo = &internaltypes.JobSchedulingInfo{ }, } +var jobSchedulingInfoWithRetryEnabled = &internaltypes.JobSchedulingInfo{ + PodRequirements: &internaltypes.PodRequirements{ + ResourceRequirements: v1.ResourceRequirements{ + Requests: v1.ResourceList{ + "cpu": k8sResource.MustParse("1"), + "storage-connections": k8sResource.MustParse("1"), + }, + }, + Annotations: map[string]string{ + configuration.PreemptionRetryEnabledAnnotation: "true", + configuration.PreemptionRetryCountMaxAnnotation: "1", + }, + }, +} + +var jobSchedulingInfoWithRetryDisabled = &internaltypes.JobSchedulingInfo{ + PodRequirements: &internaltypes.PodRequirements{ + ResourceRequirements: v1.ResourceRequirements{ + Requests: v1.ResourceList{ + "cpu": k8sResource.MustParse("1"), + "storage-connections": k8sResource.MustParse("1"), + }, + }, + Annotations: map[string]string{ + configuration.PreemptionRetryEnabledAnnotation: "false", + }, + }, +} + var baseJob, _ = jobDb.NewJob( "test-job", "test-jobSet", @@ -42,6 +74,38 @@ var baseJob, _ = jobDb.NewJob( []string{}, ) +var baseJobWithRetryEnabled, _ = jobDb.NewJob( + "test-job", + "test-jobSet", + "test-queue", + 2, + jobSchedulingInfoWithRetryEnabled, + true, + 0, + false, + false, + false, + 3, + false, + []string{}, +) + +var baseJobWithRetryDisabled, _ = jobDb.NewJob( + "test-job", + "test-jobSet", + "test-queue", + 2, + jobSchedulingInfoWithRetryDisabled, + true, + 0, + false, + false, + false, + 3, + false, + []string{}, +) + var baseRun = &JobRun{ id: uuid.New().String(), created: 3, @@ -425,3 +489,85 @@ func TestJob_TestKubernetesResourceRequirements(t *testing.T) { assert.Equal(t, int64(1000), baseJob.KubernetesResourceRequirements().GetByNameZeroIfMissing("cpu")) assert.Equal(t, int64(0), baseJob.KubernetesResourceRequirements().GetByNameZeroIfMissing("storage-connections")) } + +func TestIsEligibleForPreemptionRetry(t *testing.T) { + premptedRun1 := &JobRun{ + id: uuid.New().String(), + created: 3, + executor: "test-executor", + preempted: true, + } + + premptedRun2 := &JobRun{ + id: uuid.New().String(), + created: 5, + executor: "test-executor", + preempted: true, + } + + defaultMaxRetryCountEnabled := uint(5) + platformDefaultEnabled := configuration2.PreemptionRetryConfig{ + Enabled: true, + DefaultMaxRetryCount: &defaultMaxRetryCountEnabled, + } + + defaultMaxRetryCountDisabled := uint(0) + platformDefaultDisabled := configuration2.PreemptionRetryConfig{ + Enabled: false, + DefaultMaxRetryCount: &defaultMaxRetryCountDisabled, + } + + // no runs + t.Run("job with retry enabled and platform disabled and no runs", func(t *testing.T) { + assert.True(t, baseJobWithRetryEnabled.IsEligibleForPreemptionRetry(platformDefaultDisabled)) + }) + + t.Run("job with retry disabled and platform enabled and no runs", func(t *testing.T) { + assert.False(t, baseJobWithRetryDisabled.IsEligibleForPreemptionRetry(platformDefaultEnabled)) + }) + + t.Run("job with platform retry enabled and no runs", func(t *testing.T) { + assert.True(t, baseJob.IsEligibleForPreemptionRetry(platformDefaultEnabled)) + }) + + t.Run("job with platform retry disabled and no runs", func(t *testing.T) { + assert.False(t, baseJob.IsEligibleForPreemptionRetry(platformDefaultDisabled)) + }) + + // runs but none are preempted + t.Run("job with retry enabled and platform disabled runs but no preempted runs", func(t *testing.T) { + updatedJob := baseJobWithRetryEnabled.WithUpdatedRun(baseRun).WithUpdatedRun(baseRun) + assert.True(t, updatedJob.IsEligibleForPreemptionRetry(platformDefaultDisabled)) + }) + + t.Run("job with platform enabled runs but no preempted runs", func(t *testing.T) { + updatedJob := baseJob.WithUpdatedRun(baseRun).WithUpdatedRun(baseRun) + assert.True(t, updatedJob.IsEligibleForPreemptionRetry(platformDefaultEnabled)) + }) + + t.Run("job with retry enabled and platform disabled and one run", func(t *testing.T) { + updatedJob := baseJobWithRetryEnabled.WithUpdatedRun(premptedRun1) + assert.True(t, updatedJob.IsEligibleForPreemptionRetry(platformDefaultDisabled)) + }) + + t.Run("job with platform enabled and one run", func(t *testing.T) { + updatedJob := baseJob.WithUpdatedRun(premptedRun1) + assert.True(t, updatedJob.IsEligibleForPreemptionRetry(platformDefaultEnabled)) + }) + + // runs that are preempted + t.Run("job with retry enabled and platform disabled and out of retries", func(t *testing.T) { + updatedJob := baseJobWithRetryEnabled.WithUpdatedRun(premptedRun1).WithUpdatedRun(premptedRun2) + assert.False(t, updatedJob.IsEligibleForPreemptionRetry(platformDefaultDisabled)) + }) + + t.Run("job with retry enabled with platform enabled and out of retries", func(t *testing.T) { + updatedJob := baseJobWithRetryEnabled.WithUpdatedRun(premptedRun1).WithUpdatedRun(premptedRun2) + assert.False(t, updatedJob.IsEligibleForPreemptionRetry(platformDefaultEnabled)) + }) + + t.Run("job with platform enabled and retries left", func(t *testing.T) { + updatedJob := baseJob.WithUpdatedRun(premptedRun1).WithUpdatedRun(premptedRun2) + assert.True(t, updatedJob.IsEligibleForPreemptionRetry(platformDefaultEnabled)) + }) +} diff --git a/internal/scheduler/scheduler.go b/internal/scheduler/scheduler.go index d81bb4d344e..7c1b81384b8 100644 --- a/internal/scheduler/scheduler.go +++ b/internal/scheduler/scheduler.go @@ -506,20 +506,23 @@ func AppendEventSequencesFromPreemptedJobs(eventSequences []*armadaevents.EventS eventSequences = append(eventSequences, &armadaevents.EventSequence{ Queue: jctx.Job.Queue(), JobSetName: jctx.Job.Jobset(), - Events: createEventsForPreemptedJob(jctx.JobId, run.Id(), jctx.PreemptionDescription, time), + Events: createEventsForPreemptedJob(jctx.Job, run.Id(), jctx.PreemptionDescription, time), }) } return eventSequences, nil } -func createEventsForPreemptedJob(jobId string, runId string, reason string, time time.Time) []*armadaevents.EventSequence_Event { - return []*armadaevents.EventSequence_Event{ +func createEventsForPreemptedJob(job *jobdb.Job, runId string, reason string, time time.Time) []*armadaevents.EventSequence_Event { + // Check for job failure in case of API preemption + jobFailed := job.Failed() + + events := []*armadaevents.EventSequence_Event{ { Created: protoutil.ToTimestamp(time), Event: &armadaevents.EventSequence_Event_JobRunPreempted{ JobRunPreempted: &armadaevents.JobRunPreempted{ PreemptedRunId: runId, - PreemptedJobId: jobId, + PreemptedJobId: job.Id(), Reason: reason, }, }, @@ -529,10 +532,10 @@ func createEventsForPreemptedJob(jobId string, runId string, reason string, time Event: &armadaevents.EventSequence_Event_JobRunErrors{ JobRunErrors: &armadaevents.JobRunErrors{ RunId: runId, - JobId: jobId, + JobId: job.Id(), Errors: []*armadaevents.Error{ { - Terminal: true, + Terminal: jobFailed, Reason: &armadaevents.Error_JobRunPreemptedError{ JobRunPreemptedError: &armadaevents.JobRunPreemptedError{ Reason: reason, @@ -543,11 +546,15 @@ func createEventsForPreemptedJob(jobId string, runId string, reason string, time }, }, }, - { + } + + if jobFailed { + // Only send a job error event if the job is in a failed state + events = append(events, &armadaevents.EventSequence_Event{ Created: protoutil.ToTimestamp(time), Event: &armadaevents.EventSequence_Event_JobErrors{ JobErrors: &armadaevents.JobErrors{ - JobId: jobId, + JobId: job.Id(), Errors: []*armadaevents.Error{ { Terminal: true, @@ -560,8 +567,24 @@ func createEventsForPreemptedJob(jobId string, runId string, reason string, time }, }, }, - }, + }) + } + + if job.Queued() { + // Assume the job has already been marked as queued and the version has been bumped + events = append(events, &armadaevents.EventSequence_Event{ + Created: protoutil.ToTimestamp(time), + Event: &armadaevents.EventSequence_Event_JobRequeued{ + JobRequeued: &armadaevents.JobRequeued{ + JobId: job.Id(), + SchedulingInfo: internaltypes.ToSchedulerObjectsJobSchedulingInfo(job.JobSchedulingInfo()), + UpdateSequenceNumber: job.QueuedVersion(), + }, + }, + }) } + + return events } func AppendEventSequencesFromScheduledJobs(eventSequences []*armadaevents.EventSequence, jctxs []*schedulercontext.JobSchedulingContext) ([]*armadaevents.EventSequence, error) { @@ -813,7 +836,7 @@ func (s *Scheduler) generateUpdateMessagesFromJob(ctx *armadacontext.Context, jo } } else if lastRun.PreemptRequested() && job.PriorityClass().Preemptible { job = job.WithQueued(false).WithFailed(true).WithUpdatedRun(lastRun.WithoutTerminal().WithFailed(true)) - events = append(events, createEventsForPreemptedJob(job.Id(), lastRun.Id(), "Preempted - preemption requested via API", s.clock.Now())...) + events = append(events, createEventsForPreemptedJob(job, lastRun.Id(), "Preempted - preemption requested via API", s.clock.Now())...) } } diff --git a/internal/scheduler/scheduler_test.go b/internal/scheduler/scheduler_test.go index e640b30cdd5..a9dd5037eef 100644 --- a/internal/scheduler/scheduler_test.go +++ b/internal/scheduler/scheduler_test.go @@ -6,6 +6,8 @@ import ( "testing" "time" + "github.com/armadaproject/armada/internal/scheduler/configuration" + "github.com/gogo/protobuf/types" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgxpool" @@ -89,8 +91,26 @@ var ( }, Version: 1, } - schedulingInfoBytes = protoutil.MustMarshall(schedulingInfo) - updatedSchedulingInfo = &schedulerobjects.JobSchedulingInfo{ + schedulingInfoBytes = protoutil.MustMarshall(schedulingInfo) + schedulingInfoWithRetryEnabled = &schedulerobjects.JobSchedulingInfo{ + AtMostOnce: true, + PriorityClassName: testfixtures.PriorityClass1, + ObjectRequirements: []*schedulerobjects.ObjectRequirements{ + { + Requirements: &schedulerobjects.ObjectRequirements_PodRequirements{ + PodRequirements: &schedulerobjects.PodRequirements{ + Annotations: map[string]string{ + apiconfig.PreemptionRetryEnabledAnnotation: "true", + apiconfig.PreemptionRetryCountMaxAnnotation: "1", + }, + }, + }, + }, + }, + Version: 1, + } + schedulingInfoWithRetryEnabledBytes = protoutil.MustMarshall(schedulingInfoWithRetryEnabled) + updatedSchedulingInfo = &schedulerobjects.JobSchedulingInfo{ AtMostOnce: true, ObjectRequirements: []*schedulerobjects.ObjectRequirements{ { @@ -149,6 +169,21 @@ var leasedJob = testfixtures.NewJob( true, ).WithNewRun("testExecutor", "test-node", "node", "pool", 5) +var leasedJobWithRetryEnabled = testfixtures.NewJob( + util.NewULID(), + "testJobset", + "testQueue", + 0, + toInternalSchedulingInfo(schedulingInfoWithRetryEnabled), + false, + 1, + false, + false, + false, + 1, + true, +).WithNewRun("testExecutor", "test-node", "node", "pool", 5) + var preemptibleLeasedJob = testfixtures.NewJob( util.NewULID(), "testJobset", @@ -777,6 +812,17 @@ func TestScheduler_TestCycle(t *testing.T) { expectedTerminal: []string{leasedJob.Id()}, expectedQueuedVersion: leasedJob.QueuedVersion(), }, + "Job preempted with retry enabled": { + initialJobs: []*jobdb.Job{leasedJobWithRetryEnabled}, + expectedJobsRunsToPreempt: []string{leasedJobWithRetryEnabled.Id()}, + expectedJobRunPreempted: []string{leasedJobWithRetryEnabled.Id()}, + expectedJobErrors: []string{}, + expectedJobRunErrors: []string{leasedJobWithRetryEnabled.Id()}, + expectedTerminal: []string{}, + expectedQueued: []string{leasedJobWithRetryEnabled.Id()}, + expectedRequeued: []string{leasedJobWithRetryEnabled.Id()}, + expectedQueuedVersion: leasedJob.QueuedVersion() + 1, + }, "Fetch fails": { initialJobs: []*jobdb.Job{leasedJob}, fetchError: true, @@ -1525,11 +1571,16 @@ func (t *testSchedulingAlgo) Schedule(_ *armadacontext.Context, txn *jobdb.Txn) return nil, errors.Errorf("was asked to preempt job %s but job is still queued", job.Id()) } if run := job.LatestRun(); run != nil { - job = job.WithUpdatedRun(run.WithFailed(true)) + now := time.Now() + job = job.WithUpdatedRun(run.WithPreempted(true).WithPreemptedTime(&now)) } else { return nil, errors.Errorf("attempting to preempt job %s with no associated runs", job.Id()) } - job = job.WithQueued(false).WithFailed(true) + if job.IsEligibleForPreemptionRetry(configuration.PreemptionRetryConfig{}) { + job = job.WithQueued(true).WithQueuedVersion(job.QueuedVersion() + 1) + } else { + job = job.WithQueued(false).WithFailed(true) + } preemptedJobs = append(preemptedJobs, job) } for _, id := range t.jobsToSchedule { @@ -1669,6 +1720,16 @@ var ( Validated: true, Serial: 0, } + runningAndRetryableJobA = &database.Job{ + JobID: queuedJobA.JobID, + JobSet: "testJobSet", + Queue: "testQueue", + QueuedVersion: 1, + SchedulingInfo: schedulingInfoWithRetryEnabledBytes, + SchedulingInfoVersion: int32(schedulingInfoWithRetryEnabled.Version), + Validated: true, + Serial: 0, + } failedJobA = &database.Job{ JobID: queuedJobA.JobID, JobSet: "testJobSet", @@ -2361,6 +2422,83 @@ func TestCycleConsistency(t *testing.T) { }, }, }, + "Running job is preempted and requeued with retry enabled": { + firstSchedulerDbUpdate: schedulerDbUpdate{ + jobUpdates: []*database.Job{ + runningAndRetryableJobA, + }, + runUpdates: []*database.Run{ + newRunA, + }, + }, + idsOfJobsToPreempt: []string{queuedJobA.JobID}, + expectedJobDbCycleThree: make([]*jobdb.Job, 0), + expectedEventSequencesCycleThree: []*armadaevents.EventSequence{ + { + Queue: queuedJobA.Queue, + JobSetName: queuedJobA.JobSet, + Events: []*armadaevents.EventSequence_Event{ + { + Created: &types.Timestamp{}, + Event: &armadaevents.EventSequence_Event_JobRunPreempted{ + JobRunPreempted: &armadaevents.JobRunPreempted{ + PreemptedRunId: testfixtures.UUIDFromInt(1).String(), + PreemptedJobId: queuedJobA.JobID, + }, + }, + }, + { + Created: &types.Timestamp{}, + Event: &armadaevents.EventSequence_Event_JobRunErrors{ + JobRunErrors: &armadaevents.JobRunErrors{ + JobId: queuedJobA.JobID, + RunId: testfixtures.UUIDFromInt(1).String(), + Errors: []*armadaevents.Error{ + { + Terminal: false, + Reason: &armadaevents.Error_JobRunPreemptedError{ + JobRunPreemptedError: &armadaevents.JobRunPreemptedError{}, + }, + }, + }, + }, + }, + }, + { + Created: &types.Timestamp{}, + Event: &armadaevents.EventSequence_Event_JobRequeued{ + JobRequeued: &armadaevents.JobRequeued{ + SchedulingInfo: &schedulerobjects.JobSchedulingInfo{ + SubmitTime: &types.Timestamp{ + Seconds: -62135596800, + Nanos: 0, + }, + PriorityClassName: testfixtures.PriorityClass1, + ObjectRequirements: []*schedulerobjects.ObjectRequirements{ + { + Requirements: &schedulerobjects.ObjectRequirements_PodRequirements{ + PodRequirements: &schedulerobjects.PodRequirements{ + ResourceRequirements: &v1.ResourceRequirements{}, + NodeSelector: map[string]string{}, + Annotations: map[string]string{ + apiconfig.PreemptionRetryEnabledAnnotation: "true", + apiconfig.PreemptionRetryCountMaxAnnotation: "1", + }, + }, + }, + }, + }, + Version: 1, + }, + UpdateSequenceNumber: int32(2), + JobId: queuedJobA.JobID, + }, + }, + }, + }, + }, + }, + }, "Queued job is cancelled": { firstSchedulerDbUpdate: schedulerDbUpdate{ jobUpdates: []*database.Job{ diff --git a/internal/scheduler/scheduling/preempting_queue_scheduler_test.go b/internal/scheduler/scheduling/preempting_queue_scheduler_test.go index 41bf11a831b..fe7cce22b0a 100644 --- a/internal/scheduler/scheduling/preempting_queue_scheduler_test.go +++ b/internal/scheduler/scheduling/preempting_queue_scheduler_test.go @@ -2237,13 +2237,17 @@ func TestPreemptingQueueScheduler(t *testing.T) { var preemptedJobs []*jobdb.Job for _, jctx := range result.PreemptedJobs { job := jctx.Job - preemptedJobs = append( - preemptedJobs, - job. - WithUpdatedRun(job.LatestRun().WithFailed(true)). - WithQueued(false). - WithFailed(true), - ) + preeemptedTime := time.Now() + + updatedRun := job.LatestRun().WithPreempted(true).WithPreemptedTime(&preeemptedTime) + job = job.WithUpdatedRun(updatedRun) + + if job.IsEligibleForPreemptionRetry(configuration.PreemptionRetryConfig{}) { + job = job.WithQueued(true).WithQueuedVersion(job.QueuedVersion() + 1) + } else { + job = job.WithQueued(false).WithFailed(true) + } + preemptedJobs = append(preemptedJobs, job) } err = jobDbTxn.Upsert(preemptedJobs) require.NoError(t, err) diff --git a/internal/scheduler/scheduling/scheduling_algo.go b/internal/scheduler/scheduling/scheduling_algo.go index 9cb46378d49..ec038edb495 100644 --- a/internal/scheduler/scheduling/scheduling_algo.go +++ b/internal/scheduler/scheduling/scheduling_algo.go @@ -583,11 +583,17 @@ func (l *FairSchedulingAlgo) SchedulePool( jobDbJob := jctx.Job now := l.clock.Now() if run := jobDbJob.LatestRun(); run != nil { - jobDbJob = jobDbJob.WithUpdatedRun(run.WithFailed(true).WithPreemptedTime(&now)) + updatedRun := run.WithPreempted(true).WithPreemptedTime(&now) + jobDbJob = jobDbJob.WithUpdatedRun(updatedRun) } else { return nil, nil, errors.Errorf("attempting to preempt job %s with no associated runs", jobDbJob.Id()) } - result.PreemptedJobs[i].Job = jobDbJob.WithQueued(false).WithFailed(true) + if jobDbJob.IsEligibleForPreemptionRetry(l.schedulingConfig.DefaultPreemptionRetry) { + ctx.Infof("Handling preemption retry %d / %d for job %s. Job will be requeued.", jobDbJob.NumPreemptedRuns(), jobDbJob.MaxPreemptionRetryCount(l.schedulingConfig.DefaultPreemptionRetry), jobDbJob.Id()) + result.PreemptedJobs[i].Job = jobDbJob.WithQueued(true).WithQueuedVersion(jobDbJob.QueuedVersion() + 1) + } else { + result.PreemptedJobs[i].Job = jobDbJob.WithQueued(false).WithFailed(true) + } } for i, jctx := range result.ScheduledJobs { jobDbJob := jctx.Job diff --git a/internal/scheduler/scheduling/scheduling_algo_test.go b/internal/scheduler/scheduling/scheduling_algo_test.go index 121ac5824d1..4fd96fe3ad3 100644 --- a/internal/scheduler/scheduling/scheduling_algo_test.go +++ b/internal/scheduler/scheduling/scheduling_algo_test.go @@ -301,6 +301,26 @@ func TestSchedule(t *testing.T) { }, expectedScheduledIndices: []int{0, 1}, }, + "urgency-based preemption within a single queue with retry enabled": { + schedulingConfig: testfixtures.TestSchedulingConfig(), + executors: []*schedulerobjects.Executor{test1Node32CoreExecutor("executor1")}, + queues: []*api.Queue{{Name: "A"}}, + queuedJobs: testfixtures.WithPreemptionRetryAnnotationsJobs(testfixtures.N16Cpu128GiJobs("A", testfixtures.PriorityClass1, 2), 10), + scheduledJobsByExecutorIndexAndNodeIndex: map[int]map[int]scheduledJobs{ + 0: { + 0: scheduledJobs{ + jobs: testfixtures.WithPreemptionRetryAnnotationsJobs(testfixtures.N16Cpu128GiJobs("A", testfixtures.PriorityClass0, 1), 10), + acknowledged: true, + }, + }, + }, + expectedPreemptedJobIndicesByExecutorIndexAndNodeIndex: map[int]map[int][]int{ + 0: { + 0: {0}, + }, + }, + expectedScheduledIndices: []int{0, 1}, + }, "urgency-based preemption between queues": { schedulingConfig: testfixtures.TestSchedulingConfig(), executors: []*schedulerobjects.Executor{test1Node32CoreExecutor("executor1")}, @@ -341,6 +361,26 @@ func TestSchedule(t *testing.T) { }, expectedScheduledIndices: []int{0}, }, + "preemption to fair share with retries enabled": { + schedulingConfig: testfixtures.TestSchedulingConfig(), + executors: []*schedulerobjects.Executor{test1Node32CoreExecutor("executor1")}, + queues: []*api.Queue{{Name: "A", PriorityFactor: 0.01}, {Name: "B", PriorityFactor: 0.01}}, + queuedJobs: testfixtures.WithPreemptionRetryAnnotationsJobs(testfixtures.N16Cpu128GiJobs("A", testfixtures.PriorityClass0, 2), 10), + scheduledJobsByExecutorIndexAndNodeIndex: map[int]map[int]scheduledJobs{ + 0: { + 0: scheduledJobs{ + jobs: testfixtures.WithPreemptionRetryAnnotationsJobs(testfixtures.N16Cpu128GiJobs("B", testfixtures.PriorityClass0, 2), 10), + acknowledged: true, + }, + }, + }, + expectedPreemptedJobIndicesByExecutorIndexAndNodeIndex: map[int]map[int][]int{ + 0: { + 0: {1}, + }, + }, + expectedScheduledIndices: []int{0}, + }, "gang scheduling successful": { schedulingConfig: testfixtures.TestSchedulingConfig(), executors: []*schedulerobjects.Executor{test1Node32CoreExecutor("executor1")}, @@ -604,8 +644,18 @@ func TestSchedule(t *testing.T) { // Check that preempted jobs are marked as such consistently. for _, job := range preemptedJobs { dbJob := txn.GetById(job.Id()) - assert.True(t, dbJob.Failed()) - assert.False(t, dbJob.Queued()) + if dbJob.IsEligibleForPreemptionRetry(tc.schedulingConfig.DefaultPreemptionRetry) { + assert.False(t, dbJob.Failed()) + assert.True(t, dbJob.Queued()) + } else { + assert.True(t, dbJob.Failed()) + assert.False(t, dbJob.Queued()) + } + + // The job run is always marked for preemption + lastRun := dbJob.LatestRun() + assert.True(t, lastRun.Preempted()) + assert.NotNil(t, lastRun.PreemptedTime()) } // Check that scheduled jobs are marked as such consistently. diff --git a/internal/scheduler/testfixtures/testfixtures.go b/internal/scheduler/testfixtures/testfixtures.go index 69f48c2a84f..a4a69a43f26 100644 --- a/internal/scheduler/testfixtures/testfixtures.go +++ b/internal/scheduler/testfixtures/testfixtures.go @@ -5,6 +5,7 @@ import ( "encoding/binary" "fmt" "math" + "strconv" "sync" "sync/atomic" "time" @@ -453,6 +454,16 @@ func WithGangAnnotationsJobs(jobs []*jobdb.Job) []*jobdb.Job { ) } +func WithPreemptionRetryAnnotationsJobs(jobs []*jobdb.Job, retryCount int) []*jobdb.Job { + return WithAnnotationsJobs( + map[string]string{ + configuration.PreemptionRetryEnabledAnnotation: "true", + configuration.PreemptionRetryCountMaxAnnotation: strconv.Itoa(retryCount), + }, + jobs, + ) +} + func WithPools(jobs []*jobdb.Job, pools []string) []*jobdb.Job { result := make([]*jobdb.Job, 0, len(jobs)) for _, job := range jobs { diff --git a/internal/server/configuration/constants.go b/internal/server/configuration/constants.go index 365817ebfba..b365a02dfb5 100644 --- a/internal/server/configuration/constants.go +++ b/internal/server/configuration/constants.go @@ -16,6 +16,9 @@ const ( // Instead, the job the pod is part of fails immediately. FailFastAnnotation = "armadaproject.io/failFast" PoolAnnotation = "armadaproject.io/pool" + + PreemptionRetryCountMaxAnnotation = "armadaproject.io/preemptionRetryCountMax" + PreemptionRetryEnabledAnnotation = "armadaproject.io/preemptionRetryEnabled" ) var schedulingAnnotations = map[string]bool{ diff --git a/internal/server/submit/submit_test.go b/internal/server/submit/submit_test.go index 68bc3736694..13ac1097fde 100644 --- a/internal/server/submit/submit_test.go +++ b/internal/server/submit/submit_test.go @@ -4,6 +4,8 @@ import ( "testing" "time" + "github.com/armadaproject/armada/internal/server/configuration" + "github.com/stretchr/testify/assert" "go.uber.org/mock/gomock" v1 "k8s.io/api/core/v1" @@ -164,6 +166,21 @@ func TestSubmit_FailedValidation(t *testing.T) { Requests: v1.ResourceList{"cpu": resource.MustParse("2")}, }), }, + "Invalid Preemption Retry Enabled Annotation": { + req: withAnnotations(testfixtures.SubmitRequestWithNItems(1), map[string]string{ + configuration.PreemptionRetryEnabledAnnotation: "not a boolean", + }), + }, + "Invalid Preemption Retry Count Max Annotation": { + req: withAnnotations(testfixtures.SubmitRequestWithNItems(1), map[string]string{ + configuration.PreemptionRetryEnabledAnnotation: "not an int", + }), + }, + "Negative Preemption Retry Count Max Annotation": { + req: withAnnotations(testfixtures.SubmitRequestWithNItems(1), map[string]string{ + configuration.PreemptionRetryEnabledAnnotation: "-1", + }), + }, } for name, tc := range tests { t.Run(name, func(t *testing.T) { @@ -510,6 +527,13 @@ func withTerminationGracePeriod(req *api.JobSubmitRequest, v *int64) *api.JobSub return req } +func withAnnotations(req *api.JobSubmitRequest, annotations map[string]string) *api.JobSubmitRequest { + for _, item := range req.JobRequestItems { + item.Annotations = annotations + } + return req +} + func createTestServer(t *testing.T) (*Server, *mockObjects) { m := createMocks(t) server := NewServer( diff --git a/internal/server/submit/validation/submit_request.go b/internal/server/submit/validation/submit_request.go index 7ae87082e84..577c472eee3 100644 --- a/internal/server/submit/validation/submit_request.go +++ b/internal/server/submit/validation/submit_request.go @@ -2,10 +2,12 @@ package validation import ( "fmt" + "strconv" "github.com/pkg/errors" "k8s.io/component-helpers/scheduling/corev1/nodeaffinity" + "github.com/armadaproject/armada/internal/common/preemption" schedulercontext "github.com/armadaproject/armada/internal/scheduler/scheduling/context" "github.com/armadaproject/armada/internal/server/configuration" "github.com/armadaproject/armada/pkg/api" @@ -37,6 +39,7 @@ var ( validatePorts, validateClientId, validateTolerations, + validatePreemptionRetryConfig, } ) @@ -297,43 +300,68 @@ func (j jobAdapter) Annotations() map[string]string { return j.GetAnnotations() } +type gangDetails struct { + schedulingGangInfo schedulercontext.GangInfo + preemptionRetriesEnabled bool + preemptionRetryCountMax uint +} + // Ensures that any gang jobs defined in the request are consistent. This checks that all jobs in the same gang have // the same: // - Cardinality // - MinimumCardinality // - Priority Class // - Node Uniformity +// - Preemption Retry Configuration func validateGangs(request *api.JobSubmitRequest, _ configuration.SubmissionConfig) error { - gangDetailsByGangId := make(map[string]schedulercontext.GangInfo) + gangDetailsByGangId := make(map[string]gangDetails) for _, job := range request.JobRequestItems { - actual, err := schedulercontext.GangInfoFromLegacySchedulerJob(jobAdapter{job}) + schedulingGangInfo, err := schedulercontext.GangInfoFromLegacySchedulerJob(jobAdapter{job}) if err != nil { return fmt.Errorf("invalid gang annotations: %s", err.Error()) } - if actual.Id == "" { + + preemptionRetriesEnabled, _ := preemption.AreRetriesEnabled(job.GetAnnotations()) + preemptionRetryCountMax, _ := preemption.GetMaxRetryCount(job.GetAnnotations()) + + if schedulingGangInfo.Id == "" { continue } - if expected, ok := gangDetailsByGangId[actual.Id]; ok { - if expected.Cardinality != actual.Cardinality { + if expected, ok := gangDetailsByGangId[schedulingGangInfo.Id]; ok { + if expected.schedulingGangInfo.Cardinality != schedulingGangInfo.Cardinality { return errors.Errorf( "inconsistent gang cardinality in gang %s: expected %d but got %d", - actual.Id, expected.Cardinality, actual.Cardinality, + schedulingGangInfo.Id, expected.schedulingGangInfo.Cardinality, schedulingGangInfo.Cardinality, ) } - if expected.PriorityClassName != actual.PriorityClassName { + if expected.schedulingGangInfo.PriorityClassName != schedulingGangInfo.PriorityClassName { return errors.Errorf( "inconsistent PriorityClassName in gang %s: expected %s but got %s", - actual.Id, expected.PriorityClassName, actual.PriorityClassName, + schedulingGangInfo.Id, expected.schedulingGangInfo.PriorityClassName, schedulingGangInfo.PriorityClassName, ) } - if actual.NodeUniformity != expected.NodeUniformity { + if schedulingGangInfo.NodeUniformity != expected.schedulingGangInfo.NodeUniformity { return errors.Errorf( "inconsistent nodeUniformityLabel in gang %s: expected %s but got %s", - actual.Id, expected.NodeUniformity, actual.NodeUniformity, + schedulingGangInfo.Id, expected.schedulingGangInfo.NodeUniformity, schedulingGangInfo.NodeUniformity, ) } + if preemptionRetriesEnabled != expected.preemptionRetriesEnabled { + return errors.Errorf( + "inconsistent preemptionRetriesEnabled in gang %s: expected %t but got %t", + schedulingGangInfo.Id, expected.preemptionRetriesEnabled, preemptionRetriesEnabled) + } + if preemptionRetryCountMax != expected.preemptionRetryCountMax { + return errors.Errorf( + "inconsistent preemptionRetryCountMax in gang %s: expected %d but got %d", + schedulingGangInfo.Id, expected.preemptionRetryCountMax, preemptionRetryCountMax) + } } else { - gangDetailsByGangId[actual.Id] = actual + gangDetailsByGangId[schedulingGangInfo.Id] = gangDetails{ + schedulingGangInfo: schedulingGangInfo, + preemptionRetriesEnabled: preemptionRetriesEnabled, + preemptionRetryCountMax: preemptionRetryCountMax, + } } } return nil @@ -409,3 +437,29 @@ func validateTolerations(j *api.JobSubmitRequestItem, config configuration.Submi } return nil } + +// Ensures that if a request specified preemption retry annotations, they are valid +func validatePreemptionRetryConfig(j *api.JobSubmitRequestItem, _ configuration.SubmissionConfig) error { + preemptionRetryEnabledStr, exists := j.GetAnnotations()[configuration.PreemptionRetryEnabledAnnotation] + if exists { + _, err := strconv.ParseBool(preemptionRetryEnabledStr) + if err != nil { + return fmt.Errorf("invalid preemption retry enabled annotation value: %w", err) + } + } + + preemptionRetryCountMaxStr, exists := j.GetAnnotations()[configuration.PreemptionRetryCountMaxAnnotation] + + if exists { + preemptionRetryCountMax, err := strconv.Atoi(preemptionRetryCountMaxStr) + if err != nil { + return fmt.Errorf("invalid preemption retry count max annotation value: %w", err) + } + + if preemptionRetryCountMax <= 0 { + return fmt.Errorf("preemption retry count max must be greater than zero: %w", err) + } + } + + return nil +} From 7cf9c8531d1bb5973df08271be79f625bd03e33b Mon Sep 17 00:00:00 2001 From: Jason Parraga Date: Wed, 28 May 2025 00:31:19 +0000 Subject: [PATCH 2/3] Plumb run index to pod name Signed-off-by: Jason Parraga --- internal/executor/domain/pod_metadata.go | 1 + internal/executor/util/kubernetes_object.go | 13 +- .../executor/util/kubernetes_objects_test.go | 24 +- internal/scheduler/api.go | 13 +- internal/scheduler/database/job_repository.go | 5 +- .../migrations/022_add_job_run_index.sql | 1 + internal/scheduler/database/models.go | 1 + internal/scheduler/database/query.sql.go | 9 +- internal/scheduler/jobdb/job.go | 2 + internal/scheduler/jobdb/job_run.go | 12 + internal/scheduler/jobdb/job_run_test.go | 3 + internal/scheduler/jobdb/reconciliation.go | 1 + internal/scheduler/scheduler.go | 1 + internal/scheduler/scheduler_test.go | 4 + internal/scheduleringester/instructions.go | 1 + pkg/armadaevents/events.pb.go | 350 ++++++++++-------- pkg/armadaevents/events.proto | 1 + pkg/executorapi/executorapi.pb.go | 242 ++++++------ pkg/executorapi/executorapi.proto | 1 + 19 files changed, 398 insertions(+), 287 deletions(-) create mode 100644 internal/scheduler/database/migrations/022_add_job_run_index.sql diff --git a/internal/executor/domain/pod_metadata.go b/internal/executor/domain/pod_metadata.go index 66d1993669d..67b08bf9923 100644 --- a/internal/executor/domain/pod_metadata.go +++ b/internal/executor/domain/pod_metadata.go @@ -3,6 +3,7 @@ package domain const ( JobId = "armada_job_id" JobRunId = "armada_job_run_id" + JobRunIndex = "armada_job_run_index" PodNumber = "armada_pod_number" PodCount = "armada_pod_count" JobSetId = "armada_jobset_id" diff --git a/internal/executor/util/kubernetes_object.go b/internal/executor/util/kubernetes_object.go index ccfcf7de929..149471e89fd 100644 --- a/internal/executor/util/kubernetes_object.go +++ b/internal/executor/util/kubernetes_object.go @@ -116,11 +116,12 @@ func CreatePodFromExecutorApiJob(job *executorapi.JobRunLease, defaults *configu } labels := util.MergeMaps(job.Job.ObjectMeta.Labels, map[string]string{ - domain.JobId: jobId, - domain.JobRunId: runId, - domain.Queue: job.Queue, - domain.PodNumber: strconv.Itoa(0), - domain.PodCount: strconv.Itoa(1), + domain.JobId: jobId, + domain.JobRunId: runId, + domain.JobRunIndex: strconv.Itoa(int(job.JobRunIndex)), + domain.Queue: job.Queue, + domain.PodNumber: strconv.Itoa(0), + domain.PodCount: strconv.Itoa(1), }) annotation := util.MergeMaps(job.Job.ObjectMeta.Annotations, map[string]string{ domain.JobSetId: job.Jobset, @@ -132,7 +133,7 @@ func CreatePodFromExecutorApiJob(job *executorapi.JobRunLease, defaults *configu pod := &v1.Pod{ ObjectMeta: metav1.ObjectMeta{ - Name: common.PodNamePrefix + job.Job.JobId + "-" + strconv.Itoa(0), + Name: common.PodNamePrefix + job.Job.JobId + "-" + strconv.Itoa(0) + "-" + strconv.FormatUint(uint64(job.JobRunIndex), 10), Labels: labels, Annotations: annotation, Namespace: job.Job.ObjectMeta.Namespace, diff --git a/internal/executor/util/kubernetes_objects_test.go b/internal/executor/util/kubernetes_objects_test.go index 639aa843aab..da5dcb7c6b2 100644 --- a/internal/executor/util/kubernetes_objects_test.go +++ b/internal/executor/util/kubernetes_objects_test.go @@ -2,6 +2,7 @@ package util import ( "fmt" + "strconv" "testing" "github.com/armadaproject/armada/internal/common/util" @@ -142,12 +143,14 @@ func makePodSpec() *v1.PodSpec { func TestCreatePodFromExecutorApiJob(t *testing.T) { runId := uuid.NewString() jobId := util.NewULID() + runIndex := 0 validJobLease := &executorapi.JobRunLease{ - JobRunId: runId, - Queue: "queue", - Jobset: "job-set", - User: "user", + JobRunId: runId, + JobRunIndex: uint32(runIndex), + Queue: "queue", + Jobset: "job-set", + User: "user", Job: &armadaevents.SubmitJob{ ObjectMeta: &armadaevents.ObjectMeta{ Labels: map[string]string{}, @@ -167,14 +170,15 @@ func TestCreatePodFromExecutorApiJob(t *testing.T) { expectedPod := &v1.Pod{ ObjectMeta: metav1.ObjectMeta{ - Name: fmt.Sprintf("armada-%s-0", jobId), + Name: fmt.Sprintf("armada-%s-0-%d", jobId, runIndex), Namespace: "test-namespace", Labels: map[string]string{ - domain.JobId: jobId, - domain.JobRunId: runId, - domain.Queue: "queue", - domain.PodNumber: "0", - domain.PodCount: "1", + domain.JobId: jobId, + domain.JobRunId: runId, + domain.JobRunIndex: strconv.Itoa(runIndex), + domain.Queue: "queue", + domain.PodNumber: "0", + domain.PodCount: "1", }, Annotations: map[string]string{ domain.JobSetId: "job-set", diff --git a/internal/scheduler/api.go b/internal/scheduler/api.go index b663782769b..6918cea301b 100644 --- a/internal/scheduler/api.go +++ b/internal/scheduler/api.go @@ -173,12 +173,13 @@ func (srv *ExecutorApi) LeaseJobRuns(stream executorapi.ExecutorApi_LeaseJobRuns err := stream.Send(&executorapi.LeaseStreamMessage{ Event: &executorapi.LeaseStreamMessage_Lease{ Lease: &executorapi.JobRunLease{ - JobRunId: lease.RunID, - Queue: lease.Queue, - Jobset: lease.JobSet, - User: lease.UserID, - Groups: groups, - Job: submitMsg, + JobRunId: lease.RunID, + JobRunIndex: lease.RunIndex, + Queue: lease.Queue, + Jobset: lease.JobSet, + User: lease.UserID, + Groups: groups, + Job: submitMsg, }, }, }) diff --git a/internal/scheduler/database/job_repository.go b/internal/scheduler/database/job_repository.go index 27f4be4460a..26472aa58f6 100644 --- a/internal/scheduler/database/job_repository.go +++ b/internal/scheduler/database/job_repository.go @@ -24,6 +24,7 @@ type hasSerial interface { type JobRunLease struct { RunID string + RunIndex uint32 Queue string Pool string JobSet string @@ -352,7 +353,7 @@ func (r *PostgresJobRepository) FetchJobRunLeases(ctx *armadacontext.Context, ex } query := ` - SELECT jr.run_id, jr.node, j.queue, j.job_set, jr.pool, j.user_id, j.groups, j.submit_message, jr.pod_requirements_overlay + SELECT jr.run_id, jr.run_index, jr.node, j.queue, j.job_set, jr.pool, j.user_id, j.groups, j.submit_message, jr.pod_requirements_overlay FROM runs jr LEFT JOIN %s as tmp ON (tmp.run_id = jr.run_id) JOIN jobs j @@ -374,7 +375,7 @@ func (r *PostgresJobRepository) FetchJobRunLeases(ctx *armadacontext.Context, ex defer rows.Close() for rows.Next() { run := JobRunLease{} - err = rows.Scan(&run.RunID, &run.Node, &run.Queue, &run.JobSet, &run.Pool, &run.UserID, &run.Groups, &run.SubmitMessage, &run.PodRequirementsOverlay) + err = rows.Scan(&run.RunID, &run.RunIndex, &run.Node, &run.Queue, &run.JobSet, &run.Pool, &run.UserID, &run.Groups, &run.SubmitMessage, &run.PodRequirementsOverlay) if err != nil { return errors.WithStack(err) } diff --git a/internal/scheduler/database/migrations/022_add_job_run_index.sql b/internal/scheduler/database/migrations/022_add_job_run_index.sql new file mode 100644 index 00000000000..f1186970ef3 --- /dev/null +++ b/internal/scheduler/database/migrations/022_add_job_run_index.sql @@ -0,0 +1 @@ +ALTER TABLE runs ADD COLUMN IF NOT EXISTS run_index bigint NOT NULL DEFAULT 0; diff --git a/internal/scheduler/database/models.go b/internal/scheduler/database/models.go index 5176a5f48d6..fe83da0eb3e 100644 --- a/internal/scheduler/database/models.go +++ b/internal/scheduler/database/models.go @@ -89,4 +89,5 @@ type Run struct { PreemptRequested bool `db:"preempt_requested"` Queue string `db:"queue"` Pool string `db:"pool"` + RunIndex int64 `db:"run_index"` } diff --git a/internal/scheduler/database/query.sql.go b/internal/scheduler/database/query.sql.go index 8eb871dc04a..eecb27fb50c 100644 --- a/internal/scheduler/database/query.sql.go +++ b/internal/scheduler/database/query.sql.go @@ -468,7 +468,7 @@ func (q *Queries) SelectInitialJobs(ctx context.Context, arg SelectInitialJobsPa } const selectInitialRuns = `-- name: SelectInitialRuns :many -SELECT run_id, job_id, created, job_set, executor, node, cancelled, running, succeeded, failed, returned, run_attempted, serial, last_modified, leased_timestamp, pending_timestamp, running_timestamp, terminated_timestamp, scheduled_at_priority, preempted, pending, preempted_timestamp, pod_requirements_overlay, preempt_requested, queue, pool FROM runs WHERE serial > $1 AND job_id = ANY($3::text[]) ORDER BY serial LIMIT $2 +SELECT run_id, job_id, created, job_set, executor, node, cancelled, running, succeeded, failed, returned, run_attempted, serial, last_modified, leased_timestamp, pending_timestamp, running_timestamp, terminated_timestamp, scheduled_at_priority, preempted, pending, preempted_timestamp, pod_requirements_overlay, preempt_requested, queue, pool, run_index FROM runs WHERE serial > $1 AND job_id = ANY($3::text[]) ORDER BY serial LIMIT $2 ` type SelectInitialRunsParams struct { @@ -513,6 +513,7 @@ func (q *Queries) SelectInitialRuns(ctx context.Context, arg SelectInitialRunsPa &i.PreemptRequested, &i.Queue, &i.Pool, + &i.RunIndex, ); err != nil { return nil, err } @@ -790,7 +791,7 @@ func (q *Queries) SelectNewJobs(ctx context.Context, arg SelectNewJobsParams) ([ } const selectNewRuns = `-- name: SelectNewRuns :many -SELECT run_id, job_id, created, job_set, executor, node, cancelled, running, succeeded, failed, returned, run_attempted, serial, last_modified, leased_timestamp, pending_timestamp, running_timestamp, terminated_timestamp, scheduled_at_priority, preempted, pending, preempted_timestamp, pod_requirements_overlay, preempt_requested, queue, pool FROM runs WHERE serial > $1 ORDER BY serial LIMIT $2 +SELECT run_id, job_id, created, job_set, executor, node, cancelled, running, succeeded, failed, returned, run_attempted, serial, last_modified, leased_timestamp, pending_timestamp, running_timestamp, terminated_timestamp, scheduled_at_priority, preempted, pending, preempted_timestamp, pod_requirements_overlay, preempt_requested, queue, pool, run_index FROM runs WHERE serial > $1 ORDER BY serial LIMIT $2 ` type SelectNewRunsParams struct { @@ -834,6 +835,7 @@ func (q *Queries) SelectNewRuns(ctx context.Context, arg SelectNewRunsParams) ([ &i.PreemptRequested, &i.Queue, &i.Pool, + &i.RunIndex, ); err != nil { return nil, err } @@ -846,7 +848,7 @@ func (q *Queries) SelectNewRuns(ctx context.Context, arg SelectNewRunsParams) ([ } const selectNewRunsForJobs = `-- name: SelectNewRunsForJobs :many -SELECT run_id, job_id, created, job_set, executor, node, cancelled, running, succeeded, failed, returned, run_attempted, serial, last_modified, leased_timestamp, pending_timestamp, running_timestamp, terminated_timestamp, scheduled_at_priority, preempted, pending, preempted_timestamp, pod_requirements_overlay, preempt_requested, queue, pool FROM runs WHERE serial > $1 AND job_id = ANY($2::text[]) ORDER BY serial +SELECT run_id, job_id, created, job_set, executor, node, cancelled, running, succeeded, failed, returned, run_attempted, serial, last_modified, leased_timestamp, pending_timestamp, running_timestamp, terminated_timestamp, scheduled_at_priority, preempted, pending, preempted_timestamp, pod_requirements_overlay, preempt_requested, queue, pool, run_index FROM runs WHERE serial > $1 AND job_id = ANY($2::text[]) ORDER BY serial ` type SelectNewRunsForJobsParams struct { @@ -890,6 +892,7 @@ func (q *Queries) SelectNewRunsForJobs(ctx context.Context, arg SelectNewRunsFor &i.PreemptRequested, &i.Queue, &i.Pool, + &i.RunIndex, ); err != nil { return nil, err } diff --git a/internal/scheduler/jobdb/job.go b/internal/scheduler/jobdb/job.go index 551940f1643..abe0f6675e2 100644 --- a/internal/scheduler/jobdb/job.go +++ b/internal/scheduler/jobdb/job.go @@ -670,8 +670,10 @@ func (job *Job) ValidateResourceRequests() error { // WithNewRun creates a copy of the job with a new run on the given executor. func (job *Job) WithNewRun(executor, nodeId, nodeName, pool string, scheduledAtPriority int32) *Job { now := job.jobDb.clock.Now() + nextRunIndex := len(job.runsById) return job.WithUpdatedRun(job.jobDb.CreateRun( job.jobDb.uuidProvider.New(), + uint32(nextRunIndex), job.Id(), now.UnixNano(), executor, diff --git a/internal/scheduler/jobdb/job_run.go b/internal/scheduler/jobdb/job_run.go index 412ca46a9c0..abf6b7f164e 100644 --- a/internal/scheduler/jobdb/job_run.go +++ b/internal/scheduler/jobdb/job_run.go @@ -15,6 +15,8 @@ import ( type JobRun struct { // Unique identifier for the run. id string + + index uint32 // Id of the job this run is associated with. jobId string // Time at which the run was created. @@ -157,6 +159,9 @@ func (run *JobRun) Equal(other *JobRun) bool { if run.id != other.id { return false } + if run.index != other.index { + return false + } if run.jobId != other.jobId { return false } @@ -213,6 +218,7 @@ func MinimalRun(id string, creationTime int64) *JobRun { // CreateRun creates a new scheduler job run from a database job run func (jobDb *JobDb) CreateRun( id string, + index uint32, jobId string, creationTime int64, executor string, @@ -238,6 +244,7 @@ func (jobDb *JobDb) CreateRun( ) *JobRun { return &JobRun{ id: id, + index: index, jobId: jobId, created: creationTime, executor: jobDb.stringInterner.Intern(executor), @@ -268,6 +275,11 @@ func (run *JobRun) Id() string { return run.id } +// Index returns the index of the JobRun with respect to the Job. +func (run *JobRun) Index() uint32 { + return run.index +} + // JobId returns the id of the job this run is associated with. func (run *JobRun) JobId() string { return run.jobId diff --git a/internal/scheduler/jobdb/job_run_test.go b/internal/scheduler/jobdb/job_run_test.go index a7bc69c821f..f668941cdcb 100644 --- a/internal/scheduler/jobdb/job_run_test.go +++ b/internal/scheduler/jobdb/job_run_test.go @@ -48,6 +48,7 @@ func init() { var baseJobRun = jobDb.CreateRun( uuid.New().String(), + 1, uuid.NewString(), 5, "test-executor", @@ -125,6 +126,7 @@ func TestJobRun_TestRunAttempted(t *testing.T) { func TestDeepCopy(t *testing.T) { run := jobDb.CreateRun( uuid.NewString(), + 1, "job id", 1, "executor", @@ -150,6 +152,7 @@ func TestDeepCopy(t *testing.T) { ) expected := jobDb.CreateRun( run.id, + 1, "job id", 1, "executor", diff --git a/internal/scheduler/jobdb/reconciliation.go b/internal/scheduler/jobdb/reconciliation.go index dfcd25576e0..88b1c76e2fc 100644 --- a/internal/scheduler/jobdb/reconciliation.go +++ b/internal/scheduler/jobdb/reconciliation.go @@ -308,6 +308,7 @@ func (jobDb *JobDb) schedulerRunFromDatabaseRun(dbRun *database.Run) *JobRun { nodeId := api.NodeIdFromExecutorAndNodeName(dbRun.Executor, dbRun.Node) return jobDb.CreateRun( dbRun.RunID, + uint32(dbRun.RunIndex), dbRun.JobID, dbRun.Created, dbRun.Executor, diff --git a/internal/scheduler/scheduler.go b/internal/scheduler/scheduler.go index 7c1b81384b8..eb84f7a32b3 100644 --- a/internal/scheduler/scheduler.go +++ b/internal/scheduler/scheduler.go @@ -605,6 +605,7 @@ func AppendEventSequencesFromScheduledJobs(eventSequences []*armadaevents.EventS Event: &armadaevents.EventSequence_Event_JobRunLeased{ JobRunLeased: &armadaevents.JobRunLeased{ RunId: run.Id(), + RunIndex: run.Index(), JobId: job.Id(), ExecutorId: run.Executor(), // NodeId here refers to the unique identifier of the node in an executor cluster, diff --git a/internal/scheduler/scheduler_test.go b/internal/scheduler/scheduler_test.go index a9dd5037eef..103f2baae07 100644 --- a/internal/scheduler/scheduler_test.go +++ b/internal/scheduler/scheduler_test.go @@ -229,6 +229,7 @@ var returnedOnceLeasedJob = testfixtures.NewJob( true, ).WithUpdatedRun(testfixtures.JobDb.CreateRun( uuid.NewString(), + 0, "01h3w2wtdchtc80hgyp782shrv", 0, "testExecutor", @@ -309,6 +310,7 @@ var ( true, ).WithUpdatedRun(testfixtures.JobDb.CreateRun( uuid.NewString(), + 1, requeuedJobId, time.Now().Unix(), "testExecutor", @@ -1157,6 +1159,7 @@ func TestScheduler_TestSyncInitialState(t *testing.T) { queuedJob.WithUpdatedRun( testfixtures.JobDb.CreateRun( leasedJob.LatestRun().Id(), + leasedJob.LatestRun().Index(), queuedJob.Id(), 123, "test-executor", @@ -1313,6 +1316,7 @@ func TestScheduler_TestSyncState(t *testing.T) { queuedJob.WithUpdatedRun( testfixtures.JobDb.CreateRun( leasedJob.LatestRun().Id(), + leasedJob.LatestRun().Index(), queuedJob.Id(), 123, "test-executor", diff --git a/internal/scheduleringester/instructions.go b/internal/scheduleringester/instructions.go index 2f524890354..8917cf6aa5a 100644 --- a/internal/scheduleringester/instructions.go +++ b/internal/scheduleringester/instructions.go @@ -195,6 +195,7 @@ func (c *JobSetEventsInstructionConverter) handleJobRunLeased(jobRunLeased *arma Queue: meta.queue, DbRun: &schedulerdb.Run{ RunID: runId, + RunIndex: int64(jobRunLeased.RunIndex), JobID: jobRunLeased.JobId, Created: eventTime.UnixNano(), JobSet: meta.jobset, diff --git a/pkg/armadaevents/events.pb.go b/pkg/armadaevents/events.pb.go index 82c14c8bb99..4d0cf4d05f7 100644 --- a/pkg/armadaevents/events.pb.go +++ b/pkg/armadaevents/events.pb.go @@ -1643,7 +1643,8 @@ type JobRunLeased struct { RunId string `protobuf:"bytes,11,opt,name=run_id,json=runId,proto3" json:"runId,omitempty"` // pool is the pool this run was scheduled on to // This would be determined by the pool of the node this run was scheduled on to, at the time of scheduling - Pool string `protobuf:"bytes,12,opt,name=pool,proto3" json:"pool,omitempty"` + Pool string `protobuf:"bytes,12,opt,name=pool,proto3" json:"pool,omitempty"` + RunIndex uint32 `protobuf:"varint,13,opt,name=run_index,json=runIndex,proto3" json:"runIndex,omitempty"` } func (m *JobRunLeased) Reset() { *m = JobRunLeased{} } @@ -1742,6 +1743,13 @@ func (m *JobRunLeased) GetPool() string { return "" } +func (m *JobRunLeased) GetRunIndex() uint32 { + if m != nil { + return m.RunIndex + } + return 0 +} + // Indicates that a job has been assigned to nodes by Kubernetes. type JobRunAssigned struct { // Runtime information, e.g., which node the job is running on, its IP address etc, @@ -3661,13 +3669,13 @@ func init() { func init() { proto.RegisterFile("pkg/armadaevents/events.proto", fileDescriptor_6aab92ca59e015f8) } var fileDescriptor_6aab92ca59e015f8 = []byte{ - // 3668 bytes of a gzipped FileDescriptorProto + // 3688 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3b, 0x4d, 0x6f, 0x1b, 0x47, 0x96, 0x6e, 0x7e, 0xf3, 0x51, 0x1f, 0x74, 0xe9, 0xc3, 0xb4, 0x12, 0x8b, 0x32, 0x9d, 0x4d, 0xec, 0x20, 0xa1, 0x1c, 0x67, 0xb3, 0xc8, 0xc7, 0x22, 0x81, 0x68, 0x2b, 0xb6, 0x15, 0xcb, 0x56, 0x28, 0x2b, 0xeb, 0x5d, 0x04, 0x60, 0x9a, 0xec, 0x12, 0xd5, 0x16, 0xd9, 0xdd, 0xe9, 0x0f, 0x45, 0x02, - 0x02, 0x6c, 0xb2, 0xc8, 0xee, 0x39, 0x97, 0x05, 0x16, 0xb9, 0x6c, 0x2e, 0x7b, 0xd8, 0x05, 0xf6, - 0xb2, 0xc0, 0xce, 0x75, 0xae, 0x73, 0x18, 0x0c, 0x72, 0x19, 0x60, 0x0e, 0x13, 0x62, 0x90, 0x60, + 0x02, 0x6c, 0xb2, 0xc8, 0xee, 0x39, 0x97, 0x05, 0x16, 0xb9, 0xec, 0x5e, 0xf6, 0xb0, 0x0b, 0xec, + 0x65, 0x81, 0x99, 0xeb, 0x5c, 0xe7, 0x30, 0x18, 0xe4, 0x12, 0x60, 0x0e, 0x13, 0x62, 0x90, 0x60, 0x2e, 0x3c, 0xcc, 0x6f, 0x18, 0xd4, 0x47, 0x77, 0x57, 0x35, 0x8b, 0x16, 0xe5, 0x58, 0x86, 0x27, 0x73, 0x92, 0xfa, 0x7d, 0x56, 0xbf, 0xf7, 0xea, 0xf5, 0x7b, 0xaf, 0x8a, 0x70, 0xc1, 0xd9, 0xef, 0xae, 0xea, 0x6e, 0x5f, 0x37, 0x74, 0x7c, 0x80, 0x2d, 0xdf, 0x5b, 0x65, 0x7f, 0xea, 0x8e, 0x6b, @@ -3687,7 +3695,7 @@ var fileDescriptor_6aab92ca59e015f8 = []byte{ 0x83, 0x6a, 0x99, 0x80, 0x6e, 0x1b, 0x02, 0x4b, 0x8e, 0x41, 0xd0, 0x2b, 0x90, 0xeb, 0xba, 0x76, 0xe0, 0x78, 0x95, 0xcc, 0x4a, 0x3a, 0xa4, 0x66, 0x10, 0x91, 0x9a, 0x41, 0xd0, 0x3d, 0xc8, 0x31, 0x07, 0x56, 0xb2, 0x2b, 0xe9, 0xcb, 0xa5, 0x6b, 0x17, 0xeb, 0xa2, 0x57, 0xeb, 0xd2, 0x0b, 0xb3, - 0x27, 0x26, 0x90, 0xe1, 0x45, 0x81, 0x3c, 0x0e, 0x7e, 0x31, 0x07, 0x59, 0x4a, 0x87, 0x3e, 0x80, + 0x27, 0x26, 0x90, 0xe1, 0x45, 0x81, 0x3c, 0x0e, 0x7e, 0x39, 0x07, 0x59, 0x4a, 0x87, 0x3e, 0x80, 0x7c, 0xc7, 0xc5, 0xc4, 0xfa, 0x15, 0xb4, 0xa2, 0x5d, 0x2e, 0x5d, 0x5b, 0xaa, 0x33, 0xaf, 0xd6, 0x43, 0xaf, 0xd6, 0xef, 0x87, 0x5e, 0x6d, 0x2c, 0x0c, 0x07, 0xd5, 0xb3, 0x9c, 0x5c, 0x90, 0x1a, 0x4a, 0x40, 0x5b, 0x50, 0xf4, 0x82, 0x76, 0xdf, 0xf4, 0x37, 0xec, 0x36, 0xb5, 0x77, 0xe9, 0xda, @@ -3740,158 +3748,159 @@ var fileDescriptor_6aab92ca59e015f8 = []byte{ 0x2c, 0x6c, 0x5e, 0x85, 0x47, 0x2f, 0x43, 0x8e, 0xb4, 0x15, 0xa6, 0x41, 0xab, 0x23, 0xde, 0x82, 0x3c, 0xb4, 0xdb, 0x52, 0x63, 0x90, 0xa5, 0x00, 0x42, 0xeb, 0x06, 0x16, 0xa1, 0xcd, 0xc7, 0xb4, 0x6e, 0x60, 0xc9, 0xb4, 0x14, 0x40, 0x9d, 0xa1, 0x1f, 0x74, 0xd5, 0xce, 0x28, 0x4c, 0xea, 0x8c, - 0xb5, 0x83, 0xee, 0x23, 0x9d, 0xa1, 0xab, 0xf0, 0xa2, 0x33, 0x94, 0x04, 0x4b, 0xdf, 0x6a, 0xb0, - 0x34, 0xde, 0xcf, 0xe8, 0x12, 0xa4, 0xf7, 0xf1, 0x11, 0xef, 0xc9, 0xce, 0x0e, 0x07, 0xd5, 0xe9, - 0x7d, 0x7c, 0x24, 0x48, 0x25, 0x58, 0xf4, 0x8f, 0x90, 0x3d, 0xd0, 0x7b, 0x01, 0xe6, 0x25, 0x7f, - 0xbd, 0xce, 0xda, 0xc9, 0xba, 0xd8, 0x4e, 0xd6, 0x9d, 0xfd, 0x2e, 0x01, 0xd4, 0x43, 0x2b, 0xd4, - 0x3f, 0x0c, 0x74, 0xcb, 0x37, 0xfd, 0x23, 0x66, 0x3b, 0x2a, 0x40, 0xb4, 0x1d, 0x05, 0xbc, 0x9d, - 0x7a, 0x53, 0x5b, 0xfa, 0x4f, 0x0d, 0xce, 0x8f, 0xf5, 0xf7, 0x33, 0xb1, 0x42, 0x62, 0xc4, 0xf1, - 0xfe, 0x79, 0x16, 0x96, 0xb8, 0x91, 0x29, 0x68, 0xe5, 0xd4, 0x46, 0xa6, 0x90, 0x2a, 0xa7, 0x6b, - 0xff, 0x97, 0x83, 0x62, 0xd4, 0xe0, 0xa1, 0x5b, 0x50, 0x36, 0xb0, 0x11, 0x38, 0x3d, 0xb3, 0x43, - 0x23, 0x8d, 0x04, 0x35, 0xeb, 0xa8, 0x69, 0x76, 0x95, 0x70, 0x52, 0x78, 0xcf, 0x26, 0x50, 0xe8, - 0x1a, 0x14, 0x78, 0x23, 0x73, 0x44, 0xf3, 0xda, 0x74, 0x63, 0x71, 0x38, 0xa8, 0xa2, 0x10, 0x26, - 0xb0, 0x46, 0x74, 0xa8, 0x09, 0xc0, 0x26, 0x03, 0x9b, 0xd8, 0xd7, 0x79, 0x4b, 0x55, 0x91, 0x77, - 0xc3, 0xbd, 0x08, 0xcf, 0x7a, 0xfc, 0x98, 0x5e, 0xec, 0xf1, 0x63, 0x28, 0xfa, 0x18, 0xa0, 0xaf, - 0x9b, 0x16, 0xe3, 0xe3, 0xfd, 0x53, 0x6d, 0x5c, 0x86, 0xdd, 0x8c, 0x28, 0x99, 0xf4, 0x98, 0x53, - 0x94, 0x1e, 0x43, 0xd1, 0x3d, 0xc8, 0xf3, 0x59, 0x46, 0x25, 0x47, 0x37, 0xef, 0xf2, 0x38, 0xd1, - 0x5c, 0x2c, 0xed, 0xc6, 0x39, 0x8b, 0xd8, 0x8d, 0x73, 0x10, 0x31, 0x5b, 0xcf, 0xdc, 0xc5, 0xbe, - 0xd9, 0xc7, 0x34, 0x9b, 0x70, 0xb3, 0x85, 0x30, 0xd1, 0x6c, 0x21, 0x0c, 0xbd, 0x09, 0xa0, 0xfb, - 0x9b, 0xb6, 0xe7, 0xdf, 0xb3, 0x3a, 0x98, 0x76, 0x44, 0x05, 0xb6, 0xfc, 0x18, 0x2a, 0x2e, 0x3f, - 0x86, 0xa2, 0x77, 0xa0, 0xe4, 0xf0, 0x2f, 0x70, 0xbb, 0x87, 0x69, 0xc7, 0x53, 0x60, 0x05, 0x83, - 0x00, 0x16, 0x78, 0x45, 0x6a, 0x74, 0x13, 0x66, 0x3b, 0xb6, 0xd5, 0x09, 0x5c, 0x17, 0x5b, 0x9d, - 0xa3, 0x6d, 0x7d, 0x17, 0xd3, 0xee, 0xa6, 0xc0, 0x42, 0x25, 0x81, 0x12, 0x43, 0x25, 0x81, 0x42, - 0x6f, 0x40, 0x31, 0x9a, 0x0c, 0xd1, 0x06, 0xa6, 0xc8, 0x07, 0x0d, 0x21, 0x50, 0x60, 0x8e, 0x29, - 0xc9, 0xe2, 0x4d, 0xef, 0x06, 0x0f, 0x3a, 0x4c, 0x9b, 0x12, 0xbe, 0x78, 0x01, 0x2c, 0x2e, 0x5e, - 0x00, 0x0b, 0xf9, 0x7d, 0xe6, 0xb8, 0xfc, 0x1e, 0x6d, 0x97, 0xe9, 0xf2, 0xcc, 0x46, 0xa6, 0x30, - 0x5b, 0x2e, 0xd7, 0x7e, 0xad, 0xc1, 0xbc, 0x2a, 0x6a, 0x12, 0x11, 0xac, 0x3d, 0x91, 0x08, 0xfe, - 0x08, 0x0a, 0x8e, 0x6d, 0xb4, 0x3c, 0x07, 0x77, 0x78, 0x3e, 0x48, 0xc4, 0xef, 0x96, 0x6d, 0x6c, - 0x3b, 0xb8, 0xf3, 0x0f, 0xa6, 0xbf, 0xb7, 0x76, 0x60, 0x9b, 0xc6, 0x1d, 0xd3, 0xe3, 0x81, 0xe6, - 0x30, 0x8c, 0x54, 0xa4, 0xe4, 0x39, 0xb0, 0x51, 0x80, 0x1c, 0xd3, 0x52, 0xfb, 0x4d, 0x1a, 0xca, - 0xc9, 0x48, 0xfd, 0x4b, 0x7a, 0x15, 0xf4, 0x00, 0xf2, 0x26, 0xeb, 0x81, 0x78, 0x0d, 0xf5, 0x37, - 0x42, 0xc6, 0xac, 0xc7, 0x03, 0xd1, 0xfa, 0xc1, 0x6b, 0x75, 0xde, 0x2c, 0x51, 0x13, 0x50, 0xc9, - 0x9c, 0x53, 0x96, 0xcc, 0x81, 0xa8, 0x09, 0x79, 0x0f, 0xbb, 0x07, 0x66, 0x07, 0xf3, 0x7c, 0x54, - 0x15, 0x25, 0x77, 0x6c, 0x17, 0x13, 0x99, 0xdb, 0x8c, 0x24, 0x96, 0xc9, 0x79, 0x64, 0x99, 0x1c, - 0x88, 0x3e, 0x82, 0x62, 0xc7, 0xb6, 0x76, 0xcd, 0xee, 0xa6, 0xee, 0xf0, 0x8c, 0x74, 0x41, 0x25, - 0xf5, 0x7a, 0x48, 0xc4, 0xe7, 0x3a, 0xe1, 0x63, 0x62, 0xae, 0x13, 0x51, 0xc5, 0x0e, 0xfd, 0x53, - 0x06, 0x20, 0x76, 0x0e, 0x7a, 0x0b, 0x4a, 0xf8, 0x10, 0x77, 0x02, 0xdf, 0xa6, 0xb3, 0x4e, 0x2d, - 0x1e, 0x91, 0x86, 0x60, 0x29, 0xec, 0x21, 0x86, 0x92, 0xbd, 0x69, 0xe9, 0x7d, 0xec, 0x39, 0x7a, - 0x27, 0x9c, 0xad, 0xd2, 0xc5, 0x44, 0x40, 0x71, 0x6f, 0x46, 0x40, 0xf4, 0x22, 0x64, 0xe8, 0x34, - 0x96, 0x8d, 0x55, 0xd1, 0x70, 0x50, 0x9d, 0xb1, 0xe4, 0x39, 0x2c, 0xc5, 0xa3, 0xf7, 0x60, 0x7a, - 0x3f, 0x0a, 0x3c, 0xb2, 0xb6, 0x0c, 0x65, 0xa0, 0xc5, 0x6d, 0x8c, 0x90, 0x56, 0x37, 0x25, 0xc2, - 0xd1, 0x2e, 0x94, 0x74, 0xcb, 0xb2, 0x7d, 0xfa, 0xd9, 0x09, 0x47, 0xad, 0x57, 0xc6, 0x85, 0x69, - 0x7d, 0x2d, 0xa6, 0x65, 0xe5, 0x12, 0xcd, 0x17, 0x82, 0x04, 0x31, 0x5f, 0x08, 0x60, 0xd4, 0x84, - 0x5c, 0x4f, 0x6f, 0xe3, 0x5e, 0x98, 0xe7, 0x5f, 0x18, 0xab, 0xe2, 0x0e, 0x25, 0x63, 0xd2, 0xe9, - 0x40, 0x97, 0xf1, 0x89, 0x03, 0x5d, 0x06, 0x59, 0xda, 0x85, 0x72, 0x72, 0x3d, 0x93, 0x95, 0x07, - 0x57, 0xc4, 0xf2, 0xa0, 0x78, 0x6c, 0x45, 0xa2, 0x43, 0x49, 0x58, 0xd4, 0x69, 0xa8, 0xa8, 0xfd, - 0xb7, 0x06, 0xf3, 0xaa, 0xbd, 0x8b, 0x36, 0x85, 0x1d, 0xaf, 0xf1, 0xb1, 0x91, 0x22, 0xd4, 0x39, - 0xef, 0x98, 0xad, 0x1e, 0x6f, 0xf4, 0x06, 0xcc, 0x58, 0xb6, 0x81, 0x5b, 0x3a, 0x51, 0xd0, 0x33, - 0x3d, 0xbf, 0x92, 0xa2, 0xa3, 0x78, 0x3a, 0x6e, 0x22, 0x98, 0xb5, 0x10, 0x21, 0x70, 0x4f, 0x4b, - 0x88, 0xda, 0x67, 0x30, 0x9b, 0x18, 0x06, 0x4b, 0xc5, 0x4a, 0x6a, 0xc2, 0x62, 0x25, 0xfe, 0x82, - 0xa4, 0x27, 0xfb, 0x82, 0xd4, 0xfe, 0x35, 0x05, 0x25, 0xa1, 0x33, 0x47, 0x0f, 0x61, 0x96, 0x7f, - 0xcd, 0x4c, 0xab, 0xcb, 0x3a, 0xc0, 0x14, 0x1f, 0x13, 0x8d, 0x9c, 0x94, 0x6c, 0xd8, 0xed, 0xed, - 0x88, 0x96, 0x36, 0x80, 0x74, 0x8a, 0xe7, 0x49, 0x30, 0x41, 0xf1, 0x8c, 0x8c, 0x41, 0x0f, 0x60, - 0x31, 0x70, 0x48, 0x5f, 0xda, 0xf2, 0xf8, 0x99, 0x43, 0xcb, 0x0a, 0xfa, 0x6d, 0xec, 0xd2, 0xd5, - 0x67, 0x59, 0xa7, 0xc4, 0x28, 0xc2, 0x43, 0x89, 0xbb, 0x14, 0x2f, 0x76, 0x4a, 0x2a, 0xbc, 0x60, - 0x87, 0xcc, 0x84, 0x76, 0xb8, 0x05, 0x68, 0x74, 0x1a, 0x2f, 0xf9, 0x40, 0x9b, 0xcc, 0x07, 0xb5, - 0x43, 0x28, 0x27, 0x67, 0xec, 0x4f, 0xc9, 0x97, 0xfb, 0x50, 0x8c, 0x26, 0xe4, 0xe8, 0x15, 0xc8, - 0xb9, 0x58, 0xf7, 0x6c, 0x8b, 0xef, 0x16, 0xba, 0xed, 0x19, 0x44, 0xdc, 0xf6, 0x0c, 0xf2, 0x18, - 0xca, 0xee, 0xc3, 0x14, 0x33, 0xd2, 0xfb, 0x66, 0xcf, 0xc7, 0x2e, 0xba, 0x01, 0x39, 0xcf, 0xd7, - 0x7d, 0xec, 0x55, 0xb4, 0x95, 0xf4, 0xe5, 0x99, 0x6b, 0x8b, 0xa3, 0xe3, 0x6f, 0x82, 0x66, 0xeb, - 0x60, 0x94, 0xe2, 0x3a, 0x18, 0xa4, 0xf6, 0x2f, 0x1a, 0x4c, 0x89, 0x53, 0xfe, 0x27, 0x23, 0xf6, - 0x64, 0xc6, 0x20, 0x89, 0x63, 0x4a, 0x3c, 0x0c, 0x38, 0x3d, 0x5b, 0x92, 0xaf, 0x20, 0x3b, 0x4a, - 0x68, 0x05, 0x1e, 0x76, 0x79, 0xb4, 0xd2, 0xaf, 0x20, 0x03, 0xef, 0x78, 0x52, 0xb4, 0x43, 0x0c, - 0xe5, 0x6e, 0x20, 0x6b, 0x15, 0x8f, 0x16, 0x50, 0x37, 0x1e, 0xe0, 0x90, 0x4d, 0xe6, 0xd1, 0x64, - 0x34, 0xe9, 0x00, 0x87, 0xa6, 0x2c, 0x89, 0x5d, 0x4c, 0x59, 0x12, 0xe2, 0x31, 0x42, 0xe6, 0xdb, - 0x2c, 0x5d, 0x6b, 0x7c, 0x54, 0x90, 0xa8, 0x01, 0xd2, 0x27, 0xa8, 0x01, 0x5e, 0x85, 0x3c, 0x4d, - 0xba, 0xd1, 0x16, 0xa7, 0x3e, 0x21, 0x20, 0xf9, 0x98, 0x94, 0x41, 0x1e, 0x91, 0x6a, 0xb2, 0x3f, - 0x31, 0xd5, 0xb4, 0xe0, 0xfc, 0x9e, 0xee, 0xb5, 0xc2, 0xe4, 0x68, 0xb4, 0x74, 0xbf, 0x15, 0xed, - 0xf5, 0x1c, 0xad, 0xff, 0xe9, 0xf0, 0x71, 0x4f, 0xf7, 0xb6, 0x43, 0x9a, 0x35, 0x7f, 0x6b, 0x74, - 0xe7, 0x2f, 0xaa, 0x29, 0xd0, 0x0e, 0x2c, 0xa8, 0x85, 0xe7, 0xe9, 0xca, 0xe9, 0x6c, 0xdc, 0x7b, - 0xa4, 0xe4, 0x39, 0x05, 0x1a, 0x7d, 0xa9, 0x41, 0x85, 0x7c, 0x05, 0x5d, 0xfc, 0x69, 0x60, 0xba, - 0xb8, 0x4f, 0xc2, 0xa2, 0x65, 0x1f, 0x60, 0xb7, 0xa7, 0x1f, 0xf1, 0x63, 0xa6, 0x8b, 0xa3, 0x29, - 0x7f, 0xcb, 0x36, 0x9a, 0x02, 0x03, 0x7b, 0x35, 0x47, 0x06, 0xde, 0x63, 0x42, 0xc4, 0x57, 0x53, - 0x53, 0x08, 0x21, 0x04, 0x27, 0x18, 0x68, 0x95, 0x8e, 0x1d, 0x68, 0xbd, 0x08, 0x19, 0xc7, 0xb6, - 0x7b, 0xb4, 0xfd, 0xe2, 0x95, 0x1e, 0x79, 0x16, 0x2b, 0x3d, 0xf2, 0x2c, 0xce, 0x1c, 0x36, 0x32, - 0x85, 0x42, 0xb9, 0x58, 0xfb, 0x5e, 0x83, 0x19, 0xf9, 0x64, 0x6a, 0x74, 0x43, 0xa5, 0x4f, 0x7d, - 0x43, 0x65, 0x4e, 0x60, 0x8d, 0xec, 0x71, 0xd6, 0x90, 0x26, 0x2b, 0xbf, 0xd7, 0x60, 0x5a, 0x3a, - 0x14, 0xfb, 0x79, 0xbd, 0xde, 0x7f, 0xa4, 0x60, 0x51, 0xbd, 0xd4, 0x53, 0x69, 0x1d, 0x6f, 0x01, - 0x29, 0x02, 0x6f, 0xc7, 0x45, 0xd2, 0xc2, 0x48, 0xe7, 0x48, 0xcd, 0x14, 0x56, 0x90, 0x23, 0xe7, - 0x65, 0x21, 0x3b, 0x7a, 0x00, 0x25, 0x53, 0x38, 0x99, 0x4b, 0xab, 0x0e, 0x50, 0xc4, 0xf3, 0x38, - 0x36, 0x52, 0x18, 0x73, 0x0a, 0x27, 0x8a, 0x6a, 0xe4, 0x20, 0x43, 0xaa, 0xb8, 0xda, 0x01, 0xe4, - 0xf9, 0x72, 0xd0, 0xeb, 0x50, 0xa4, 0xb9, 0x93, 0x76, 0x43, 0xac, 0xe4, 0xa6, 0xe5, 0x08, 0x01, - 0x26, 0x6e, 0xa6, 0x14, 0x42, 0x18, 0xfa, 0x3b, 0x00, 0x92, 0x2e, 0x78, 0xd6, 0x4c, 0xd1, 0xdc, - 0x43, 0xbb, 0x2e, 0xc7, 0x36, 0x46, 0x52, 0x65, 0x31, 0x02, 0xd6, 0xfe, 0x37, 0x05, 0x25, 0xf1, - 0x2c, 0xf0, 0xb1, 0x94, 0x7f, 0x0e, 0x61, 0x47, 0xdc, 0xd2, 0x0d, 0x83, 0xfc, 0xc5, 0xe1, 0x87, - 0x6d, 0x75, 0xac, 0x91, 0xc2, 0xff, 0xd7, 0x42, 0x0e, 0xd6, 0xff, 0xd0, 0xfb, 0x0e, 0x66, 0x02, - 0x25, 0x68, 0x2d, 0x27, 0x71, 0x4b, 0xfb, 0xb0, 0xa0, 0x14, 0x25, 0x76, 0x2d, 0xd9, 0x27, 0xd5, - 0xb5, 0xfc, 0x57, 0x16, 0x16, 0x94, 0x67, 0xb0, 0x89, 0x08, 0x4e, 0x3f, 0x91, 0x08, 0xfe, 0x37, - 0x4d, 0x65, 0x59, 0x76, 0x00, 0xf3, 0xd6, 0x04, 0x07, 0xc3, 0x4f, 0xca, 0xc6, 0x72, 0x58, 0x64, - 0x1f, 0x2b, 0x26, 0x73, 0x93, 0xc6, 0x24, 0xba, 0xca, 0x1a, 0x40, 0xaa, 0x8b, 0x1d, 0x8f, 0x84, - 0x3b, 0x34, 0xa1, 0x2a, 0xcf, 0x41, 0xe8, 0x3d, 0x98, 0x0e, 0x39, 0xd8, 0xd8, 0xa1, 0x10, 0xcf, - 0x04, 0x38, 0x4d, 0x72, 0xf2, 0x30, 0x25, 0xc2, 0x85, 0xec, 0x57, 0x3c, 0x41, 0xf6, 0x83, 0xe3, - 0xb2, 0xdf, 0x53, 0x8d, 0x4d, 0x29, 0xd5, 0x0e, 0x34, 0x98, 0x4d, 0x5c, 0x7d, 0xf8, 0x79, 0x7d, - 0x4b, 0xbe, 0xd0, 0xa0, 0x18, 0xdd, 0xac, 0x41, 0x6b, 0x90, 0xc3, 0xec, 0x76, 0x06, 0x4b, 0x3b, - 0x73, 0x89, 0x9b, 0x73, 0x04, 0xc7, 0xef, 0xca, 0x25, 0x2e, 0x64, 0x34, 0x39, 0xe3, 0x63, 0x14, - 0xcc, 0xff, 0xaf, 0x85, 0x05, 0xf3, 0xc8, 0x2a, 0xd2, 0x3f, 0x7d, 0x15, 0xa7, 0x67, 0xba, 0x5f, - 0x16, 0x21, 0x4b, 0xd7, 0x42, 0x1a, 0x5f, 0x1f, 0xbb, 0x7d, 0xd3, 0xd2, 0x7b, 0x34, 0x14, 0x0b, - 0x6c, 0x57, 0x87, 0x30, 0x71, 0x57, 0x87, 0x30, 0xb4, 0x07, 0xb3, 0xf1, 0x38, 0x8d, 0x8a, 0x51, - 0x5f, 0xd5, 0xfb, 0x40, 0x26, 0x62, 0x23, 0xfe, 0x04, 0xa7, 0x7c, 0xd6, 0x9e, 0x40, 0x22, 0x03, - 0x66, 0x3a, 0xb6, 0xe5, 0xeb, 0xa6, 0x85, 0x5d, 0xa6, 0x28, 0xad, 0xba, 0xaa, 0x74, 0x5d, 0xa2, - 0x61, 0x43, 0x0e, 0x99, 0x4f, 0xbe, 0xaa, 0x24, 0xe3, 0xd0, 0x27, 0x30, 0x1d, 0x36, 0x2e, 0x4c, - 0x49, 0x46, 0x75, 0x55, 0x69, 0x5d, 0x24, 0x61, 0x9b, 0x41, 0xe2, 0x92, 0xaf, 0x2a, 0x49, 0x28, - 0xd4, 0x83, 0xb2, 0x63, 0x1b, 0x3b, 0x16, 0x2f, 0xd7, 0xf5, 0x76, 0x0f, 0xf3, 0x19, 0xee, 0xf2, - 0x48, 0x41, 0x22, 0x51, 0xb1, 0x44, 0x9d, 0xe4, 0x95, 0x2f, 0xff, 0x25, 0xb1, 0xe8, 0x63, 0x98, - 0xea, 0x91, 0xfe, 0x6d, 0xfd, 0xd0, 0x31, 0x5d, 0x6c, 0xa8, 0xaf, 0xea, 0xdd, 0x11, 0x28, 0x58, - 0x9a, 0x14, 0x79, 0xe4, 0x1b, 0x0a, 0x22, 0x86, 0x78, 0xbf, 0xaf, 0x1f, 0x36, 0x03, 0xcb, 0x5b, - 0x3f, 0xe4, 0xd7, 0xae, 0xf2, 0x2a, 0xef, 0x6f, 0xca, 0x44, 0xcc, 0xfb, 0x09, 0x4e, 0xd9, 0xfb, - 0x09, 0x24, 0xba, 0x43, 0xbf, 0x02, 0xcc, 0x25, 0xec, 0xca, 0xde, 0xe2, 0x88, 0xb5, 0x98, 0x37, - 0xd8, 0xb0, 0x86, 0x3f, 0x49, 0x42, 0x23, 0x09, 0xdc, 0x07, 0xf4, 0xb5, 0x9b, 0xd8, 0x0f, 0x5c, - 0x0b, 0x1b, 0xbc, 0x8d, 0x1a, 0xf5, 0x81, 0x44, 0x15, 0xf9, 0x40, 0x82, 0x8e, 0xf8, 0x40, 0xc2, - 0xa2, 0xcf, 0x61, 0x3e, 0x71, 0x01, 0x89, 0xbd, 0x47, 0x49, 0x75, 0x80, 0xb1, 0xa1, 0xa0, 0x64, - 0x1d, 0xaf, 0x4a, 0x86, 0xa4, 0x59, 0xa9, 0x85, 0x68, 0xef, 0xea, 0x56, 0x77, 0xc3, 0x6e, 0xcb, - 0x31, 0x37, 0xa5, 0xd2, 0x7e, 0x53, 0x41, 0xc9, 0xb4, 0xab, 0x64, 0xc8, 0xda, 0x55, 0x14, 0xd1, - 0x65, 0x23, 0x52, 0xc4, 0x44, 0x97, 0xf2, 0x54, 0x97, 0x8d, 0x18, 0x81, 0x70, 0xd9, 0x88, 0x01, - 0x14, 0x97, 0x8d, 0x38, 0x65, 0x21, 0x9c, 0xf3, 0xd4, 0x3e, 0x84, 0xd9, 0x44, 0x7a, 0x41, 0xef, - 0x42, 0x74, 0x85, 0xe5, 0xfe, 0x91, 0x13, 0xd6, 0xae, 0xd2, 0x95, 0x17, 0x02, 0x57, 0x5d, 0x79, - 0x21, 0xf0, 0xda, 0xd7, 0x19, 0x28, 0x84, 0x11, 0x75, 0x2a, 0xdd, 0xc8, 0x2a, 0xe4, 0xfb, 0xd8, - 0xa3, 0xd7, 0x54, 0x52, 0x71, 0x51, 0xc3, 0x41, 0x62, 0x51, 0xc3, 0x41, 0x72, 0xcd, 0x95, 0x7e, - 0xac, 0x9a, 0x2b, 0x33, 0x71, 0xcd, 0x85, 0xe9, 0xc9, 0xac, 0x90, 0x17, 0xc3, 0x83, 0x91, 0x47, - 0x27, 0xdb, 0xf0, 0xdc, 0x56, 0x64, 0x4c, 0x9c, 0xdb, 0x8a, 0x28, 0xb4, 0x0f, 0x67, 0x85, 0xc3, - 0x1b, 0x3e, 0xb5, 0x23, 0x19, 0x6a, 0x66, 0xfc, 0x31, 0x78, 0x93, 0x52, 0xb1, 0x7d, 0xb8, 0x9f, - 0x80, 0x8a, 0x45, 0x6b, 0x12, 0x47, 0x42, 0xc2, 0xc0, 0xed, 0xa0, 0xbb, 0xc9, 0xcd, 0x9e, 0x8f, - 0x43, 0x42, 0x84, 0x8b, 0x21, 0x21, 0xc2, 0x6b, 0x7f, 0x4c, 0xc1, 0x8c, 0xfc, 0xbe, 0xa7, 0x12, - 0x18, 0xaf, 0x43, 0x11, 0x1f, 0x9a, 0x7e, 0xab, 0x63, 0x1b, 0x98, 0x77, 0x6e, 0xd4, 0xcf, 0x04, - 0x78, 0xdd, 0x36, 0x24, 0x3f, 0x87, 0x30, 0x31, 0x9a, 0xd2, 0x13, 0x45, 0x53, 0x3c, 0x24, 0xcd, - 0x4c, 0x30, 0x24, 0x55, 0xfa, 0xa9, 0x78, 0x3a, 0x7e, 0xaa, 0x7d, 0x97, 0x82, 0x72, 0x32, 0xed, - 0x3e, 0x1b, 0x5b, 0x50, 0xde, 0x4d, 0xe9, 0x89, 0x77, 0xd3, 0x7b, 0x30, 0x4d, 0x2a, 0x33, 0xdd, - 0xf7, 0xf9, 0xad, 0xd6, 0x0c, 0x2d, 0xae, 0x58, 0x36, 0x0a, 0xac, 0xb5, 0x10, 0x2e, 0x65, 0x23, - 0x01, 0x3e, 0x12, 0xba, 0xd9, 0x13, 0x86, 0xee, 0x97, 0x29, 0x98, 0xde, 0xb2, 0x8d, 0xfb, 0xac, - 0x68, 0xf3, 0x9f, 0x15, 0x7b, 0x3e, 0xcd, 0x94, 0x56, 0x9b, 0x85, 0x69, 0xa9, 0x6a, 0xab, 0x7d, - 0xc5, 0xe2, 0x4c, 0xfe, 0x5c, 0xfd, 0xf5, 0xd9, 0x65, 0x06, 0xa6, 0xc4, 0xf2, 0xaf, 0xd6, 0x80, - 0xd9, 0x44, 0xb5, 0x26, 0xbe, 0x80, 0x36, 0xc9, 0x0b, 0xd4, 0x6e, 0xc0, 0xbc, 0xaa, 0x8c, 0x11, - 0xb2, 0x8e, 0x36, 0xc1, 0xc9, 0xce, 0x4d, 0x98, 0x57, 0x95, 0x23, 0x27, 0x5f, 0xce, 0xbb, 0xfc, - 0xd4, 0x94, 0x15, 0x0e, 0x27, 0xe7, 0xff, 0x6d, 0xd4, 0x3d, 0xc7, 0x37, 0xc8, 0xdf, 0x87, 0xb2, - 0x13, 0x3e, 0xb4, 0x78, 0x8f, 0xc6, 0xb6, 0x25, 0xed, 0x38, 0x22, 0xdc, 0x46, 0xa2, 0x59, 0x9b, - 0x91, 0x31, 0xb2, 0x1c, 0xde, 0xbf, 0xe5, 0x14, 0x72, 0x9a, 0x89, 0x46, 0x6e, 0x46, 0xc6, 0x08, - 0xa6, 0xcd, 0x1f, 0x6f, 0x5a, 0xda, 0xff, 0x65, 0x49, 0xd3, 0x3c, 0x9b, 0xb8, 0xe1, 0x8e, 0xae, - 0x42, 0x81, 0xfe, 0xfc, 0x2c, 0xee, 0x7c, 0xa9, 0x75, 0x28, 0x4c, 0x5a, 0x40, 0x9e, 0x83, 0xd0, - 0x1b, 0x50, 0x8c, 0x2e, 0xbd, 0xf3, 0xf3, 0x52, 0x16, 0x77, 0x21, 0x50, 0x8a, 0xbb, 0x10, 0xc8, - 0x9b, 0xe6, 0x7f, 0x86, 0xf3, 0x63, 0xaf, 0xbb, 0x9f, 0xe8, 0x6c, 0x2e, 0xee, 0x7e, 0x33, 0x27, - 0xea, 0x7e, 0x0f, 0x61, 0x51, 0x7d, 0x0b, 0x5d, 0xd0, 0x9e, 0x3a, 0x56, 0x7b, 0x6c, 0xfd, 0xf4, - 0x84, 0xd6, 0x4f, 0xd5, 0xf6, 0xe9, 0xb8, 0x20, 0xba, 0xed, 0x8d, 0xae, 0x40, 0xd6, 0xb1, 0xed, - 0x9e, 0xc7, 0x2f, 0x24, 0x50, 0x75, 0x14, 0x20, 0xaa, 0xa3, 0x80, 0xc7, 0x18, 0x4e, 0x04, 0x61, - 0x04, 0xc7, 0x77, 0xd7, 0x9f, 0x82, 0x75, 0x5f, 0xbe, 0x0a, 0x85, 0xf0, 0xd0, 0x17, 0x01, 0xe4, - 0x3e, 0xdc, 0x59, 0xdf, 0x59, 0xbf, 0x51, 0x3e, 0x83, 0x4a, 0x90, 0xdf, 0x5a, 0xbf, 0x7b, 0xe3, - 0xf6, 0xdd, 0x9b, 0x65, 0x8d, 0x3c, 0x34, 0x77, 0xee, 0xde, 0x25, 0x0f, 0xa9, 0x97, 0xef, 0x88, - 0x17, 0xc9, 0x78, 0xe5, 0x36, 0x05, 0x85, 0x35, 0xc7, 0xa1, 0x29, 0x84, 0xf1, 0xae, 0x1f, 0x98, - 0x64, 0x27, 0x97, 0x35, 0x94, 0x87, 0xf4, 0xbd, 0x7b, 0x9b, 0xe5, 0x14, 0x9a, 0x87, 0xf2, 0x0d, - 0xac, 0x1b, 0x3d, 0xd3, 0xc2, 0x61, 0xde, 0x2a, 0xa7, 0x1b, 0x0f, 0x7f, 0xf5, 0xc3, 0xb2, 0xf6, - 0xdd, 0x0f, 0xcb, 0xda, 0x1f, 0x7e, 0x58, 0xd6, 0xbe, 0xfe, 0x71, 0xf9, 0xcc, 0x77, 0x3f, 0x2e, - 0x9f, 0xf9, 0xdd, 0x8f, 0xcb, 0x67, 0xfe, 0xe9, 0x6a, 0xd7, 0xf4, 0xf7, 0x82, 0x76, 0xbd, 0x63, - 0xf7, 0xf9, 0xef, 0x68, 0x1d, 0xd7, 0x26, 0x09, 0x82, 0x3f, 0xad, 0x26, 0x7f, 0x60, 0xfb, 0x3f, - 0xa9, 0x0b, 0x6b, 0xf4, 0x71, 0x8b, 0xd1, 0xd5, 0x6f, 0xdb, 0x75, 0x06, 0xa0, 0x3f, 0xa9, 0xf4, - 0xda, 0x39, 0xfa, 0xd3, 0xc9, 0xd7, 0xff, 0x1c, 0x00, 0x00, 0xff, 0xff, 0x6d, 0x29, 0x28, 0x70, - 0x9b, 0x3b, 0x00, 0x00, + 0xb5, 0x83, 0xee, 0x23, 0x9d, 0xa1, 0xab, 0xf0, 0xa2, 0x33, 0x94, 0x04, 0x4b, 0xff, 0xa5, 0xc1, + 0xd2, 0x78, 0x3f, 0xa3, 0x4b, 0x90, 0xde, 0xc7, 0x47, 0xbc, 0x27, 0x3b, 0x3b, 0x1c, 0x54, 0xa7, + 0xf7, 0xf1, 0x91, 0x20, 0x95, 0x60, 0xd1, 0x3f, 0x42, 0xf6, 0x40, 0xef, 0x05, 0x98, 0x97, 0xfc, + 0xf5, 0x3a, 0x6b, 0x27, 0xeb, 0x62, 0x3b, 0x59, 0x77, 0xf6, 0xbb, 0x04, 0x50, 0x0f, 0xad, 0x50, + 0xff, 0x30, 0xd0, 0x2d, 0xdf, 0xf4, 0x8f, 0x98, 0xed, 0xa8, 0x00, 0xd1, 0x76, 0x14, 0xf0, 0x76, + 0xea, 0x4d, 0x6d, 0xe9, 0x3f, 0x35, 0x38, 0x3f, 0xd6, 0xdf, 0xcf, 0xc4, 0x0a, 0x89, 0x11, 0xc7, + 0xfb, 0xe7, 0x59, 0x58, 0xe2, 0x46, 0xa6, 0xa0, 0x95, 0x53, 0x1b, 0x99, 0x42, 0xaa, 0x9c, 0xae, + 0xfd, 0x7f, 0x0e, 0x8a, 0x51, 0x83, 0x87, 0x6e, 0x41, 0xd9, 0xc0, 0x46, 0xe0, 0xf4, 0xcc, 0x0e, + 0x8d, 0x34, 0x12, 0xd4, 0xac, 0xa3, 0xa6, 0xd9, 0x55, 0xc2, 0x49, 0xe1, 0x3d, 0x9b, 0x40, 0xa1, + 0x6b, 0x50, 0xe0, 0x8d, 0xcc, 0x11, 0xcd, 0x6b, 0xd3, 0x8d, 0xc5, 0xe1, 0xa0, 0x8a, 0x42, 0x98, + 0xc0, 0x1a, 0xd1, 0xa1, 0x26, 0x00, 0x9b, 0x0c, 0x6c, 0x62, 0x5f, 0xe7, 0x2d, 0x55, 0x45, 0xde, + 0x0d, 0xf7, 0x22, 0x3c, 0xeb, 0xf1, 0x63, 0x7a, 0xb1, 0xc7, 0x8f, 0xa1, 0xe8, 0x63, 0x80, 0xbe, + 0x6e, 0x5a, 0x8c, 0x8f, 0xf7, 0x4f, 0xb5, 0x71, 0x19, 0x76, 0x33, 0xa2, 0x64, 0xd2, 0x63, 0x4e, + 0x51, 0x7a, 0x0c, 0x45, 0xf7, 0x20, 0xcf, 0x67, 0x19, 0x95, 0x1c, 0xdd, 0xbc, 0xcb, 0xe3, 0x44, + 0x73, 0xb1, 0xb4, 0x1b, 0xe7, 0x2c, 0x62, 0x37, 0xce, 0x41, 0xc4, 0x6c, 0x3d, 0x73, 0x17, 0xfb, + 0x66, 0x1f, 0xd3, 0x6c, 0xc2, 0xcd, 0x16, 0xc2, 0x44, 0xb3, 0x85, 0x30, 0xf4, 0x26, 0x80, 0xee, + 0x6f, 0xda, 0x9e, 0x7f, 0xcf, 0xea, 0x60, 0xda, 0x11, 0x15, 0xd8, 0xf2, 0x63, 0xa8, 0xb8, 0xfc, + 0x18, 0x8a, 0xde, 0x81, 0x92, 0xc3, 0xbf, 0xc0, 0xed, 0x1e, 0xa6, 0x1d, 0x4f, 0x81, 0x15, 0x0c, + 0x02, 0x58, 0xe0, 0x15, 0xa9, 0xd1, 0x4d, 0x98, 0xed, 0xd8, 0x56, 0x27, 0x70, 0x5d, 0x6c, 0x75, + 0x8e, 0xb6, 0xf5, 0x5d, 0x4c, 0xbb, 0x9b, 0x02, 0x0b, 0x95, 0x04, 0x4a, 0x0c, 0x95, 0x04, 0x0a, + 0xbd, 0x01, 0xc5, 0x68, 0x32, 0x44, 0x1b, 0x98, 0x22, 0x1f, 0x34, 0x84, 0x40, 0x81, 0x39, 0xa6, + 0x24, 0x8b, 0x37, 0xbd, 0x1b, 0x3c, 0xe8, 0x30, 0x6d, 0x4a, 0xf8, 0xe2, 0x05, 0xb0, 0xb8, 0x78, + 0x01, 0x2c, 0xe4, 0xf7, 0x99, 0xe3, 0xf2, 0x7b, 0xb4, 0x5d, 0xa6, 0xcb, 0x33, 0x1b, 0x99, 0xc2, + 0x6c, 0xb9, 0x5c, 0xfb, 0x8d, 0x06, 0xf3, 0xaa, 0xa8, 0x49, 0x44, 0xb0, 0xf6, 0x44, 0x22, 0xf8, + 0x23, 0x28, 0x38, 0xb6, 0xd1, 0xf2, 0x1c, 0xdc, 0xe1, 0xf9, 0x20, 0x11, 0xbf, 0x5b, 0xb6, 0xb1, + 0xed, 0xe0, 0xce, 0x3f, 0x98, 0xfe, 0xde, 0xda, 0x81, 0x6d, 0x1a, 0x77, 0x4c, 0x8f, 0x07, 0x9a, + 0xc3, 0x30, 0x52, 0x91, 0x92, 0xe7, 0xc0, 0x46, 0x01, 0x72, 0x4c, 0x4b, 0xed, 0xb7, 0x69, 0x28, + 0x27, 0x23, 0xf5, 0x2f, 0xe9, 0x55, 0xd0, 0x03, 0xc8, 0x9b, 0xac, 0x07, 0xe2, 0x35, 0xd4, 0xdf, + 0x08, 0x19, 0xb3, 0x1e, 0x0f, 0x44, 0xeb, 0x07, 0xaf, 0xd5, 0x79, 0xb3, 0x44, 0x4d, 0x40, 0x25, + 0x73, 0x4e, 0x59, 0x32, 0x07, 0xa2, 0x26, 0xe4, 0x3d, 0xec, 0x1e, 0x98, 0x1d, 0xcc, 0xf3, 0x51, + 0x55, 0x94, 0xdc, 0xb1, 0x5d, 0x4c, 0x64, 0x6e, 0x33, 0x92, 0x58, 0x26, 0xe7, 0x91, 0x65, 0x72, + 0x20, 0xfa, 0x08, 0x8a, 0x1d, 0xdb, 0xda, 0x35, 0xbb, 0x9b, 0xba, 0xc3, 0x33, 0xd2, 0x05, 0x95, + 0xd4, 0xeb, 0x21, 0x11, 0x9f, 0xeb, 0x84, 0x8f, 0x89, 0xb9, 0x4e, 0x44, 0x15, 0x3b, 0xf4, 0x4f, + 0x19, 0x80, 0xd8, 0x39, 0xe8, 0x2d, 0x28, 0xe1, 0x43, 0xdc, 0x09, 0x7c, 0x9b, 0xce, 0x3a, 0xb5, + 0x78, 0x44, 0x1a, 0x82, 0xa5, 0xb0, 0x87, 0x18, 0x4a, 0xf6, 0xa6, 0xa5, 0xf7, 0xb1, 0xe7, 0xe8, + 0x9d, 0x70, 0xb6, 0x4a, 0x17, 0x13, 0x01, 0xc5, 0xbd, 0x19, 0x01, 0xd1, 0x8b, 0x90, 0xa1, 0xd3, + 0x58, 0x36, 0x56, 0x45, 0xc3, 0x41, 0x75, 0xc6, 0x92, 0xe7, 0xb0, 0x14, 0x8f, 0xde, 0x83, 0xe9, + 0xfd, 0x28, 0xf0, 0xc8, 0xda, 0x32, 0x94, 0x81, 0x16, 0xb7, 0x31, 0x42, 0x5a, 0xdd, 0x94, 0x08, + 0x47, 0xbb, 0x50, 0xd2, 0x2d, 0xcb, 0xf6, 0xe9, 0x67, 0x27, 0x1c, 0xb5, 0x5e, 0x19, 0x17, 0xa6, + 0xf5, 0xb5, 0x98, 0x96, 0x95, 0x4b, 0x34, 0x5f, 0x08, 0x12, 0xc4, 0x7c, 0x21, 0x80, 0x51, 0x13, + 0x72, 0x3d, 0xbd, 0x8d, 0x7b, 0x61, 0x9e, 0x7f, 0x61, 0xac, 0x8a, 0x3b, 0x94, 0x8c, 0x49, 0xa7, + 0x03, 0x5d, 0xc6, 0x27, 0x0e, 0x74, 0x19, 0x64, 0x69, 0x17, 0xca, 0xc9, 0xf5, 0x4c, 0x56, 0x1e, + 0x5c, 0x11, 0xcb, 0x83, 0xe2, 0xb1, 0x15, 0x89, 0x0e, 0x25, 0x61, 0x51, 0xa7, 0xa1, 0xa2, 0xf6, + 0x3f, 0x1a, 0xcc, 0xab, 0xf6, 0x2e, 0xda, 0x14, 0x76, 0xbc, 0xc6, 0xc7, 0x46, 0x8a, 0x50, 0xe7, + 0xbc, 0x63, 0xb6, 0x7a, 0xbc, 0xd1, 0x1b, 0x30, 0x63, 0xd9, 0x06, 0x6e, 0xe9, 0x44, 0x41, 0xcf, + 0xf4, 0xfc, 0x4a, 0x8a, 0x8e, 0xe2, 0xe9, 0xb8, 0x89, 0x60, 0xd6, 0x42, 0x84, 0xc0, 0x3d, 0x2d, + 0x21, 0x6a, 0x9f, 0xc1, 0x6c, 0x62, 0x18, 0x2c, 0x15, 0x2b, 0xa9, 0x09, 0x8b, 0x95, 0xf8, 0x0b, + 0x92, 0x9e, 0xec, 0x0b, 0x52, 0xfb, 0xd7, 0x14, 0x94, 0x84, 0xce, 0x1c, 0x3d, 0x84, 0x59, 0xfe, + 0x35, 0x33, 0xad, 0x2e, 0xeb, 0x00, 0x53, 0x7c, 0x4c, 0x34, 0x72, 0x52, 0xb2, 0x61, 0xb7, 0xb7, + 0x23, 0x5a, 0xda, 0x00, 0xd2, 0x29, 0x9e, 0x27, 0xc1, 0x04, 0xc5, 0x33, 0x32, 0x06, 0x3d, 0x80, + 0xc5, 0xc0, 0x21, 0x7d, 0x69, 0xcb, 0xe3, 0x67, 0x0e, 0x2d, 0x2b, 0xe8, 0xb7, 0xb1, 0x4b, 0x57, + 0x9f, 0x65, 0x9d, 0x12, 0xa3, 0x08, 0x0f, 0x25, 0xee, 0x52, 0xbc, 0xd8, 0x29, 0xa9, 0xf0, 0x82, + 0x1d, 0x32, 0x13, 0xda, 0xe1, 0x16, 0xa0, 0xd1, 0x69, 0xbc, 0xe4, 0x03, 0x6d, 0x32, 0x1f, 0xd4, + 0x0e, 0xa1, 0x9c, 0x9c, 0xb1, 0x3f, 0x25, 0x5f, 0xee, 0x43, 0x31, 0x9a, 0x90, 0xa3, 0x57, 0x20, + 0xe7, 0x62, 0xdd, 0xb3, 0x2d, 0xbe, 0x5b, 0xe8, 0xb6, 0x67, 0x10, 0x71, 0xdb, 0x33, 0xc8, 0x63, + 0x28, 0xbb, 0x0f, 0x53, 0xcc, 0x48, 0xef, 0x9b, 0x3d, 0x1f, 0xbb, 0xe8, 0x06, 0xe4, 0x3c, 0x5f, + 0xf7, 0xb1, 0x57, 0xd1, 0x56, 0xd2, 0x97, 0x67, 0xae, 0x2d, 0x8e, 0x8e, 0xbf, 0x09, 0x9a, 0xad, + 0x83, 0x51, 0x8a, 0xeb, 0x60, 0x90, 0xda, 0xbf, 0x68, 0x30, 0x25, 0x4e, 0xf9, 0x9f, 0x8c, 0xd8, + 0x93, 0x19, 0x83, 0x24, 0x8e, 0x29, 0xf1, 0x30, 0xe0, 0xf4, 0x6c, 0x49, 0xbe, 0x82, 0xec, 0x28, + 0xa1, 0x15, 0x78, 0xd8, 0xe5, 0xd1, 0x4a, 0xbf, 0x82, 0x0c, 0xbc, 0xe3, 0x49, 0xd1, 0x0e, 0x31, + 0x94, 0xbb, 0x81, 0xac, 0x55, 0x3c, 0x5a, 0x40, 0xdd, 0x78, 0x80, 0x43, 0x36, 0x99, 0x47, 0x93, + 0xd1, 0xa4, 0x03, 0x1c, 0x9a, 0xb2, 0x24, 0x76, 0x31, 0x65, 0x49, 0x88, 0xc7, 0x08, 0x99, 0xef, + 0xb2, 0x74, 0xad, 0xf1, 0x51, 0x41, 0xa2, 0x06, 0x48, 0x9f, 0xa0, 0x06, 0x78, 0x15, 0xf2, 0x34, + 0xe9, 0x46, 0x5b, 0x9c, 0xfa, 0x84, 0x80, 0xe4, 0x63, 0x52, 0x06, 0x79, 0x44, 0xaa, 0xc9, 0xfe, + 0xc4, 0x54, 0xd3, 0x82, 0xf3, 0x7b, 0xba, 0xd7, 0x0a, 0x93, 0xa3, 0xd1, 0xd2, 0xfd, 0x56, 0xb4, + 0xd7, 0x73, 0xb4, 0xfe, 0xa7, 0xc3, 0xc7, 0x3d, 0xdd, 0xdb, 0x0e, 0x69, 0xd6, 0xfc, 0xad, 0xd1, + 0x9d, 0xbf, 0xa8, 0xa6, 0x40, 0x3b, 0xb0, 0xa0, 0x16, 0x9e, 0xa7, 0x2b, 0xa7, 0xb3, 0x71, 0xef, + 0x91, 0x92, 0xe7, 0x14, 0x68, 0xf4, 0xa5, 0x06, 0x15, 0xf2, 0x15, 0x74, 0xf1, 0xa7, 0x81, 0xe9, + 0xe2, 0x3e, 0x09, 0x8b, 0x96, 0x7d, 0x80, 0xdd, 0x9e, 0x7e, 0xc4, 0x8f, 0x99, 0x2e, 0x8e, 0xa6, + 0xfc, 0x2d, 0xdb, 0x68, 0x0a, 0x0c, 0xec, 0xd5, 0x1c, 0x19, 0x78, 0x8f, 0x09, 0x11, 0x5f, 0x4d, + 0x4d, 0x21, 0x84, 0x10, 0x9c, 0x60, 0xa0, 0x55, 0x3a, 0x76, 0xa0, 0xf5, 0x22, 0x64, 0x1c, 0xdb, + 0xee, 0xd1, 0xf6, 0x8b, 0x57, 0x7a, 0xe4, 0x59, 0xac, 0xf4, 0xc8, 0x33, 0x7a, 0x1d, 0x8a, 0x54, + 0xa6, 0x65, 0xe0, 0x43, 0x7a, 0xb8, 0xc3, 0xf3, 0x32, 0x91, 0x42, 0x60, 0x62, 0x5e, 0x0e, 0x61, + 0xe2, 0xa0, 0x62, 0x23, 0x53, 0x28, 0x94, 0x8b, 0xb5, 0xef, 0x35, 0x98, 0x91, 0x8f, 0xb3, 0x46, + 0x77, 0x61, 0xfa, 0xd4, 0x77, 0x61, 0xe6, 0x04, 0x26, 0xcc, 0x1e, 0x67, 0x42, 0x69, 0x1c, 0xf3, + 0x7b, 0x0d, 0xa6, 0xa5, 0x93, 0xb4, 0x9f, 0xd7, 0xeb, 0xfd, 0x47, 0x0a, 0x16, 0xd5, 0x4b, 0x3d, + 0x95, 0x7e, 0xf3, 0x16, 0x90, 0xca, 0xf1, 0x76, 0x5c, 0x59, 0x2d, 0x8c, 0xb4, 0x9b, 0xd4, 0x4c, + 0x61, 0xd9, 0x39, 0x72, 0xc8, 0x16, 0xb2, 0xa3, 0x07, 0x50, 0x32, 0x85, 0xe3, 0xbc, 0xb4, 0xea, + 0xd4, 0x45, 0x3c, 0xc4, 0x63, 0x73, 0x88, 0x31, 0x47, 0x77, 0xa2, 0xa8, 0x46, 0x0e, 0x32, 0xa4, + 0xf4, 0xab, 0x1d, 0x40, 0x9e, 0x2f, 0x87, 0xec, 0x15, 0x9a, 0x70, 0x69, 0x0b, 0xc5, 0xea, 0x74, + 0xba, 0x57, 0x08, 0x30, 0x71, 0x9d, 0xa5, 0x10, 0xc2, 0xd0, 0xdf, 0x01, 0x90, 0x1c, 0xc3, 0x53, + 0x6d, 0x8a, 0x26, 0x2c, 0xda, 0xaa, 0x39, 0xb6, 0x31, 0x92, 0x5f, 0x8b, 0x11, 0xb0, 0xf6, 0x7f, + 0x29, 0x28, 0x89, 0x07, 0x88, 0x8f, 0xa5, 0xfc, 0x73, 0x08, 0xdb, 0xe8, 0x96, 0x6e, 0x18, 0xe4, + 0x2f, 0x0e, 0xbf, 0x86, 0xab, 0x63, 0x8d, 0x14, 0xfe, 0xbf, 0x16, 0x72, 0xb0, 0xa6, 0x89, 0x5e, + 0x92, 0x30, 0x13, 0x28, 0x41, 0x6b, 0x39, 0x89, 0x5b, 0xda, 0x87, 0x05, 0xa5, 0x28, 0xb1, 0xd5, + 0xc9, 0x3e, 0xa9, 0x56, 0xe7, 0xbf, 0xb3, 0xb0, 0xa0, 0x3c, 0xb8, 0x4d, 0x44, 0x70, 0xfa, 0x89, + 0x44, 0xf0, 0xbf, 0x69, 0x2a, 0xcb, 0xb2, 0x53, 0x9b, 0xb7, 0x26, 0x38, 0x4d, 0x7e, 0x52, 0x36, + 0x96, 0xc3, 0x22, 0xfb, 0x58, 0x31, 0x99, 0x9b, 0x34, 0x26, 0xd1, 0x55, 0xd6, 0x35, 0x52, 0x5d, + 0xec, 0x4c, 0x25, 0xdc, 0xa1, 0x09, 0x55, 0x79, 0x0e, 0x42, 0xef, 0xc1, 0x74, 0xc8, 0xc1, 0x66, + 0x15, 0x85, 0x78, 0x90, 0xc0, 0x69, 0x92, 0xe3, 0x8a, 0x29, 0x11, 0x2e, 0x64, 0xbf, 0xe2, 0x09, + 0xb2, 0x1f, 0x1c, 0x97, 0xfd, 0x9e, 0x6a, 0x6c, 0x4a, 0xa9, 0x76, 0xa0, 0xc1, 0x6c, 0xe2, 0xbe, + 0xc4, 0xcf, 0xeb, 0x5b, 0xf2, 0x85, 0x06, 0xc5, 0xe8, 0x3a, 0x0e, 0x5a, 0x83, 0x1c, 0x66, 0x57, + 0x3a, 0x58, 0xda, 0x99, 0x4b, 0x5c, 0xb7, 0x23, 0x38, 0x7e, 0xc1, 0x2e, 0x71, 0x8b, 0xa3, 0xc9, + 0x19, 0x1f, 0xa3, 0xca, 0xfe, 0x85, 0x16, 0x56, 0xd9, 0x23, 0xab, 0x48, 0xff, 0xf4, 0x55, 0x9c, + 0x9e, 0xe9, 0x7e, 0x55, 0x84, 0x2c, 0x5d, 0x0b, 0xe9, 0x96, 0x7d, 0xec, 0xf6, 0x4d, 0x4b, 0xef, + 0xd1, 0x50, 0x2c, 0xb0, 0x5d, 0x1d, 0xc2, 0xc4, 0x5d, 0x1d, 0xc2, 0xd0, 0x1e, 0xcc, 0xc6, 0x33, + 0x38, 0x2a, 0x46, 0x7d, 0xbf, 0xef, 0x03, 0x99, 0x88, 0x9d, 0x0b, 0x24, 0x38, 0xe5, 0x03, 0xfa, + 0x04, 0x12, 0x19, 0x30, 0xd3, 0xb1, 0x2d, 0x5f, 0x37, 0x2d, 0xec, 0x32, 0x45, 0x69, 0xd5, 0xfd, + 0xa6, 0xeb, 0x12, 0x0d, 0x9b, 0x8c, 0xc8, 0x7c, 0xf2, 0xfd, 0x26, 0x19, 0x87, 0x3e, 0x81, 0xe9, + 0xb0, 0xdb, 0x61, 0x4a, 0x32, 0xaa, 0xfb, 0x4d, 0xeb, 0x22, 0x09, 0xdb, 0x0c, 0x12, 0x97, 0x7c, + 0xbf, 0x49, 0x42, 0xa1, 0x1e, 0x94, 0x1d, 0xdb, 0xd8, 0xb1, 0x78, 0x8d, 0xaf, 0xb7, 0x7b, 0x98, + 0x0f, 0x7e, 0x97, 0x47, 0x0a, 0x12, 0x89, 0x8a, 0x25, 0xea, 0x24, 0xaf, 0x7c, 0x63, 0x30, 0x89, + 0x45, 0x1f, 0xc3, 0x54, 0x8f, 0x34, 0x7d, 0xeb, 0x87, 0x8e, 0xe9, 0x62, 0x43, 0x7d, 0xbf, 0xef, + 0x8e, 0x40, 0xc1, 0xd2, 0xa4, 0xc8, 0x23, 0x5f, 0x6b, 0x10, 0x31, 0xc4, 0xfb, 0x7d, 0xfd, 0xb0, + 0x19, 0x58, 0xde, 0xfa, 0x21, 0xbf, 0xab, 0x95, 0x57, 0x79, 0x7f, 0x53, 0x26, 0x62, 0xde, 0x4f, + 0x70, 0xca, 0xde, 0x4f, 0x20, 0xd1, 0x1d, 0xfa, 0x15, 0x60, 0x2e, 0x61, 0xf7, 0xfc, 0x16, 0x47, + 0xac, 0xc5, 0xbc, 0xc1, 0x26, 0x3c, 0xfc, 0x49, 0x12, 0x1a, 0x49, 0xe0, 0x3e, 0xa0, 0xaf, 0xdd, + 0xc4, 0x7e, 0xe0, 0x5a, 0xd8, 0xe0, 0xbd, 0xd7, 0xa8, 0x0f, 0x24, 0xaa, 0xc8, 0x07, 0x12, 0x74, + 0xc4, 0x07, 0x12, 0x16, 0x7d, 0x0e, 0xf3, 0x89, 0x5b, 0x4b, 0xec, 0x3d, 0x4a, 0xaa, 0x53, 0x8f, + 0x0d, 0x05, 0x25, 0x6b, 0x93, 0x55, 0x32, 0x24, 0xcd, 0x4a, 0x2d, 0x44, 0x7b, 0x57, 0xb7, 0xba, + 0x1b, 0x76, 0x5b, 0x8e, 0xb9, 0x29, 0x95, 0xf6, 0x9b, 0x0a, 0x4a, 0xa6, 0x5d, 0x25, 0x43, 0xd6, + 0xae, 0xa2, 0x88, 0x6e, 0x28, 0x91, 0x22, 0x26, 0xba, 0xc9, 0xa7, 0xba, 0xa1, 0xc4, 0x08, 0x84, + 0x1b, 0x4a, 0x0c, 0xa0, 0xb8, 0xa1, 0xc4, 0x29, 0x0b, 0xe1, 0x70, 0xa8, 0xf6, 0x21, 0xcc, 0x26, + 0xd2, 0x0b, 0x7a, 0x17, 0xa2, 0x7b, 0x2f, 0xf7, 0x8f, 0x9c, 0xb0, 0x76, 0x95, 0xee, 0xc9, 0x10, + 0xb8, 0xea, 0x9e, 0x0c, 0x81, 0xd7, 0xbe, 0xce, 0x40, 0x21, 0x8c, 0xa8, 0x53, 0xe9, 0x46, 0x56, + 0x21, 0xdf, 0xc7, 0x1e, 0xbd, 0xdb, 0x92, 0x8a, 0x8b, 0x1a, 0x0e, 0x12, 0x8b, 0x1a, 0x0e, 0x92, + 0x6b, 0xae, 0xf4, 0x63, 0xd5, 0x5c, 0x99, 0x89, 0x6b, 0x2e, 0x4c, 0x8f, 0x73, 0x85, 0xbc, 0x18, + 0x9e, 0xa6, 0x3c, 0x3a, 0xd9, 0x86, 0x87, 0xbd, 0x22, 0x63, 0xe2, 0xb0, 0x57, 0x44, 0xa1, 0x7d, + 0x38, 0x2b, 0x9c, 0xf8, 0xf0, 0x51, 0x1f, 0xc9, 0x50, 0x33, 0xe3, 0xcf, 0xce, 0x9b, 0x94, 0x8a, + 0xed, 0xc3, 0xfd, 0x04, 0x54, 0x2c, 0x5a, 0x93, 0x38, 0x12, 0x12, 0x06, 0x6e, 0x07, 0xdd, 0x4d, + 0x6e, 0xf6, 0x7c, 0x1c, 0x12, 0x22, 0x5c, 0x0c, 0x09, 0x11, 0x5e, 0xfb, 0x63, 0x0a, 0x66, 0xe4, + 0xf7, 0x3d, 0x95, 0xc0, 0x78, 0x1d, 0x8a, 0xf8, 0xd0, 0xf4, 0x5b, 0x1d, 0xdb, 0xc0, 0xbc, 0x73, + 0xa3, 0x7e, 0x26, 0xc0, 0xeb, 0xb6, 0x21, 0xf9, 0x39, 0x84, 0x89, 0xd1, 0x94, 0x9e, 0x28, 0x9a, + 0xe2, 0xc9, 0x6a, 0x66, 0x82, 0xc9, 0xaa, 0xd2, 0x4f, 0xc5, 0xd3, 0xf1, 0x53, 0xed, 0xdb, 0x14, + 0x94, 0x93, 0x69, 0xf7, 0xd9, 0xd8, 0x82, 0xf2, 0x6e, 0x4a, 0x4f, 0xbc, 0x9b, 0xde, 0x83, 0x69, + 0x52, 0x99, 0xe9, 0xbe, 0xcf, 0xaf, 0xc2, 0x66, 0x68, 0x71, 0xc5, 0xb2, 0x51, 0x60, 0xad, 0x85, + 0x70, 0x29, 0x1b, 0x09, 0xf0, 0x91, 0xd0, 0xcd, 0x9e, 0x30, 0x74, 0xbf, 0x4c, 0xc1, 0xf4, 0x96, + 0x6d, 0xdc, 0x67, 0x45, 0x9b, 0xff, 0xac, 0xd8, 0xf3, 0x69, 0xa6, 0xb4, 0xda, 0x2c, 0x4c, 0x4b, + 0x55, 0x5b, 0xed, 0x2b, 0x16, 0x67, 0xf2, 0xe7, 0xea, 0xaf, 0xcf, 0x2e, 0x33, 0x30, 0x25, 0x96, + 0x7f, 0xb5, 0x06, 0xcc, 0x26, 0xaa, 0x35, 0xf1, 0x05, 0xb4, 0x49, 0x5e, 0xa0, 0x76, 0x03, 0xe6, + 0x55, 0x65, 0x8c, 0x90, 0x75, 0xb4, 0x09, 0x8e, 0x83, 0x6e, 0xc2, 0xbc, 0xaa, 0x1c, 0x39, 0xf9, + 0x72, 0xde, 0xe5, 0x47, 0xad, 0xac, 0x70, 0x38, 0x39, 0xff, 0x77, 0x51, 0xf7, 0x1c, 0x5f, 0x3b, + 0x7f, 0x1f, 0xca, 0x4e, 0xf8, 0xd0, 0xe2, 0x3d, 0x1a, 0xdb, 0x96, 0xb4, 0xe3, 0x88, 0x70, 0x1b, + 0x89, 0x66, 0x6d, 0x46, 0xc6, 0xc8, 0x72, 0x78, 0xff, 0x96, 0x53, 0xc8, 0x69, 0x26, 0x1a, 0xb9, + 0x19, 0x19, 0x23, 0x98, 0x36, 0x7f, 0xbc, 0x69, 0x69, 0xff, 0x97, 0x25, 0x4d, 0xf3, 0x6c, 0xe2, + 0x5a, 0x3c, 0xba, 0x0a, 0x05, 0xfa, 0x9b, 0xb5, 0xb8, 0xf3, 0xa5, 0xd6, 0xa1, 0x30, 0x69, 0x01, + 0x79, 0x0e, 0x42, 0x6f, 0x40, 0x31, 0xba, 0x29, 0xcf, 0x0f, 0x59, 0x59, 0xdc, 0x85, 0x40, 0x29, + 0xee, 0x42, 0x20, 0x6f, 0x9a, 0xff, 0x19, 0xce, 0x8f, 0xbd, 0x23, 0x7f, 0xa2, 0x03, 0xbd, 0xb8, + 0xfb, 0xcd, 0x9c, 0xa8, 0xfb, 0x3d, 0x84, 0x45, 0xf5, 0xd5, 0x75, 0x41, 0x7b, 0xea, 0x58, 0xed, + 0xb1, 0xf5, 0xd3, 0x13, 0x5a, 0x3f, 0x55, 0xdb, 0xa7, 0xe3, 0x82, 0xe8, 0x8a, 0x38, 0xba, 0x02, + 0x59, 0xc7, 0xb6, 0x7b, 0x1e, 0xbf, 0xc5, 0x40, 0xd5, 0x51, 0x80, 0xa8, 0x8e, 0x02, 0x1e, 0x63, + 0x38, 0x11, 0x84, 0x11, 0x1c, 0x5f, 0x78, 0x7f, 0x0a, 0xd6, 0x7d, 0xf9, 0x2a, 0x14, 0xc2, 0x93, + 0x62, 0x04, 0x90, 0xfb, 0x70, 0x67, 0x7d, 0x67, 0xfd, 0x46, 0xf9, 0x0c, 0x2a, 0x41, 0x7e, 0x6b, + 0xfd, 0xee, 0x8d, 0xdb, 0x77, 0x6f, 0x96, 0x35, 0xf2, 0xd0, 0xdc, 0xb9, 0x7b, 0x97, 0x3c, 0xa4, + 0x5e, 0xbe, 0x23, 0xde, 0x3e, 0xe3, 0x95, 0xdb, 0x14, 0x14, 0xd6, 0x1c, 0x87, 0xa6, 0x10, 0xc6, + 0xbb, 0x7e, 0x60, 0x92, 0x9d, 0x5c, 0xd6, 0x50, 0x1e, 0xd2, 0xf7, 0xee, 0x6d, 0x96, 0x53, 0x68, + 0x1e, 0xca, 0x37, 0xb0, 0x6e, 0xf4, 0x4c, 0x0b, 0x87, 0x79, 0xab, 0x9c, 0x6e, 0x3c, 0xfc, 0xf5, + 0x0f, 0xcb, 0xda, 0xb7, 0x3f, 0x2c, 0x6b, 0x7f, 0xf8, 0x61, 0x59, 0xfb, 0xfa, 0xc7, 0xe5, 0x33, + 0xdf, 0xfe, 0xb8, 0x7c, 0xe6, 0x77, 0x3f, 0x2e, 0x9f, 0xf9, 0xa7, 0xab, 0x5d, 0xd3, 0xdf, 0x0b, + 0xda, 0xf5, 0x8e, 0xdd, 0xe7, 0x3f, 0xbe, 0x75, 0x5c, 0x9b, 0x24, 0x08, 0xfe, 0xb4, 0x9a, 0xfc, + 0x55, 0xee, 0xff, 0xa6, 0x2e, 0xac, 0xd1, 0xc7, 0x2d, 0x46, 0x57, 0xbf, 0x6d, 0xd7, 0x19, 0x80, + 0xfe, 0x0e, 0xd3, 0x6b, 0xe7, 0xe8, 0xef, 0x2d, 0x5f, 0xff, 0x73, 0x00, 0x00, 0x00, 0xff, 0xff, + 0xad, 0x03, 0xc2, 0xae, 0xd0, 0x3b, 0x00, 0x00, } func (m *EventSequence) Marshal() (dAtA []byte, err error) { @@ -5472,6 +5481,11 @@ func (m *JobRunLeased) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.RunIndex != 0 { + i = encodeVarintEvents(dAtA, i, uint64(m.RunIndex)) + i-- + dAtA[i] = 0x68 + } if len(m.Pool) > 0 { i -= len(m.Pool) copy(dAtA[i:], m.Pool) @@ -7856,6 +7870,9 @@ func (m *JobRunLeased) Size() (n int) { if l > 0 { n += 1 + l + sovEvents(uint64(l)) } + if m.RunIndex != 0 { + n += 1 + sovEvents(uint64(m.RunIndex)) + } return n } @@ -12817,6 +12834,25 @@ func (m *JobRunLeased) Unmarshal(dAtA []byte) error { } m.Pool = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RunIndex", wireType) + } + m.RunIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RunIndex |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipEvents(dAtA[iNdEx:]) diff --git a/pkg/armadaevents/events.proto b/pkg/armadaevents/events.proto index 8d23a565d96..ca6ea78b15d 100644 --- a/pkg/armadaevents/events.proto +++ b/pkg/armadaevents/events.proto @@ -309,6 +309,7 @@ message JobRunLeased { // pool is the pool this run was scheduled on to // This would be determined by the pool of the node this run was scheduled on to, at the time of scheduling string pool = 12; + uint32 run_index = 13; } // Indicates that a job has been assigned to nodes by Kubernetes. diff --git a/pkg/executorapi/executorapi.pb.go b/pkg/executorapi/executorapi.pb.go index f4794a87256..9fc8103638b 100644 --- a/pkg/executorapi/executorapi.pb.go +++ b/pkg/executorapi/executorapi.pb.go @@ -428,12 +428,13 @@ func (m *LeaseRequest) GetUnassignedJobRunIds() []string { // Indicates that a job run is now leased. type JobRunLease struct { - Queue string `protobuf:"bytes,2,opt,name=queue,proto3" json:"queue,omitempty"` - Jobset string `protobuf:"bytes,3,opt,name=jobset,proto3" json:"jobset,omitempty"` - User string `protobuf:"bytes,4,opt,name=user,proto3" json:"user,omitempty"` - Groups []string `protobuf:"bytes,5,rep,name=groups,proto3" json:"groups,omitempty"` - Job *armadaevents.SubmitJob `protobuf:"bytes,6,opt,name=job,proto3" json:"job,omitempty"` - JobRunId string `protobuf:"bytes,7,opt,name=job_run_id,json=jobRunId,proto3" json:"jobRunId,omitempty"` + Queue string `protobuf:"bytes,2,opt,name=queue,proto3" json:"queue,omitempty"` + Jobset string `protobuf:"bytes,3,opt,name=jobset,proto3" json:"jobset,omitempty"` + User string `protobuf:"bytes,4,opt,name=user,proto3" json:"user,omitempty"` + Groups []string `protobuf:"bytes,5,rep,name=groups,proto3" json:"groups,omitempty"` + Job *armadaevents.SubmitJob `protobuf:"bytes,6,opt,name=job,proto3" json:"job,omitempty"` + JobRunId string `protobuf:"bytes,7,opt,name=job_run_id,json=jobRunId,proto3" json:"jobRunId,omitempty"` + JobRunIndex uint32 `protobuf:"varint,8,opt,name=job_run_index,json=jobRunIndex,proto3" json:"jobRunIndex,omitempty"` } func (m *JobRunLease) Reset() { *m = JobRunLease{} } @@ -511,6 +512,13 @@ func (m *JobRunLease) GetJobRunId() string { return "" } +func (m *JobRunLease) GetJobRunIndex() uint32 { + if m != nil { + return m.JobRunIndex + } + return 0 +} + // Indicates that the job runs with the given ids should be cancelled. type CancelRuns struct { JobRunIdsToCancel []string `protobuf:"bytes,2,rep,name=job_run_ids_to_cancel,json=jobRunIdsToCancel,proto3" json:"jobRunIdsToCancel,omitempty"` @@ -775,103 +783,104 @@ func init() { func init() { proto.RegisterFile("pkg/executorapi/executorapi.proto", fileDescriptor_57e0d9d0e484e459) } var fileDescriptor_57e0d9d0e484e459 = []byte{ - // 1526 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x58, 0x3b, 0x6f, 0xdc, 0xc6, - 0x16, 0x16, 0x57, 0xd2, 0x5a, 0x3b, 0xab, 0xe7, 0xe8, 0x45, 0x49, 0xf6, 0x52, 0xde, 0x8b, 0x7b, - 0x21, 0xe1, 0xda, 0xdc, 0x6b, 0xd9, 0xb8, 0x70, 0x82, 0x24, 0x88, 0xd6, 0x10, 0x1c, 0x09, 0xb6, - 0x63, 0x4b, 0x8a, 0x91, 0xa4, 0x21, 0x86, 0xcb, 0xf1, 0x9a, 0x12, 0x39, 0x43, 0x93, 0x43, 0xc5, - 0xeb, 0x2a, 0x65, 0x8a, 0x14, 0x2e, 0xd2, 0xa4, 0x30, 0xd2, 0xa5, 0xce, 0xcf, 0x48, 0x69, 0x20, - 0x4d, 0x2a, 0x26, 0xb0, 0x3a, 0xf6, 0xe9, 0x83, 0x99, 0x21, 0xb5, 0xc3, 0x7d, 0x58, 0x4e, 0x2a, - 0x21, 0x95, 0x34, 0xe7, 0xf5, 0x7d, 0x73, 0xce, 0x9c, 0xb3, 0x67, 0x17, 0x5c, 0x0d, 0x8e, 0xdb, - 0x0d, 0xfc, 0x1c, 0xb7, 0x62, 0x46, 0x43, 0x14, 0xb8, 0xea, 0xff, 0x66, 0x10, 0x52, 0x46, 0x61, - 0x55, 0x11, 0xad, 0x5e, 0xe1, 0xf6, 0x28, 0xf4, 0x91, 0x83, 0xf0, 0x09, 0x26, 0x2c, 0x6a, 0xc8, - 0x3f, 0xd2, 0x76, 0x75, 0x41, 0xa8, 0x03, 0xb7, 0x11, 0xc5, 0xb6, 0xef, 0xb2, 0x4c, 0xba, 0xd6, - 0xa6, 0xb4, 0xed, 0xe1, 0x86, 0x38, 0xd9, 0xf1, 0x93, 0x06, 0xf6, 0x03, 0xd6, 0xc9, 0x94, 0xf5, - 0xe3, 0xdb, 0x91, 0xe9, 0x52, 0xe1, 0xd5, 0xa2, 0x21, 0x6e, 0x9c, 0xdc, 0x68, 0xb4, 0x31, 0xc1, - 0x21, 0x62, 0xd8, 0xc9, 0x6c, 0x6e, 0x75, 0x6d, 0x7c, 0xd4, 0x7a, 0xea, 0x12, 0x1c, 0x76, 0x1a, - 0x39, 0x54, 0x88, 0x23, 0x1a, 0x87, 0x2d, 0xdc, 0xeb, 0x55, 0xff, 0x63, 0x06, 0x4c, 0x3c, 0xa0, - 0x0e, 0xde, 0x25, 0x4f, 0x28, 0xfc, 0x0f, 0x18, 0x23, 0xc8, 0xc7, 0xba, 0xb6, 0xae, 0x6d, 0x54, - 0x9a, 0x30, 0x4d, 0x8c, 0x69, 0x7e, 0xbe, 0x46, 0x7d, 0x97, 0x09, 0x3a, 0xfb, 0x42, 0x0f, 0xef, - 0x82, 0x32, 0x43, 0x2e, 0x61, 0x91, 0x5e, 0x5a, 0x1f, 0xdd, 0xa8, 0x6e, 0xad, 0x98, 0x12, 0xdb, - 0xe4, 0x09, 0xe1, 0xfc, 0xcc, 0x93, 0x1b, 0xe6, 0x21, 0xb7, 0x68, 0x2e, 0xa4, 0x89, 0x31, 0x2b, - 0x8d, 0x95, 0x30, 0x99, 0x3b, 0xfc, 0x14, 0x94, 0x3d, 0x64, 0x63, 0x2f, 0xd2, 0x47, 0x45, 0xa0, - 0xab, 0xa6, 0x9a, 0xda, 0x9c, 0x97, 0x79, 0x4f, 0xd8, 0xec, 0x10, 0x16, 0x76, 0x64, 0x40, 0xe9, - 0xa4, 0x06, 0x94, 0x12, 0xf8, 0x8d, 0x06, 0x16, 0x91, 0xe7, 0xd1, 0x16, 0x62, 0xc8, 0xf6, 0xb0, - 0x95, 0xdf, 0x3b, 0xd2, 0xc7, 0x04, 0x40, 0x63, 0x30, 0xc0, 0x76, 0xd7, 0x65, 0x3f, 0xf7, 0x90, - 0x70, 0xf5, 0x34, 0x31, 0x6a, 0x68, 0x80, 0x5a, 0x01, 0x5f, 0x18, 0xa4, 0x87, 0x5f, 0x6b, 0x60, - 0x1e, 0x9d, 0x20, 0xd7, 0xeb, 0x21, 0x32, 0x2e, 0x88, 0x5c, 0x1f, 0x42, 0x24, 0x77, 0xe8, 0xa1, - 0xb1, 0x9e, 0x26, 0xc6, 0x65, 0xd4, 0xa7, 0x54, 0x48, 0xc0, 0x7e, 0x2d, 0x0c, 0xc0, 0x0c, 0xa3, - 0x0c, 0x79, 0x0a, 0x7a, 0x59, 0xa0, 0x6f, 0x0e, 0x46, 0x3f, 0xe4, 0xc6, 0x3d, 0xc8, 0x97, 0xd3, - 0xc4, 0xd0, 0x59, 0x41, 0xa1, 0xa0, 0x4e, 0x17, 0x35, 0x90, 0x80, 0xd9, 0x30, 0x26, 0x96, 0xeb, - 0x44, 0x96, 0xdd, 0xb1, 0x22, 0x86, 0x18, 0xd6, 0x27, 0x04, 0xe4, 0xc6, 0x60, 0xc8, 0xfd, 0x98, - 0xec, 0x3a, 0x51, 0xb3, 0x73, 0xc0, 0x4d, 0x25, 0xe2, 0x5a, 0x9a, 0x18, 0xcb, 0xa1, 0x2a, 0x57, - 0x00, 0xa7, 0x0a, 0x0a, 0xf8, 0xa3, 0x06, 0x6a, 0x84, 0x12, 0x4b, 0x76, 0x9b, 0x95, 0x15, 0x02, - 0x3b, 0xca, 0x8d, 0x2b, 0x02, 0xfe, 0xff, 0x83, 0xe1, 0x1f, 0x50, 0xb2, 0x2d, 0x5c, 0xb7, 0x73, - 0xcf, 0x9e, 0xeb, 0x6f, 0xa6, 0x89, 0xf1, 0x6f, 0x32, 0xdc, 0x4a, 0xa1, 0xb6, 0xf6, 0x16, 0x33, - 0xb8, 0x0d, 0xa6, 0x62, 0x12, 0xb5, 0x9e, 0x62, 0x27, 0x16, 0x45, 0xd2, 0xc1, 0xba, 0xb6, 0x31, - 0x21, 0xef, 0x5a, 0x50, 0xa8, 0x77, 0x2d, 0x28, 0xe0, 0x4d, 0x50, 0x21, 0xd4, 0xc1, 0x16, 0xeb, - 0x04, 0x58, 0x9f, 0x14, 0x2d, 0xba, 0x94, 0x26, 0x06, 0xe4, 0xc2, 0xc3, 0x4e, 0xa0, 0x7a, 0x4e, - 0xe4, 0x32, 0xde, 0xd2, 0x01, 0xa5, 0x9e, 0x3e, 0xd5, 0x6d, 0x69, 0x7e, 0x56, 0x5b, 0x9a, 0x9f, - 0xe1, 0x4b, 0x0d, 0xac, 0xe7, 0x39, 0xb3, 0xe2, 0x08, 0xb5, 0x31, 0x2f, 0xe0, 0xb3, 0x18, 0xc7, - 0xd8, 0x42, 0xc4, 0xb1, 0x44, 0x90, 0x69, 0x91, 0xca, 0x5a, 0x21, 0x95, 0x0f, 0x29, 0xf5, 0x1e, - 0x71, 0xb3, 0xfc, 0xae, 0x32, 0x65, 0x79, 0xac, 0xcf, 0x78, 0xa8, 0x66, 0x47, 0x58, 0x6c, 0x13, - 0xe7, 0x61, 0x11, 0x7b, 0xed, 0x2d, 0x66, 0xab, 0x08, 0x54, 0x95, 0xc6, 0x87, 0xff, 0x02, 0xa3, - 0xc7, 0xb8, 0x93, 0xcd, 0xa6, 0xb9, 0x34, 0x31, 0xa6, 0x8e, 0x71, 0x47, 0x89, 0xc5, 0xb5, 0x70, - 0x13, 0x8c, 0x9f, 0x20, 0x2f, 0xc6, 0x7a, 0x49, 0x98, 0xcd, 0xa7, 0x89, 0x31, 0x23, 0x04, 0x8a, - 0xa1, 0xb4, 0x78, 0xbf, 0x74, 0x5b, 0x5b, 0xfd, 0x41, 0x03, 0x2b, 0x43, 0x7b, 0xff, 0xdd, 0x10, - 0xbf, 0x50, 0x11, 0xab, 0x5b, 0xa6, 0x32, 0x0a, 0xcf, 0xc6, 0xb0, 0x19, 0x1c, 0xb7, 0xc5, 0x6c, - 0xcc, 0xaf, 0x6b, 0x3e, 0x8a, 0x11, 0x61, 0x2e, 0xeb, 0x9c, 0xcb, 0xf0, 0x95, 0x06, 0x96, 0x87, - 0x0c, 0x85, 0x0b, 0xc1, 0xef, 0x7b, 0x0d, 0xcc, 0x0f, 0x18, 0x1b, 0x17, 0x82, 0xdb, 0x57, 0x00, - 0xf6, 0x8f, 0x97, 0x77, 0x63, 0x76, 0x5b, 0x65, 0x36, 0xbd, 0x35, 0x25, 0x18, 0xec, 0x51, 0x5b, - 0xc4, 0x39, 0x17, 0xf8, 0x3b, 0x0d, 0xac, 0x9f, 0x37, 0x59, 0x54, 0x1e, 0xe3, 0x43, 0x79, 0xdc, - 0x2d, 0x66, 0xe8, 0x72, 0xa1, 0xf5, 0xee, 0x50, 0x3f, 0x88, 0x59, 0xb7, 0xf1, 0xce, 0xa1, 0xb5, - 0x37, 0x36, 0x71, 0x69, 0x76, 0x62, 0x6f, 0x6c, 0xa2, 0x3a, 0x3b, 0x59, 0xff, 0xb6, 0x04, 0x66, - 0x7a, 0xfc, 0xa1, 0x0d, 0x2a, 0xdd, 0xb1, 0xa9, 0x89, 0x5e, 0xff, 0xef, 0xdb, 0x00, 0xcd, 0x9e, - 0x59, 0xb9, 0x9c, 0x26, 0xc6, 0x7c, 0x38, 0x60, 0x32, 0x76, 0xc3, 0xf2, 0xd4, 0x4c, 0x5f, 0xbc, - 0xa7, 0x52, 0x3f, 0x2d, 0x81, 0xb9, 0xbe, 0x49, 0x76, 0x36, 0x3c, 0xb5, 0x73, 0x86, 0xe7, 0x26, - 0x18, 0x17, 0x93, 0x52, 0x9d, 0x3a, 0x42, 0xa0, 0x82, 0x09, 0x01, 0x74, 0xd4, 0x1c, 0x8f, 0x0e, - 0x58, 0x05, 0xfa, 0x58, 0xfc, 0x83, 0xb2, 0xfc, 0x18, 0x54, 0x76, 0xf8, 0x26, 0x7c, 0xcf, 0x8d, - 0x18, 0xdc, 0x05, 0x65, 0xb9, 0x16, 0x67, 0x4f, 0x6d, 0xcd, 0x54, 0x57, 0x66, 0x53, 0x18, 0x1e, - 0xe0, 0x67, 0x31, 0x26, 0x2d, 0x2c, 0xb7, 0x3e, 0xa9, 0x51, 0xb7, 0x3e, 0x29, 0xa9, 0xff, 0x56, - 0x06, 0x93, 0xf7, 0x30, 0x8a, 0xf0, 0x3e, 0xb7, 0x8f, 0x18, 0x7c, 0x0f, 0x9c, 0x2d, 0xe4, 0x96, - 0xeb, 0x64, 0x97, 0xd6, 0xd3, 0xc4, 0x58, 0xc8, 0xc5, 0xbb, 0x8e, 0x12, 0x07, 0x74, 0xa5, 0x67, - 0x35, 0x2f, 0x9d, 0x53, 0x73, 0xab, 0xbf, 0x90, 0xc5, 0x15, 0x47, 0x25, 0xf4, 0x37, 0x6a, 0x08, - 0x63, 0x30, 0xeb, 0xbb, 0xc4, 0xf5, 0x63, 0xdf, 0x3a, 0xa2, 0xb6, 0x15, 0xb9, 0x2f, 0x70, 0xb6, - 0xc4, 0x5e, 0x1f, 0x8e, 0x73, 0x5f, 0x7a, 0xf0, 0x49, 0xe5, 0xbe, 0xc0, 0xca, 0x06, 0xe7, 0x17, - 0x14, 0xea, 0x06, 0x57, 0xd4, 0xc0, 0x8f, 0xc1, 0x38, 0x5f, 0x1e, 0xf2, 0x3d, 0x75, 0x71, 0xe0, - 0xde, 0x24, 0x2b, 0x2d, 0xec, 0xd4, 0x4a, 0x0b, 0x01, 0xbc, 0x0b, 0xe6, 0x7c, 0xf4, 0x9c, 0x93, - 0x8e, 0x2c, 0x46, 0x2d, 0x8f, 0xf3, 0xd3, 0x2f, 0xad, 0x6b, 0x1b, 0x53, 0x19, 0x15, 0xf4, 0x7c, - 0x8f, 0xda, 0xd1, 0x21, 0x15, 0xcc, 0x0b, 0x54, 0x0a, 0x1a, 0xf8, 0x18, 0x2c, 0xc5, 0x04, 0x45, - 0x91, 0xdb, 0x26, 0xd8, 0x11, 0x49, 0xc8, 0x76, 0x4b, 0xb1, 0x52, 0x56, 0x9a, 0x57, 0xd3, 0xc4, - 0xb8, 0xd2, 0xb5, 0xd8, 0xa3, 0xb6, 0x1c, 0xf7, 0x4a, 0xc8, 0xf9, 0x01, 0xea, 0x0b, 0xda, 0x1d, - 0xe2, 0xa3, 0x74, 0x40, 0xfd, 0x2e, 0x44, 0xe7, 0xfe, 0x54, 0x02, 0x55, 0x99, 0x40, 0x59, 0x9a, - 0xbf, 0x30, 0xf1, 0xae, 0x81, 0x32, 0x7f, 0x0a, 0x98, 0xe9, 0xa3, 0xc2, 0x56, 0xb4, 0xb2, 0x94, - 0xa8, 0xad, 0x2c, 0x25, 0xbc, 0xfd, 0xe2, 0x08, 0x87, 0xfa, 0x58, 0xb7, 0xfd, 0xf8, 0x59, 0x6d, - 0x3f, 0x7e, 0xe6, 0x51, 0xdb, 0x21, 0x8d, 0x03, 0xf9, 0x4e, 0xb3, 0xa8, 0x52, 0xa2, 0x46, 0x95, - 0x12, 0xf8, 0x01, 0x18, 0x3d, 0xa2, 0xb6, 0x5e, 0x16, 0xb9, 0x59, 0x2e, 0x0e, 0x9a, 0x03, 0xf1, - 0x2d, 0x7c, 0x8f, 0xda, 0x32, 0xb7, 0x47, 0xd4, 0x56, 0x73, 0x7b, 0x44, 0x6d, 0x78, 0x0b, 0x80, - 0xee, 0xe3, 0x13, 0x2f, 0x39, 0xdb, 0xbc, 0x8f, 0xb2, 0x27, 0xa5, 0x6e, 0xde, 0xb9, 0xac, 0x6e, - 0x01, 0x70, 0x07, 0x91, 0x16, 0xf6, 0xf6, 0x63, 0x12, 0xc1, 0x47, 0x60, 0x51, 0x79, 0xc0, 0xbc, - 0x2f, 0x5a, 0x42, 0x29, 0xbe, 0x41, 0x57, 0x9a, 0x46, 0x9a, 0x18, 0x6b, 0xb9, 0x6b, 0x74, 0x48, - 0xa5, 0xa7, 0x12, 0x77, 0xae, 0x4f, 0x59, 0x6f, 0x81, 0xea, 0xc3, 0x10, 0x73, 0xb5, 0x40, 0x38, - 0x04, 0x4b, 0x3d, 0x08, 0x81, 0xd4, 0x66, 0x10, 0xe2, 0x2b, 0xa4, 0x12, 0x25, 0xf3, 0x55, 0xbf, - 0x42, 0xf6, 0x6b, 0xeb, 0x55, 0x50, 0xd9, 0x21, 0xce, 0x7d, 0x14, 0x1e, 0xe3, 0xb0, 0xfe, 0x4b, - 0x09, 0x40, 0x51, 0xff, 0x03, 0x16, 0x62, 0xe4, 0xdf, 0xc7, 0x11, 0x5f, 0xdb, 0xe1, 0x0e, 0x18, - 0x97, 0x4d, 0xae, 0x89, 0xfc, 0xea, 0x85, 0x91, 0xa1, 0xbc, 0x1a, 0xf9, 0x4c, 0xbc, 0x62, 0xd7, - 0x7f, 0x32, 0xb2, 0x2f, 0xbd, 0xe1, 0x21, 0xa8, 0xca, 0x9c, 0xf0, 0x3b, 0x44, 0xd9, 0x43, 0x5e, - 0x2e, 0x2e, 0x20, 0x67, 0x09, 0x95, 0xd3, 0xbc, 0x75, 0x76, 0x2e, 0x04, 0x04, 0x5d, 0x39, 0xfc, - 0x10, 0x8c, 0x62, 0xe2, 0x88, 0xb7, 0x57, 0xdd, 0x5a, 0x2a, 0x44, 0x3b, 0xbb, 0x98, 0xac, 0x3c, - 0x26, 0x4e, 0x21, 0x0a, 0xf7, 0x83, 0x9f, 0x83, 0xc9, 0x2c, 0x8d, 0x92, 0xd5, 0xd8, 0x80, 0x2b, - 0x2a, 0x55, 0x68, 0xae, 0xa4, 0x89, 0xb1, 0x18, 0x74, 0x05, 0x85, 0x88, 0x55, 0x45, 0xd1, 0xbc, - 0x04, 0xc6, 0xc5, 0x0b, 0xdc, 0x7a, 0xa5, 0x81, 0xea, 0x4e, 0x16, 0x6e, 0x3b, 0x70, 0xe1, 0x83, - 0xec, 0xc3, 0x4c, 0x66, 0x2e, 0x82, 0x2b, 0x43, 0xc7, 0xfd, 0xaa, 0xd1, 0xaf, 0x2a, 0x94, 0x66, - 0x43, 0xfb, 0x9f, 0x06, 0x3f, 0x02, 0x93, 0xfb, 0x38, 0xa0, 0x21, 0x13, 0x1f, 0xa9, 0x11, 0xec, - 0x49, 0x42, 0xfe, 0x81, 0xbc, 0xba, 0x64, 0xca, 0x9f, 0xa0, 0xcc, 0xfc, 0x27, 0x28, 0x73, 0x87, - 0xf3, 0x6e, 0xee, 0xfe, 0xfc, 0xa6, 0xa6, 0xbd, 0x7e, 0x53, 0xd3, 0x7e, 0x7f, 0x53, 0xd3, 0x5e, - 0x9e, 0xd6, 0x46, 0x5e, 0x9f, 0xd6, 0x46, 0x7e, 0x3d, 0xad, 0x8d, 0x7c, 0xd9, 0x68, 0xbb, 0xec, - 0x69, 0x6c, 0x9b, 0x2d, 0xea, 0x67, 0xbf, 0x77, 0x05, 0x21, 0x3d, 0xc2, 0x2d, 0x96, 0x9d, 0x1a, - 0x3d, 0x3f, 0x9c, 0xd9, 0x65, 0x11, 0xfa, 0xe6, 0x9f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x80, 0x1b, - 0x8b, 0x62, 0x52, 0x13, 0x00, 0x00, + // 1546 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x58, 0xbd, 0x6f, 0xdb, 0x46, + 0x1b, 0x37, 0x65, 0x5b, 0x91, 0x4e, 0xfe, 0x3c, 0x7f, 0xd1, 0x76, 0x22, 0x3a, 0x7a, 0xf1, 0xbe, + 0xb0, 0xf1, 0x26, 0x54, 0xe3, 0x04, 0x45, 0x5a, 0x34, 0x45, 0xad, 0xc0, 0x48, 0x6d, 0x24, 0x69, + 0x62, 0xbb, 0x41, 0xdb, 0x85, 0x38, 0x8a, 0x17, 0x85, 0xb6, 0x78, 0xc7, 0x90, 0x47, 0xd7, 0xca, + 0xd4, 0xb1, 0x43, 0x87, 0x0c, 0x5d, 0x3a, 0x04, 0xdd, 0xfa, 0xb7, 0x74, 0x0c, 0xd0, 0xa5, 0x13, + 0x5b, 0xc4, 0x1b, 0x97, 0x4e, 0xdd, 0x8b, 0xbb, 0x23, 0xa5, 0xa3, 0x24, 0xc7, 0x69, 0x27, 0xa3, + 0x93, 0x7d, 0xcf, 0xd7, 0xef, 0xb9, 0xe7, 0xeb, 0x1e, 0x11, 0x5c, 0xf5, 0x8f, 0x5a, 0x75, 0x7c, + 0x82, 0x9b, 0x11, 0xa3, 0x01, 0xf2, 0x5d, 0xf5, 0x7f, 0xd3, 0x0f, 0x28, 0xa3, 0xb0, 0xa2, 0x90, + 0x56, 0xae, 0x70, 0x79, 0x14, 0x78, 0xc8, 0x41, 0xf8, 0x18, 0x13, 0x16, 0xd6, 0xe5, 0x1f, 0x29, + 0xbb, 0x32, 0x2f, 0xd8, 0xbe, 0x5b, 0x0f, 0x23, 0xdb, 0x73, 0x59, 0x4a, 0x5d, 0x6d, 0x51, 0xda, + 0x6a, 0xe3, 0xba, 0x38, 0xd9, 0xd1, 0xd3, 0x3a, 0xf6, 0x7c, 0xd6, 0x49, 0x99, 0xb5, 0xa3, 0xdb, + 0xa1, 0xe9, 0x52, 0xa1, 0xd5, 0xa4, 0x01, 0xae, 0x1f, 0xdf, 0xa8, 0xb7, 0x30, 0xc1, 0x01, 0x62, + 0xd8, 0x49, 0x65, 0x6e, 0xf5, 0x64, 0x3c, 0xd4, 0x7c, 0xe6, 0x12, 0x1c, 0x74, 0xea, 0x19, 0x54, + 0x80, 0x43, 0x1a, 0x05, 0x4d, 0xdc, 0xaf, 0x55, 0xfb, 0x73, 0x1a, 0x94, 0x1e, 0x52, 0x07, 0xef, + 0x90, 0xa7, 0x14, 0xfe, 0x0f, 0x8c, 0x11, 0xe4, 0x61, 0x5d, 0x5b, 0xd3, 0xd6, 0xcb, 0x0d, 0x98, + 0xc4, 0xc6, 0x14, 0x3f, 0x5f, 0xa3, 0x9e, 0xcb, 0x84, 0x3b, 0x7b, 0x82, 0x0f, 0xef, 0x81, 0x22, + 0x43, 0x2e, 0x61, 0xa1, 0x5e, 0x58, 0x1b, 0x5d, 0xaf, 0x6c, 0x2e, 0x9b, 0x12, 0xdb, 0xe4, 0x01, + 0xe1, 0xfe, 0x99, 0xc7, 0x37, 0xcc, 0x03, 0x2e, 0xd1, 0x98, 0x4f, 0x62, 0x63, 0x46, 0x0a, 0x2b, + 0x66, 0x52, 0x75, 0xf8, 0x19, 0x28, 0xb6, 0x91, 0x8d, 0xdb, 0xa1, 0x3e, 0x2a, 0x0c, 0x5d, 0x35, + 0xd5, 0xd0, 0x66, 0x7e, 0x99, 0xf7, 0x85, 0xcc, 0x36, 0x61, 0x41, 0x47, 0x1a, 0x94, 0x4a, 0xaa, + 0x41, 0x49, 0x81, 0xdf, 0x6a, 0x60, 0x01, 0xb5, 0xdb, 0xb4, 0x89, 0x18, 0xb2, 0xdb, 0xd8, 0xca, + 0xee, 0x1d, 0xea, 0x63, 0x02, 0xa0, 0x3e, 0x1c, 0x60, 0xab, 0xa7, 0xb2, 0x97, 0x69, 0x48, 0xb8, + 0x5a, 0x12, 0x1b, 0x55, 0x34, 0x84, 0xad, 0x80, 0xcf, 0x0f, 0xe3, 0xc3, 0x6f, 0x34, 0x30, 0x87, + 0x8e, 0x91, 0xdb, 0xee, 0x73, 0x64, 0x5c, 0x38, 0x72, 0xfd, 0x0c, 0x47, 0x32, 0x85, 0x3e, 0x37, + 0xd6, 0x92, 0xd8, 0xb8, 0x8c, 0x06, 0x98, 0x8a, 0x13, 0x70, 0x90, 0x0b, 0x7d, 0x30, 0xcd, 0x28, + 0x43, 0x6d, 0x05, 0xbd, 0x28, 0xd0, 0x37, 0x86, 0xa3, 0x1f, 0x70, 0xe1, 0x3e, 0xe4, 0xcb, 0x49, + 0x6c, 0xe8, 0x2c, 0xc7, 0x50, 0x50, 0xa7, 0xf2, 0x1c, 0x48, 0xc0, 0x4c, 0x10, 0x11, 0xcb, 0x75, + 0x42, 0xcb, 0xee, 0x58, 0x21, 0x43, 0x0c, 0xeb, 0x25, 0x01, 0xb9, 0x3e, 0x1c, 0x72, 0x2f, 0x22, + 0x3b, 0x4e, 0xd8, 0xe8, 0xec, 0x73, 0x51, 0x89, 0xb8, 0x9a, 0xc4, 0xc6, 0x52, 0xa0, 0xd2, 0x15, + 0xc0, 0xc9, 0x1c, 0x03, 0xfe, 0xa4, 0x81, 0x2a, 0xa1, 0xc4, 0x92, 0xdd, 0x66, 0xa5, 0x89, 0xc0, + 0x8e, 0x72, 0xe3, 0xb2, 0x80, 0x7f, 0x7f, 0x38, 0xfc, 0x43, 0x4a, 0xb6, 0x84, 0xea, 0x56, 0xa6, + 0xd9, 0x77, 0xfd, 0x8d, 0x24, 0x36, 0xfe, 0x4b, 0xce, 0x96, 0x52, 0x5c, 0x5b, 0x7d, 0x8b, 0x18, + 0xdc, 0x02, 0x93, 0x11, 0x09, 0x9b, 0xcf, 0xb0, 0x13, 0x89, 0x24, 0xe9, 0x60, 0x4d, 0x5b, 0x2f, + 0xc9, 0xbb, 0xe6, 0x18, 0xea, 0x5d, 0x73, 0x0c, 0x78, 0x13, 0x94, 0x09, 0x75, 0xb0, 0xc5, 0x3a, + 0x3e, 0xd6, 0x27, 0x44, 0x8b, 0x2e, 0x26, 0xb1, 0x01, 0x39, 0xf1, 0xa0, 0xe3, 0xab, 0x9a, 0xa5, + 0x8c, 0xc6, 0x5b, 0xda, 0xa7, 0xb4, 0xad, 0x4f, 0xf6, 0x5a, 0x9a, 0x9f, 0xd5, 0x96, 0xe6, 0x67, + 0xf8, 0x52, 0x03, 0x6b, 0x59, 0xcc, 0xac, 0x28, 0x44, 0x2d, 0xcc, 0x13, 0xf8, 0x3c, 0xc2, 0x11, + 0xb6, 0x10, 0x71, 0x2c, 0x61, 0x64, 0x4a, 0x84, 0xb2, 0x9a, 0x0b, 0xe5, 0x23, 0x4a, 0xdb, 0x8f, + 0xb9, 0x58, 0x76, 0x57, 0x19, 0xb2, 0xcc, 0xd6, 0xe7, 0xdc, 0x54, 0xa3, 0x23, 0x24, 0xb6, 0x88, + 0xf3, 0x28, 0x8f, 0xbd, 0xfa, 0x16, 0xb1, 0x15, 0x04, 0x2a, 0x4a, 0xe3, 0xc3, 0xff, 0x80, 0xd1, + 0x23, 0xdc, 0x49, 0x67, 0xd3, 0x6c, 0x12, 0x1b, 0x93, 0x47, 0xb8, 0xa3, 0xd8, 0xe2, 0x5c, 0xb8, + 0x01, 0xc6, 0x8f, 0x51, 0x3b, 0xc2, 0x7a, 0x41, 0x88, 0xcd, 0x25, 0xb1, 0x31, 0x2d, 0x08, 0x8a, + 0xa0, 0x94, 0xf8, 0xb0, 0x70, 0x5b, 0x5b, 0xf9, 0x51, 0x03, 0xcb, 0x67, 0xf6, 0xfe, 0xbb, 0x21, + 0x7e, 0xa9, 0x22, 0x56, 0x36, 0x4d, 0x65, 0x14, 0x76, 0xc7, 0xb0, 0xe9, 0x1f, 0xb5, 0xc4, 0x6c, + 0xcc, 0xae, 0x6b, 0x3e, 0x8e, 0x10, 0x61, 0x2e, 0xeb, 0x9c, 0xeb, 0xe1, 0x2b, 0x0d, 0x2c, 0x9d, + 0x31, 0x14, 0x2e, 0x84, 0x7f, 0x3f, 0x68, 0x60, 0x6e, 0xc8, 0xd8, 0xb8, 0x10, 0xbe, 0x7d, 0x0d, + 0xe0, 0xe0, 0x78, 0x79, 0x37, 0xcf, 0x6e, 0xab, 0x9e, 0x4d, 0x6d, 0x4e, 0x0a, 0x0f, 0x76, 0xa9, + 0x2d, 0xec, 0x9c, 0x0b, 0xfc, 0xbd, 0x06, 0xd6, 0xce, 0x9b, 0x2c, 0xaa, 0x1f, 0xe3, 0x67, 0xfa, + 0x71, 0x2f, 0x1f, 0xa1, 0xcb, 0xb9, 0xd6, 0xbb, 0x4b, 0x3d, 0x3f, 0x62, 0xbd, 0xc6, 0x3b, 0xc7, + 0xad, 0xdd, 0xb1, 0xd2, 0xa5, 0x99, 0xd2, 0xee, 0x58, 0xa9, 0x32, 0x33, 0x51, 0xfb, 0xae, 0x00, + 0xa6, 0xfb, 0xf4, 0xa1, 0x0d, 0xca, 0xbd, 0xb1, 0xa9, 0x89, 0x5e, 0xff, 0xff, 0xdb, 0x00, 0xcd, + 0xbe, 0x59, 0xb9, 0x94, 0xc4, 0xc6, 0x5c, 0x30, 0x64, 0x32, 0xf6, 0xcc, 0xf2, 0xd0, 0x4c, 0x5d, + 0xbc, 0x52, 0xa9, 0x9d, 0x16, 0xc0, 0xec, 0xc0, 0x24, 0xeb, 0x0e, 0x4f, 0xed, 0x9c, 0xe1, 0xb9, + 0x01, 0xc6, 0xc5, 0xa4, 0x54, 0xa7, 0x8e, 0x20, 0xa8, 0x60, 0x82, 0x00, 0x1d, 0x35, 0xc6, 0xa3, + 0x43, 0x56, 0x81, 0x01, 0x2f, 0xfe, 0x45, 0x51, 0x7e, 0x02, 0xca, 0xdb, 0x7c, 0x13, 0xbe, 0xef, + 0x86, 0x0c, 0xee, 0x80, 0xa2, 0x5c, 0x8b, 0xd3, 0x52, 0x5b, 0x35, 0xd5, 0x95, 0xd9, 0x14, 0x82, + 0xfb, 0xf8, 0x79, 0x84, 0x49, 0x13, 0xcb, 0xad, 0x4f, 0x72, 0xd4, 0xad, 0x4f, 0x52, 0x6a, 0xbf, + 0x15, 0xc1, 0xc4, 0x7d, 0x8c, 0x42, 0xbc, 0xc7, 0xe5, 0x43, 0x06, 0x3f, 0x00, 0xdd, 0x85, 0xdc, + 0x72, 0x9d, 0xf4, 0xd2, 0x7a, 0x12, 0x1b, 0xf3, 0x19, 0x79, 0xc7, 0x51, 0xec, 0x80, 0x1e, 0xb5, + 0x9b, 0xf3, 0xc2, 0x39, 0x39, 0xb7, 0x06, 0x13, 0x99, 0x5f, 0x71, 0x54, 0x87, 0xfe, 0x41, 0x0e, + 0x61, 0x04, 0x66, 0x3c, 0x97, 0xb8, 0x5e, 0xe4, 0x59, 0x87, 0xd4, 0xb6, 0x42, 0xf7, 0x05, 0x4e, + 0x97, 0xd8, 0xeb, 0x67, 0xe3, 0x3c, 0x90, 0x1a, 0x7c, 0x52, 0xb9, 0x2f, 0xb0, 0xb2, 0xc1, 0x79, + 0x39, 0x86, 0xba, 0xc1, 0xe5, 0x39, 0xf0, 0x13, 0x30, 0xce, 0x97, 0x87, 0x6c, 0x4f, 0x5d, 0x18, + 0xba, 0x37, 0xc9, 0x4c, 0x0b, 0x39, 0x35, 0xd3, 0x82, 0x00, 0xef, 0x81, 0x59, 0x0f, 0x9d, 0x70, + 0xa7, 0x43, 0x8b, 0x51, 0xab, 0xcd, 0xfd, 0xd3, 0x2f, 0xad, 0x69, 0xeb, 0x93, 0xa9, 0x2b, 0xe8, + 0x64, 0x97, 0xda, 0xe1, 0x01, 0x15, 0x9e, 0xe7, 0x5c, 0xc9, 0x71, 0xe0, 0x13, 0xb0, 0x18, 0x11, + 0x14, 0x86, 0x6e, 0x8b, 0x60, 0x47, 0x04, 0x21, 0xdd, 0x2d, 0xc5, 0x4a, 0x59, 0x6e, 0x5c, 0x4d, + 0x62, 0xe3, 0x4a, 0x4f, 0x62, 0x97, 0xda, 0x72, 0xdc, 0x2b, 0x26, 0xe7, 0x86, 0xb0, 0x2f, 0x68, + 0x77, 0x88, 0xa7, 0x74, 0x48, 0xfe, 0x2e, 0x44, 0xe7, 0xfe, 0x51, 0x00, 0x15, 0x19, 0x40, 0x99, + 0x9a, 0xbf, 0x31, 0xf1, 0xae, 0x81, 0x22, 0x2f, 0x05, 0xcc, 0xf4, 0x51, 0x21, 0x2b, 0x5a, 0x59, + 0x52, 0xd4, 0x56, 0x96, 0x14, 0xde, 0x7e, 0x51, 0x88, 0x03, 0x7d, 0xac, 0xd7, 0x7e, 0xfc, 0xac, + 0xb6, 0x1f, 0x3f, 0x73, 0xab, 0xad, 0x80, 0x46, 0xbe, 0xac, 0xd3, 0xd4, 0xaa, 0xa4, 0xa8, 0x56, + 0x25, 0x05, 0x7e, 0x04, 0x46, 0x0f, 0xa9, 0xad, 0x17, 0x45, 0x6c, 0x96, 0xf2, 0x83, 0x66, 0x5f, + 0xfc, 0x0a, 0xdf, 0xa5, 0xb6, 0x8c, 0xed, 0x21, 0xb5, 0xd5, 0xd8, 0x1e, 0x52, 0x1b, 0xde, 0x02, + 0xa0, 0x57, 0x7c, 0xa2, 0x92, 0xd3, 0xcd, 0xfb, 0x30, 0x2d, 0x29, 0x75, 0xf3, 0xce, 0x68, 0xf0, + 0x0e, 0x98, 0xec, 0x6a, 0x11, 0x07, 0x9f, 0xe8, 0x25, 0xd1, 0x02, 0xcb, 0x49, 0x6c, 0x2c, 0xa4, + 0x42, 0x9c, 0xac, 0xe8, 0x56, 0x14, 0x72, 0xcd, 0x02, 0xe0, 0x2e, 0x22, 0x4d, 0xdc, 0xde, 0x8b, + 0x48, 0x08, 0x1f, 0x83, 0x05, 0xa5, 0xfe, 0x79, 0x5b, 0x35, 0x05, 0x53, 0xfc, 0x00, 0x2f, 0x37, + 0x8c, 0x24, 0x36, 0x56, 0x33, 0xe4, 0xf0, 0x80, 0x4a, 0x4d, 0xc5, 0xf4, 0xec, 0x00, 0xb3, 0xd6, + 0x04, 0x95, 0x47, 0x01, 0xe6, 0x6c, 0x81, 0x70, 0x00, 0x16, 0xfb, 0x10, 0x7c, 0xc9, 0x4d, 0x21, + 0xc4, 0x2f, 0x50, 0xc5, 0x4a, 0xaa, 0xab, 0xfe, 0x02, 0x1d, 0xe4, 0xd6, 0x2a, 0xa0, 0xbc, 0x4d, + 0x9c, 0x07, 0x28, 0x38, 0xc2, 0x41, 0xed, 0x97, 0x02, 0x80, 0xa2, 0x7c, 0xf6, 0x59, 0x80, 0x91, + 0xf7, 0x00, 0x87, 0x7c, 0xeb, 0x87, 0xdb, 0x60, 0x5c, 0xce, 0x08, 0x4d, 0xa4, 0x47, 0xcf, 0x4d, + 0x1c, 0xa5, 0xe8, 0x64, 0x95, 0xb5, 0xf3, 0x43, 0xe3, 0xd3, 0x91, 0x3d, 0xa9, 0x0d, 0x0f, 0x40, + 0x45, 0xc6, 0x84, 0xdf, 0x21, 0x4c, 0xfb, 0x60, 0x29, 0xbf, 0xbf, 0x74, 0x03, 0x2a, 0x1f, 0x83, + 0x66, 0xf7, 0x9c, 0x33, 0x08, 0x7a, 0x74, 0x78, 0x07, 0x8c, 0x62, 0xe2, 0x88, 0xd2, 0xad, 0x6c, + 0x2e, 0xe6, 0xac, 0x75, 0x2f, 0x26, 0x0b, 0x07, 0x13, 0x27, 0x67, 0x85, 0xeb, 0xc1, 0x2f, 0xc0, + 0x44, 0x1a, 0x46, 0xe9, 0xd5, 0xd8, 0x90, 0x2b, 0x2a, 0x59, 0x90, 0xd5, 0xe1, 0xf7, 0x08, 0x39, + 0x8b, 0x15, 0x85, 0xd1, 0xb8, 0x04, 0xc6, 0x45, 0x01, 0x6f, 0xbe, 0xd2, 0x40, 0x65, 0x3b, 0x35, + 0xb7, 0xe5, 0xbb, 0xf0, 0x61, 0xfa, 0x16, 0xca, 0xc8, 0x85, 0x70, 0xf9, 0xcc, 0xd7, 0x62, 0xc5, + 0x18, 0x64, 0xe5, 0x52, 0xb3, 0xae, 0xbd, 0xa7, 0xc1, 0x8f, 0xc1, 0xc4, 0x1e, 0xf6, 0x69, 0xc0, + 0xc4, 0x8b, 0x1c, 0xc2, 0xbe, 0x20, 0x64, 0xef, 0xf9, 0xca, 0xa2, 0x29, 0xbf, 0x60, 0x99, 0xd9, + 0x17, 0x2c, 0x73, 0x9b, 0xfb, 0xdd, 0xd8, 0xf9, 0xf9, 0x4d, 0x55, 0x7b, 0xfd, 0xa6, 0xaa, 0xfd, + 0xfe, 0xa6, 0xaa, 0xbd, 0x3c, 0xad, 0x8e, 0xbc, 0x3e, 0xad, 0x8e, 0xfc, 0x7a, 0x5a, 0x1d, 0xf9, + 0xaa, 0xde, 0x72, 0xd9, 0xb3, 0xc8, 0x36, 0x9b, 0xd4, 0x4b, 0x3f, 0x97, 0xf9, 0x01, 0x3d, 0xc4, + 0x4d, 0x96, 0x9e, 0xea, 0x7d, 0xdf, 0xdd, 0xec, 0xa2, 0x30, 0x7d, 0xf3, 0xaf, 0x00, 0x00, 0x00, + 0xff, 0xff, 0x50, 0x7f, 0x67, 0x18, 0x91, 0x13, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -1543,6 +1552,11 @@ func (m *JobRunLease) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.JobRunIndex != 0 { + i = encodeVarintExecutorapi(dAtA, i, uint64(m.JobRunIndex)) + i-- + dAtA[i] = 0x40 + } if len(m.JobRunId) > 0 { i -= len(m.JobRunId) copy(dAtA[i:], m.JobRunId) @@ -2070,6 +2084,9 @@ func (m *JobRunLease) Size() (n int) { if l > 0 { n += 1 + l + sovExecutorapi(uint64(l)) } + if m.JobRunIndex != 0 { + n += 1 + sovExecutorapi(uint64(m.JobRunIndex)) + } return n } @@ -4343,6 +4360,25 @@ func (m *JobRunLease) Unmarshal(dAtA []byte) error { } m.JobRunId = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field JobRunIndex", wireType) + } + m.JobRunIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowExecutorapi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.JobRunIndex |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipExecutorapi(dAtA[iNdEx:]) diff --git a/pkg/executorapi/executorapi.proto b/pkg/executorapi/executorapi.proto index 58fa4aa7cf6..d34f4262408 100644 --- a/pkg/executorapi/executorapi.proto +++ b/pkg/executorapi/executorapi.proto @@ -79,6 +79,7 @@ message JobRunLease{ repeated string groups = 5; armadaevents.SubmitJob job = 6; string job_run_id = 7; + uint32 job_run_index = 8; } // Indicates that the job runs with the given ids should be cancelled. From 71c1009a243f6cb8e23172a5ea3d849ebe5f7c08 Mon Sep 17 00:00:00 2001 From: Jason Parraga Date: Thu, 29 May 2025 17:49:31 +0000 Subject: [PATCH 3/3] Make migration smoother with nullable value Signed-off-by: Jason Parraga --- internal/common/ingest/testfixtures/event.go | 1 + internal/scheduler/api.go | 7 ++++++- internal/scheduler/database/job_repository.go | 2 +- .../database/migrations/022_add_job_run_index.sql | 2 +- internal/scheduler/database/models.go | 2 +- internal/scheduler/jobdb/reconciliation.go | 6 +++++- internal/scheduleringester/instructions.go | 3 ++- internal/scheduleringester/instructions_test.go | 1 + 8 files changed, 18 insertions(+), 6 deletions(-) diff --git a/internal/common/ingest/testfixtures/event.go b/internal/common/ingest/testfixtures/event.go index 7d5e45eef82..1fb9736f866 100644 --- a/internal/common/ingest/testfixtures/event.go +++ b/internal/common/ingest/testfixtures/event.go @@ -48,6 +48,7 @@ const ( ) var ( + RunIndex = int64(0) PartitionMarkerGroupIdUuid = uuid.MustParse(PartitionMarkerGroupId) PriorityClassName = "test-priority" Groups = []string{"group1", "group2"} diff --git a/internal/scheduler/api.go b/internal/scheduler/api.go index 6918cea301b..bddf99bac9f 100644 --- a/internal/scheduler/api.go +++ b/internal/scheduler/api.go @@ -170,11 +170,16 @@ func (srv *ExecutorApi) LeaseJobRuns(stream executorapi.ExecutorApi_LeaseJobRuns } } + var runIndex uint32 = 0 + if lease.RunIndex != nil { + runIndex = uint32(*lease.RunIndex) + } + err := stream.Send(&executorapi.LeaseStreamMessage{ Event: &executorapi.LeaseStreamMessage_Lease{ Lease: &executorapi.JobRunLease{ JobRunId: lease.RunID, - JobRunIndex: lease.RunIndex, + JobRunIndex: runIndex, Queue: lease.Queue, Jobset: lease.JobSet, User: lease.UserID, diff --git a/internal/scheduler/database/job_repository.go b/internal/scheduler/database/job_repository.go index 26472aa58f6..81f7a622647 100644 --- a/internal/scheduler/database/job_repository.go +++ b/internal/scheduler/database/job_repository.go @@ -24,7 +24,7 @@ type hasSerial interface { type JobRunLease struct { RunID string - RunIndex uint32 + RunIndex *int64 Queue string Pool string JobSet string diff --git a/internal/scheduler/database/migrations/022_add_job_run_index.sql b/internal/scheduler/database/migrations/022_add_job_run_index.sql index f1186970ef3..28737c31bd5 100644 --- a/internal/scheduler/database/migrations/022_add_job_run_index.sql +++ b/internal/scheduler/database/migrations/022_add_job_run_index.sql @@ -1 +1 @@ -ALTER TABLE runs ADD COLUMN IF NOT EXISTS run_index bigint NOT NULL DEFAULT 0; +ALTER TABLE runs ADD COLUMN IF NOT EXISTS run_index bigint; diff --git a/internal/scheduler/database/models.go b/internal/scheduler/database/models.go index fe83da0eb3e..386e99d1777 100644 --- a/internal/scheduler/database/models.go +++ b/internal/scheduler/database/models.go @@ -89,5 +89,5 @@ type Run struct { PreemptRequested bool `db:"preempt_requested"` Queue string `db:"queue"` Pool string `db:"pool"` - RunIndex int64 `db:"run_index"` + RunIndex *int64 `db:"run_index"` } diff --git a/internal/scheduler/jobdb/reconciliation.go b/internal/scheduler/jobdb/reconciliation.go index 88b1c76e2fc..88f4f471aa2 100644 --- a/internal/scheduler/jobdb/reconciliation.go +++ b/internal/scheduler/jobdb/reconciliation.go @@ -306,9 +306,13 @@ func (jobDb *JobDb) schedulerJobFromDatabaseJob(dbJob *database.Job) (*Job, erro // schedulerRunFromDatabaseRun creates a new scheduler job run from a database job run func (jobDb *JobDb) schedulerRunFromDatabaseRun(dbRun *database.Run) *JobRun { nodeId := api.NodeIdFromExecutorAndNodeName(dbRun.Executor, dbRun.Node) + var runIndex uint32 = 0 + if dbRun.RunIndex != nil { + runIndex = uint32(*dbRun.RunIndex) + } return jobDb.CreateRun( dbRun.RunID, - uint32(dbRun.RunIndex), + runIndex, dbRun.JobID, dbRun.Created, dbRun.Executor, diff --git a/internal/scheduleringester/instructions.go b/internal/scheduleringester/instructions.go index 8917cf6aa5a..c68f447cdea 100644 --- a/internal/scheduleringester/instructions.go +++ b/internal/scheduleringester/instructions.go @@ -186,6 +186,7 @@ func (c *JobSetEventsInstructionConverter) handleJobRunLeased(jobRunLeased *arma if jobRunLeased.HasScheduledAtPriority { scheduledAtPriority = &jobRunLeased.ScheduledAtPriority } + runIndex := int64(jobRunLeased.RunIndex) PodRequirementsOverlay, err := proto.Marshal(jobRunLeased.GetPodRequirementsOverlay()) if err != nil { return nil, errors.WithStack(err) @@ -195,7 +196,7 @@ func (c *JobSetEventsInstructionConverter) handleJobRunLeased(jobRunLeased *arma Queue: meta.queue, DbRun: &schedulerdb.Run{ RunID: runId, - RunIndex: int64(jobRunLeased.RunIndex), + RunIndex: &runIndex, JobID: jobRunLeased.JobId, Created: eventTime.UnixNano(), JobSet: meta.jobset, diff --git a/internal/scheduleringester/instructions_test.go b/internal/scheduleringester/instructions_test.go index d51d729b48a..94d625d9583 100644 --- a/internal/scheduleringester/instructions_test.go +++ b/internal/scheduleringester/instructions_test.go @@ -54,6 +54,7 @@ func TestConvertEventSequence(t *testing.T) { expected: []DbOperation{ InsertRuns{f.RunId: &JobRunDetails{Queue: f.Queue, DbRun: &schedulerdb.Run{ RunID: f.RunId, + RunIndex: &f.RunIndex, JobID: f.JobId, JobSet: f.JobsetName, Queue: f.Queue,