You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2014/08/08 03:00:27 UTC

git commit: Drop HostAttributes with no slaveId when recovering from persistent storage.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master b0fef4b20 -> 5154f61e7


Drop HostAttributes with no slaveId when recovering from persistent storage.

Bugs closed: AURORA-641

Reviewed at https://reviews.apache.org/r/24486/


Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/5154f61e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/5154f61e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/5154f61e

Branch: refs/heads/master
Commit: 5154f61e7534f7f9c22d4760a6e12302b89c7dc5
Parents: b0fef4b
Author: Bill Farner <wf...@apache.org>
Authored: Thu Aug 7 17:57:59 2014 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Thu Aug 7 17:57:59 2014 -0700

----------------------------------------------------------------------
 .../scheduler/storage/log/LogStorage.java       | 12 +++++++++--
 .../storage/log/SnapshotStoreImpl.java          |  9 ++++++++-
 .../scheduler/storage/log/LogStorageTest.java   | 21 ++++++++++++++++++--
 .../storage/log/SnapshotStoreImplTest.java      | 14 +++++++++----
 4 files changed, 47 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5154f61e/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java b/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
index 3454420..342bab0 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
@@ -36,6 +36,7 @@ import com.twitter.common.quantity.Time;
 import com.twitter.common.util.concurrent.ExecutorServiceShutdown;
 
 import org.apache.aurora.codec.ThriftBinaryCodec.CodingException;
+import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.storage.LogEntry;
 import org.apache.aurora.gen.storage.Op;
 import org.apache.aurora.gen.storage.RewriteTask;
@@ -439,8 +440,15 @@ public class LogStorage implements NonVolatileStorage, DistributedSnapshotStore
         break;
 
       case SAVE_HOST_ATTRIBUTES:
-        writeBehindAttributeStore.saveHostAttributes(
-            IHostAttributes.build(op.getSaveHostAttributes().hostAttributes));
+        HostAttributes attributes = op.getSaveHostAttributes().hostAttributes;
+        // Prior to commit 5cf760b, the store would persist maintenance mode changes for
+        // unknown hosts.  5cf760b began rejecting these, but the replicated log may still
+        // contain entries with a null slave ID.
+        if (attributes.isSetSlaveId()) {
+          writeBehindAttributeStore.saveHostAttributes(IHostAttributes.build(attributes));
+        } else {
+          LOG.info("Dropping host attributes with no slave ID: " + attributes);
+        }
         break;
 
       case SAVE_LOCK:

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5154f61e/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java b/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
index 5d3ec36..e921e71 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
@@ -80,7 +80,14 @@ public class SnapshotStoreImpl implements SnapshotStore<Snapshot> {
 
           if (snapshot.isSetHostAttributes()) {
             for (HostAttributes attributes : snapshot.getHostAttributes()) {
-              store.getAttributeStore().saveHostAttributes(IHostAttributes.build(attributes));
+              // Prior to commit 5cf760b, the store would persist maintenance mode changes for
+              // unknown hosts.  5cf760b began rejecting these, but the replicated log may still
+              // contain entries with a null slave ID.
+              if (attributes.isSetSlaveId()) {
+                store.getAttributeStore().saveHostAttributes(IHostAttributes.build(attributes));
+              } else {
+                LOG.info("Dropping host attributes with no slave ID: " + attributes);
+              }
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5154f61e/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java b/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
index a9545e1..8f2be53 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
@@ -45,6 +45,7 @@ import org.apache.aurora.gen.JobUpdateStatus;
 import org.apache.aurora.gen.JobUpdateSummary;
 import org.apache.aurora.gen.Lock;
 import org.apache.aurora.gen.LockKey;
+import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.gen.Range;
 import org.apache.aurora.gen.ResourceAggregate;
 import org.apache.aurora.gen.ScheduleStatus;
@@ -186,15 +187,30 @@ public class LogStorageTest extends EasyMockTest {
 
     Entry entry1 = createMock(Entry.class);
     Entry entry2 = createMock(Entry.class);
+    Entry entry3 = createMock(Entry.class);
+    Entry entry4 = createMock(Entry.class);
     String frameworkId1 = "bob";
     LogEntry recoveredEntry1 =
         createTransaction(Op.saveFrameworkId(new SaveFrameworkId(frameworkId1)));
     String frameworkId2 = "jim";
     LogEntry recoveredEntry2 =
         createTransaction(Op.saveFrameworkId(new SaveFrameworkId(frameworkId2)));
+    // This entry lacks a slave ID, and should therefore be discarded.
+    LogEntry recoveredEntry3 =
+        createTransaction(Op.saveHostAttributes(new SaveHostAttributes(new HostAttributes()
+            .setHost("host1")
+            .setMode(MaintenanceMode.DRAINED))));
+    IHostAttributes attributes = IHostAttributes.build(new HostAttributes()
+        .setHost("host2")
+        .setSlaveId("slave2")
+        .setMode(MaintenanceMode.DRAINED));
+    LogEntry recoveredEntry4 =
+        createTransaction(Op.saveHostAttributes(new SaveHostAttributes(attributes.newBuilder())));
     expect(entry1.contents()).andReturn(ThriftBinaryCodec.encodeNonNull(recoveredEntry1));
     expect(entry2.contents()).andReturn(ThriftBinaryCodec.encodeNonNull(recoveredEntry2));
-    expect(stream.readAll()).andReturn(Iterators.forArray(entry1, entry2));
+    expect(entry3.contents()).andReturn(ThriftBinaryCodec.encodeNonNull(recoveredEntry3));
+    expect(entry4.contents()).andReturn(ThriftBinaryCodec.encodeNonNull(recoveredEntry4));
+    expect(stream.readAll()).andReturn(Iterators.forArray(entry1, entry2, entry3, entry4));
 
     final Capture<MutateWork<Void, RuntimeException>> recoveryWork = createCapture();
     expect(storageUtil.storage.write(capture(recoveryWork))).andAnswer(
@@ -207,6 +223,7 @@ public class LogStorageTest extends EasyMockTest {
         });
     storageUtil.schedulerStore.saveFrameworkId(frameworkId1);
     storageUtil.schedulerStore.saveFrameworkId(frameworkId2);
+    storageUtil.attributeStore.saveHostAttributes(attributes);
 
     final Capture<MutateWork<Void, RuntimeException>> initializationWork = createCapture();
     expect(storageUtil.storage.write(capture(initializationWork))).andAnswer(
@@ -238,7 +255,7 @@ public class LogStorageTest extends EasyMockTest {
             snapshotWork.getValue().apply(storageUtil.mutableStoreProvider);
             return null;
           }
-        }).times(2);
+        }).times(4);
 
     control.replay();
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5154f61e/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
index 48600b5..9b5ba5b 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
@@ -80,8 +80,13 @@ public class SnapshotStoreImplTest extends EasyMockTest {
     Set<QuotaConfiguration> quotas =
         ImmutableSet.of(
             new QuotaConfiguration("steve", ResourceAggregates.none().newBuilder()));
-    IHostAttributes attribute = IHostAttributes.build(new HostAttributes("host",
-        ImmutableSet.of(new Attribute("attr", ImmutableSet.of("value")))));
+    IHostAttributes attribute = IHostAttributes.build(
+        new HostAttributes("host", ImmutableSet.of(new Attribute("attr", ImmutableSet.of("value"))))
+            .setSlaveId("slave id"));
+    // A legacy attribute that has a maintenance mode set, but nothing else.  These should be
+    // dropped.
+    IHostAttributes legacyAttribute = IHostAttributes.build(
+        new HostAttributes("host", ImmutableSet.<Attribute>of()));
     StoredJob job = new StoredJob(
         "jobManager",
         new JobConfiguration().setKey(new JobKey("owner", "env", "name")));
@@ -104,7 +109,8 @@ public class SnapshotStoreImplTest extends EasyMockTest {
     expect(storageUtil.taskStore.fetchTasks(Query.unscoped())).andReturn(tasks);
     expect(storageUtil.quotaStore.fetchQuotas())
         .andReturn(ImmutableMap.of("steve", ResourceAggregates.none()));
-    expect(storageUtil.attributeStore.getHostAttributes()).andReturn(ImmutableSet.of(attribute));
+    expect(storageUtil.attributeStore.getHostAttributes())
+        .andReturn(ImmutableSet.of(attribute, legacyAttribute));
     expect(storageUtil.jobStore.fetchManagerIds()).andReturn(ImmutableSet.of("jobManager"));
     expect(storageUtil.jobStore.fetchJobs("jobManager"))
         .andReturn(ImmutableSet.of(IJobConfiguration.build(job.getJobConfiguration())));
@@ -136,7 +142,7 @@ public class SnapshotStoreImplTest extends EasyMockTest {
         .setTimestamp(NOW)
         .setTasks(IScheduledTask.toBuildersSet(tasks))
         .setQuotaConfigurations(quotas)
-        .setHostAttributes(ImmutableSet.of(attribute.newBuilder()))
+        .setHostAttributes(ImmutableSet.of(attribute.newBuilder(), legacyAttribute.newBuilder()))
         .setJobs(ImmutableSet.of(job))
         .setSchedulerMetadata(metadata)
         .setLocks(ImmutableSet.of(lock.newBuilder()))