ephemeral disk: `migrate` should imply `sticky` (#16826)

The `ephemeral_disk` block's `migrate` field allows for best-effort migration of
the ephemeral disk data to new nodes. The documentation says the `migrate` field
is only respected if `sticky=true`, but in fact if client ACLs are not set the
data is migrated even if `sticky=false`.

The existing behavior when client ACLs are disabled has existed since the early
implementation, so "fixing" that case now would silently break backwards
compatibility. Additionally, having `migrate` not imply `sticky` seems
nonsensical: it suggests that if we place on a new node we migrate the data but
if we place on the same node, we throw the data away!

Update so that `migrate=true` implies `sticky=true` as follows:

* The failure mode when client ACLs are enabled comes from the server not passing
  along a migration token. Update the server so that the server provides a
  migration token whenever `migrate=true` and not just when `sticky=true` too.
* Update the scheduler so that `migrate` implies `sticky`.
* Update the client so that we check for `migrate || sticky` where appropriate.
* Refactor the E2E tests to move them off the old framework and make the intention
  of the test more clear.
This commit is contained in:
Tim Gross 2023-04-07 16:33:45 -04:00 committed by GitHub
parent e7eae66cf1
commit 1335543731
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 175 additions and 189 deletions

3
.changelog/16826.txt Normal file
View File

@ -0,0 +1,3 @@
```release-note:improvement
ephemeral disk: migrate=true now implies sticky=true
```

View File

@ -98,8 +98,8 @@ func newMigratorForAlloc(c Config, tg *structs.TaskGroup, watchedAllocID string,
logger := c.Logger.Named("alloc_migrator").With("alloc_id", c.Alloc.ID).With("previous_alloc", watchedAllocID)
tasks := tg.Tasks
sticky := tg.EphemeralDisk != nil && tg.EphemeralDisk.Sticky
migrate := tg.EphemeralDisk != nil && tg.EphemeralDisk.Migrate
sticky := tg.EphemeralDisk != nil && (tg.EphemeralDisk.Sticky || migrate)
if m != nil {
// Local Allocation because there's an alloc runner

View File

@ -1,5 +1,4 @@
job "drain_migrate" {
datacenters = ["dc1", "dc2"]
constraint {
attribute = "${attr.kernel.name}"
@ -19,22 +18,7 @@ job "drain_migrate" {
config {
image = "busybox:1"
command = "/bin/sh"
args = ["local/test.sh"]
}
template {
data = <<EOT
#!/bin/sh
if [ ! -f /alloc/data/{{ env "NOMAD_JOB_NAME" }} ]; then
echo writing {{ env "NOMAD_ALLOC_ID" }} to /alloc/data/{{ env "NOMAD_JOB_NAME" }}
echo {{ env "NOMAD_ALLOC_ID" }} > /alloc/data/{{ env "NOMAD_JOB_NAME" }}
else
echo /alloc/data/{{ env "NOMAD_JOB_NAME" }} already exists
fi
sleep 3600
EOT
destination = "local/test.sh"
args = ["-c", "echo \"data from $NOMAD_ALLOC_ID\" >> /alloc/data/migrate.txt && sleep 120"]
}
resources {

View File

@ -3,6 +3,7 @@ package nodedrain
import (
"fmt"
"os"
"strings"
"testing"
"time"
@ -24,6 +25,7 @@ func TestNodeDrain(t *testing.T) {
e2eutil.WaitForNodesReady(t, nomadClient, 2) // needs at least 2 to test migration
t.Run("IgnoreSystem", testIgnoreSystem)
t.Run("EphemeralMigrate", testEphemeralMigrate)
t.Run("KeepIneligible", testKeepIneligible)
}
@ -78,6 +80,69 @@ func testIgnoreSystem(t *testing.T) {
}
}
// testEphemeralMigrate tests that ephermeral_disk migrations work as expected
// even during a node drain.
func testEphemeralMigrate(t *testing.T) {
t.Cleanup(cleanupDrainState(t))
nomadClient := e2eutil.NomadClient(t)
jobID := "drain-migrate-" + uuid.Short()
allocs := registerAndWaitForRunning(t, nomadClient, jobID, "./input/drain_migrate.nomad", 1)
t.Cleanup(cleanupJobState(t, jobID))
oldAllocID := allocs[0].ID
oldNodeID := allocs[0].NodeID
// make sure the allocation has written its ID to disk so we have something to migrate
must.Wait(t, wait.InitialSuccess(
wait.ErrorFunc(func() error {
got, err := e2eutil.Command("nomad", "alloc", "fs", oldAllocID,
"alloc/data/migrate.txt")
if err != nil {
return fmt.Errorf("did not expect error reading alloc fs: %v", err)
}
if !strings.Contains(got, oldAllocID) {
return fmt.Errorf("expected data to be written for alloc %q", oldAllocID)
}
return nil
}),
wait.Timeout(10*time.Second),
wait.Gap(500*time.Millisecond),
))
out, err := e2eutil.Command("nomad", "node", "drain", "-enable", "-yes", "-detach", oldNodeID)
must.NoError(t, err, must.Sprintf("expected no error when marking node for drain: %v", out))
newAllocs := waitForAllocDrain(t, nomadClient, jobID, oldAllocID, oldNodeID)
must.Len(t, 1, newAllocs, must.Sprint("expected 1 new alloc"))
newAllocID := newAllocs[0].ID
newNodeID := newAllocs[0].NodeID
// although migrate=true implies sticky=true, the drained node is ineligible
// for scheduling so the alloc should have been migrated
must.NotEq(t, oldNodeID, newNodeID, must.Sprint("new alloc was placed on draining node"))
// once the new allocation is running, it should quickly have the right data
must.Wait(t, wait.InitialSuccess(
wait.ErrorFunc(func() error {
got, err := e2eutil.Command("nomad", "alloc", "fs", newAllocID,
"alloc/data/migrate.txt")
if err != nil {
return fmt.Errorf("did not expect error reading alloc fs: %v", err)
}
if !strings.Contains(got, oldAllocID) || !strings.Contains(got, newAllocID) {
return fmt.Errorf(
"expected data to be migrated from alloc=%s on node=%s to alloc=%s on node=%s but got:\n%q",
oldAllocID[:8], oldNodeID[:8], newAllocID[:8], newNodeID[:8], got)
}
return nil
}),
wait.Timeout(10*time.Second),
wait.Gap(500*time.Millisecond),
))
}
// testKeepIneligible tests that nodes can be kept ineligible for scheduling after
// disabling drain.
func testKeepIneligible(t *testing.T) {
@ -172,7 +237,7 @@ func waitForAllocDrain(t *testing.T, nomadClient *api.Client, jobID, oldAllocID,
}
}
t.Logf("alloc has drained from node=%s after %v",
oldNodeID, time.Now().Sub(start))
oldNodeID[:8], time.Now().Sub(start))
return nil
}),
wait.Timeout(120*time.Second),

View File

@ -3,7 +3,6 @@ package nodedrain
import (
"fmt"
"os"
"strings"
"time"
e2e "github.com/hashicorp/nomad/e2e/e2eutil"
@ -95,73 +94,6 @@ func waitForNodeDrain(nodeID string, comparison func([]map[string]string) bool,
return err
}
// TestNodeDrainEphemeralMigrate tests that ephermeral_disk migrations work as
// expected even during a node drain.
func (tc *NodeDrainE2ETest) TestNodeDrainEphemeralMigrate(f *framework.F) {
jobID := "test-node-drain-" + uuid.Generate()[0:8]
f.NoError(e2e.Register(jobID, "nodedrain/input/drain_migrate.nomad"))
tc.jobIDs = append(tc.jobIDs, jobID)
expected := []string{"running"}
f.NoError(e2e.WaitForAllocStatusExpected(jobID, ns, expected), "job should be running")
allocs, err := e2e.AllocsForJob(jobID, ns)
f.NoError(err, "could not get allocs for job")
f.Len(allocs, 1, "could not get allocs for job")
oldAllocID := allocs[0]["ID"]
nodes, err := nodesForJob(jobID)
f.NoError(err, "could not get nodes for job")
f.Len(nodes, 1, "could not get nodes for job")
nodeID := nodes[0]
out, err := e2e.Command("nomad", "node", "drain", "-enable", "-yes", "-detach", nodeID)
f.NoError(err, fmt.Sprintf("'nomad node drain' failed: %v\n%v", err, out))
tc.nodeIDs = append(tc.nodeIDs, nodeID)
f.NoError(waitForNodeDrain(nodeID,
func(got []map[string]string) bool {
for _, alloc := range got {
if alloc["ID"] == oldAllocID && alloc["Status"] == "complete" {
return true
}
}
return false
}, &e2e.WaitConfig{Interval: time.Millisecond * 100, Retries: 500},
), "node did not drain")
// wait for the allocation to be migrated
expected = []string{"running", "complete"}
f.NoError(e2e.WaitForAllocStatusExpected(jobID, ns, expected), "job should be running")
allocs, err = e2e.AllocsForJob(jobID, ns)
f.NoError(err, "could not get allocations for job")
// the task writes its alloc ID to a file if it hasn't been previously
// written, so find the contents of the migrated file and make sure they
// match the old allocation, not the running one
var got string
var fsErr error
testutil.WaitForResultRetries(10, func() (bool, error) {
time.Sleep(time.Millisecond * 100)
for _, alloc := range allocs {
if alloc["Status"] == "running" && alloc["Node ID"] != nodeID && alloc["ID"] != oldAllocID {
got, fsErr = e2e.Command("nomad", "alloc", "fs",
alloc["ID"], fmt.Sprintf("alloc/data/%s", jobID))
if err != nil {
return false, err
}
return true, nil
}
}
return false, fmt.Errorf("missing expected allocation")
}, func(e error) {
fsErr = e
})
f.NoError(fsErr, "could not get allocation data")
f.Equal(oldAllocID, strings.TrimSpace(got), "node drained but migration failed")
}
// TestNodeDrainDeadline tests the enforcement of the node drain deadline so
// that allocations are terminated even if they haven't gracefully exited.
func (tc *NodeDrainE2ETest) TestNodeDrainDeadline(f *framework.F) {

View File

@ -10634,13 +10634,8 @@ func (a *Allocation) ShouldMigrate() bool {
return false
}
// We won't migrate any data is the user hasn't enabled migration or the
// disk is not marked as sticky
if !tg.EphemeralDisk.Migrate || !tg.EphemeralDisk.Sticky {
return false
}
return true
// We won't migrate any data if the user hasn't enabled migration
return tg.EphemeralDisk.Migrate
}
// SetEventDisplayMessages populates the display message if its not already set,

View File

@ -4408,97 +4408,100 @@ func TestTaskArtifact_Hash(t *testing.T) {
func TestAllocation_ShouldMigrate(t *testing.T) {
ci.Parallel(t)
alloc := Allocation{
PreviousAllocation: "123",
TaskGroup: "foo",
Job: &Job{
TaskGroups: []*TaskGroup{
{
Name: "foo",
EphemeralDisk: &EphemeralDisk{
Migrate: true,
Sticky: true,
testCases := []struct {
name string
expect bool
alloc Allocation
}{
{
name: "should migrate with previous alloc and migrate=true sticky=true",
expect: true,
alloc: Allocation{
PreviousAllocation: "123",
TaskGroup: "foo",
Job: &Job{
TaskGroups: []*TaskGroup{
{
Name: "foo",
EphemeralDisk: &EphemeralDisk{
Migrate: true,
Sticky: true,
},
},
},
},
},
},
{
name: "should not migrate with migrate=false sticky=false",
expect: false,
alloc: Allocation{
PreviousAllocation: "123",
TaskGroup: "foo",
Job: &Job{
TaskGroups: []*TaskGroup{
{
Name: "foo",
EphemeralDisk: &EphemeralDisk{},
},
},
},
},
},
{
name: "should migrate with migrate=true sticky=false",
expect: true,
alloc: Allocation{
PreviousAllocation: "123",
TaskGroup: "foo",
Job: &Job{
TaskGroups: []*TaskGroup{
{
Name: "foo",
EphemeralDisk: &EphemeralDisk{
Sticky: false,
Migrate: true,
},
},
},
},
},
},
{
name: "should not migrate with nil ephemeral disk",
expect: false,
alloc: Allocation{
PreviousAllocation: "123",
TaskGroup: "foo",
Job: &Job{
TaskGroups: []*TaskGroup{{Name: "foo"}},
},
},
},
{
name: "should not migrate without previous alloc",
expect: false,
alloc: Allocation{
TaskGroup: "foo",
Job: &Job{
TaskGroups: []*TaskGroup{
{
Name: "foo",
EphemeralDisk: &EphemeralDisk{
Migrate: true,
Sticky: true,
},
},
},
},
},
},
}
if !alloc.ShouldMigrate() {
t.Fatalf("bad: %v", alloc)
}
alloc1 := Allocation{
PreviousAllocation: "123",
TaskGroup: "foo",
Job: &Job{
TaskGroups: []*TaskGroup{
{
Name: "foo",
EphemeralDisk: &EphemeralDisk{},
},
},
},
}
if alloc1.ShouldMigrate() {
t.Fatalf("bad: %v", alloc)
}
alloc2 := Allocation{
PreviousAllocation: "123",
TaskGroup: "foo",
Job: &Job{
TaskGroups: []*TaskGroup{
{
Name: "foo",
EphemeralDisk: &EphemeralDisk{
Sticky: false,
Migrate: true,
},
},
},
},
}
if alloc2.ShouldMigrate() {
t.Fatalf("bad: %v", alloc)
}
alloc3 := Allocation{
PreviousAllocation: "123",
TaskGroup: "foo",
Job: &Job{
TaskGroups: []*TaskGroup{
{
Name: "foo",
},
},
},
}
if alloc3.ShouldMigrate() {
t.Fatalf("bad: %v", alloc)
}
// No previous
alloc4 := Allocation{
TaskGroup: "foo",
Job: &Job{
TaskGroups: []*TaskGroup{
{
Name: "foo",
EphemeralDisk: &EphemeralDisk{
Migrate: true,
Sticky: true,
},
},
},
},
}
if alloc4.ShouldMigrate() {
t.Fatalf("bad: %v", alloc4)
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
must.Eq(t, tc.expect, tc.alloc.ShouldMigrate())
})
}
}

View File

@ -789,7 +789,11 @@ func updateRescheduleTracker(alloc *structs.Allocation, prev *structs.Allocation
// findPreferredNode finds the preferred node for an allocation
func (s *GenericScheduler) findPreferredNode(place placementResult) (*structs.Node, error) {
if prev := place.PreviousAllocation(); prev != nil && place.TaskGroup().EphemeralDisk.Sticky {
prev := place.PreviousAllocation()
if prev == nil {
return nil, nil
}
if place.TaskGroup().EphemeralDisk.Sticky || place.TaskGroup().EphemeralDisk.Migrate {
var preferredNode *structs.Node
ws := memdb.NewWatchSet()
preferredNode, err := s.state.NodeByID(ws, prev.NodeID)

View File

@ -32,15 +32,15 @@ be found in the [filesystem internals][].
## `ephemeral_disk` Parameters
- `migrate` `(bool: false)` - When `sticky` is true, this specifies that the
Nomad client should make a best-effort attempt to migrate the data from a
remote machine if placement cannot be made on the original node. During data
migration, the task will block starting until the data migration has
completed. Migration is atomic and any partially migrated data will be removed
if an error is encountered. Note that data migration will not take place if a
client garbage collects a failed allocation or if the allocation has been
intentionally stopped via `nomad alloc stop`, because the original allocation
has already been removed.
- `migrate` `(bool: false)` - This specifies that the Nomad client should make a
best-effort attempt to migrate the data from the previous allocation, even if
the previous allocation was on another client. Enabling `migrate`
automatically enables `sticky` as well. During data migration, the task will
block starting until the data migration has completed. Migration is atomic and
any partially migrated data will be removed if an error is encountered. Note
that data migration will not take place if a client garbage collects a failed
allocation or if the allocation has been intentionally stopped via `nomad
alloc stop`, because the original allocation has already been removed.
- `size` `(int: 300)` - Specifies the size of the ephemeral disk in MB. The
current Nomad ephemeral storage implementation does not enforce this limit;