diff --git a/.changelog/24909.txt b/.changelog/24909.txt
new file mode 100644
index 00000000000..6d000d3fd79
--- /dev/null
+++ b/.changelog/24909.txt
@@ -0,0 +1,3 @@
+```release-note:breaking-change
+drivers: remove remote task support for task drivers
+```
diff --git a/api/tasks.go b/api/tasks.go
index b39c55ad56b..cdf747a7290 100644
--- a/api/tasks.go
+++ b/api/tasks.go
@@ -1112,17 +1112,6 @@ type TaskState struct {
StartedAt time.Time
FinishedAt time.Time
Events []*TaskEvent
-
- // Experimental - TaskHandle is based on drivers.TaskHandle and used
- // by remote task drivers to migrate task handles between allocations.
- TaskHandle *TaskHandle
-}
-
-// Experimental - TaskHandle is based on drivers.TaskHandle and used by remote
-// task drivers to migrate task handles between allocations.
-type TaskHandle struct {
- Version int
- DriverState []byte
}
const (
diff --git a/client/allocrunner/taskrunner/remotetask_hook.go b/client/allocrunner/taskrunner/remotetask_hook.go
deleted file mode 100644
index edbbda48b62..00000000000
--- a/client/allocrunner/taskrunner/remotetask_hook.go
+++ /dev/null
@@ -1,128 +0,0 @@
-// Copyright (c) HashiCorp, Inc.
-// SPDX-License-Identifier: BUSL-1.1
-
-package taskrunner
-
-import (
- "context"
-
- hclog "github.com/hashicorp/go-hclog"
- "github.com/hashicorp/nomad/client/allocrunner/interfaces"
- "github.com/hashicorp/nomad/nomad/structs"
- "github.com/hashicorp/nomad/plugins/drivers"
-)
-
-var _ interfaces.TaskPrestartHook = (*remoteTaskHook)(nil)
-var _ interfaces.TaskPreKillHook = (*remoteTaskHook)(nil)
-
-// remoteTaskHook reattaches to remotely executing tasks.
-// Deprecated: remote tasks drivers are no longer developed or supported.
-type remoteTaskHook struct {
- tr *TaskRunner
-
- logger hclog.Logger
-}
-
-func newRemoteTaskHook(tr *TaskRunner, logger hclog.Logger) interfaces.TaskHook {
- h := &remoteTaskHook{
- tr: tr,
- }
- h.logger = logger.Named(h.Name())
- return h
-}
-
-func (h *remoteTaskHook) Name() string {
- return "remote_task"
-}
-
-// Prestart performs 2 remote task driver related tasks:
-// 1. If there is no local handle, see if there is a handle propagated from a
-// previous alloc to be restored.
-// 2. If the alloc is lost make sure the task signal is set to detach instead
-// of kill.
-func (h *remoteTaskHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error {
- if h.tr.getDriverHandle() != nil {
- // Driver handle already exists so don't try to load remote
- // task handle
- return nil
- }
-
- h.tr.stateLock.Lock()
- th := drivers.NewTaskHandleFromState(h.tr.state)
- h.tr.stateLock.Unlock()
-
- // Task handle will be nil if there was no previous allocation or if
- // this is a destructive update
- if th == nil {
- resp.Done = true
- return nil
- }
-
- // The task config is unique per invocation so recreate it here
- th.Config = h.tr.buildTaskConfig()
-
- if err := h.tr.driver.RecoverTask(th); err != nil {
- // Soft error here to let a new instance get started instead of
- // failing the task since retrying is unlikely to help.
- h.logger.Error("error recovering task state", "error", err)
- return nil
- }
-
- taskInfo, err := h.tr.driver.InspectTask(th.Config.ID)
- if err != nil {
- // Soft error here to let a new instance get started instead of
- // failing the task since retrying is unlikely to help.
- h.logger.Error("error inspecting recovered task state", "error", err)
- return nil
- }
-
- h.tr.setDriverHandle(NewDriverHandle(h.tr.driver, th.Config.ID, h.tr.Task(), h.tr.clientConfig.MaxKillTimeout, taskInfo.NetworkOverride))
-
- h.tr.stateLock.Lock()
- h.tr.localState.TaskHandle = th
- h.tr.localState.DriverNetwork = taskInfo.NetworkOverride
- h.tr.stateLock.Unlock()
-
- // Ensure the signal is set according to the allocation's state
- h.setSignal(h.tr.Alloc())
-
- // Emit TaskStarted manually since the normal task runner logic will
- // treat this task like a restored task and skip emitting started.
- h.tr.UpdateState(structs.TaskStateRunning, structs.NewTaskEvent(structs.TaskStarted))
-
- return nil
-}
-
-// PreKilling tells the remote task driver to detach a remote task instead of
-// stopping it.
-func (h *remoteTaskHook) PreKilling(ctx context.Context, req *interfaces.TaskPreKillRequest, resp *interfaces.TaskPreKillResponse) error {
- alloc := h.tr.Alloc()
- h.setSignal(alloc)
- return nil
-}
-
-// setSignal to detach if the allocation is lost or draining. Safe to call
-// multiple times as it only transitions to using detach -- never back to kill.
-func (h *remoteTaskHook) setSignal(alloc *structs.Allocation) {
- driverHandle := h.tr.getDriverHandle()
- if driverHandle == nil {
- // Nothing to do exit early
- return
- }
-
- switch {
- case alloc.ClientStatus == structs.AllocClientStatusLost:
- // Continue on; lost allocs should just detach
- h.logger.Debug("detaching from remote task since alloc was lost")
- case alloc.DesiredTransition.ShouldMigrate():
- // Continue on; migrating allocs should just detach
- h.logger.Debug("detaching from remote task since alloc was drained")
- default:
- // Nothing to do exit early
- return
- }
-
- // Set DetachSignal to indicate to the remote task driver that it
- // should detach this remote task and ignore it.
- driverHandle.SetKillSignal(drivers.DetachSignal)
-}
diff --git a/client/allocrunner/taskrunner/task_runner.go b/client/allocrunner/taskrunner/task_runner.go
index b93b75e3d83..a45def36b53 100644
--- a/client/allocrunner/taskrunner/task_runner.go
+++ b/client/allocrunner/taskrunner/task_runner.go
@@ -1364,12 +1364,6 @@ func (tr *TaskRunner) UpdateState(state string, event *structs.TaskEvent) {
tr.logger.Error("error persisting task state", "error", err, "event", event, "state", state)
}
- // Store task handle for remote tasks
- if tr.driverCapabilities != nil && tr.driverCapabilities.RemoteTasks {
- tr.logger.Trace("storing remote task handle state")
- tr.localState.TaskHandle.Store(tr.state)
- }
-
// Notify the alloc runner of the transition
tr.stateUpdater.TaskStateUpdated()
}
diff --git a/client/allocrunner/taskrunner/task_runner_hooks.go b/client/allocrunner/taskrunner/task_runner_hooks.go
index 86b1cebd41a..b49b686d2a6 100644
--- a/client/allocrunner/taskrunner/task_runner_hooks.go
+++ b/client/allocrunner/taskrunner/task_runner_hooks.go
@@ -187,12 +187,6 @@ func (tr *TaskRunner) initHooks() {
logger: hookLogger,
}))
- // If this task driver has remote capabilities, add the remote task
- // hook.
- if tr.driverCapabilities.RemoteTasks {
- tr.runnerHooks = append(tr.runnerHooks, newRemoteTaskHook(tr, hookLogger))
- }
-
// If this task has a pause schedule, initialize the pause (Enterprise)
if task.Schedule != nil {
tr.runnerHooks = append(tr.runnerHooks, newPauseHook(tr, hookLogger))
diff --git a/drivers/docker/config_test.go b/drivers/docker/config_test.go
index 7b7fd5a1e5f..d30e7cb544a 100644
--- a/drivers/docker/config_test.go
+++ b/drivers/docker/config_test.go
@@ -646,7 +646,6 @@ func TestConfig_Capabilities(t *testing.T) {
NetIsolationModes: []drivers.NetIsolationMode{"host", "group", "task"},
MustInitiateNetwork: true,
MountConfigs: 0,
- RemoteTasks: false,
DisableLogCollection: false,
},
},
@@ -660,7 +659,6 @@ func TestConfig_Capabilities(t *testing.T) {
NetIsolationModes: []drivers.NetIsolationMode{"host", "group", "task"},
MustInitiateNetwork: true,
MountConfigs: 0,
- RemoteTasks: false,
DisableLogCollection: true,
},
},
@@ -674,7 +672,6 @@ func TestConfig_Capabilities(t *testing.T) {
NetIsolationModes: []drivers.NetIsolationMode{"host", "group", "task"},
MustInitiateNetwork: true,
MountConfigs: 0,
- RemoteTasks: false,
DisableLogCollection: false,
},
},
diff --git a/e2e/e2e_test.go b/e2e/e2e_test.go
index fb31c9ad65c..8a304a1b460 100644
--- a/e2e/e2e_test.go
+++ b/e2e/e2e_test.go
@@ -23,7 +23,6 @@ import (
_ "github.com/hashicorp/nomad/e2e/parameterized"
_ "github.com/hashicorp/nomad/e2e/periodic"
_ "github.com/hashicorp/nomad/e2e/quotas"
- _ "github.com/hashicorp/nomad/e2e/remotetasks"
_ "github.com/hashicorp/nomad/e2e/scaling"
_ "github.com/hashicorp/nomad/e2e/scalingpolicies"
_ "github.com/hashicorp/nomad/e2e/scheduler_sysbatch"
diff --git a/e2e/remotetasks/input/ecs.nomad b/e2e/remotetasks/input/ecs.nomad
deleted file mode 100644
index 2448c287ed6..00000000000
--- a/e2e/remotetasks/input/ecs.nomad
+++ /dev/null
@@ -1,46 +0,0 @@
-# Copyright (c) HashiCorp, Inc.
-# SPDX-License-Identifier: BUSL-1.1
-
-variable "subnets" {
- type = list(string)
- description = "Subnet IDs task will run in."
-}
-
-variable "security_groups" {
- type = list(string)
- description = "Security Group IDs task will run in."
-}
-
-job "nomad-ecs-e2e" {
- datacenters = ["dc1"]
-
- group "ecs-remote-task-e2e" {
- restart {
- attempts = 0
- mode = "fail"
- }
-
- reschedule {
- delay = "5s"
- }
-
- task "http-server" {
- driver = "ecs"
- kill_timeout = "1m" // increased from default to accomodate ECS.
-
- config {
- task {
- launch_type = "FARGATE"
- task_definition = "nomad-rtd-e2e"
- network_configuration {
- aws_vpc_configuration {
- assign_public_ip = "ENABLED"
- security_groups = var.security_groups
- subnets = var.subnets
- }
- }
- }
- }
- }
- }
-}
diff --git a/e2e/remotetasks/remotetasks.go b/e2e/remotetasks/remotetasks.go
deleted file mode 100644
index a71eafcb7af..00000000000
--- a/e2e/remotetasks/remotetasks.go
+++ /dev/null
@@ -1,446 +0,0 @@
-// Copyright (c) HashiCorp, Inc.
-// SPDX-License-Identifier: BUSL-1.1
-
-package remotetasks
-
-import (
- "context"
- "fmt"
- "os"
- "testing"
- "time"
-
- "github.com/aws/aws-sdk-go-v2/aws"
- "github.com/aws/aws-sdk-go-v2/config"
- "github.com/aws/aws-sdk-go-v2/service/ecs"
- "github.com/hashicorp/nomad/api"
- "github.com/hashicorp/nomad/e2e/e2eutil"
- "github.com/hashicorp/nomad/e2e/framework"
- "github.com/hashicorp/nomad/helper/uuid"
- "github.com/hashicorp/nomad/jobspec2"
- "github.com/hashicorp/nomad/plugins/base"
- "github.com/hashicorp/nomad/testutil"
- "github.com/stretchr/testify/assert"
- "github.com/stretchr/testify/require"
-)
-
-const (
- // ECS Task Statuses (currently unused statuses commented out to
- // appease linter)
- //ecsTaskStatusDeactivating = "DEACTIVATING"
- //ecsTaskStatusStopping = "STOPPING"
- //ecsTaskStatusDeprovisioning = "DEPROVISIONING"
- ecsTaskStatusStopped = "STOPPED"
- ecsTaskStatusRunning = "RUNNING"
-)
-
-type RemoteTasksTest struct {
- framework.TC
- jobIDs []string
-}
-
-func init() {
- framework.AddSuites(&framework.TestSuite{
- Component: "RemoteTasks",
- CanRunLocal: true,
- Cases: []framework.TestCase{
- new(RemoteTasksTest),
- },
- })
-}
-
-func (tc *RemoteTasksTest) BeforeAll(f *framework.F) {
- e2eutil.WaitForLeader(f.T(), tc.Nomad())
- e2eutil.WaitForNodesReady(f.T(), tc.Nomad(), 2)
-}
-
-func (tc *RemoteTasksTest) AfterEach(f *framework.F) {
- nomadClient := tc.Nomad()
-
- // Mark all nodes eligible
- nodesAPI := tc.Nomad().Nodes()
- nodes, _, _ := nodesAPI.List(nil)
- for _, node := range nodes {
- nodesAPI.ToggleEligibility(node.ID, true, nil)
- }
-
- jobs := nomadClient.Jobs()
- // Stop all jobs in test
- for _, id := range tc.jobIDs {
- jobs.Deregister(id, true, nil)
- }
- tc.jobIDs = []string{}
-
- // Garbage collect
- nomadClient.System().GarbageCollect()
-}
-
-// TestECSJob asserts an ECS job may be started and is cleaned up when stopped.
-func (tc *RemoteTasksTest) TestECSJob(f *framework.F) {
- t := f.T()
-
- ctx := context.Background()
-
- ecsClient := ecsOrSkip(ctx, t, tc.Nomad())
-
- jobID := "ecsjob-" + uuid.Generate()[0:8]
- tc.jobIDs = append(tc.jobIDs, jobID)
- _, allocs := registerECSJobs(t, tc.Nomad(), jobID)
- require.Len(t, allocs, 1)
- allocID := allocs[0].ID
- e2eutil.WaitForAllocsRunning(t, tc.Nomad(), []string{allocID})
-
- // We need to go from Allocation -> ECS ARN, so grab the updated
- // allocation's task state.
- arn := arnForAlloc(t, tc.Nomad().Allocations(), allocID)
-
- // Use ARN to lookup status of ECS task in AWS
- ensureECSRunning(ctx, t, ecsClient, arn)
-
- t.Logf("Task %s is running!", arn)
-
- // Stop the job
- e2eutil.WaitForJobStopped(t, tc.Nomad(), jobID)
-
- // Ensure it is stopped in ECS
- input := ecs.DescribeTasksInput{
- Cluster: aws.String("nomad-rtd-e2e"),
- Tasks: []string{arn},
- }
- testutil.WaitForResult(func() (bool, error) {
- resp, err := ecsClient.DescribeTasks(ctx, &input)
- if err != nil {
- return false, err
- }
- status := *resp.Tasks[0].LastStatus
- return status == ecsTaskStatusStopped, fmt.Errorf("ecs task is not stopped: %s", status)
- }, func(err error) {
- t.Fatalf("error retrieving ecs task status: %v", err)
- })
-}
-
-// TestECSDrain asserts an ECS job may be started, drained from one node, and
-// is managed by a new node without stopping and restarting the remote task.
-func (tc *RemoteTasksTest) TestECSDrain(f *framework.F) {
- t := f.T()
-
- ctx := context.Background()
-
- ecsClient := ecsOrSkip(ctx, t, tc.Nomad())
-
- jobID := "ecsjob-" + uuid.Generate()[0:8]
- tc.jobIDs = append(tc.jobIDs, jobID)
- _, allocs := registerECSJobs(t, tc.Nomad(), jobID)
- require.Len(t, allocs, 1)
- origNode := allocs[0].NodeID
- origAlloc := allocs[0].ID
- e2eutil.WaitForAllocsRunning(t, tc.Nomad(), []string{origAlloc})
-
- arn := arnForAlloc(t, tc.Nomad().Allocations(), origAlloc)
- ensureECSRunning(ctx, t, ecsClient, arn)
-
- t.Logf("Task %s is running! Now to drain the node.", arn)
-
- // Drain the node
- _, err := tc.Nomad().Nodes().UpdateDrain(
- origNode,
- &api.DrainSpec{Deadline: 30 * time.Second},
- false,
- nil,
- )
- require.NoError(t, err, "error draining original node")
-
- // Wait for new alloc to be running
- var newAlloc *api.AllocationListStub
- qopts := &api.QueryOptions{}
- testutil.WaitForResult(func() (bool, error) {
- allocs, resp, err := tc.Nomad().Jobs().Allocations(jobID, false, qopts)
- if err != nil {
- return false, fmt.Errorf("error retrieving allocations for job: %w", err)
- }
-
- qopts.WaitIndex = resp.LastIndex
-
- if len(allocs) > 2 {
- return false, fmt.Errorf("expected 1 or 2 allocs but found %d", len(allocs))
- }
-
- for _, alloc := range allocs {
- if alloc.ID == origAlloc {
- // This is the old alloc, skip it
- continue
- }
-
- newAlloc = alloc
-
- if newAlloc.ClientStatus == "running" {
- break
- }
- }
-
- if newAlloc == nil {
- return false, fmt.Errorf("no new alloc found")
- }
- if newAlloc.ClientStatus != "running" {
- return false, fmt.Errorf("expected new alloc (%s) to be running but found: %s",
- newAlloc.ID, newAlloc.ClientStatus)
- }
-
- return true, nil
- }, func(err error) {
- t.Fatalf("error waiting for new alloc to be running: %v", err)
- })
-
- // Make sure the ARN hasn't changed by looking up the new alloc's ARN
- newARN := arnForAlloc(t, tc.Nomad().Allocations(), newAlloc.ID)
-
- assert.Equal(t, arn, newARN, "unexpected new ARN")
-}
-
-// TestECSDeployment asserts a new ECS task is started when an ECS job is
-// deployed.
-func (tc *RemoteTasksTest) TestECSDeployment(f *framework.F) {
- t := f.T()
-
- ctx := context.Background()
-
- ecsClient := ecsOrSkip(ctx, t, tc.Nomad())
-
- jobID := "ecsjob-" + uuid.Generate()[0:8]
- tc.jobIDs = append(tc.jobIDs, jobID)
- job, origAllocs := registerECSJobs(t, tc.Nomad(), jobID)
- require.Len(t, origAllocs, 1)
- origAllocID := origAllocs[0].ID
- e2eutil.WaitForAllocsRunning(t, tc.Nomad(), []string{origAllocID})
-
- // We need to go from Allocation -> ECS ARN, so grab the updated
- // allocation's task state.
- origARN := arnForAlloc(t, tc.Nomad().Allocations(), origAllocID)
-
- // Use ARN to lookup status of ECS task in AWS
- ensureECSRunning(ctx, t, ecsClient, origARN)
-
- t.Logf("Task %s is running! Updating...", origARN)
-
- // Force a deployment by updating meta
- job.Meta = map[string]string{
- "updated": time.Now().Format(time.RFC3339Nano),
- }
-
- // Register updated job
- resp, _, err := tc.Nomad().Jobs().Register(job, nil)
- require.NoError(t, err, "error registering updated job")
- require.NotEmpty(t, resp.EvalID, "no eval id created when registering updated job")
-
- // Wait for new alloc to be running
- var newAlloc *api.AllocationListStub
- testutil.WaitForResult(func() (bool, error) {
- allocs, _, err := tc.Nomad().Jobs().Allocations(jobID, false, nil)
- if err != nil {
- return false, err
- }
-
- for _, a := range allocs {
- if a.ID == origAllocID {
- if a.ClientStatus == "complete" {
- // Original alloc stopped as expected!
- continue
- }
-
- // Original alloc is still running
- newAlloc = nil
- return false, fmt.Errorf("original alloc not yet terminal. "+
- "client status: %s; desired status: %s",
- a.ClientStatus, a.DesiredStatus)
- }
-
- if a.ClientStatus != "running" {
- return false, fmt.Errorf("new alloc is not running: %s", a.ClientStatus)
- }
-
- if newAlloc != nil {
- return false, fmt.Errorf("found 2 replacement allocs: %s and %s",
- a.ID, newAlloc.ID)
- }
-
- newAlloc = a
- }
-
- return newAlloc != nil, fmt.Errorf("no new alloc found for updated job")
- }, func(err error) {
- require.NoError(t, err, "error waiting for updated alloc")
- })
-
- newARN := arnForAlloc(t, tc.Nomad().Allocations(), newAlloc.ID)
- t.Logf("Task %s is updated!", newARN)
- require.NotEqual(t, origARN, newARN, "expected new ARN")
-
- // Ensure original ARN is stopped in ECS
- input := ecs.DescribeTasksInput{
- Cluster: aws.String("nomad-rtd-e2e"),
- Tasks: []string{origARN},
- }
- testutil.WaitForResult(func() (bool, error) {
- resp, err := ecsClient.DescribeTasks(ctx, &input)
- if err != nil {
- return false, err
- }
- status := *resp.Tasks[0].LastStatus
- return status == ecsTaskStatusStopped, fmt.Errorf("original ecs task is not stopped: %s", status)
- }, func(err error) {
- t.Fatalf("error retrieving ecs task status for original ARN: %v", err)
- })
-}
-
-// ecsOrSkip returns an AWS ECS client or skips the test if ECS is unreachable
-// by the test runner or the ECS remote task driver isn't healthy.
-func ecsOrSkip(ctx context.Context, t *testing.T, nomadClient *api.Client) *ecs.Client {
- cfg, err := config.LoadDefaultConfig(ctx, config.WithRegion("us-east-1"))
- require.NoError(t, err)
-
- ecsClient := ecs.NewFromConfig(cfg)
-
- _, err = ecsClient.ListClusters(ctx, &ecs.ListClustersInput{})
- if err != nil {
- t.Skipf("Skipping ECS Remote Task Driver Task. Error querying AWS ECS API: %v", err)
- }
-
- testutil.WaitForResult(func() (bool, error) {
- nodes, _, err := nomadClient.Nodes().List(nil)
- if err != nil {
- return false, fmt.Errorf("error retrieving node listing: %w", err)
- }
-
- notReady := 0
- notEligible := 0
- noECS := 0
- notHealthy := 0
- ready := 0
- for _, n := range nodes {
- if n.Status != "ready" {
- notReady++
- continue
- }
- if n.SchedulingEligibility != "eligible" {
- notEligible++
- continue
- }
- ecsDriver, ok := n.Drivers["ecs"]
- if !ok {
- noECS++
- continue
- }
- if !ecsDriver.Healthy {
- notHealthy++
- continue
- }
- ready++
- }
-
- return ready > 1, fmt.Errorf("expected 2 nodes with healthy ecs drivers but found: "+
- "not_ready=%d ineligible=%d no_driver=%d unhealthy=%d ok=%d",
- notReady, notEligible, noECS, notHealthy, ready)
- }, func(err error) {
- if err != nil {
- t.Skipf("Skipping Remote Task Driver tests due to: %v", err)
- }
- })
-
- return ecsClient
-}
-
-// arnForAlloc retrieves the ARN for a running allocation.
-func arnForAlloc(t *testing.T, allocAPI *api.Allocations, allocID string) string {
- t.Logf("Retrieving ARN for alloc=%s", allocID)
- ecsState := struct {
- ARN string
- }{}
- testutil.WaitForResult(func() (bool, error) {
- alloc, _, err := allocAPI.Info(allocID, nil)
- if err != nil {
- return false, err
- }
- state := alloc.TaskStates["http-server"]
- if state == nil {
- return false, fmt.Errorf("no task state for http-server (%d task states)", len(alloc.TaskStates))
- }
- if state.TaskHandle == nil {
- return false, fmt.Errorf("no task handle for http-server")
- }
- if len(state.TaskHandle.DriverState) == 0 {
- return false, fmt.Errorf("no driver state for task handle")
- }
- if err := base.MsgPackDecode(state.TaskHandle.DriverState, &ecsState); err != nil {
- return false, fmt.Errorf("error decoding driver state: %w", err)
- }
- if ecsState.ARN == "" {
- return false, fmt.Errorf("ARN is empty despite DriverState being %d bytes", len(state.TaskHandle.DriverState))
- }
- return true, nil
- }, func(err error) {
- t.Fatalf("error getting ARN: %v", err)
- })
- t.Logf("Retrieved ARN=%s for alloc=%s", ecsState.ARN, allocID)
-
- return ecsState.ARN
-}
-
-// ensureECSRunning asserts that the given ARN is a running ECS task.
-func ensureECSRunning(ctx context.Context, t *testing.T, ecsClient *ecs.Client, arn string) {
- t.Logf("Ensuring ARN=%s is running", arn)
- input := ecs.DescribeTasksInput{
- Cluster: aws.String("nomad-rtd-e2e"),
- Tasks: []string{arn},
- }
- testutil.WaitForResult(func() (bool, error) {
- resp, err := ecsClient.DescribeTasks(ctx, &input)
- if err != nil {
- return false, err
- }
- status := *resp.Tasks[0].LastStatus
- return status == ecsTaskStatusRunning, fmt.Errorf("ecs task is not running: %s", status)
- }, func(err error) {
- t.Fatalf("error retrieving ecs task status: %v", err)
- })
- t.Logf("ARN=%s is running", arn)
-}
-
-// registerECSJobs registers an ECS job and returns it and its allocation
-// stubs.
-func registerECSJobs(t *testing.T, nomadClient *api.Client, jobID string) (*api.Job, []*api.AllocationListStub) {
- const (
- jobPath = "remotetasks/input/ecs.nomad"
- varPath = "remotetasks/input/ecs.vars"
- )
-
- jobBytes, err := os.ReadFile(jobPath)
- require.NoError(t, err, "error reading job file")
-
- job, err := jobspec2.ParseWithConfig(&jobspec2.ParseConfig{
- Path: jobPath,
- Body: jobBytes,
- VarFiles: []string{varPath},
- Strict: true,
- })
- require.NoErrorf(t, err, "error parsing jobspec from %s with var file %s", jobPath, varPath)
-
- job.ID = &jobID
- job.Name = &jobID
-
- // Register job
- resp, _, err := nomadClient.Jobs().Register(job, nil)
- require.NoError(t, err, "error registering job")
- require.NotEmpty(t, resp.EvalID, "no eval id created when registering job")
-
- var allocs []*api.AllocationListStub
- testutil.WaitForResult(func() (bool, error) {
- allocs, _, err = nomadClient.Jobs().Allocations(jobID, false, nil)
- if err != nil {
- return false, err
- }
- return len(allocs) > 0, fmt.Errorf("no allocs found")
- }, func(err error) {
- require.NoErrorf(t, err, "error retrieving allocations for %s", jobID)
- })
- return job, allocs
-}
diff --git a/e2e/terraform/provision-infra/ecs-task.json b/e2e/terraform/provision-infra/ecs-task.json
deleted file mode 100644
index cd0f0db9f3e..00000000000
--- a/e2e/terraform/provision-infra/ecs-task.json
+++ /dev/null
@@ -1,21 +0,0 @@
-[
- {
- "command": [
- "/bin/sh -c \"echo '
Amazon ECS Sample App Amazon ECS Sample App
Congratulations!
Your application is now running on a container in Amazon ECS.
' > /usr/local/apache2/htdocs/index.html && httpd-foreground\""
- ],
- "entryPoint": [
- "sh",
- "-c"
- ],
- "essential": true,
- "image": "httpd:2.4",
- "name": "nomad-remote-driver-demo",
- "portMappings": [
- {
- "containerPort": 80,
- "hostPort": 80,
- "protocol": "tcp"
- }
- ]
- }
-]
diff --git a/e2e/terraform/provision-infra/ecs.tf b/e2e/terraform/provision-infra/ecs.tf
deleted file mode 100644
index b0854806e8d..00000000000
--- a/e2e/terraform/provision-infra/ecs.tf
+++ /dev/null
@@ -1,28 +0,0 @@
-# Copyright (c) HashiCorp, Inc.
-# SPDX-License-Identifier: BUSL-1.1
-
-# Nomad ECS Remote Task Driver E2E
-resource "aws_ecs_cluster" "nomad_rtd_e2e" {
- name = "nomad-rtd-e2e-${random_pet.e2e.id}"
-}
-
-resource "aws_ecs_task_definition" "nomad_rtd_e2e" {
- family = "nomad-rtd-e2e"
- container_definitions = file("${path.module}/ecs-task.json")
-
- # Don't need a network for e2e tests
- network_mode = "awsvpc"
-
- requires_compatibilities = ["FARGATE"]
- cpu = 256
- memory = 512
-}
-
-resource "local_file" "ecs_vars_hcl" {
- content = templatefile("${path.module}/ecs.tftpl", {
- sg_id = aws_security_group.clients.id,
- subnet_id = data.aws_subnet.default.id,
- })
- filename = "${path.module}/../remotetasks/input/ecs.vars"
- file_permission = "0664"
-}
diff --git a/e2e/terraform/provision-infra/ecs.tftpl b/e2e/terraform/provision-infra/ecs.tftpl
deleted file mode 100644
index bb5a10430ba..00000000000
--- a/e2e/terraform/provision-infra/ecs.tftpl
+++ /dev/null
@@ -1,2 +0,0 @@
-security_groups = ["${sg_id}"]
-subnets = ["${subnet_id}"]
diff --git a/go.mod b/go.mod
index 92d5e62fedc..ed32d8e500d 100644
--- a/go.mod
+++ b/go.mod
@@ -18,10 +18,8 @@ require (
github.com/Microsoft/go-winio v0.6.1
github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e
github.com/armon/go-metrics v0.5.3
- github.com/aws/aws-sdk-go-v2 v1.32.7
github.com/aws/aws-sdk-go-v2/config v1.28.6
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.21
- github.com/aws/aws-sdk-go-v2/service/ecs v1.53.0
github.com/aws/smithy-go v1.22.1
github.com/brianvoe/gofakeit/v6 v6.20.1
github.com/container-storage-interface/spec v1.10.0
@@ -186,6 +184,7 @@ require (
github.com/apparentlymart/go-textseg/v15 v15.0.0 // indirect
github.com/armon/go-radix v1.0.0 // indirect
github.com/aws/aws-sdk-go v1.55.6 // indirect
+ github.com/aws/aws-sdk-go-v2 v1.32.7 // indirect
github.com/aws/aws-sdk-go-v2/credentials v1.17.47 // indirect
github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.26 // indirect
github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.26 // indirect
diff --git a/go.sum b/go.sum
index 260ba280e48..4ca6b173497 100644
--- a/go.sum
+++ b/go.sum
@@ -755,8 +755,6 @@ github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.26 h1:zXFLuEuMMUOvEARXFU
github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.26/go.mod h1:3o2Wpy0bogG1kyOPrgkXA8pgIfEEv0+m19O9D5+W8y8=
github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 h1:VaRN3TlFdd6KxX1x3ILT5ynH6HvKgqdiXoTxAF4HQcQ=
github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1/go.mod h1:FbtygfRFze9usAadmnGJNc8KsP346kEe+y2/oyhGAGc=
-github.com/aws/aws-sdk-go-v2/service/ecs v1.53.0 h1:TCQZX4ztlcWXAcZouKh9qJMcVaH/qTidFTfsvJwUI30=
-github.com/aws/aws-sdk-go-v2/service/ecs v1.53.0/go.mod h1:Ghi1OWUv4+VMEULWiHsKH2gNA3KAcMoLWsvU0eRXvIA=
github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.1 h1:iXtILhvDxB6kPvEXgsDhGaZCSC6LQET5ZHSdJozeI0Y=
github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.1/go.mod h1:9nu0fVANtYiAePIBh2/pFUSwtJ402hLnp854CNoDOeE=
github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.6 h1:50+XsN70RS7dwJ2CkVNXzj7U2L1HKP8nqTd3XWEXBN4=
diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go
index deb3f25d96f..734e9241306 100644
--- a/nomad/structs/structs.go
+++ b/nomad/structs/structs.go
@@ -9117,44 +9117,6 @@ type AllocState struct {
Time time.Time
}
-// TaskHandle is optional handle to a task propogated to the servers for use
-// by remote tasks. Since remote tasks are not implicitly lost when the node
-// they are assigned to is down, their state is migrated to the replacement
-// allocation.
-//
-// Minimal set of fields from plugins/drivers/task_handle.go:TaskHandle
-type TaskHandle struct {
- // Version of driver state. Used by the driver to gracefully handle
- // plugin upgrades.
- Version int
-
- // Driver-specific state containing a handle to the remote task.
- DriverState []byte
-}
-
-func (h *TaskHandle) Copy() *TaskHandle {
- if h == nil {
- return nil
- }
-
- newTH := TaskHandle{
- Version: h.Version,
- DriverState: make([]byte, len(h.DriverState)),
- }
- copy(newTH.DriverState, h.DriverState)
- return &newTH
-}
-
-func (h *TaskHandle) Equal(o *TaskHandle) bool {
- if h == nil || o == nil {
- return h == o
- }
- if h.Version != o.Version {
- return false
- }
- return bytes.Equal(h.DriverState, o.DriverState)
-}
-
// Set of possible states for a task.
const (
TaskStatePending = "pending" // The task is waiting to be run.
@@ -9189,9 +9151,9 @@ type TaskState struct {
// Series of task events that transition the state of the task.
Events []*TaskEvent
- // Experimental - TaskHandle is based on drivers.TaskHandle and used
- // by remote task drivers to migrate task handles between allocations.
- TaskHandle *TaskHandle
+ // // Experimental - TaskHandle is based on drivers.TaskHandle and used
+ // // by remote task drivers to migrate task handles between allocations.
+ // TaskHandle *TaskHandle
// Enterprise Only - Paused is set to the paused state of the task. See
// task_sched.go
@@ -9227,7 +9189,6 @@ func (ts *TaskState) Copy() *TaskState {
}
}
- newTS.TaskHandle = ts.TaskHandle.Copy()
return newTS
}
@@ -9262,9 +9223,6 @@ func (ts *TaskState) Equal(o *TaskState) bool {
}) {
return false
}
- if !ts.TaskHandle.Equal(o.TaskHandle) {
- return false
- }
return true
}
diff --git a/plugins/drivers/client.go b/plugins/drivers/client.go
index 3c27df33003..b8d468e366e 100644
--- a/plugins/drivers/client.go
+++ b/plugins/drivers/client.go
@@ -79,7 +79,6 @@ func (d *driverPluginClient) Capabilities() (*Capabilities, error) {
}
caps.MountConfigs = MountConfigSupport(resp.Capabilities.MountConfigs)
- caps.RemoteTasks = resp.Capabilities.RemoteTasks
caps.DisableLogCollection = resp.Capabilities.DisableLogCollection
caps.DynamicWorkloadUsers = resp.Capabilities.DynamicWorkloadUsers
}
diff --git a/plugins/drivers/driver.go b/plugins/drivers/driver.go
index e7ec790afcf..8f9c6418632 100644
--- a/plugins/drivers/driver.go
+++ b/plugins/drivers/driver.go
@@ -168,14 +168,6 @@ type Capabilities struct {
// MountConfigs tells Nomad which mounting config options the driver supports.
MountConfigs MountConfigSupport
- // RemoteTasks indicates this driver runs tasks on remote systems
- // instead of locally. The Nomad client can use this information to
- // adjust behavior such as propagating task handles between allocations
- // to avoid downtime when a client is lost.
- //
- // Deprecated: remote tasks drivers are no longer developed or supported.
- RemoteTasks bool
-
// DisableLogCollection indicates this driver has disabled log collection
// and the client should not start a logmon process.
DisableLogCollection bool
diff --git a/plugins/drivers/proto/driver.pb.go b/plugins/drivers/proto/driver.pb.go
index 86103cd32af..b0e017e02cc 100644
--- a/plugins/drivers/proto/driver.pb.go
+++ b/plugins/drivers/proto/driver.pb.go
@@ -1849,9 +1849,6 @@ type DriverCapabilities struct {
MustCreateNetwork bool `protobuf:"varint,5,opt,name=must_create_network,json=mustCreateNetwork,proto3" json:"must_create_network,omitempty"`
// MountConfigs indicates whether the driver supports mount configurations.
MountConfigs DriverCapabilities_MountConfigs `protobuf:"varint,6,opt,name=mount_configs,json=mountConfigs,proto3,enum=hashicorp.nomad.plugins.drivers.proto.DriverCapabilities_MountConfigs" json:"mount_configs,omitempty"`
- // remote_tasks indicates whether the driver executes tasks remotely such
- // on cloud runtimes like AWS ECS.
- RemoteTasks bool `protobuf:"varint,7,opt,name=remote_tasks,json=remoteTasks,proto3" json:"remote_tasks,omitempty"` // Deprecated: Do not use.
// disable_log_collection indicates whether the driver has the capability of
// disabling log collection
DisableLogCollection bool `protobuf:"varint,8,opt,name=disable_log_collection,json=disableLogCollection,proto3" json:"disable_log_collection,omitempty"`
@@ -1930,14 +1927,6 @@ func (m *DriverCapabilities) GetMountConfigs() DriverCapabilities_MountConfigs {
return DriverCapabilities_UNKNOWN_MOUNTS
}
-// Deprecated: Do not use.
-func (m *DriverCapabilities) GetRemoteTasks() bool {
- if m != nil {
- return m.RemoteTasks
- }
- return false
-}
-
func (m *DriverCapabilities) GetDisableLogCollection() bool {
if m != nil {
return m.DisableLogCollection
@@ -3769,255 +3758,254 @@ func init() {
}
var fileDescriptor_4a8f45747846a74d = []byte{
- // 3953 bytes of a gzipped FileDescriptorProto
+ // 3938 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x5a, 0x4f, 0x73, 0xdb, 0x48,
- 0x76, 0x17, 0x08, 0x92, 0x22, 0x1f, 0x29, 0x0a, 0x6a, 0x4b, 0x1e, 0x9a, 0xb3, 0xc9, 0x78, 0xb0,
- 0x35, 0x29, 0x67, 0x77, 0x86, 0x9e, 0xd5, 0x26, 0xe3, 0xb1, 0xd7, 0xb3, 0x1e, 0x9a, 0xa2, 0x2d,
- 0xda, 0x12, 0xa5, 0x34, 0xa9, 0x78, 0x1d, 0x27, 0x83, 0x40, 0x40, 0x9b, 0x82, 0x45, 0x02, 0x18,
- 0x34, 0x28, 0x4b, 0x9b, 0x4a, 0x25, 0xb5, 0xa9, 0x4a, 0x6d, 0xaa, 0x92, 0x4a, 0x2e, 0x93, 0xbd,
- 0xe4, 0xb4, 0x55, 0x39, 0xa5, 0x72, 0x4f, 0x6d, 0x6a, 0x4f, 0x39, 0xe4, 0x4b, 0xe4, 0x92, 0x5b,
- 0xae, 0xa9, 0x7c, 0x80, 0x6c, 0xf5, 0x1f, 0x80, 0x80, 0x48, 0x8f, 0x41, 0xca, 0x27, 0xf2, 0xbd,
- 0xee, 0xfe, 0xf5, 0xc3, 0x7b, 0xaf, 0x5f, 0xbf, 0xee, 0x7e, 0xa0, 0xfb, 0xa3, 0xc9, 0xd0, 0x71,
- 0xe9, 0x6d, 0x3b, 0x70, 0xce, 0x48, 0x40, 0x6f, 0xfb, 0x81, 0x17, 0x7a, 0x92, 0x6a, 0x72, 0x02,
- 0x7d, 0x74, 0x62, 0xd2, 0x13, 0xc7, 0xf2, 0x02, 0xbf, 0xe9, 0x7a, 0x63, 0xd3, 0x6e, 0xca, 0x31,
- 0x4d, 0x39, 0x46, 0x74, 0x6b, 0xfc, 0xf6, 0xd0, 0xf3, 0x86, 0x23, 0x22, 0x10, 0x8e, 0x27, 0x2f,
- 0x6f, 0xdb, 0x93, 0xc0, 0x0c, 0x1d, 0xcf, 0x95, 0xed, 0x1f, 0x5c, 0x6e, 0x0f, 0x9d, 0x31, 0xa1,
- 0xa1, 0x39, 0xf6, 0x65, 0x87, 0x8f, 0x22, 0x59, 0xe8, 0x89, 0x19, 0x10, 0xfb, 0xf6, 0x89, 0x35,
- 0xa2, 0x3e, 0xb1, 0xd8, 0xaf, 0xc1, 0xfe, 0xc8, 0x6e, 0x1f, 0x5f, 0xea, 0x46, 0xc3, 0x60, 0x62,
- 0x85, 0x91, 0xe4, 0x66, 0x18, 0x06, 0xce, 0xf1, 0x24, 0x24, 0xa2, 0xb7, 0x7e, 0x03, 0xde, 0x1b,
- 0x98, 0xf4, 0xb4, 0xed, 0xb9, 0x2f, 0x9d, 0x61, 0xdf, 0x3a, 0x21, 0x63, 0x13, 0x93, 0xaf, 0x27,
- 0x84, 0x86, 0xfa, 0x1f, 0x43, 0x7d, 0xb6, 0x89, 0xfa, 0x9e, 0x4b, 0x09, 0xfa, 0x12, 0xf2, 0x6c,
- 0xca, 0xba, 0x72, 0x53, 0xb9, 0x55, 0xd9, 0xfe, 0xb8, 0xf9, 0x26, 0x15, 0x08, 0x19, 0x9a, 0x52,
- 0xd4, 0x66, 0xdf, 0x27, 0x16, 0xe6, 0x23, 0xf5, 0x2d, 0xb8, 0xd6, 0x36, 0x7d, 0xf3, 0xd8, 0x19,
- 0x39, 0xa1, 0x43, 0x68, 0x34, 0xe9, 0x04, 0x36, 0xd3, 0x6c, 0x39, 0xe1, 0x9f, 0x40, 0xd5, 0x4a,
- 0xf0, 0xe5, 0xc4, 0x77, 0x9b, 0x99, 0x74, 0xdf, 0xdc, 0xe1, 0x54, 0x0a, 0x38, 0x05, 0xa7, 0x6f,
- 0x02, 0x7a, 0xe4, 0xb8, 0x43, 0x12, 0xf8, 0x81, 0xe3, 0x86, 0x91, 0x30, 0xbf, 0x56, 0xe1, 0x5a,
- 0x8a, 0x2d, 0x85, 0x79, 0x05, 0x10, 0xeb, 0x91, 0x89, 0xa2, 0xde, 0xaa, 0x6c, 0x3f, 0xc9, 0x28,
- 0xca, 0x1c, 0xbc, 0x66, 0x2b, 0x06, 0xeb, 0xb8, 0x61, 0x70, 0x81, 0x13, 0xe8, 0xe8, 0x2b, 0x28,
- 0x9e, 0x10, 0x73, 0x14, 0x9e, 0xd4, 0x73, 0x37, 0x95, 0x5b, 0xb5, 0xed, 0x47, 0x57, 0x98, 0x67,
- 0x97, 0x03, 0xf5, 0x43, 0x33, 0x24, 0x58, 0xa2, 0xa2, 0x4f, 0x00, 0x89, 0x7f, 0x86, 0x4d, 0xa8,
- 0x15, 0x38, 0x3e, 0x73, 0xc9, 0xba, 0x7a, 0x53, 0xb9, 0x55, 0xc6, 0x1b, 0xa2, 0x65, 0x67, 0xda,
- 0xd0, 0xf0, 0x61, 0xfd, 0x92, 0xb4, 0x48, 0x03, 0xf5, 0x94, 0x5c, 0x70, 0x8b, 0x94, 0x31, 0xfb,
- 0x8b, 0x1e, 0x43, 0xe1, 0xcc, 0x1c, 0x4d, 0x08, 0x17, 0xb9, 0xb2, 0xfd, 0x83, 0xb7, 0xb9, 0x87,
- 0x74, 0xd1, 0xa9, 0x1e, 0xb0, 0x18, 0x7f, 0x2f, 0xf7, 0xb9, 0xa2, 0xdf, 0x85, 0x4a, 0x42, 0x6e,
- 0x54, 0x03, 0x38, 0xea, 0xed, 0x74, 0x06, 0x9d, 0xf6, 0xa0, 0xb3, 0xa3, 0xad, 0xa0, 0x35, 0x28,
- 0x1f, 0xf5, 0x76, 0x3b, 0xad, 0xbd, 0xc1, 0xee, 0x73, 0x4d, 0x41, 0x15, 0x58, 0x8d, 0x88, 0x9c,
- 0x7e, 0x0e, 0x08, 0x13, 0xcb, 0x3b, 0x23, 0x01, 0x73, 0x64, 0x69, 0x55, 0xf4, 0x1e, 0xac, 0x86,
- 0x26, 0x3d, 0x35, 0x1c, 0x5b, 0xca, 0x5c, 0x64, 0x64, 0xd7, 0x46, 0x5d, 0x28, 0x9e, 0x98, 0xae,
- 0x3d, 0x7a, 0xbb, 0xdc, 0x69, 0x55, 0x33, 0xf0, 0x5d, 0x3e, 0x10, 0x4b, 0x00, 0xe6, 0xdd, 0xa9,
- 0x99, 0x85, 0x01, 0xf4, 0xe7, 0xa0, 0xf5, 0x43, 0x33, 0x08, 0x93, 0xe2, 0x74, 0x20, 0xcf, 0xe6,
- 0x97, 0x1e, 0xbd, 0xc8, 0x9c, 0x62, 0x65, 0x62, 0x3e, 0x5c, 0xff, 0xdf, 0x1c, 0x6c, 0x24, 0xb0,
- 0xa5, 0xa7, 0x3e, 0x83, 0x62, 0x40, 0xe8, 0x64, 0x14, 0x72, 0xf8, 0xda, 0xf6, 0x83, 0x8c, 0xf0,
- 0x33, 0x48, 0x4d, 0xcc, 0x61, 0xb0, 0x84, 0x43, 0xb7, 0x40, 0x13, 0x23, 0x0c, 0x12, 0x04, 0x5e,
- 0x60, 0x8c, 0xe9, 0x90, 0x6b, 0xad, 0x8c, 0x6b, 0x82, 0xdf, 0x61, 0xec, 0x7d, 0x3a, 0x4c, 0x68,
- 0x55, 0xbd, 0xa2, 0x56, 0x91, 0x09, 0x9a, 0x4b, 0xc2, 0xd7, 0x5e, 0x70, 0x6a, 0x30, 0xd5, 0x06,
- 0x8e, 0x4d, 0xea, 0x79, 0x0e, 0xfa, 0x59, 0x46, 0xd0, 0x9e, 0x18, 0x7e, 0x20, 0x47, 0xe3, 0x75,
- 0x37, 0xcd, 0xd0, 0xbf, 0x0f, 0x45, 0xf1, 0xa5, 0xcc, 0x93, 0xfa, 0x47, 0xed, 0x76, 0xa7, 0xdf,
- 0xd7, 0x56, 0x50, 0x19, 0x0a, 0xb8, 0x33, 0xc0, 0xcc, 0xc3, 0xca, 0x50, 0x78, 0xd4, 0x1a, 0xb4,
- 0xf6, 0xb4, 0x9c, 0xfe, 0x3d, 0x58, 0x7f, 0x66, 0x3a, 0x61, 0x16, 0xe7, 0xd2, 0x3d, 0xd0, 0xa6,
- 0x7d, 0xa5, 0x75, 0xba, 0x29, 0xeb, 0x64, 0x57, 0x4d, 0xe7, 0xdc, 0x09, 0x2f, 0xd9, 0x43, 0x03,
- 0x95, 0x04, 0x81, 0x34, 0x01, 0xfb, 0xab, 0xbf, 0x86, 0xf5, 0x7e, 0xe8, 0xf9, 0x99, 0x3c, 0xff,
- 0x87, 0xb0, 0xca, 0x76, 0x1b, 0x6f, 0x12, 0x4a, 0xd7, 0xbf, 0xd1, 0x14, 0xbb, 0x51, 0x33, 0xda,
- 0x8d, 0x9a, 0x3b, 0x72, 0xb7, 0xc2, 0x51, 0x4f, 0x74, 0x1d, 0x8a, 0xd4, 0x19, 0xba, 0xe6, 0x48,
- 0x46, 0x0b, 0x49, 0xe9, 0x88, 0x39, 0x79, 0x34, 0xb1, 0x74, 0xfc, 0x36, 0xa0, 0x1d, 0x42, 0xc3,
- 0xc0, 0xbb, 0xc8, 0x24, 0xcf, 0x26, 0x14, 0x5e, 0x7a, 0x81, 0x25, 0x16, 0x62, 0x09, 0x0b, 0x82,
- 0x2d, 0xaa, 0x14, 0x88, 0xc4, 0xfe, 0x04, 0x50, 0xd7, 0x65, 0x7b, 0x4a, 0x36, 0x43, 0xfc, 0x43,
- 0x0e, 0xae, 0xa5, 0xfa, 0x4b, 0x63, 0x2c, 0xbf, 0x0e, 0x59, 0x60, 0x9a, 0x50, 0xb1, 0x0e, 0xd1,
- 0x01, 0x14, 0x45, 0x0f, 0xa9, 0xc9, 0x3b, 0x0b, 0x00, 0x89, 0x6d, 0x4a, 0xc2, 0x49, 0x98, 0xb9,
- 0x4e, 0xaf, 0xbe, 0x5b, 0xa7, 0x7f, 0x0d, 0x5a, 0xf4, 0x1d, 0xf4, 0xad, 0xb6, 0x79, 0x02, 0xd7,
- 0x2c, 0x6f, 0x34, 0x22, 0x16, 0xf3, 0x06, 0xc3, 0x71, 0x43, 0x12, 0x9c, 0x99, 0xa3, 0xb7, 0xfb,
- 0x0d, 0x9a, 0x8e, 0xea, 0xca, 0x41, 0xfa, 0x0b, 0xd8, 0x48, 0x4c, 0x2c, 0x0d, 0xf1, 0x08, 0x0a,
- 0x94, 0x31, 0xa4, 0x25, 0x3e, 0x5d, 0xd0, 0x12, 0x14, 0x8b, 0xe1, 0xfa, 0x35, 0x01, 0xde, 0x39,
- 0x23, 0x6e, 0xfc, 0x59, 0xfa, 0x0e, 0x6c, 0xf4, 0xb9, 0x9b, 0x66, 0xf2, 0xc3, 0xa9, 0x8b, 0xe7,
- 0x52, 0x2e, 0xbe, 0x09, 0x28, 0x89, 0x22, 0x1d, 0xf1, 0x02, 0xd6, 0x3b, 0xe7, 0xc4, 0xca, 0x84,
- 0x5c, 0x87, 0x55, 0xcb, 0x1b, 0x8f, 0x4d, 0xd7, 0xae, 0xe7, 0x6e, 0xaa, 0xb7, 0xca, 0x38, 0x22,
- 0x93, 0x6b, 0x51, 0xcd, 0xba, 0x16, 0xf5, 0xbf, 0x53, 0x40, 0x9b, 0xce, 0x2d, 0x15, 0xc9, 0xa4,
- 0x0f, 0x6d, 0x06, 0xc4, 0xe6, 0xae, 0x62, 0x49, 0x49, 0x7e, 0x14, 0x2e, 0x04, 0x9f, 0x04, 0x41,
- 0x22, 0x1c, 0xa9, 0x57, 0x0c, 0x47, 0xfa, 0x2e, 0x7c, 0x27, 0x12, 0xa7, 0x1f, 0x06, 0xc4, 0x1c,
- 0x3b, 0xee, 0xb0, 0x7b, 0x70, 0xe0, 0x13, 0x21, 0x38, 0x42, 0x90, 0xb7, 0xcd, 0xd0, 0x94, 0x82,
- 0xf1, 0xff, 0x6c, 0xd1, 0x5b, 0x23, 0x8f, 0xc6, 0x8b, 0x9e, 0x13, 0xfa, 0x7f, 0xaa, 0x50, 0x9f,
- 0x81, 0x8a, 0xd4, 0xfb, 0x02, 0x0a, 0x94, 0x84, 0x13, 0x5f, 0xba, 0x4a, 0x27, 0xb3, 0xc0, 0xf3,
- 0xf1, 0x9a, 0x7d, 0x06, 0x86, 0x05, 0x26, 0x1a, 0x42, 0x29, 0x0c, 0x2f, 0x0c, 0xea, 0xfc, 0x34,
- 0x4a, 0x08, 0xf6, 0xae, 0x8a, 0x3f, 0x20, 0xc1, 0xd8, 0x71, 0xcd, 0x51, 0xdf, 0xf9, 0x29, 0xc1,
- 0xab, 0x61, 0x78, 0xc1, 0xfe, 0xa0, 0xe7, 0xcc, 0xe1, 0x6d, 0xc7, 0x95, 0x6a, 0x6f, 0x2f, 0x3b,
- 0x4b, 0x42, 0xc1, 0x58, 0x20, 0x36, 0xf6, 0xa0, 0xc0, 0xbf, 0x69, 0x19, 0x47, 0xd4, 0x40, 0x0d,
- 0xc3, 0x0b, 0x2e, 0x54, 0x09, 0xb3, 0xbf, 0x8d, 0xfb, 0x50, 0x4d, 0x7e, 0x01, 0x73, 0xa4, 0x13,
- 0xe2, 0x0c, 0x4f, 0x84, 0x83, 0x15, 0xb0, 0xa4, 0x98, 0x25, 0x5f, 0x3b, 0xb6, 0x4c, 0x59, 0x0b,
- 0x58, 0x10, 0xfa, 0xbf, 0xe5, 0xe0, 0xc6, 0x1c, 0xcd, 0x48, 0x67, 0x7d, 0x91, 0x72, 0xd6, 0x77,
- 0xa4, 0x85, 0xc8, 0xe3, 0x5f, 0xa4, 0x3c, 0xfe, 0x1d, 0x82, 0xb3, 0x65, 0x73, 0x1d, 0x8a, 0xe4,
- 0xdc, 0x09, 0x89, 0x2d, 0x55, 0x25, 0xa9, 0xc4, 0x72, 0xca, 0x5f, 0x75, 0x39, 0xed, 0xc3, 0x66,
- 0x3b, 0x20, 0x66, 0x48, 0x64, 0x28, 0x8f, 0xfc, 0xff, 0x06, 0x94, 0xcc, 0xd1, 0xc8, 0xb3, 0xa6,
- 0x66, 0x5d, 0xe5, 0x74, 0xd7, 0x46, 0x0d, 0x28, 0x9d, 0x78, 0x34, 0x74, 0xcd, 0x31, 0x91, 0xc1,
- 0x2b, 0xa6, 0xf5, 0x6f, 0x14, 0xd8, 0xba, 0x84, 0x27, 0xad, 0x70, 0x0c, 0x35, 0x87, 0x7a, 0x23,
- 0xfe, 0x81, 0x46, 0xe2, 0x84, 0xf7, 0xa3, 0xc5, 0xb6, 0x9a, 0x6e, 0x84, 0xc1, 0x0f, 0x7c, 0x6b,
- 0x4e, 0x92, 0xe4, 0x1e, 0xc7, 0x27, 0xb7, 0xe5, 0x4a, 0x8f, 0x48, 0xfd, 0x1f, 0x15, 0xd8, 0x92,
- 0x3b, 0x7c, 0xf6, 0x0f, 0x9d, 0x15, 0x39, 0xf7, 0xae, 0x45, 0xd6, 0xeb, 0x70, 0xfd, 0xb2, 0x5c,
- 0x32, 0xe6, 0xff, 0x5f, 0x01, 0xd0, 0xec, 0xe9, 0x12, 0x7d, 0x08, 0x55, 0x4a, 0x5c, 0xdb, 0x10,
- 0xfb, 0x85, 0xd8, 0xca, 0x4a, 0xb8, 0xc2, 0x78, 0x62, 0xe3, 0xa0, 0x2c, 0x04, 0x92, 0x73, 0x29,
- 0x6d, 0x09, 0xf3, 0xff, 0xe8, 0x04, 0xaa, 0x2f, 0xa9, 0x11, 0xcf, 0xcd, 0x1d, 0xaa, 0x96, 0x39,
- 0xac, 0xcd, 0xca, 0xd1, 0x7c, 0xd4, 0x8f, 0xbf, 0x0b, 0x57, 0x5e, 0xd2, 0x98, 0x40, 0x3f, 0x57,
- 0xe0, 0xbd, 0x28, 0xad, 0x98, 0xaa, 0x6f, 0xec, 0xd9, 0x84, 0xd6, 0xf3, 0x37, 0xd5, 0x5b, 0xb5,
- 0xed, 0xc3, 0x2b, 0xe8, 0x6f, 0x86, 0xb9, 0xef, 0xd9, 0x04, 0x6f, 0xb9, 0x73, 0xb8, 0x14, 0x35,
- 0xe1, 0xda, 0x78, 0x42, 0x43, 0x43, 0x78, 0x81, 0x21, 0x3b, 0xd5, 0x0b, 0x5c, 0x2f, 0x1b, 0xac,
- 0x29, 0xe5, 0xab, 0xe8, 0x14, 0xd6, 0xc6, 0xde, 0xc4, 0x0d, 0x0d, 0x8b, 0x9f, 0x7f, 0x68, 0xbd,
- 0xb8, 0xd0, 0xc1, 0x78, 0x8e, 0x96, 0xf6, 0x19, 0x9c, 0x38, 0x4d, 0x51, 0x5c, 0x1d, 0x27, 0x28,
- 0xf4, 0x11, 0x54, 0x03, 0x32, 0xf6, 0x42, 0x62, 0xb0, 0x78, 0x49, 0xeb, 0xab, 0x4c, 0xaa, 0x87,
- 0xb9, 0xba, 0x82, 0x2b, 0x82, 0xcf, 0xc2, 0x03, 0x45, 0xbf, 0x07, 0xd7, 0x6d, 0x87, 0x9a, 0xc7,
- 0x23, 0x62, 0x8c, 0xbc, 0xa1, 0x31, 0x4d, 0x75, 0xea, 0x25, 0xfe, 0x19, 0x9b, 0xb2, 0x75, 0xcf,
- 0x1b, 0xb6, 0xe3, 0x36, 0x3e, 0xea, 0xc2, 0x35, 0xc7, 0x8e, 0x65, 0xb0, 0x2f, 0x1b, 0x79, 0xa6,
- 0x6d, 0x4c, 0x28, 0x09, 0x68, 0xbd, 0x2c, 0x47, 0x89, 0xd6, 0x67, 0xb2, 0xf1, 0x88, 0xb5, 0xe9,
- 0xf7, 0xa0, 0x92, 0x30, 0x2b, 0x2a, 0x41, 0xbe, 0x77, 0xd0, 0xeb, 0x68, 0x2b, 0x08, 0xa0, 0xd8,
- 0xde, 0xc5, 0x07, 0x07, 0x03, 0x71, 0x4a, 0xe9, 0xee, 0xb7, 0x1e, 0x77, 0xb4, 0x1c, 0x63, 0x1f,
- 0xf5, 0xfe, 0xb0, 0xd3, 0xdd, 0xd3, 0x54, 0xbd, 0x03, 0xd5, 0xe4, 0xc7, 0x22, 0x04, 0xb5, 0xa3,
- 0xde, 0xd3, 0xde, 0xc1, 0xb3, 0x9e, 0xb1, 0x7f, 0x70, 0xd4, 0x1b, 0xb0, 0xb3, 0x4e, 0x0d, 0xa0,
- 0xd5, 0x7b, 0x3e, 0xa5, 0xd7, 0xa0, 0xdc, 0x3b, 0x88, 0x48, 0xa5, 0x91, 0xd3, 0x14, 0xfd, 0x3f,
- 0x54, 0xd8, 0x9c, 0x67, 0x77, 0x64, 0x43, 0x9e, 0xf9, 0x90, 0x3c, 0x6d, 0xbe, 0x7b, 0x17, 0xe2,
- 0xe8, 0x6c, 0xe9, 0xf8, 0xa6, 0xdc, 0x5e, 0xca, 0x98, 0xff, 0x47, 0x06, 0x14, 0x47, 0xe6, 0x31,
- 0x19, 0xd1, 0xba, 0xca, 0xef, 0x63, 0x1e, 0x5f, 0x65, 0xee, 0x3d, 0x8e, 0x24, 0x2e, 0x63, 0x24,
- 0x2c, 0x1a, 0x40, 0x85, 0x05, 0x50, 0x2a, 0x54, 0x27, 0x63, 0xfa, 0x76, 0xc6, 0x59, 0x76, 0xa7,
- 0x23, 0x71, 0x12, 0xa6, 0x71, 0x17, 0x2a, 0x89, 0xc9, 0xe6, 0xdc, 0xa5, 0x6c, 0x26, 0xef, 0x52,
- 0xca, 0xc9, 0x8b, 0x91, 0x07, 0xb3, 0x36, 0x60, 0x3a, 0x62, 0x0e, 0xb1, 0x7b, 0xd0, 0x1f, 0x88,
- 0x53, 0xeb, 0x63, 0x7c, 0x70, 0x74, 0xa8, 0x29, 0x8c, 0x39, 0x68, 0xf5, 0x9f, 0x6a, 0xb9, 0xd8,
- 0x5f, 0x54, 0xbd, 0x0d, 0x95, 0x84, 0x5c, 0xa9, 0x1d, 0x43, 0x49, 0xef, 0x18, 0x2c, 0x66, 0x9b,
- 0xb6, 0x1d, 0x10, 0x4a, 0xa5, 0x1c, 0x11, 0xa9, 0xbf, 0x80, 0xf2, 0x4e, 0xaf, 0x2f, 0x21, 0xea,
- 0xb0, 0x4a, 0x49, 0xc0, 0xbe, 0x9b, 0xdf, 0x8a, 0x95, 0x71, 0x44, 0x32, 0x70, 0x4a, 0xcc, 0xc0,
- 0x3a, 0x21, 0x54, 0xe6, 0x19, 0x31, 0xcd, 0x46, 0x79, 0xfc, 0x76, 0x49, 0xd8, 0xae, 0x8c, 0x23,
- 0x52, 0xff, 0xff, 0x12, 0xc0, 0xf4, 0xa6, 0x03, 0xd5, 0x20, 0x17, 0xc7, 0xff, 0x9c, 0x63, 0x33,
- 0x3f, 0x48, 0xec, 0x6f, 0xfc, 0x3f, 0xda, 0x86, 0xad, 0x31, 0x1d, 0xfa, 0xa6, 0x75, 0x6a, 0xc8,
- 0x0b, 0x0a, 0x11, 0x26, 0x78, 0x2c, 0xad, 0xe2, 0x6b, 0xb2, 0x51, 0x46, 0x01, 0x81, 0xbb, 0x07,
- 0x2a, 0x71, 0xcf, 0x78, 0xdc, 0xab, 0x6c, 0xdf, 0x5b, 0xf8, 0x06, 0xa6, 0xd9, 0x71, 0xcf, 0x84,
- 0xaf, 0x30, 0x18, 0x64, 0x00, 0xd8, 0xe4, 0xcc, 0xb1, 0x88, 0xc1, 0x40, 0x0b, 0x1c, 0xf4, 0xcb,
- 0xc5, 0x41, 0x77, 0x38, 0x46, 0x0c, 0x5d, 0xb6, 0x23, 0x1a, 0xf5, 0xa0, 0x1c, 0x10, 0xea, 0x4d,
- 0x02, 0x8b, 0x88, 0xe0, 0x97, 0xfd, 0x90, 0x84, 0xa3, 0x71, 0x78, 0x0a, 0x81, 0x76, 0xa0, 0xc8,
- 0x63, 0x1e, 0x8b, 0x6e, 0xea, 0xb7, 0x5e, 0xe7, 0xa6, 0xc1, 0x78, 0x24, 0xc1, 0x72, 0x2c, 0x7a,
- 0x0c, 0xab, 0x42, 0x44, 0x5a, 0x2f, 0x71, 0x98, 0x4f, 0xb2, 0x06, 0x64, 0x3e, 0x0a, 0x47, 0xa3,
- 0x99, 0x55, 0x59, 0x10, 0xe4, 0x31, 0xb0, 0x8c, 0xf9, 0x7f, 0xf4, 0x3e, 0x94, 0xc5, 0xfe, 0x6f,
- 0x3b, 0x41, 0x1d, 0x84, 0x73, 0x72, 0xc6, 0x8e, 0x13, 0xa0, 0x0f, 0xa0, 0x22, 0xf2, 0x3c, 0x83,
- 0x47, 0x85, 0x0a, 0x6f, 0x06, 0xc1, 0x3a, 0x64, 0xb1, 0x41, 0x74, 0x20, 0x41, 0x20, 0x3a, 0x54,
- 0xe3, 0x0e, 0x24, 0x08, 0x78, 0x87, 0xdf, 0x81, 0x75, 0x9e, 0x1d, 0x0f, 0x03, 0x6f, 0xe2, 0x1b,
- 0xdc, 0xa7, 0xd6, 0x78, 0xa7, 0x35, 0xc6, 0x7e, 0xcc, 0xb8, 0x3d, 0xe6, 0x5c, 0x37, 0xa0, 0xf4,
- 0xca, 0x3b, 0x16, 0x1d, 0x6a, 0x62, 0x1d, 0xbc, 0xf2, 0x8e, 0xa3, 0xa6, 0x38, 0x43, 0x59, 0x4f,
- 0x67, 0x28, 0x5f, 0xc3, 0xf5, 0xd9, 0xad, 0x96, 0x67, 0x2a, 0xda, 0xd5, 0x33, 0x95, 0x4d, 0x77,
- 0x5e, 0x1c, 0x7e, 0x08, 0xaa, 0xed, 0xd2, 0xfa, 0xc6, 0x42, 0xce, 0x11, 0xaf, 0x63, 0xcc, 0x06,
- 0xa3, 0x2d, 0x28, 0xb2, 0x8f, 0x75, 0xec, 0x3a, 0x12, 0xa1, 0xe7, 0x95, 0x77, 0xdc, 0xb5, 0xd1,
- 0x77, 0xa0, 0xcc, 0xbe, 0x9f, 0xfa, 0xa6, 0x45, 0xea, 0xd7, 0x78, 0xcb, 0x94, 0xc1, 0x0c, 0xe5,
- 0x7a, 0x36, 0x11, 0x2a, 0xda, 0x14, 0x86, 0x62, 0x0c, 0xae, 0xa3, 0xf7, 0x60, 0x95, 0x37, 0x3a,
- 0x76, 0x7d, 0x4b, 0x1c, 0x42, 0x18, 0xd9, 0xb5, 0x91, 0x0e, 0x6b, 0xbe, 0x19, 0x10, 0x37, 0x34,
- 0xe4, 0x8c, 0xd7, 0x79, 0x73, 0x45, 0x30, 0x9f, 0xb0, 0x79, 0x1b, 0x9f, 0x41, 0x29, 0x5a, 0x0c,
- 0x8b, 0x84, 0xc9, 0xc6, 0x7d, 0xa8, 0xa5, 0x97, 0xd2, 0x42, 0x41, 0xf6, 0x9f, 0x73, 0x50, 0x8e,
- 0x17, 0x0d, 0x72, 0xe1, 0x1a, 0x37, 0x2a, 0xcb, 0x56, 0x8d, 0xe9, 0x1a, 0x14, 0x39, 0xf2, 0x17,
- 0x19, 0xd5, 0xdc, 0x8a, 0x10, 0xe4, 0x61, 0x5d, 0x2e, 0x48, 0x14, 0x23, 0x4f, 0xe7, 0xfb, 0x0a,
- 0xd6, 0x47, 0x8e, 0x3b, 0x39, 0x4f, 0xcc, 0x25, 0x92, 0xdb, 0xdf, 0xcf, 0x38, 0xd7, 0x1e, 0x1b,
- 0x3d, 0x9d, 0xa3, 0x36, 0x4a, 0xd1, 0x68, 0x17, 0x0a, 0xbe, 0x17, 0x84, 0xd1, 0x9e, 0x99, 0x75,
- 0x37, 0x3b, 0xf4, 0x82, 0x70, 0xdf, 0xf4, 0x7d, 0x76, 0x7e, 0x13, 0x00, 0xfa, 0x37, 0x39, 0xb8,
- 0x3e, 0xff, 0xc3, 0x50, 0x0f, 0x54, 0xcb, 0x9f, 0x48, 0x25, 0xdd, 0x5f, 0x54, 0x49, 0x6d, 0x7f,
- 0x32, 0x95, 0x9f, 0x01, 0xa1, 0x67, 0x50, 0x1c, 0x93, 0xb1, 0x17, 0x5c, 0x48, 0x5d, 0x3c, 0x58,
- 0x14, 0x72, 0x9f, 0x8f, 0x9e, 0xa2, 0x4a, 0x38, 0x84, 0xa1, 0x24, 0x17, 0x13, 0x95, 0x61, 0x7b,
- 0xc1, 0x1b, 0xb6, 0x08, 0x12, 0xc7, 0x38, 0xfa, 0x67, 0xb0, 0x35, 0xf7, 0x53, 0xd0, 0x6f, 0x01,
- 0x58, 0xfe, 0xc4, 0xe0, 0x2f, 0x20, 0xc2, 0x83, 0x54, 0x5c, 0xb6, 0xfc, 0x49, 0x9f, 0x33, 0xf4,
- 0x17, 0x50, 0x7f, 0x93, 0xbc, 0x6c, 0x8d, 0x09, 0x89, 0x8d, 0xf1, 0x31, 0xd7, 0x81, 0x8a, 0x4b,
- 0x82, 0xb1, 0x7f, 0xcc, 0x96, 0x52, 0xd4, 0x68, 0x9e, 0xb3, 0x0e, 0x2a, 0xef, 0x50, 0x91, 0x1d,
- 0xcc, 0xf3, 0xfd, 0x63, 0xfd, 0x17, 0x39, 0x58, 0xbf, 0x24, 0x32, 0x3b, 0xc5, 0x8a, 0x00, 0x1c,
- 0xdd, 0x0f, 0x08, 0x8a, 0x45, 0x63, 0xcb, 0xb1, 0xa3, 0x9b, 0x65, 0xfe, 0x9f, 0xef, 0xc3, 0xbe,
- 0xbc, 0xf5, 0xcd, 0x39, 0x3e, 0x5b, 0x3e, 0xe3, 0x63, 0x27, 0xa4, 0x3c, 0x29, 0x2a, 0x60, 0x41,
- 0xa0, 0xe7, 0x50, 0x0b, 0x08, 0xdf, 0xff, 0x6d, 0x43, 0x78, 0x59, 0x61, 0x21, 0x2f, 0x93, 0x12,
- 0x32, 0x67, 0xc3, 0x6b, 0x11, 0x12, 0xa3, 0x28, 0x7a, 0x06, 0x6b, 0x51, 0xe2, 0x2c, 0x90, 0x8b,
- 0x4b, 0x23, 0x57, 0x25, 0x10, 0x07, 0xd6, 0xef, 0x42, 0x25, 0xd1, 0xc8, 0x3e, 0x8c, 0x67, 0x7f,
- 0x52, 0x27, 0x82, 0x48, 0x47, 0x8b, 0x82, 0x8c, 0x16, 0xfa, 0x31, 0x54, 0x12, 0xeb, 0x62, 0x91,
- 0xa1, 0x4c, 0x9f, 0xa1, 0xc7, 0xf5, 0x59, 0xc0, 0xb9, 0xd0, 0x63, 0x71, 0x92, 0x65, 0x5e, 0x86,
- 0xe3, 0x73, 0x8d, 0x96, 0x71, 0x91, 0x91, 0x5d, 0x5f, 0xff, 0x55, 0x0e, 0x6a, 0xe9, 0x25, 0x1d,
- 0xf9, 0x91, 0x4f, 0x02, 0xc7, 0xb3, 0x13, 0x7e, 0x74, 0xc8, 0x19, 0xcc, 0x57, 0x58, 0xf3, 0xd7,
- 0x13, 0x2f, 0x34, 0x23, 0x5f, 0xb1, 0xfc, 0xc9, 0x1f, 0x30, 0xfa, 0x92, 0x0f, 0xaa, 0x97, 0x7c,
- 0x10, 0x7d, 0x0c, 0x48, 0xba, 0xd2, 0xc8, 0x19, 0x3b, 0xa1, 0x71, 0x7c, 0x11, 0x12, 0x61, 0x63,
- 0x15, 0x6b, 0xa2, 0x65, 0x8f, 0x35, 0x3c, 0x64, 0x7c, 0xe6, 0x78, 0x9e, 0x37, 0x36, 0xa8, 0xe5,
- 0x05, 0xc4, 0x30, 0xed, 0x57, 0xfc, 0x00, 0xa7, 0xe2, 0x8a, 0xe7, 0x8d, 0xfb, 0x8c, 0xd7, 0xb2,
- 0x5f, 0xb1, 0x8d, 0xd8, 0xf2, 0x27, 0x94, 0x84, 0x06, 0xfb, 0xe1, 0xb9, 0x4b, 0x19, 0x83, 0x60,
- 0xb5, 0xfd, 0x09, 0x45, 0xdf, 0x85, 0xb5, 0xa8, 0x03, 0xdf, 0x8b, 0x65, 0x12, 0x50, 0x95, 0x5d,
- 0x38, 0x0f, 0xe9, 0x50, 0x3d, 0x24, 0x81, 0x45, 0xdc, 0x70, 0xe0, 0x58, 0xa7, 0x94, 0x1f, 0xb1,
- 0x14, 0x9c, 0xe2, 0x3d, 0xc9, 0x97, 0x56, 0xb5, 0x12, 0x8e, 0x66, 0x1b, 0x93, 0x31, 0xd5, 0xff,
- 0x55, 0x81, 0x02, 0x4f, 0x59, 0x98, 0x52, 0xf8, 0x76, 0xcf, 0xb3, 0x01, 0x99, 0xea, 0x32, 0x06,
- 0xcf, 0x05, 0xde, 0x87, 0x32, 0x57, 0x7e, 0xe2, 0x84, 0xc1, 0xf3, 0x60, 0xde, 0xd8, 0x80, 0x52,
- 0x40, 0x4c, 0xdb, 0x73, 0x47, 0xd1, 0xc5, 0x58, 0x4c, 0xa3, 0xdf, 0x05, 0xcd, 0x0f, 0x3c, 0xdf,
- 0x1c, 0x4e, 0xcf, 0xd2, 0xd2, 0x7c, 0xeb, 0x09, 0x3e, 0x4f, 0xd1, 0xbf, 0x0b, 0x6b, 0x94, 0x88,
- 0xc8, 0x2e, 0x9c, 0xa4, 0x20, 0x3e, 0x53, 0x32, 0xf9, 0x89, 0x40, 0xff, 0x1a, 0x8a, 0x62, 0xe3,
- 0xba, 0x82, 0xbc, 0x9f, 0x00, 0x12, 0x8a, 0x64, 0x0e, 0x32, 0x76, 0x28, 0x95, 0x59, 0x36, 0x7f,
- 0xdd, 0x15, 0x2d, 0x87, 0xd3, 0x06, 0xfd, 0xbf, 0x14, 0x91, 0x6f, 0x8b, 0x77, 0x37, 0x96, 0x98,
- 0xb3, 0x55, 0xc3, 0x8e, 0xb1, 0xe2, 0x82, 0x2f, 0x22, 0x51, 0x17, 0x8a, 0x32, 0xad, 0xce, 0x2d,
- 0xfb, 0x6c, 0x29, 0x01, 0xa2, 0xeb, 0x7e, 0x22, 0x2f, 0x3b, 0x16, 0xbd, 0xee, 0x27, 0xe2, 0xba,
- 0x9f, 0xa0, 0x0f, 0xa1, 0x2a, 0x13, 0x7e, 0x01, 0x97, 0xe7, 0xf9, 0x7e, 0xc5, 0x8e, 0xdf, 0x54,
- 0x88, 0xfe, 0x3f, 0x4a, 0x1c, 0xf7, 0xa2, 0xb7, 0x0f, 0xf4, 0x15, 0x94, 0x58, 0x08, 0x31, 0xc6,
- 0xa6, 0x2f, 0x5f, 0xf2, 0xdb, 0xcb, 0x3d, 0xab, 0x44, 0xbb, 0xa2, 0x48, 0xd7, 0x57, 0x7d, 0x41,
- 0xb1, 0xf8, 0xc9, 0x8e, 0x4a, 0x51, 0xfc, 0x64, 0xff, 0xd1, 0x47, 0x50, 0x33, 0x27, 0xa1, 0x67,
- 0x98, 0xf6, 0x19, 0x09, 0x42, 0x87, 0x12, 0xe9, 0x4b, 0x6b, 0x8c, 0xdb, 0x8a, 0x98, 0x8d, 0x7b,
- 0x50, 0x4d, 0x62, 0xbe, 0x2d, 0x6f, 0x29, 0x24, 0xf3, 0x96, 0x3f, 0x05, 0x98, 0xde, 0x23, 0x32,
- 0x1f, 0x21, 0xe7, 0x4e, 0x68, 0x58, 0xd1, 0xd9, 0xbc, 0x80, 0x4b, 0x8c, 0xd1, 0x66, 0xce, 0x98,
- 0x7e, 0xe4, 0x28, 0x44, 0x8f, 0x1c, 0x2c, 0x3a, 0xb0, 0x05, 0x7d, 0xea, 0x8c, 0x46, 0xf1, 0xdd,
- 0x66, 0xd9, 0xf3, 0xc6, 0x4f, 0x39, 0x43, 0xff, 0x75, 0x4e, 0xf8, 0x8a, 0x78, 0xae, 0xca, 0x74,
- 0x36, 0x7b, 0x57, 0xa6, 0xbe, 0x0b, 0x40, 0x43, 0x33, 0x60, 0x49, 0x98, 0x19, 0xdd, 0xae, 0x36,
- 0x66, 0x5e, 0x49, 0x06, 0x51, 0xfd, 0x0c, 0x2e, 0xcb, 0xde, 0xad, 0x10, 0x7d, 0x01, 0x55, 0xcb,
- 0x1b, 0xfb, 0x23, 0x22, 0x07, 0x17, 0xde, 0x3a, 0xb8, 0x12, 0xf7, 0x6f, 0x85, 0x89, 0x3b, 0xdd,
- 0xe2, 0x55, 0xef, 0x74, 0x7f, 0xa5, 0x88, 0x57, 0xb7, 0xe4, 0xa3, 0x1f, 0x1a, 0xce, 0xa9, 0x2c,
- 0x79, 0xbc, 0xe4, 0x0b, 0xe2, 0xb7, 0x95, 0x95, 0x34, 0xbe, 0xc8, 0x52, 0xc7, 0xf1, 0xe6, 0xb4,
- 0xf8, 0xdf, 0x55, 0x28, 0xc7, 0x0f, 0x6e, 0x33, 0xb6, 0xff, 0x1c, 0xca, 0x71, 0xf1, 0x92, 0x0c,
- 0x10, 0xdf, 0x6a, 0x9e, 0xb8, 0x33, 0x7a, 0x09, 0xc8, 0x1c, 0x0e, 0xe3, 0x74, 0xd7, 0x98, 0x50,
- 0x73, 0x18, 0x3d, 0x77, 0x7e, 0xbe, 0x80, 0x1e, 0xa2, 0xfd, 0xf1, 0x88, 0x8d, 0xc7, 0x9a, 0x39,
- 0x1c, 0xa6, 0x38, 0xe8, 0xcf, 0x60, 0x2b, 0x3d, 0x87, 0x71, 0x7c, 0x61, 0xf8, 0x8e, 0x2d, 0xef,
- 0x00, 0x76, 0x17, 0x7d, 0x73, 0x6c, 0xa6, 0xe0, 0x1f, 0x5e, 0x1c, 0x3a, 0xb6, 0xd0, 0x39, 0x0a,
- 0x66, 0x1a, 0x1a, 0x7f, 0x01, 0xef, 0xbd, 0xa1, 0xfb, 0x1c, 0x1b, 0xf4, 0xd2, 0xb5, 0x34, 0xcb,
- 0x2b, 0x21, 0x61, 0xbd, 0x5f, 0x2a, 0xe2, 0x69, 0x34, 0xad, 0x93, 0x56, 0x32, 0x4f, 0xbf, 0x9d,
- 0x71, 0x9e, 0xf6, 0xe1, 0x91, 0x80, 0xe7, 0xa9, 0xf9, 0x93, 0x4b, 0xa9, 0x79, 0xd6, 0x84, 0x4c,
- 0x64, 0xb8, 0x02, 0x48, 0x22, 0xe8, 0xff, 0xa2, 0x42, 0x29, 0x42, 0xe7, 0x27, 0xf8, 0x0b, 0x1a,
- 0x92, 0xb1, 0x11, 0x5f, 0x2f, 0x2a, 0x18, 0x04, 0x8b, 0xef, 0xa8, 0xef, 0x43, 0x79, 0x42, 0x49,
- 0x20, 0x9a, 0x73, 0xbc, 0xb9, 0xc4, 0x18, 0xbc, 0xf1, 0x03, 0xa8, 0x84, 0x5e, 0x68, 0x8e, 0x8c,
- 0x90, 0xe7, 0x0b, 0xaa, 0x18, 0xcd, 0x59, 0x3c, 0x5b, 0x40, 0xdf, 0x87, 0x8d, 0xf0, 0x24, 0xf0,
- 0xc2, 0x70, 0xc4, 0x72, 0x55, 0x9e, 0x39, 0x89, 0x44, 0x27, 0x8f, 0xb5, 0xb8, 0x41, 0x64, 0x54,
- 0x94, 0x45, 0xef, 0x69, 0x67, 0xe6, 0xba, 0x3c, 0x88, 0xe4, 0xf1, 0x5a, 0xcc, 0x65, 0xae, 0xcd,
- 0x36, 0x4f, 0x5f, 0x64, 0x24, 0x3c, 0x56, 0x28, 0x38, 0x22, 0x91, 0x01, 0xeb, 0x63, 0x62, 0xd2,
- 0x49, 0x40, 0x6c, 0xe3, 0xa5, 0x43, 0x46, 0xb6, 0xb8, 0x78, 0xa9, 0x65, 0x3e, 0x6e, 0x44, 0x6a,
- 0x69, 0x3e, 0xe2, 0xa3, 0x71, 0x2d, 0x82, 0x13, 0x34, 0xcb, 0x1c, 0xc4, 0x3f, 0xb4, 0x0e, 0x95,
- 0xfe, 0xf3, 0xfe, 0xa0, 0xb3, 0x6f, 0xec, 0x1f, 0xec, 0x74, 0x64, 0xb9, 0x54, 0xbf, 0x83, 0x05,
- 0xa9, 0xb0, 0xf6, 0xc1, 0xc1, 0xa0, 0xb5, 0x67, 0x0c, 0xba, 0xed, 0xa7, 0x7d, 0x2d, 0x87, 0xb6,
- 0x60, 0x63, 0xb0, 0x8b, 0x0f, 0x06, 0x83, 0xbd, 0xce, 0x8e, 0x71, 0xd8, 0xc1, 0xdd, 0x83, 0x9d,
- 0xbe, 0xa6, 0x22, 0x04, 0xb5, 0x29, 0x7b, 0xd0, 0xdd, 0xef, 0x68, 0x79, 0x54, 0x81, 0xd5, 0xc3,
- 0x0e, 0x6e, 0x77, 0x7a, 0x03, 0xad, 0xa0, 0xff, 0x42, 0x85, 0x4a, 0xc2, 0x8a, 0xcc, 0x91, 0x03,
- 0x2a, 0xce, 0x35, 0x79, 0xcc, 0xfe, 0xf2, 0xe7, 0x5d, 0xd3, 0x3a, 0x11, 0xd6, 0xc9, 0x63, 0x41,
- 0xf0, 0xb3, 0x8c, 0x79, 0x9e, 0x58, 0xe7, 0x79, 0x5c, 0x1a, 0x9b, 0xe7, 0x02, 0xe4, 0x43, 0xa8,
- 0x9e, 0x92, 0xc0, 0x25, 0x23, 0xd9, 0x2e, 0x2c, 0x52, 0x11, 0x3c, 0xd1, 0xe5, 0x16, 0x68, 0xb2,
- 0xcb, 0x14, 0x46, 0x98, 0xa3, 0x26, 0xf8, 0xfb, 0x11, 0xd8, 0x26, 0x14, 0x44, 0xf3, 0xaa, 0x98,
- 0x9f, 0x13, 0x6c, 0x9b, 0xa2, 0xaf, 0x4d, 0x9f, 0xe7, 0x90, 0x79, 0xcc, 0xff, 0xa3, 0xe3, 0x59,
- 0xfb, 0x14, 0xb9, 0x7d, 0xee, 0x2e, 0xee, 0xce, 0x6f, 0x32, 0xd1, 0x49, 0x6c, 0xa2, 0x55, 0x50,
- 0x71, 0x54, 0x63, 0xd4, 0x6e, 0xb5, 0x77, 0x99, 0x59, 0xd6, 0xa0, 0xbc, 0xdf, 0xfa, 0x89, 0x71,
- 0xd4, 0x17, 0x37, 0xf8, 0x1a, 0x54, 0x9f, 0x76, 0x70, 0xaf, 0xb3, 0x27, 0x39, 0x2a, 0xda, 0x04,
- 0x4d, 0x72, 0xa6, 0xfd, 0xf2, 0x0c, 0x41, 0xfc, 0x2d, 0xa0, 0x12, 0xe4, 0xfb, 0xcf, 0x5a, 0x87,
- 0x5a, 0x51, 0xff, 0xef, 0x1c, 0xac, 0x8b, 0x6d, 0x21, 0xae, 0x86, 0x78, 0xf3, 0x6b, 0x70, 0xf2,
- 0x16, 0x2b, 0x97, 0xbe, 0xc5, 0x8a, 0x92, 0x50, 0xbe, 0xab, 0xab, 0xd3, 0x24, 0x94, 0xdf, 0xec,
- 0xa4, 0x22, 0x7e, 0x7e, 0x91, 0x88, 0x5f, 0x87, 0xd5, 0x31, 0xa1, 0xb1, 0xdd, 0xca, 0x38, 0x22,
- 0x91, 0x03, 0x15, 0xd3, 0x75, 0xbd, 0xd0, 0x14, 0x57, 0xc3, 0xc5, 0x85, 0x36, 0xc3, 0x4b, 0x5f,
- 0xdc, 0x6c, 0x4d, 0x91, 0x44, 0x60, 0x4e, 0x62, 0x37, 0x7e, 0x0c, 0xda, 0xe5, 0x0e, 0x8b, 0x6c,
- 0x87, 0xdf, 0xfb, 0xc1, 0x74, 0x37, 0x24, 0x6c, 0x5d, 0xc8, 0x37, 0x15, 0x6d, 0x85, 0x11, 0xf8,
- 0xa8, 0xd7, 0xeb, 0xf6, 0x1e, 0x6b, 0x0a, 0x02, 0x28, 0x76, 0x7e, 0xd2, 0x1d, 0x74, 0x76, 0xb4,
- 0xdc, 0xf6, 0x2f, 0x37, 0xa0, 0x28, 0x84, 0x44, 0xdf, 0xc8, 0x4c, 0x20, 0x59, 0x69, 0x8b, 0x7e,
- 0xbc, 0x70, 0x46, 0x9d, 0xaa, 0xde, 0x6d, 0x3c, 0x58, 0x7a, 0xbc, 0x7c, 0xd9, 0x5c, 0x41, 0x7f,
- 0xa3, 0x40, 0x35, 0xf5, 0xaa, 0x99, 0xf5, 0x6a, 0x7c, 0x4e, 0x61, 0x6f, 0xe3, 0x47, 0x4b, 0x8d,
- 0x8d, 0x65, 0xf9, 0xb9, 0x02, 0x95, 0x44, 0x49, 0x2b, 0xba, 0xbb, 0x4c, 0x19, 0xac, 0x90, 0xe4,
- 0xde, 0xf2, 0x15, 0xb4, 0xfa, 0xca, 0xa7, 0x0a, 0xfa, 0x6b, 0x05, 0x2a, 0x89, 0xe2, 0xce, 0xcc,
- 0xa2, 0xcc, 0x96, 0xa2, 0x66, 0x16, 0x65, 0x5e, 0x2d, 0xe9, 0x0a, 0xfa, 0x4b, 0x05, 0xca, 0x71,
- 0xa1, 0x26, 0xba, 0xb3, 0x78, 0x69, 0xa7, 0x10, 0xe2, 0xf3, 0x65, 0x6b, 0x42, 0xf5, 0x15, 0xf4,
- 0xe7, 0x50, 0x8a, 0xaa, 0x1a, 0x51, 0xd6, 0xdd, 0xeb, 0x52, 0xc9, 0x64, 0xe3, 0xce, 0xc2, 0xe3,
- 0x92, 0xd3, 0x47, 0xa5, 0x86, 0x99, 0xa7, 0xbf, 0x54, 0x14, 0xd9, 0xb8, 0xb3, 0xf0, 0xb8, 0x78,
- 0x7a, 0xe6, 0x09, 0x89, 0x8a, 0xc4, 0xcc, 0x9e, 0x30, 0x5b, 0x0a, 0x99, 0xd9, 0x13, 0xe6, 0x15,
- 0x40, 0x0a, 0x41, 0x12, 0x35, 0x8d, 0x99, 0x05, 0x99, 0xad, 0x9b, 0xcc, 0x2c, 0xc8, 0x9c, 0x12,
- 0x4a, 0x7d, 0x05, 0xfd, 0x4c, 0x49, 0x9e, 0x0b, 0xee, 0x2c, 0x5c, 0xba, 0xb7, 0xa0, 0x4b, 0xce,
- 0x14, 0x0f, 0xf2, 0x05, 0xfa, 0x33, 0x79, 0x8b, 0x21, 0x2a, 0xff, 0xd0, 0x22, 0x60, 0xa9, 0x62,
- 0xc1, 0xc6, 0x67, 0xcb, 0x6d, 0x36, 0x5c, 0x88, 0xbf, 0x52, 0x00, 0xa6, 0x35, 0x82, 0x99, 0x85,
- 0x98, 0x29, 0x4e, 0x6c, 0xdc, 0x5d, 0x62, 0x64, 0x72, 0x81, 0x44, 0x35, 0x4c, 0x99, 0x17, 0xc8,
- 0xa5, 0x1a, 0xc6, 0xcc, 0x0b, 0xe4, 0x72, 0xfd, 0xa1, 0xbe, 0x82, 0xfe, 0x49, 0x81, 0x8d, 0x99,
- 0x1a, 0x2a, 0xf4, 0xe0, 0x8a, 0x65, 0x74, 0x8d, 0x2f, 0x97, 0x07, 0x88, 0x44, 0xbb, 0xa5, 0x7c,
- 0xaa, 0xa0, 0xbf, 0x55, 0x60, 0x2d, 0x5d, 0x5b, 0x92, 0x79, 0x97, 0x9a, 0x53, 0x8d, 0xd5, 0xb8,
- 0xbf, 0xdc, 0xe0, 0x58, 0x5b, 0x7f, 0xaf, 0x40, 0x2d, 0x5d, 0x66, 0x84, 0xee, 0x2f, 0x16, 0x16,
- 0x2e, 0x09, 0xf4, 0xc5, 0x92, 0xa3, 0x23, 0x89, 0x1e, 0xae, 0xfe, 0x51, 0x41, 0x64, 0x6f, 0x45,
- 0xfe, 0xf3, 0xc3, 0xdf, 0x04, 0x00, 0x00, 0xff, 0xff, 0x51, 0x2f, 0x79, 0xbc, 0x10, 0x35, 0x00,
- 0x00,
+ 0x76, 0x17, 0xf8, 0x4f, 0xe4, 0x23, 0x45, 0x41, 0x2d, 0xc9, 0xa6, 0x39, 0x9b, 0x8c, 0x07, 0x5b,
+ 0x93, 0x52, 0x76, 0x67, 0xe8, 0x59, 0x6d, 0x32, 0x1e, 0x7b, 0x3d, 0xeb, 0xe1, 0x50, 0xb4, 0x45,
+ 0x5b, 0xa2, 0x94, 0x26, 0x15, 0xaf, 0xe3, 0x64, 0x10, 0x08, 0x68, 0x53, 0xb0, 0x48, 0x00, 0x83,
+ 0x06, 0x65, 0x69, 0x53, 0xa9, 0xa4, 0x36, 0x55, 0xa9, 0x4d, 0x55, 0x52, 0xc9, 0x65, 0xb2, 0x97,
+ 0x9c, 0xb6, 0x2a, 0xa7, 0x54, 0xee, 0xa9, 0x4d, 0xed, 0x29, 0x87, 0x7c, 0x89, 0x1c, 0x92, 0x5b,
+ 0xae, 0xf9, 0x04, 0xd9, 0xea, 0x3f, 0x00, 0x01, 0x92, 0x1e, 0x83, 0x94, 0x4f, 0xe4, 0x7b, 0xdd,
+ 0xfd, 0xeb, 0x87, 0xf7, 0x5e, 0xbf, 0x7e, 0xdd, 0xfd, 0x40, 0xf3, 0x86, 0xe3, 0x81, 0xed, 0xd0,
+ 0x3b, 0x96, 0x6f, 0x5f, 0x10, 0x9f, 0xde, 0xf1, 0x7c, 0x37, 0x70, 0x25, 0xd5, 0xe0, 0x04, 0xfa,
+ 0xf0, 0xcc, 0xa0, 0x67, 0xb6, 0xe9, 0xfa, 0x5e, 0xc3, 0x71, 0x47, 0x86, 0xd5, 0x90, 0x63, 0x1a,
+ 0x72, 0x8c, 0xe8, 0x56, 0xff, 0xed, 0x81, 0xeb, 0x0e, 0x86, 0x44, 0x20, 0x9c, 0x8e, 0x5f, 0xde,
+ 0xb1, 0xc6, 0xbe, 0x11, 0xd8, 0xae, 0x23, 0xdb, 0xdf, 0x9f, 0x6e, 0x0f, 0xec, 0x11, 0xa1, 0x81,
+ 0x31, 0xf2, 0x64, 0x87, 0x0f, 0x43, 0x59, 0xe8, 0x99, 0xe1, 0x13, 0xeb, 0xce, 0x99, 0x39, 0xa4,
+ 0x1e, 0x31, 0xd9, 0xaf, 0xce, 0xfe, 0xc8, 0x6e, 0x1f, 0x4d, 0x75, 0xa3, 0x81, 0x3f, 0x36, 0x83,
+ 0x50, 0x72, 0x23, 0x08, 0x7c, 0xfb, 0x74, 0x1c, 0x10, 0xd1, 0x5b, 0xbb, 0x05, 0x37, 0xfb, 0x06,
+ 0x3d, 0x6f, 0xb9, 0xce, 0x4b, 0x7b, 0xd0, 0x33, 0xcf, 0xc8, 0xc8, 0xc0, 0xe4, 0xeb, 0x31, 0xa1,
+ 0x81, 0xf6, 0xc7, 0x50, 0x9b, 0x6d, 0xa2, 0x9e, 0xeb, 0x50, 0x82, 0xbe, 0x80, 0x1c, 0x9b, 0xb2,
+ 0xa6, 0xdc, 0x56, 0x76, 0xca, 0xbb, 0x1f, 0x35, 0xde, 0xa4, 0x02, 0x21, 0x43, 0x43, 0x8a, 0xda,
+ 0xe8, 0x79, 0xc4, 0xc4, 0x7c, 0xa4, 0xb6, 0x0d, 0x9b, 0x2d, 0xc3, 0x33, 0x4e, 0xed, 0xa1, 0x1d,
+ 0xd8, 0x84, 0x86, 0x93, 0x8e, 0x61, 0x2b, 0xc9, 0x96, 0x13, 0xfe, 0x09, 0x54, 0xcc, 0x18, 0x5f,
+ 0x4e, 0x7c, 0xaf, 0x91, 0x4a, 0xf7, 0x8d, 0x3d, 0x4e, 0x25, 0x80, 0x13, 0x70, 0xda, 0x16, 0xa0,
+ 0x47, 0xb6, 0x33, 0x20, 0xbe, 0xe7, 0xdb, 0x4e, 0x10, 0x0a, 0xf3, 0xeb, 0x2c, 0x6c, 0x26, 0xd8,
+ 0x52, 0x98, 0x57, 0x00, 0x91, 0x1e, 0x99, 0x28, 0xd9, 0x9d, 0xf2, 0xee, 0x93, 0x94, 0xa2, 0xcc,
+ 0xc1, 0x6b, 0x34, 0x23, 0xb0, 0xb6, 0x13, 0xf8, 0x57, 0x38, 0x86, 0x8e, 0xbe, 0x82, 0xc2, 0x19,
+ 0x31, 0x86, 0xc1, 0x59, 0x2d, 0x73, 0x5b, 0xd9, 0xa9, 0xee, 0x3e, 0xba, 0xc6, 0x3c, 0xfb, 0x1c,
+ 0xa8, 0x17, 0x18, 0x01, 0xc1, 0x12, 0x15, 0x7d, 0x0c, 0x48, 0xfc, 0xd3, 0x2d, 0x42, 0x4d, 0xdf,
+ 0xf6, 0x98, 0x4b, 0xd6, 0xb2, 0xb7, 0x95, 0x9d, 0x12, 0xde, 0x10, 0x2d, 0x7b, 0x93, 0x86, 0xba,
+ 0x07, 0xeb, 0x53, 0xd2, 0x22, 0x15, 0xb2, 0xe7, 0xe4, 0x8a, 0x5b, 0xa4, 0x84, 0xd9, 0x5f, 0xf4,
+ 0x18, 0xf2, 0x17, 0xc6, 0x70, 0x4c, 0xb8, 0xc8, 0xe5, 0xdd, 0x1f, 0xbc, 0xcd, 0x3d, 0xa4, 0x8b,
+ 0x4e, 0xf4, 0x80, 0xc5, 0xf8, 0xfb, 0x99, 0xcf, 0x14, 0xed, 0x1e, 0x94, 0x63, 0x72, 0xa3, 0x2a,
+ 0xc0, 0x49, 0x77, 0xaf, 0xdd, 0x6f, 0xb7, 0xfa, 0xed, 0x3d, 0x75, 0x05, 0xad, 0x41, 0xe9, 0xa4,
+ 0xbb, 0xdf, 0x6e, 0x1e, 0xf4, 0xf7, 0x9f, 0xab, 0x0a, 0x2a, 0xc3, 0x6a, 0x48, 0x64, 0xb4, 0x4b,
+ 0x40, 0x98, 0x98, 0xee, 0x05, 0xf1, 0x99, 0x23, 0x4b, 0xab, 0xa2, 0x9b, 0xb0, 0x1a, 0x18, 0xf4,
+ 0x5c, 0xb7, 0x2d, 0x29, 0x73, 0x81, 0x91, 0x1d, 0x0b, 0x75, 0xa0, 0x70, 0x66, 0x38, 0xd6, 0xf0,
+ 0xed, 0x72, 0x27, 0x55, 0xcd, 0xc0, 0xf7, 0xf9, 0x40, 0x2c, 0x01, 0x98, 0x77, 0x27, 0x66, 0x16,
+ 0x06, 0xd0, 0x9e, 0x83, 0xda, 0x0b, 0x0c, 0x3f, 0x88, 0x8b, 0xd3, 0x86, 0x1c, 0x9b, 0x5f, 0x7a,
+ 0xf4, 0x22, 0x73, 0x8a, 0x95, 0x89, 0xf9, 0x70, 0xed, 0xff, 0x32, 0xb0, 0x11, 0xc3, 0x96, 0x9e,
+ 0xfa, 0x0c, 0x0a, 0x3e, 0xa1, 0xe3, 0x61, 0xc0, 0xe1, 0xab, 0xbb, 0x0f, 0x53, 0xc2, 0xcf, 0x20,
+ 0x35, 0x30, 0x87, 0xc1, 0x12, 0x0e, 0xed, 0x80, 0x2a, 0x46, 0xe8, 0xc4, 0xf7, 0x5d, 0x5f, 0x1f,
+ 0xd1, 0x01, 0xd7, 0x5a, 0x09, 0x57, 0x05, 0xbf, 0xcd, 0xd8, 0x87, 0x74, 0x10, 0xd3, 0x6a, 0xf6,
+ 0x9a, 0x5a, 0x45, 0x06, 0xa8, 0x0e, 0x09, 0x5e, 0xbb, 0xfe, 0xb9, 0xce, 0x54, 0xeb, 0xdb, 0x16,
+ 0xa9, 0xe5, 0x38, 0xe8, 0xa7, 0x29, 0x41, 0xbb, 0x62, 0xf8, 0x91, 0x1c, 0x8d, 0xd7, 0x9d, 0x24,
+ 0x43, 0xfb, 0x3e, 0x14, 0xc4, 0x97, 0x32, 0x4f, 0xea, 0x9d, 0xb4, 0x5a, 0xed, 0x5e, 0x4f, 0x5d,
+ 0x41, 0x25, 0xc8, 0xe3, 0x76, 0x1f, 0x33, 0x0f, 0x2b, 0x41, 0xfe, 0x51, 0xb3, 0xdf, 0x3c, 0x50,
+ 0x33, 0xda, 0xf7, 0x60, 0xfd, 0x99, 0x61, 0x07, 0x69, 0x9c, 0x4b, 0x73, 0x41, 0x9d, 0xf4, 0x95,
+ 0xd6, 0xe9, 0x24, 0xac, 0x93, 0x5e, 0x35, 0xed, 0x4b, 0x3b, 0x98, 0xb2, 0x87, 0x0a, 0x59, 0xe2,
+ 0xfb, 0xd2, 0x04, 0xec, 0xaf, 0xf6, 0x1a, 0xd6, 0x7b, 0x81, 0xeb, 0xa5, 0xf2, 0xfc, 0x1f, 0xc2,
+ 0x2a, 0xdb, 0x6d, 0xdc, 0x71, 0x20, 0x5d, 0xff, 0x56, 0x43, 0xec, 0x46, 0x8d, 0x70, 0x37, 0x6a,
+ 0xec, 0xc9, 0xdd, 0x0a, 0x87, 0x3d, 0xd1, 0x0d, 0x28, 0x50, 0x7b, 0xe0, 0x18, 0x43, 0x19, 0x2d,
+ 0x24, 0xa5, 0x21, 0xe6, 0xe4, 0xe1, 0xc4, 0xd2, 0xf1, 0x5b, 0x80, 0xf6, 0x08, 0x0d, 0x7c, 0xf7,
+ 0x2a, 0x95, 0x3c, 0x5b, 0x90, 0x7f, 0xe9, 0xfa, 0xa6, 0x58, 0x88, 0x45, 0x2c, 0x08, 0xb6, 0xa8,
+ 0x12, 0x20, 0x12, 0xfb, 0x63, 0x40, 0x1d, 0x87, 0xed, 0x29, 0xe9, 0x0c, 0xf1, 0x0f, 0x19, 0xd8,
+ 0x4c, 0xf4, 0x97, 0xc6, 0x58, 0x7e, 0x1d, 0xb2, 0xc0, 0x34, 0xa6, 0x62, 0x1d, 0xa2, 0x23, 0x28,
+ 0x88, 0x1e, 0x52, 0x93, 0x77, 0x17, 0x00, 0x12, 0xdb, 0x94, 0x84, 0x93, 0x30, 0x73, 0x9d, 0x3e,
+ 0xfb, 0x6e, 0x9d, 0xfe, 0x35, 0xa8, 0xe1, 0x77, 0xd0, 0xb7, 0xda, 0xe6, 0x09, 0x6c, 0x9a, 0xee,
+ 0x70, 0x48, 0x4c, 0xe6, 0x0d, 0xba, 0xed, 0x04, 0xc4, 0xbf, 0x30, 0x86, 0x6f, 0xf7, 0x1b, 0x34,
+ 0x19, 0xd5, 0x91, 0x83, 0xb4, 0x17, 0xb0, 0x11, 0x9b, 0x58, 0x1a, 0xe2, 0x11, 0xe4, 0x29, 0x63,
+ 0x48, 0x4b, 0x7c, 0xb2, 0xa0, 0x25, 0x28, 0x16, 0xc3, 0xb5, 0x4d, 0x01, 0xde, 0xbe, 0x20, 0x4e,
+ 0xf4, 0x59, 0xda, 0x1e, 0x6c, 0xf4, 0xb8, 0x9b, 0xa6, 0xf2, 0xc3, 0x89, 0x8b, 0x67, 0x12, 0x2e,
+ 0xbe, 0x05, 0x28, 0x8e, 0x22, 0x1d, 0xf1, 0x0a, 0xd6, 0xdb, 0x97, 0xc4, 0x4c, 0x85, 0x5c, 0x83,
+ 0x55, 0xd3, 0x1d, 0x8d, 0x0c, 0xc7, 0xaa, 0x65, 0x6e, 0x67, 0x77, 0x4a, 0x38, 0x24, 0xe3, 0x6b,
+ 0x31, 0x9b, 0x76, 0x2d, 0x6a, 0x7f, 0xa7, 0x80, 0x3a, 0x99, 0x5b, 0x2a, 0x92, 0x49, 0x1f, 0x58,
+ 0x0c, 0x88, 0xcd, 0x5d, 0xc1, 0x92, 0x92, 0xfc, 0x30, 0x5c, 0x08, 0x3e, 0xf1, 0xfd, 0x58, 0x38,
+ 0xca, 0x5e, 0x33, 0x1c, 0x69, 0xfb, 0xf0, 0x9d, 0x50, 0x9c, 0x5e, 0xe0, 0x13, 0x63, 0x64, 0x3b,
+ 0x83, 0xce, 0xd1, 0x91, 0x47, 0x84, 0xe0, 0x08, 0x41, 0xce, 0x32, 0x02, 0x43, 0x0a, 0xc6, 0xff,
+ 0xb3, 0x45, 0x6f, 0x0e, 0x5d, 0x1a, 0x2d, 0x7a, 0x4e, 0x68, 0xff, 0x99, 0x85, 0xda, 0x0c, 0x54,
+ 0xa8, 0xde, 0x17, 0x90, 0xa7, 0x24, 0x18, 0x7b, 0xd2, 0x55, 0xda, 0xa9, 0x05, 0x9e, 0x8f, 0xd7,
+ 0xe8, 0x31, 0x30, 0x2c, 0x30, 0xd1, 0x00, 0x8a, 0x41, 0x70, 0xa5, 0x53, 0xfb, 0xa7, 0x61, 0x42,
+ 0x70, 0x70, 0x5d, 0xfc, 0x3e, 0xf1, 0x47, 0xb6, 0x63, 0x0c, 0x7b, 0xf6, 0x4f, 0x09, 0x5e, 0x0d,
+ 0x82, 0x2b, 0xf6, 0x07, 0x3d, 0x67, 0x0e, 0x6f, 0xd9, 0x8e, 0x54, 0x7b, 0x6b, 0xd9, 0x59, 0x62,
+ 0x0a, 0xc6, 0x02, 0xb1, 0x7e, 0x00, 0x79, 0xfe, 0x4d, 0xcb, 0x38, 0xa2, 0x0a, 0xd9, 0x20, 0xb8,
+ 0xe2, 0x42, 0x15, 0x31, 0xfb, 0x5b, 0x7f, 0x00, 0x95, 0xf8, 0x17, 0x30, 0x47, 0x3a, 0x23, 0xf6,
+ 0xe0, 0x4c, 0x38, 0x58, 0x1e, 0x4b, 0x8a, 0x59, 0xf2, 0xb5, 0x6d, 0xc9, 0x94, 0x35, 0x8f, 0x05,
+ 0xa1, 0xfd, 0x5b, 0x06, 0x6e, 0xcd, 0xd1, 0x8c, 0x74, 0xd6, 0x17, 0x09, 0x67, 0x7d, 0x47, 0x5a,
+ 0x08, 0x3d, 0xfe, 0x45, 0xc2, 0xe3, 0xdf, 0x21, 0x38, 0x5b, 0x36, 0x37, 0xa0, 0x40, 0x2e, 0xed,
+ 0x80, 0x58, 0x52, 0x55, 0x92, 0x8a, 0x2d, 0xa7, 0xdc, 0x75, 0x97, 0xd3, 0x21, 0x6c, 0xb5, 0x7c,
+ 0x62, 0x04, 0x44, 0x86, 0xf2, 0xd0, 0xff, 0x6f, 0x41, 0xd1, 0x18, 0x0e, 0x5d, 0x73, 0x62, 0xd6,
+ 0x55, 0x4e, 0x77, 0x2c, 0x54, 0x87, 0xe2, 0x99, 0x4b, 0x03, 0xc7, 0x18, 0x11, 0x19, 0xbc, 0x22,
+ 0x5a, 0xfb, 0x46, 0x81, 0xed, 0x29, 0x3c, 0x69, 0x85, 0x53, 0xa8, 0xda, 0xd4, 0x1d, 0xf2, 0x0f,
+ 0xd4, 0x63, 0x27, 0xbc, 0x1f, 0x2d, 0xb6, 0xd5, 0x74, 0x42, 0x0c, 0x7e, 0xe0, 0x5b, 0xb3, 0xe3,
+ 0x24, 0xf7, 0x38, 0x3e, 0xb9, 0x25, 0x57, 0x7a, 0x48, 0x6a, 0xff, 0xa8, 0xc0, 0xb6, 0xdc, 0xe1,
+ 0xd3, 0x7f, 0xe8, 0xac, 0xc8, 0x99, 0x77, 0x2d, 0xb2, 0x56, 0x83, 0x1b, 0xd3, 0x72, 0xc9, 0x98,
+ 0xff, 0xdf, 0x79, 0x40, 0xb3, 0xa7, 0x4b, 0xf4, 0x01, 0x54, 0x28, 0x71, 0x2c, 0x5d, 0xec, 0x17,
+ 0x62, 0x2b, 0x2b, 0xe2, 0x32, 0xe3, 0x89, 0x8d, 0x83, 0xb2, 0x10, 0x48, 0x2e, 0xa5, 0xb4, 0x45,
+ 0xcc, 0xff, 0xa3, 0x33, 0xa8, 0xbc, 0xa4, 0x7a, 0x34, 0x37, 0x77, 0xa8, 0x6a, 0xea, 0xb0, 0x36,
+ 0x2b, 0x47, 0xe3, 0x51, 0x2f, 0xfa, 0x2e, 0x5c, 0x7e, 0x49, 0x23, 0x02, 0xfd, 0x5c, 0x81, 0x9b,
+ 0x61, 0x5a, 0x31, 0x51, 0xdf, 0xc8, 0xb5, 0x08, 0xad, 0xe5, 0x6e, 0x67, 0x77, 0xaa, 0xbb, 0xc7,
+ 0xd7, 0xd0, 0xdf, 0x0c, 0xf3, 0xd0, 0xb5, 0x08, 0xde, 0x76, 0xe6, 0x70, 0x29, 0x6a, 0xc0, 0xe6,
+ 0x68, 0x4c, 0x03, 0x5d, 0x78, 0x81, 0x2e, 0x3b, 0xd5, 0xf2, 0x5c, 0x2f, 0x1b, 0xac, 0x29, 0xe1,
+ 0xab, 0xe8, 0x1c, 0xd6, 0x46, 0xee, 0xd8, 0x09, 0x74, 0x93, 0x9f, 0x7f, 0x68, 0xad, 0xb0, 0xd0,
+ 0xc1, 0x78, 0x8e, 0x96, 0x0e, 0x19, 0x9c, 0x38, 0x4d, 0x51, 0x5c, 0x19, 0xc5, 0x28, 0xf4, 0x7b,
+ 0x70, 0xc3, 0xb2, 0xa9, 0x71, 0x3a, 0x24, 0xfa, 0xd0, 0x1d, 0xe8, 0x93, 0x1c, 0xa6, 0x56, 0xe4,
+ 0xf2, 0x6d, 0xc9, 0xd6, 0x03, 0x77, 0xd0, 0x8a, 0xda, 0xf8, 0xa8, 0x2b, 0xc7, 0x18, 0xd9, 0xa6,
+ 0xce, 0x44, 0x1e, 0xba, 0x86, 0xa5, 0x8f, 0x29, 0xf1, 0x69, 0xad, 0x24, 0x47, 0x89, 0xd6, 0x67,
+ 0xb2, 0xf1, 0x84, 0xb5, 0x69, 0xf7, 0xa1, 0x1c, 0xb3, 0x17, 0x2a, 0x42, 0xae, 0x7b, 0xd4, 0x6d,
+ 0xab, 0x2b, 0x08, 0xa0, 0xd0, 0xda, 0xc7, 0x47, 0x47, 0x7d, 0x71, 0xfc, 0xe8, 0x1c, 0x36, 0x1f,
+ 0xb7, 0xd5, 0x0c, 0x63, 0x9f, 0x74, 0xff, 0xb0, 0xdd, 0x39, 0x50, 0xb3, 0x5a, 0x1b, 0x2a, 0xf1,
+ 0xaf, 0x40, 0x08, 0xaa, 0x27, 0xdd, 0xa7, 0xdd, 0xa3, 0x67, 0x5d, 0xfd, 0xf0, 0xe8, 0xa4, 0xdb,
+ 0x67, 0x87, 0x98, 0x2a, 0x40, 0xb3, 0xfb, 0x7c, 0x42, 0xaf, 0x41, 0xa9, 0x7b, 0x14, 0x92, 0x4a,
+ 0x3d, 0xa3, 0x2a, 0x4f, 0x72, 0xc5, 0x55, 0xb5, 0x88, 0x2b, 0x3e, 0x19, 0xb9, 0x01, 0xd1, 0xd9,
+ 0x16, 0x41, 0xb5, 0xff, 0xc8, 0xc2, 0xd6, 0x3c, 0x23, 0x23, 0x0b, 0x72, 0xcc, 0x61, 0xe4, 0xd1,
+ 0xf2, 0xdd, 0xfb, 0x0b, 0x47, 0x67, 0xeb, 0xc4, 0x33, 0xe4, 0x5e, 0x52, 0xc2, 0xfc, 0x3f, 0xd2,
+ 0xa1, 0x30, 0x34, 0x4e, 0xc9, 0x90, 0xd6, 0xb2, 0xfc, 0xf2, 0xe5, 0xf1, 0x75, 0xe6, 0x3e, 0xe0,
+ 0x48, 0xe2, 0xe6, 0x45, 0xc2, 0xa2, 0x3e, 0x94, 0x59, 0xb4, 0xa4, 0x42, 0x9d, 0x32, 0x80, 0xef,
+ 0xa6, 0x9c, 0x65, 0x7f, 0x32, 0x12, 0xc7, 0x61, 0xea, 0xf7, 0xa0, 0x1c, 0x9b, 0x6c, 0xce, 0xc5,
+ 0xc9, 0x56, 0xfc, 0xe2, 0xa4, 0x14, 0xbf, 0x05, 0x79, 0x38, 0x6b, 0x03, 0xa6, 0x23, 0xe6, 0x24,
+ 0xfb, 0x47, 0xbd, 0xbe, 0x38, 0xa2, 0x3e, 0xc6, 0x47, 0x27, 0xc7, 0xaa, 0xc2, 0x98, 0xfd, 0x66,
+ 0xef, 0xa9, 0x9a, 0x89, 0x7c, 0x28, 0xab, 0xb5, 0xa0, 0x1c, 0x93, 0x2b, 0xb1, 0x3d, 0x28, 0xc9,
+ 0xed, 0x81, 0x05, 0x68, 0xc3, 0xb2, 0x7c, 0x42, 0xa9, 0x94, 0x23, 0x24, 0xb5, 0x17, 0x50, 0xda,
+ 0xeb, 0xf6, 0x24, 0x44, 0x0d, 0x56, 0x29, 0xf1, 0xd9, 0x77, 0xf3, 0x2b, 0xb0, 0x12, 0x0e, 0x49,
+ 0x06, 0x4e, 0x89, 0xe1, 0x9b, 0x67, 0x84, 0xca, 0xa4, 0x22, 0xa2, 0xd9, 0x28, 0x97, 0x5f, 0x25,
+ 0x09, 0xdb, 0x95, 0x70, 0x48, 0x6a, 0xff, 0x5f, 0x04, 0x98, 0x5c, 0x6b, 0xa0, 0x2a, 0x64, 0xa2,
+ 0x60, 0x9f, 0xb1, 0x2d, 0xe6, 0x07, 0xb1, 0xcd, 0x8c, 0xff, 0x47, 0xbb, 0xb0, 0x3d, 0xa2, 0x03,
+ 0xcf, 0x30, 0xcf, 0x75, 0x79, 0x1b, 0x21, 0x62, 0x02, 0x0f, 0x9c, 0x15, 0xbc, 0x29, 0x1b, 0xe5,
+ 0x92, 0x17, 0xb8, 0x07, 0x90, 0x25, 0xce, 0x05, 0x0f, 0x72, 0xe5, 0xdd, 0xfb, 0x0b, 0x5f, 0xb7,
+ 0x34, 0xda, 0xce, 0x85, 0xf0, 0x15, 0x06, 0x83, 0x74, 0x00, 0x8b, 0x5c, 0xd8, 0x26, 0xd1, 0x19,
+ 0x68, 0x9e, 0x83, 0x7e, 0xb1, 0x38, 0xe8, 0x1e, 0xc7, 0x88, 0xa0, 0x4b, 0x56, 0x48, 0xa3, 0x2e,
+ 0x94, 0x7c, 0x42, 0xdd, 0xb1, 0x6f, 0x12, 0x11, 0xe9, 0xd2, 0x9f, 0x88, 0x70, 0x38, 0x0e, 0x4f,
+ 0x20, 0xd0, 0x1e, 0x14, 0x78, 0x80, 0xa3, 0xb5, 0x55, 0x2e, 0xec, 0x47, 0x29, 0xc1, 0x78, 0x74,
+ 0xc1, 0x72, 0x2c, 0x7a, 0x0c, 0xab, 0x42, 0x44, 0x5a, 0x2b, 0x72, 0x98, 0x8f, 0xd3, 0x46, 0x5f,
+ 0x3e, 0x0a, 0x87, 0xa3, 0x99, 0x55, 0x59, 0x60, 0xe4, 0x71, 0xb1, 0x84, 0xf9, 0x7f, 0xf4, 0x1e,
+ 0x94, 0xc4, 0x66, 0x6f, 0xd9, 0x7e, 0x0d, 0x84, 0x73, 0x72, 0xc6, 0x9e, 0xed, 0xa3, 0xf7, 0xa1,
+ 0x2c, 0x92, 0x3a, 0x9d, 0x47, 0x85, 0x32, 0x6f, 0x06, 0xc1, 0x3a, 0x66, 0xb1, 0x41, 0x74, 0x20,
+ 0xbe, 0x2f, 0x3a, 0x54, 0xa2, 0x0e, 0xc4, 0xf7, 0x79, 0x87, 0xdf, 0x81, 0x75, 0x9e, 0x0a, 0x0f,
+ 0x7c, 0x77, 0xec, 0xe9, 0xdc, 0xa7, 0xd6, 0x78, 0xa7, 0x35, 0xc6, 0x7e, 0xcc, 0xb8, 0x5d, 0xe6,
+ 0x5c, 0xb7, 0xa0, 0xf8, 0xca, 0x3d, 0x15, 0x1d, 0xaa, 0x62, 0x1d, 0xbc, 0x72, 0x4f, 0xc3, 0xa6,
+ 0x28, 0x1d, 0x59, 0x4f, 0xa6, 0x23, 0x5f, 0xc3, 0x8d, 0xd9, 0x7d, 0x95, 0xa7, 0x25, 0xea, 0xf5,
+ 0xd3, 0x92, 0x2d, 0x67, 0x5e, 0x1c, 0xfe, 0x12, 0xb2, 0x96, 0x43, 0x6b, 0x1b, 0x0b, 0x39, 0x47,
+ 0xb4, 0x8e, 0x31, 0x1b, 0x8c, 0xb6, 0xa1, 0xc0, 0x3e, 0xd6, 0xb6, 0x6a, 0x48, 0x84, 0x9e, 0x57,
+ 0xee, 0x69, 0xc7, 0x42, 0xdf, 0x81, 0x12, 0xfb, 0x7e, 0xea, 0x19, 0x26, 0xa9, 0x6d, 0xf2, 0x96,
+ 0x09, 0x83, 0x19, 0xca, 0x71, 0x2d, 0x22, 0x54, 0xb4, 0x25, 0x0c, 0xc5, 0x18, 0x5c, 0x47, 0x37,
+ 0x61, 0x95, 0x37, 0xda, 0x56, 0x6d, 0x5b, 0x9c, 0x38, 0x18, 0xd9, 0xb1, 0x90, 0x06, 0x6b, 0x9e,
+ 0xe1, 0x13, 0x27, 0xd0, 0xe5, 0x8c, 0x37, 0x78, 0x73, 0x59, 0x30, 0x9f, 0xb0, 0x79, 0xeb, 0x9f,
+ 0x42, 0x31, 0x5c, 0x0c, 0x8b, 0x84, 0xc9, 0xfa, 0x03, 0xa8, 0x26, 0x97, 0xd2, 0x42, 0x41, 0xf6,
+ 0x9f, 0x33, 0x50, 0x8a, 0x16, 0x0d, 0x72, 0x60, 0x93, 0x1b, 0x95, 0xa5, 0xa6, 0xfa, 0x64, 0x0d,
+ 0x8a, 0x84, 0xf8, 0xf3, 0x94, 0x6a, 0x6e, 0x86, 0x08, 0xf2, 0x64, 0x2e, 0x17, 0x24, 0x8a, 0x90,
+ 0x27, 0xf3, 0x7d, 0x05, 0xeb, 0x43, 0xdb, 0x19, 0x5f, 0xc6, 0xe6, 0x12, 0x99, 0xec, 0xef, 0xa7,
+ 0x9c, 0xeb, 0x80, 0x8d, 0x9e, 0xcc, 0x51, 0x1d, 0x26, 0x68, 0xb4, 0x0f, 0x79, 0xcf, 0xf5, 0x83,
+ 0x70, 0xcf, 0x4c, 0xbb, 0x9b, 0x1d, 0xbb, 0x7e, 0x70, 0x68, 0x78, 0x1e, 0x3b, 0xac, 0x09, 0x00,
+ 0xed, 0x9b, 0x0c, 0xdc, 0x98, 0xff, 0x61, 0xa8, 0x0b, 0x59, 0xd3, 0x1b, 0x4b, 0x25, 0x3d, 0x58,
+ 0x54, 0x49, 0x2d, 0x6f, 0x3c, 0x91, 0x9f, 0x01, 0xa1, 0x67, 0x50, 0x18, 0x91, 0x91, 0xeb, 0x5f,
+ 0x49, 0x5d, 0x3c, 0x5c, 0x14, 0xf2, 0x90, 0x8f, 0x9e, 0xa0, 0x4a, 0x38, 0x84, 0xa1, 0x28, 0x17,
+ 0x13, 0x95, 0x61, 0x7b, 0xc1, 0xeb, 0xb4, 0x10, 0x12, 0x47, 0x38, 0xda, 0xa7, 0xb0, 0x3d, 0xf7,
+ 0x53, 0xd0, 0x6f, 0x01, 0x98, 0xde, 0x58, 0xe7, 0xcf, 0x1d, 0xc2, 0x83, 0xb2, 0xb8, 0x64, 0x7a,
+ 0xe3, 0x1e, 0x67, 0x68, 0x2f, 0xa0, 0xf6, 0x26, 0x79, 0xd9, 0x1a, 0x13, 0x12, 0xeb, 0xa3, 0x53,
+ 0xae, 0x83, 0x2c, 0x2e, 0x0a, 0xc6, 0xe1, 0x29, 0x5b, 0x4a, 0x61, 0xa3, 0x71, 0xc9, 0x3a, 0x64,
+ 0x79, 0x87, 0xb2, 0xec, 0x60, 0x5c, 0x1e, 0x9e, 0x6a, 0xbf, 0xc8, 0xc0, 0xfa, 0x94, 0xc8, 0xec,
+ 0xc8, 0x2a, 0x02, 0x70, 0x78, 0x19, 0x20, 0x28, 0x16, 0x8d, 0x4d, 0xdb, 0x0a, 0xaf, 0x91, 0xf9,
+ 0x7f, 0xbe, 0x0f, 0x7b, 0xf2, 0x8a, 0x37, 0x63, 0x7b, 0x6c, 0xf9, 0x8c, 0x4e, 0xed, 0x80, 0xf2,
+ 0xa4, 0x28, 0x8f, 0x05, 0x81, 0x9e, 0x43, 0xd5, 0x27, 0x7c, 0xff, 0xb7, 0x74, 0xe1, 0x65, 0xf9,
+ 0x85, 0xbc, 0x4c, 0x4a, 0xc8, 0x9c, 0x0d, 0xaf, 0x85, 0x48, 0x8c, 0xa2, 0xe8, 0x19, 0xac, 0x85,
+ 0xc9, 0xb4, 0x40, 0x2e, 0x2c, 0x8d, 0x5c, 0x91, 0x40, 0x1c, 0x58, 0xbb, 0x07, 0xe5, 0x58, 0x23,
+ 0xfb, 0x30, 0x9e, 0xfd, 0x49, 0x9d, 0x08, 0x22, 0x19, 0x2d, 0xf2, 0x32, 0x5a, 0x68, 0xa7, 0x50,
+ 0x8e, 0xad, 0x8b, 0x45, 0x86, 0x32, 0x7d, 0x06, 0x2e, 0xd7, 0x67, 0x1e, 0x67, 0x02, 0x97, 0xc5,
+ 0x49, 0x96, 0x79, 0xe9, 0xb6, 0xc7, 0x35, 0x5a, 0xc2, 0x05, 0x46, 0x76, 0x3c, 0xed, 0x57, 0x19,
+ 0xa8, 0x26, 0x97, 0x74, 0xe8, 0x47, 0x1e, 0xf1, 0x6d, 0xd7, 0x8a, 0xf9, 0xd1, 0x31, 0x67, 0x30,
+ 0x5f, 0x61, 0xcd, 0x5f, 0x8f, 0xdd, 0xc0, 0x08, 0x7d, 0xc5, 0xf4, 0xc6, 0x7f, 0xc0, 0xe8, 0x29,
+ 0x1f, 0xcc, 0x4e, 0xf9, 0x20, 0xfa, 0x08, 0x90, 0x74, 0xa5, 0xa1, 0x3d, 0xb2, 0x03, 0xfd, 0xf4,
+ 0x2a, 0x20, 0xc2, 0xc6, 0x59, 0xac, 0x8a, 0x96, 0x03, 0xd6, 0xf0, 0x25, 0xe3, 0x33, 0xc7, 0x73,
+ 0xdd, 0x91, 0x4e, 0x4d, 0xd7, 0x27, 0xba, 0x61, 0xbd, 0xe2, 0xa7, 0xb5, 0x2c, 0x2e, 0xbb, 0xee,
+ 0xa8, 0xc7, 0x78, 0x4d, 0xeb, 0x15, 0xdb, 0x88, 0x4d, 0x6f, 0x4c, 0x49, 0xa0, 0xb3, 0x1f, 0x9e,
+ 0xbb, 0x94, 0x30, 0x08, 0x56, 0xcb, 0x1b, 0x53, 0xf4, 0x5d, 0x58, 0x0b, 0x3b, 0xf0, 0xbd, 0x58,
+ 0x26, 0x01, 0x15, 0xd9, 0x85, 0xf3, 0x90, 0x06, 0x95, 0x63, 0xe2, 0x9b, 0xc4, 0x09, 0xfa, 0xb6,
+ 0x79, 0x4e, 0xf9, 0xb1, 0x4b, 0xc1, 0x09, 0x9e, 0x3c, 0xb5, 0x84, 0xb3, 0x8d, 0xc8, 0x88, 0x6a,
+ 0xff, 0xaa, 0x40, 0x9e, 0xa7, 0x2c, 0x4c, 0x29, 0x7c, 0xbb, 0xe7, 0xd9, 0x80, 0x4c, 0x75, 0x19,
+ 0x83, 0xe7, 0x02, 0xef, 0x41, 0x89, 0x2b, 0x3f, 0x76, 0xc2, 0xe0, 0x79, 0x30, 0x6f, 0xac, 0x43,
+ 0xd1, 0x27, 0x86, 0xe5, 0x3a, 0xc3, 0xf0, 0x16, 0x2c, 0xa2, 0xd1, 0xef, 0x82, 0xea, 0xf9, 0xae,
+ 0x67, 0x0c, 0x26, 0x07, 0x67, 0x69, 0xbe, 0xf5, 0x18, 0x9f, 0xa7, 0xe8, 0xdf, 0x85, 0x35, 0x4a,
+ 0x44, 0x64, 0x17, 0x4e, 0x92, 0x17, 0x9f, 0x29, 0x99, 0xfc, 0x44, 0xa0, 0x7d, 0x0d, 0x05, 0xb1,
+ 0x71, 0x5d, 0x43, 0xde, 0x8f, 0x01, 0x09, 0x45, 0x32, 0x07, 0x19, 0xd9, 0x94, 0xca, 0x2c, 0x9b,
+ 0x3f, 0xe5, 0x8a, 0x96, 0xe3, 0x49, 0x83, 0xf6, 0x5f, 0x8a, 0xc8, 0xb7, 0xc5, 0x23, 0x1b, 0x4b,
+ 0xcc, 0xd9, 0xaa, 0x61, 0x47, 0x5b, 0x71, 0x9b, 0x17, 0x92, 0xa8, 0x03, 0x05, 0x99, 0x56, 0x67,
+ 0x96, 0x7d, 0xa3, 0x94, 0x00, 0xe1, 0xdd, 0x3e, 0x91, 0x37, 0x1b, 0x8b, 0xde, 0xed, 0x13, 0x71,
+ 0xb7, 0x4f, 0xd0, 0x07, 0x50, 0x91, 0x09, 0xbf, 0x80, 0xcb, 0xf1, 0x7c, 0xbf, 0x6c, 0x45, 0x0f,
+ 0x28, 0x44, 0xfb, 0x5f, 0x25, 0x8a, 0x7b, 0xe1, 0x43, 0x07, 0xfa, 0x0a, 0x8a, 0x2c, 0x84, 0xe8,
+ 0x23, 0xc3, 0x93, 0xcf, 0xf6, 0xad, 0xe5, 0xde, 0x50, 0xc2, 0x5d, 0x51, 0xa4, 0xeb, 0xab, 0x9e,
+ 0xa0, 0x58, 0xfc, 0x64, 0x47, 0xa5, 0x30, 0x7e, 0xb2, 0xff, 0xe8, 0x43, 0xa8, 0x1a, 0xe3, 0xc0,
+ 0xd5, 0x0d, 0xeb, 0x82, 0xf8, 0x81, 0x4d, 0x89, 0xf4, 0xa5, 0x35, 0xc6, 0x6d, 0x86, 0xcc, 0xfa,
+ 0x7d, 0xa8, 0xc4, 0x31, 0xdf, 0x96, 0xb7, 0xe4, 0xe3, 0x79, 0xcb, 0x9f, 0x02, 0x4c, 0x2e, 0x0d,
+ 0x99, 0x8f, 0x90, 0x4b, 0x3b, 0xd0, 0xcd, 0xf0, 0x6c, 0x9e, 0xc7, 0x45, 0xc6, 0x68, 0x31, 0x67,
+ 0x4c, 0xbe, 0x68, 0xe4, 0xc3, 0x17, 0x0d, 0x16, 0x1d, 0xd8, 0x82, 0x3e, 0xb7, 0x87, 0xc3, 0xe8,
+ 0x22, 0xb3, 0xe4, 0xba, 0xa3, 0xa7, 0x9c, 0xa1, 0xfd, 0x3a, 0x23, 0x7c, 0x45, 0xbc, 0x4d, 0xa5,
+ 0x3a, 0x9b, 0xbd, 0x2b, 0x53, 0xdf, 0x03, 0xa0, 0x81, 0xe1, 0xb3, 0x24, 0xcc, 0x08, 0xaf, 0x52,
+ 0xeb, 0x33, 0x4f, 0x22, 0xfd, 0xb0, 0x58, 0x06, 0x97, 0x64, 0xef, 0x66, 0x80, 0x3e, 0x87, 0x8a,
+ 0xe9, 0x8e, 0xbc, 0x21, 0x91, 0x83, 0xf3, 0x6f, 0x1d, 0x5c, 0x8e, 0xfa, 0x37, 0x83, 0xd8, 0x05,
+ 0x6e, 0xe1, 0xba, 0x17, 0xb8, 0xbf, 0x52, 0xc4, 0x13, 0x5b, 0xfc, 0x85, 0x0f, 0x0d, 0xe6, 0x94,
+ 0x91, 0x3c, 0x5e, 0xf2, 0xb9, 0xf0, 0xdb, 0x6a, 0x48, 0xea, 0x9f, 0xa7, 0x29, 0xda, 0x78, 0x73,
+ 0x5a, 0xfc, 0xef, 0x59, 0x28, 0x45, 0xaf, 0x6b, 0x33, 0xb6, 0xff, 0x0c, 0x4a, 0x51, 0xa5, 0x92,
+ 0x0c, 0x10, 0xdf, 0x6a, 0x9e, 0xa8, 0x33, 0x7a, 0x09, 0xc8, 0x18, 0x0c, 0xa2, 0x74, 0x57, 0x1f,
+ 0x53, 0x63, 0x10, 0xbe, 0x6d, 0x7e, 0xb6, 0x80, 0x1e, 0xc2, 0xfd, 0xf1, 0x84, 0x8d, 0xc7, 0xaa,
+ 0x31, 0x18, 0x24, 0x38, 0xe8, 0xcf, 0x60, 0x3b, 0x39, 0x87, 0x7e, 0x7a, 0xa5, 0x7b, 0xb6, 0x25,
+ 0xef, 0x00, 0xf6, 0x17, 0x7d, 0x60, 0x6c, 0x24, 0xe0, 0xbf, 0xbc, 0x3a, 0xb6, 0x2d, 0xa1, 0x73,
+ 0xe4, 0xcf, 0x34, 0xd4, 0xff, 0x02, 0x6e, 0xbe, 0xa1, 0xfb, 0x1c, 0x1b, 0x74, 0x93, 0x85, 0x33,
+ 0xcb, 0x2b, 0x21, 0x66, 0xbd, 0x5f, 0x2a, 0xe2, 0x1d, 0x34, 0xa9, 0x93, 0x66, 0x3c, 0x4f, 0xbf,
+ 0x93, 0x72, 0x9e, 0xd6, 0xf1, 0x89, 0x80, 0xe7, 0xa9, 0xf9, 0x93, 0xa9, 0xd4, 0x3c, 0x6d, 0x42,
+ 0x26, 0x32, 0x5c, 0x01, 0x24, 0x11, 0xb4, 0x7f, 0xc9, 0x42, 0x31, 0x44, 0xe7, 0x27, 0xf8, 0x2b,
+ 0x1a, 0x90, 0x91, 0x1e, 0x5d, 0x2f, 0x2a, 0x18, 0x04, 0x8b, 0xef, 0xa8, 0xef, 0x41, 0x69, 0x4c,
+ 0x89, 0x2f, 0x9a, 0x33, 0xbc, 0xb9, 0xc8, 0x18, 0xbc, 0xf1, 0x7d, 0x28, 0x07, 0x6e, 0x60, 0x0c,
+ 0xf5, 0x80, 0xe7, 0x0b, 0x59, 0x31, 0x9a, 0xb3, 0x78, 0xb6, 0x80, 0xbe, 0x0f, 0x1b, 0xc1, 0x99,
+ 0xef, 0x06, 0xc1, 0x90, 0xe5, 0xaa, 0x3c, 0x73, 0x12, 0x89, 0x4e, 0x0e, 0xab, 0x51, 0x83, 0xc8,
+ 0xa8, 0x28, 0x8b, 0xde, 0x93, 0xce, 0xcc, 0x75, 0x79, 0x10, 0xc9, 0xe1, 0xb5, 0x88, 0xcb, 0x5c,
+ 0x9b, 0x6d, 0x9e, 0x9e, 0xc8, 0x48, 0x78, 0xac, 0x50, 0x70, 0x48, 0x22, 0x1d, 0xd6, 0x47, 0xc4,
+ 0xa0, 0x63, 0x9f, 0x58, 0xfa, 0x4b, 0x9b, 0x0c, 0x2d, 0x71, 0xf1, 0x52, 0x4d, 0x7d, 0xdc, 0x08,
+ 0xd5, 0xd2, 0x78, 0xc4, 0x47, 0xe3, 0x6a, 0x08, 0x27, 0x68, 0x96, 0x39, 0x88, 0x7f, 0x68, 0x1d,
+ 0xca, 0xbd, 0xe7, 0xbd, 0x7e, 0xfb, 0x50, 0x3f, 0x3c, 0xda, 0x6b, 0xcb, 0xda, 0xa8, 0x5e, 0x1b,
+ 0x0b, 0x52, 0x61, 0xed, 0xfd, 0xa3, 0x7e, 0xf3, 0x40, 0xef, 0x77, 0x5a, 0x4f, 0x7b, 0x6a, 0x06,
+ 0x6d, 0xc3, 0x46, 0x7f, 0x1f, 0x1f, 0xf5, 0xfb, 0x07, 0xed, 0x3d, 0xfd, 0xb8, 0x8d, 0x3b, 0x47,
+ 0x7b, 0x3d, 0x35, 0x8b, 0x10, 0x54, 0x27, 0xec, 0x7e, 0xe7, 0xb0, 0xad, 0xe6, 0x50, 0x19, 0x56,
+ 0x8f, 0xdb, 0xb8, 0xd5, 0xee, 0xf6, 0xd5, 0xbc, 0xf6, 0x8b, 0x2c, 0x94, 0x63, 0x56, 0x64, 0x8e,
+ 0xec, 0x53, 0x71, 0xae, 0xc9, 0x61, 0xf6, 0x97, 0xbf, 0xe5, 0x1a, 0xe6, 0x99, 0xb0, 0x4e, 0x0e,
+ 0x0b, 0x82, 0x9f, 0x65, 0x8c, 0xcb, 0xd8, 0x3a, 0xcf, 0xe1, 0xe2, 0xc8, 0xb8, 0x14, 0x20, 0x1f,
+ 0x40, 0xe5, 0x9c, 0xf8, 0x0e, 0x19, 0xca, 0x76, 0x61, 0x91, 0xb2, 0xe0, 0x89, 0x2e, 0x3b, 0xa0,
+ 0xca, 0x2e, 0x13, 0x18, 0x61, 0x8e, 0xaa, 0xe0, 0x1f, 0x86, 0x60, 0x5b, 0x90, 0x17, 0xcd, 0xab,
+ 0x62, 0x7e, 0x4e, 0xb0, 0x6d, 0x8a, 0xbe, 0x36, 0x3c, 0x9e, 0x43, 0xe6, 0x30, 0xff, 0x8f, 0x4e,
+ 0x67, 0xed, 0x53, 0xe0, 0xf6, 0xb9, 0xb7, 0xb8, 0x3b, 0xbf, 0xc9, 0x44, 0x67, 0x91, 0x89, 0x56,
+ 0x21, 0x8b, 0xc3, 0x82, 0xa2, 0x56, 0xb3, 0xb5, 0xcf, 0xcc, 0xb2, 0x06, 0xa5, 0xc3, 0xe6, 0x4f,
+ 0xf4, 0x93, 0x9e, 0xb8, 0xd5, 0x57, 0xa1, 0xf2, 0xb4, 0x8d, 0xbb, 0xed, 0x03, 0xc9, 0xc9, 0xa2,
+ 0x2d, 0x50, 0x25, 0x67, 0xd2, 0x2f, 0xc7, 0x10, 0xc4, 0xdf, 0x3c, 0x2a, 0x42, 0xae, 0xf7, 0xac,
+ 0x79, 0xac, 0x16, 0xb4, 0xff, 0xc9, 0xc0, 0xba, 0xd8, 0x16, 0xa2, 0xd2, 0x87, 0x37, 0x3f, 0xfd,
+ 0xc6, 0x6f, 0xb1, 0x32, 0xc9, 0x5b, 0xac, 0x30, 0x09, 0xe5, 0xbb, 0x7a, 0x76, 0x92, 0x84, 0xf2,
+ 0x9b, 0x9d, 0x44, 0xc4, 0xcf, 0x2d, 0x12, 0xf1, 0x6b, 0xb0, 0x3a, 0x22, 0x34, 0xb2, 0x5b, 0x09,
+ 0x87, 0x24, 0xb2, 0xa1, 0x6c, 0x38, 0x8e, 0x1b, 0x18, 0xe2, 0x6a, 0xb8, 0xb0, 0xd0, 0x66, 0x38,
+ 0xf5, 0xc5, 0x8d, 0xe6, 0x04, 0x49, 0x04, 0xe6, 0x38, 0x76, 0xfd, 0xc7, 0xa0, 0x4e, 0x77, 0x58,
+ 0x64, 0x3b, 0xfc, 0xde, 0x0f, 0x26, 0xbb, 0x21, 0x61, 0xeb, 0x42, 0xbe, 0xb3, 0xa8, 0x2b, 0x8c,
+ 0xc0, 0x27, 0xdd, 0x6e, 0xa7, 0xfb, 0x58, 0x55, 0x10, 0x40, 0xa1, 0xfd, 0x93, 0x4e, 0xbf, 0xbd,
+ 0xa7, 0x66, 0x76, 0x7f, 0xb9, 0x01, 0x05, 0x21, 0x24, 0xfa, 0x46, 0x66, 0x02, 0xf1, 0xb2, 0x5a,
+ 0xf4, 0xe3, 0x85, 0x33, 0xea, 0x44, 0xa9, 0x6e, 0xfd, 0xe1, 0xd2, 0xe3, 0xe5, 0x33, 0xe6, 0x0a,
+ 0xfa, 0x1b, 0x05, 0x2a, 0x89, 0x27, 0xcc, 0xb4, 0x57, 0xe3, 0x73, 0xaa, 0x78, 0xeb, 0x3f, 0x5a,
+ 0x6a, 0x6c, 0x24, 0xcb, 0xcf, 0x15, 0x28, 0xc7, 0xea, 0x57, 0xd1, 0xbd, 0x65, 0x6a, 0x5e, 0x85,
+ 0x24, 0xf7, 0x97, 0x2f, 0x97, 0xd5, 0x56, 0x3e, 0x51, 0xd0, 0x5f, 0x2b, 0x50, 0x8e, 0x55, 0x72,
+ 0xa6, 0x16, 0x65, 0xb6, 0xee, 0x34, 0xb5, 0x28, 0xf3, 0x0a, 0x47, 0x57, 0xd0, 0x5f, 0x2a, 0x50,
+ 0x8a, 0xaa, 0x32, 0xd1, 0xdd, 0xc5, 0xeb, 0x38, 0x85, 0x10, 0x9f, 0x2d, 0x5b, 0x00, 0xaa, 0xad,
+ 0xa0, 0x3f, 0x87, 0x62, 0x58, 0xc2, 0x88, 0xd2, 0xee, 0x5e, 0x53, 0xf5, 0x91, 0xf5, 0xbb, 0x0b,
+ 0x8f, 0x8b, 0x4f, 0x1f, 0xd6, 0x15, 0xa6, 0x9e, 0x7e, 0xaa, 0x02, 0xb2, 0x7e, 0x77, 0xe1, 0x71,
+ 0xd1, 0xf4, 0xcc, 0x13, 0x62, 0xe5, 0x87, 0xa9, 0x3d, 0x61, 0xb6, 0xee, 0x31, 0xb5, 0x27, 0xcc,
+ 0xab, 0x76, 0x14, 0x82, 0xc4, 0x0a, 0x18, 0x53, 0x0b, 0x32, 0x5b, 0x24, 0x99, 0x5a, 0x90, 0x39,
+ 0xf5, 0x92, 0xda, 0x0a, 0xfa, 0x99, 0x12, 0x3f, 0x17, 0xdc, 0x5d, 0xb8, 0x4e, 0x6f, 0x41, 0x97,
+ 0x9c, 0xa9, 0x14, 0xe4, 0x0b, 0xf4, 0x67, 0xf2, 0x16, 0x43, 0x94, 0xf9, 0xa1, 0x45, 0xc0, 0x12,
+ 0x95, 0x81, 0xf5, 0x4f, 0x97, 0xdb, 0x6c, 0xb8, 0x10, 0x7f, 0xa5, 0x00, 0x4c, 0x0a, 0x02, 0x53,
+ 0x0b, 0x31, 0x53, 0x89, 0x58, 0xbf, 0xb7, 0xc4, 0xc8, 0xf8, 0x02, 0x09, 0x0b, 0x96, 0x52, 0x2f,
+ 0x90, 0xa9, 0x82, 0xc5, 0xd4, 0x0b, 0x64, 0xba, 0xd8, 0x50, 0x5b, 0x41, 0xff, 0xa4, 0xc0, 0xc6,
+ 0x4c, 0xc1, 0x14, 0x7a, 0x78, 0xcd, 0x9a, 0xb9, 0xfa, 0x17, 0xcb, 0x03, 0x84, 0xa2, 0xed, 0x28,
+ 0x9f, 0x28, 0xe8, 0x6f, 0x15, 0x58, 0x4b, 0x16, 0x92, 0xa4, 0xde, 0xa5, 0xe6, 0x94, 0x5e, 0xd5,
+ 0x1f, 0x2c, 0x37, 0x38, 0xd2, 0xd6, 0xdf, 0x2b, 0x50, 0x4d, 0xd6, 0x14, 0xa1, 0x07, 0x8b, 0x85,
+ 0x85, 0x29, 0x81, 0x3e, 0x5f, 0x72, 0x74, 0x28, 0xd1, 0x97, 0xab, 0x7f, 0x94, 0x17, 0xd9, 0x5b,
+ 0x81, 0xff, 0xfc, 0xf0, 0x37, 0x01, 0x00, 0x00, 0xff, 0xff, 0x4f, 0xda, 0xef, 0xe8, 0xfd, 0x34,
+ 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
diff --git a/plugins/drivers/proto/driver.proto b/plugins/drivers/proto/driver.proto
index 707aeee9444..1a8ca941a36 100644
--- a/plugins/drivers/proto/driver.proto
+++ b/plugins/drivers/proto/driver.proto
@@ -390,9 +390,9 @@ message DriverCapabilities {
// MountConfigs indicates whether the driver supports mount configurations.
MountConfigs mount_configs = 6;
- // remote_tasks indicates whether the driver executes tasks remotely such
- // on cloud runtimes like AWS ECS.
- bool remote_tasks = 7 [deprecated=true];
+ // previous remote_tasks field no longer used by nomad clients
+ reserved 7;
+ reserved "remote_tasks";
// disable_log_collection indicates whether the driver has the capability of
// disabling log collection
diff --git a/plugins/drivers/server.go b/plugins/drivers/server.go
index 36a9d96dc5b..1bd105e81e2 100644
--- a/plugins/drivers/server.go
+++ b/plugins/drivers/server.go
@@ -49,7 +49,6 @@ func (b *driverPluginServer) Capabilities(ctx context.Context, req *proto.Capabi
Exec: caps.Exec,
MustCreateNetwork: caps.MustInitiateNetwork,
NetworkIsolationModes: []proto.NetworkIsolationSpec_NetworkIsolationMode{},
- RemoteTasks: caps.RemoteTasks,
DynamicWorkloadUsers: caps.DynamicWorkloadUsers,
},
}
diff --git a/plugins/drivers/task_handle.go b/plugins/drivers/task_handle.go
index 701fb12a8a4..488134ae7f3 100644
--- a/plugins/drivers/task_handle.go
+++ b/plugins/drivers/task_handle.go
@@ -4,7 +4,6 @@
package drivers
import (
- "github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/base"
)
@@ -49,34 +48,3 @@ func (h *TaskHandle) Copy() *TaskHandle {
copy(handle.DriverState, h.DriverState)
return handle
}
-
-// Store this TaskHandle on the given TaskState.
-func (h *TaskHandle) Store(ts *structs.TaskState) {
- if h == nil || len(h.DriverState) == 0 {
- // No handle or state, clear existing state
- ts.TaskHandle = nil
- return
- }
-
- ds := make([]byte, len(h.DriverState))
- copy(ds, h.DriverState)
- ts.TaskHandle = &structs.TaskHandle{
- Version: h.Version,
- DriverState: ds,
- }
-}
-
-// NewTaskHandleFromState returns the TaskHandle stored in a TaskState or nil
-// if no handle was stored.
-func NewTaskHandleFromState(ts *structs.TaskState) *TaskHandle {
- if ts.TaskHandle == nil {
- return nil
- }
-
- th := TaskHandle{
- Version: ts.TaskHandle.Version,
- DriverState: make([]byte, len(ts.TaskHandle.DriverState)),
- }
- copy(th.DriverState, ts.TaskHandle.DriverState)
- return &th
-}
diff --git a/scheduler/generic_sched.go b/scheduler/generic_sched.go
index 60b4f7f1eed..f15c5e629a7 100644
--- a/scheduler/generic_sched.go
+++ b/scheduler/generic_sched.go
@@ -709,10 +709,6 @@ func (s *GenericScheduler) computePlacements(destructive, place []placementResul
if len(prevAllocation.HostVolumeIDs) > 0 {
alloc.HostVolumeIDs = prevAllocation.HostVolumeIDs
}
-
- // If the allocation has task handles,
- // copy them to the new allocation
- propagateTaskState(alloc, prevAllocation, missing.PreviousLost())
}
// If we are placing a canary and we found a match, add the canary
@@ -802,46 +798,6 @@ func needsToSetNodes(a, b *structs.Job) bool {
a.NodePool != b.NodePool
}
-// propagateTaskState copies task handles from previous allocations to
-// replacement allocations when the previous allocation is being drained or was
-// lost. Remote task drivers rely on this to reconnect to remote tasks when the
-// allocation managing them changes due to a down or draining node.
-//
-// The previous allocation will be marked as lost after task state has been
-// propagated (when the plan is applied), so its ClientStatus is not yet marked
-// as lost. Instead, we use the `prevLost` flag to track whether the previous
-// allocation will be marked lost.
-func propagateTaskState(newAlloc, prev *structs.Allocation, prevLost bool) {
- // Don't transfer state from client terminal allocs
- if prev.ClientTerminalStatus() {
- return
- }
-
- // If previous allocation is not lost and not draining, do not copy
- // task handles.
- if !prevLost && !prev.DesiredTransition.ShouldMigrate() {
- return
- }
-
- newAlloc.TaskStates = make(map[string]*structs.TaskState, len(newAlloc.AllocatedResources.Tasks))
- for taskName, prevState := range prev.TaskStates {
- if prevState.TaskHandle == nil {
- // No task handle, skip
- continue
- }
-
- if _, ok := newAlloc.AllocatedResources.Tasks[taskName]; !ok {
- // Task dropped in update, skip
- continue
- }
-
- // Copy state
- newState := structs.NewTaskState()
- newState.TaskHandle = prevState.TaskHandle.Copy()
- newAlloc.TaskStates[taskName] = newState
- }
-}
-
// getSelectOptions sets up preferred nodes and penalty nodes
func getSelectOptions(prevAllocation *structs.Allocation, preferredNode *structs.Node) *SelectOptions {
selectOptions := &SelectOptions{}
diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go
index 3d236b5d289..b1622767458 100644
--- a/scheduler/generic_sched_test.go
+++ b/scheduler/generic_sched_test.go
@@ -4264,100 +4264,6 @@ func TestServiceSched_NodeDrain_Queued_Allocations(t *testing.T) {
}
}
-// TestServiceSched_NodeDrain_TaskHandle asserts that allocations with task
-// handles have them propagated to replacement allocations when drained.
-func TestServiceSched_NodeDrain_TaskHandle(t *testing.T) {
- ci.Parallel(t)
-
- h := NewHarness(t)
-
- node := mock.Node()
- require.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), node))
-
- // Create some nodes
- for i := 0; i < 10; i++ {
- node := mock.Node()
- require.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), node))
- }
-
- // Generate a fake job with allocations and an update policy.
- job := mock.Job()
- require.NoError(t, h.State.UpsertJob(structs.MsgTypeTestSetup, h.NextIndex(), nil, job))
-
- var allocs []*structs.Allocation
- for i := 0; i < 10; i++ {
- alloc := mock.Alloc()
- alloc.Job = job
- alloc.JobID = job.ID
- alloc.NodeID = node.ID
- alloc.Name = fmt.Sprintf("my-job.web[%d]", i)
- alloc.DesiredTransition.Migrate = pointer.Of(true)
- alloc.TaskStates = map[string]*structs.TaskState{
- "web": {
- TaskHandle: &structs.TaskHandle{
- Version: 1,
- DriverState: []byte("test-driver-state"),
- },
- },
- }
- allocs = append(allocs, alloc)
- }
- require.NoError(t, h.State.UpsertAllocs(structs.MsgTypeTestSetup, h.NextIndex(), allocs))
-
- node.DrainStrategy = mock.DrainNode().DrainStrategy
- require.NoError(t, h.State.UpsertNode(structs.MsgTypeTestSetup, h.NextIndex(), node))
-
- // Create a mock evaluation to deal with drain
- eval := &structs.Evaluation{
- Namespace: structs.DefaultNamespace,
- ID: uuid.Generate(),
- Priority: 50,
- TriggeredBy: structs.EvalTriggerNodeUpdate,
- JobID: job.ID,
- NodeID: node.ID,
- Status: structs.EvalStatusPending,
- }
- require.NoError(t, h.State.UpsertEvals(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Evaluation{eval}))
-
- // Process the evaluation
- err := h.Process(NewServiceScheduler, eval)
- require.NoError(t, err)
-
- // Ensure a single plan
- require.Len(t, h.Plans, 1)
- plan := h.Plans[0]
-
- // Ensure the plan evicted all allocs
- require.Len(t, plan.NodeUpdate[node.ID], len(allocs))
-
- // Ensure the plan allocated
- var planned []*structs.Allocation
- for _, allocList := range plan.NodeAllocation {
- planned = append(planned, allocList...)
- }
- require.Len(t, planned, len(allocs))
-
- // Lookup the allocations by JobID
- ws := memdb.NewWatchSet()
- out, err := h.State.AllocsByJob(ws, job.Namespace, job.ID, false)
- require.NoError(t, err)
-
- // Ensure all allocations placed
- out, _ = structs.FilterTerminalAllocs(out)
- require.Len(t, out, len(allocs))
-
- // Ensure task states were propagated
- for _, a := range out {
- require.NotEmpty(t, a.TaskStates)
- require.NotEmpty(t, a.TaskStates["web"])
- require.NotNil(t, a.TaskStates["web"].TaskHandle)
- assert.Equal(t, 1, a.TaskStates["web"].TaskHandle.Version)
- assert.Equal(t, []byte("test-driver-state"), a.TaskStates["web"].TaskHandle.DriverState)
- }
-
- h.AssertEvalStatus(t, structs.EvalStatusComplete)
-}
-
func TestServiceSched_RetryLimit(t *testing.T) {
ci.Parallel(t)
@@ -7440,124 +7346,6 @@ func TestServiceSched_CSITopology(t *testing.T) {
}
-// TestPropagateTaskState asserts that propagateTaskState only copies state
-// when the previous allocation is lost or draining.
-func TestPropagateTaskState(t *testing.T) {
- ci.Parallel(t)
-
- const taskName = "web"
- taskHandle := &structs.TaskHandle{
- Version: 1,
- DriverState: []byte("driver-state"),
- }
-
- cases := []struct {
- name string
- prevAlloc *structs.Allocation
- prevLost bool
- copied bool
- }{
- {
- name: "LostWithState",
- prevAlloc: &structs.Allocation{
- ClientStatus: structs.AllocClientStatusRunning,
- DesiredTransition: structs.DesiredTransition{},
- TaskStates: map[string]*structs.TaskState{
- taskName: {
- TaskHandle: taskHandle,
- },
- },
- },
- prevLost: true,
- copied: true,
- },
- {
- name: "DrainedWithState",
- prevAlloc: &structs.Allocation{
- ClientStatus: structs.AllocClientStatusRunning,
- DesiredTransition: structs.DesiredTransition{
- Migrate: pointer.Of(true),
- },
- TaskStates: map[string]*structs.TaskState{
- taskName: {
- TaskHandle: taskHandle,
- },
- },
- },
- prevLost: false,
- copied: true,
- },
- {
- name: "LostWithoutState",
- prevAlloc: &structs.Allocation{
- ClientStatus: structs.AllocClientStatusRunning,
- DesiredTransition: structs.DesiredTransition{},
- TaskStates: map[string]*structs.TaskState{
- taskName: {},
- },
- },
- prevLost: true,
- copied: false,
- },
- {
- name: "DrainedWithoutState",
- prevAlloc: &structs.Allocation{
- ClientStatus: structs.AllocClientStatusRunning,
- DesiredTransition: structs.DesiredTransition{
- Migrate: pointer.Of(true),
- },
- TaskStates: map[string]*structs.TaskState{
- taskName: {},
- },
- },
- prevLost: false,
- copied: false,
- },
- {
- name: "TerminalWithState",
- prevAlloc: &structs.Allocation{
- ClientStatus: structs.AllocClientStatusComplete,
- DesiredTransition: structs.DesiredTransition{},
- TaskStates: map[string]*structs.TaskState{
- taskName: {
- TaskHandle: taskHandle,
- },
- },
- },
- prevLost: false,
- copied: false,
- },
- }
-
- for i := range cases {
- tc := cases[i]
- t.Run(tc.name, func(t *testing.T) {
- newAlloc := &structs.Allocation{
- // Required by propagateTaskState and populated
- // by the scheduler's node iterator.
- AllocatedResources: &structs.AllocatedResources{
- Tasks: map[string]*structs.AllocatedTaskResources{
- taskName: nil, // value isn't used
- },
- },
- }
-
- propagateTaskState(newAlloc, tc.prevAlloc, tc.prevLost)
-
- if tc.copied {
- // Assert state was copied
- require.NotNil(t, newAlloc.TaskStates)
- require.Contains(t, newAlloc.TaskStates, taskName)
- require.Equal(t, taskHandle, newAlloc.TaskStates[taskName].TaskHandle)
- } else {
- // Assert state was *not* copied
- require.Empty(t, newAlloc.TaskStates,
- "expected task states not to be copied")
- }
- })
- }
-}
-
// Tests that a client disconnect generates attribute updates and follow up evals.
func TestServiceSched_Client_Disconnect_Creates_Updates_and_Evals(t *testing.T) {
diff --git a/website/content/docs/concepts/plugins/task-drivers.mdx b/website/content/docs/concepts/plugins/task-drivers.mdx
index 4d457ec0e5f..a63c7af4920 100644
--- a/website/content/docs/concepts/plugins/task-drivers.mdx
+++ b/website/content/docs/concepts/plugins/task-drivers.mdx
@@ -65,12 +65,6 @@ type Capabilities struct {
// MountConfigs tells Nomad which mounting config options the driver supports.
MountConfigs MountConfigSupport
- // RemoteTasks indicates this driver runs tasks on remote systems
- // instead of locally. The Nomad client can use this information to
- // adjust behavior such as propagating task handles between allocations
- // to avoid downtime when a client is lost.
- RemoteTasks bool
-
// DisableLogCollection indicates this driver has disabled log collection
// and the client should not start a logmon process.
DisableLogCollection bool
diff --git a/website/content/docs/upgrade/upgrade-specific.mdx b/website/content/docs/upgrade/upgrade-specific.mdx
index 07163542297..60bb14df431 100644
--- a/website/content/docs/upgrade/upgrade-specific.mdx
+++ b/website/content/docs/upgrade/upgrade-specific.mdx
@@ -30,6 +30,13 @@ In Nomad 1.10.0, the Go API for quotas has a breaking change. The
`Resources`. The `QuotaSpec.VariablesLimit` field is deprecated in lieu of
`QuotaSpec.RegionLimit.Storage.Variables` and will be removed in Nomad 1.12.0.
+#### Remote task driver support removed
+
+All support for remote task driver capabilities has been removed in Nomad 1.10.0.
+Drivers with the `RemoteTasks` capability will no longer be detached in the event
+the allocation is lost, nor will remote tasks be detached when a node is drained.
+Workloads running as remote tasks should be migrated prior to upgrading.
+
## Nomad 1.9.5
#### CNI plugins