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 2018/01/04 16:06:38 UTC

[2/2] aurora git commit: Add a test to detect incompatible storage changes

Add a test to detect incompatible storage changes

This is intended as a safeguard against future compatibility regressions like
[AURORA-1959](https://issues.apache.org/jira/browse/AURORA-1959).

I approached this with a few goals:

  - golden files should be text-based and human-readable.  This allows for
    non-opaque code reviews, and simpler remedy when it's necessary to update
    the goldens (i.e. copy-pasteable)
  - guidance for schema evolution should be included directly in test failures
  - separate detection of 'what the scheduler _can_ read' and 'what the
    scheduler writes'
  - reasonably-complete schema coverage with minimal manual labor.  These tests
    auto-generate structs to mitigate maintenance burden of test code as
    schemas evolve.

This is not a replacement for vigilance with data compatibility, but it should
at least

1. mitigate unintentional breakages in compatibility, especially for new
   contributors
2. draw code reviewer attention to compatibility changes in a patch (signaled by
   changes to golden files)

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


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

Branch: refs/heads/master
Commit: 8e5e08ebfe6d848beb4e6037d3ab67e49321ae42
Parents: f1d9caf
Author: Bill Farner <wf...@apache.org>
Authored: Thu Jan 4 08:02:55 2018 -0800
Committer: Bill Farner <wf...@apache.org>
Committed: Thu Jan 4 08:02:55 2018 -0800

----------------------------------------------------------------------
 .../durability/DataCompatibilityTest.java       | 376 +++++++++++++++
 .../scheduler/storage/durability/Generator.java | 142 ++++++
 .../goldens/current/pruneJobUpdateHistory       |  12 +
 .../durability/goldens/current/removeJob        |  19 +
 .../durability/goldens/current/removeJobUpdate  |  30 ++
 .../durability/goldens/current/removeLock       |  23 +
 .../durability/goldens/current/removeQuota      |   9 +
 .../durability/goldens/current/removeTasks      |  13 +
 .../durability/goldens/current/saveCronJob      | 200 ++++++++
 .../durability/goldens/current/saveFrameworkId  |   9 +
 .../goldens/current/saveHostAttributes          |  37 ++
 .../goldens/current/saveJobInstanceUpdateEvent  |  39 ++
 .../durability/goldens/current/saveJobUpdate    | 469 +++++++++++++++++++
 .../goldens/current/saveJobUpdateEvent          |  42 ++
 .../storage/durability/goldens/current/saveLock |  39 ++
 .../durability/goldens/current/saveQuota        |  33 ++
 .../durability/goldens/current/saveTasks        | 233 +++++++++
 .../read-compatible/1-pruneJobUpdateHistory     |  12 +
 .../goldens/read-compatible/10-saveJobUpdate    | 469 +++++++++++++++++++
 .../read-compatible/11-saveJobUpdateEvent       |  42 ++
 .../12-saveJobInstanceUpdateEvent               |  39 ++
 .../goldens/read-compatible/13-saveLock         |  39 ++
 .../goldens/read-compatible/14-saveQuota        |  43 ++
 .../goldens/read-compatible/15-saveTasks        | 233 +++++++++
 .../goldens/read-compatible/2-removeJob         |  19 +
 .../goldens/read-compatible/3-removeJobUpdate   |  30 ++
 .../goldens/read-compatible/4-removeLock        |  23 +
 .../goldens/read-compatible/5-removeQuota       |   9 +
 .../goldens/read-compatible/6-removeTasks       |  13 +
 .../goldens/read-compatible/7-saveCronJob       | 200 ++++++++
 .../goldens/read-compatible/8-saveFrameworkId   |   9 +
 .../read-compatible/9-saveHostAttributes        |  37 ++
 32 files changed, 2942 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/java/org/apache/aurora/scheduler/storage/durability/DataCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/durability/DataCompatibilityTest.java b/src/test/java/org/apache/aurora/scheduler/storage/durability/DataCompatibilityTest.java
new file mode 100644
index 0000000..31f9545
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/storage/durability/DataCompatibilityTest.java
@@ -0,0 +1,376 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.aurora.scheduler.storage.durability;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Type;
+import java.net.URL;
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.MapDifference.ValueDifference;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Ordering;
+import com.google.common.collect.Streams;
+import com.google.common.io.Files;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonParser;
+import com.google.inject.AbstractModule;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+
+import org.apache.aurora.common.inject.Bindings;
+import org.apache.aurora.common.stats.StatsProvider;
+import org.apache.aurora.gen.Resource;
+import org.apache.aurora.gen.ResourceAggregate;
+import org.apache.aurora.gen.storage.Op;
+import org.apache.aurora.gen.storage.PruneJobUpdateHistory;
+import org.apache.aurora.gen.storage.RemoveJob;
+import org.apache.aurora.gen.storage.RemoveJobUpdates;
+import org.apache.aurora.gen.storage.RemoveLock;
+import org.apache.aurora.gen.storage.RemoveQuota;
+import org.apache.aurora.gen.storage.RemoveTasks;
+import org.apache.aurora.gen.storage.SaveCronJob;
+import org.apache.aurora.gen.storage.SaveFrameworkId;
+import org.apache.aurora.gen.storage.SaveHostAttributes;
+import org.apache.aurora.gen.storage.SaveJobInstanceUpdateEvent;
+import org.apache.aurora.gen.storage.SaveJobUpdate;
+import org.apache.aurora.gen.storage.SaveJobUpdateEvent;
+import org.apache.aurora.gen.storage.SaveLock;
+import org.apache.aurora.gen.storage.SaveQuota;
+import org.apache.aurora.gen.storage.SaveTasks;
+import org.apache.aurora.scheduler.TierInfo;
+import org.apache.aurora.scheduler.TierManager.TierManagerImpl.TierConfig;
+import org.apache.aurora.scheduler.TierModule;
+import org.apache.aurora.scheduler.app.LifecycleModule;
+import org.apache.aurora.scheduler.events.EventSink;
+import org.apache.aurora.scheduler.storage.Storage.NonVolatileStorage;
+import org.apache.aurora.scheduler.storage.Storage.Volatile;
+import org.apache.aurora.scheduler.storage.mem.MemStorageModule;
+import org.apache.aurora.scheduler.testing.FakeStatsProvider;
+import org.apache.thrift.TDeserializer;
+import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
+import org.apache.thrift.TUnion;
+import org.apache.thrift.protocol.TJSONProtocol;
+import org.junit.Test;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import static org.apache.aurora.scheduler.storage.durability.Generator.newStruct;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class DataCompatibilityTest {
+
+  private NonVolatileStorage createStorage(Persistence persistence) {
+    Injector injector = Guice.createInjector(
+        new DurableStorageModule(),
+        new MemStorageModule(Bindings.annotatedKeyFactory(Volatile.class)),
+        new LifecycleModule(),
+        new TierModule(new TierConfig(
+            "string-value",
+            ImmutableMap.of("string-value", new TierInfo(false, false)))),
+        new AbstractModule() {
+          @Override
+          protected void configure() {
+            bind(StatsProvider.class).toInstance(new FakeStatsProvider());
+            bind(EventSink.class).toInstance(event -> { });
+            bind(Persistence.class).toInstance(persistence);
+          }
+        });
+    return injector.getInstance(NonVolatileStorage.class);
+  }
+
+  /**
+   * Ops to serve as a reference for the replacement golden values when read compatibility changes.
+   */
+  private static final List<Op> READ_COMPATIBILITY_OPS = ImmutableList.of(
+      Op.pruneJobUpdateHistory(newStruct(PruneJobUpdateHistory.class)),
+      Op.removeJob(newStruct(RemoveJob.class)),
+      Op.removeJobUpdate(newStruct(RemoveJobUpdates.class)),
+      Op.removeLock(newStruct(RemoveLock.class)),
+      Op.removeQuota(newStruct(RemoveQuota.class)),
+      Op.removeTasks(newStruct(RemoveTasks.class)),
+      Op.saveCronJob(newStruct(SaveCronJob.class)),
+      Op.saveFrameworkId(newStruct(SaveFrameworkId.class)),
+      Op.saveHostAttributes(newStruct(SaveHostAttributes.class)),
+      Op.saveJobUpdate(newStruct(SaveJobUpdate.class)),
+      Op.saveJobInstanceUpdateEvent(newStruct(SaveJobInstanceUpdateEvent.class)),
+      Op.saveJobUpdateEvent(newStruct(SaveJobUpdateEvent.class)),
+      Op.saveLock(newStruct(SaveLock.class)),
+      Op.saveQuota(new SaveQuota()
+          .setRole("role")
+          .setQuota(new ResourceAggregate()
+              .setResources(ImmutableSet.of(
+                  Resource.numCpus(2.0),
+                  Resource.diskMb(1),
+                  Resource.ramMb(1))))),
+      Op.saveTasks(newStruct(SaveTasks.class)));
+
+  @Test
+  public void testReadCompatibility() {
+    // Verifies that storage can recover known-good serialized records.  A failure of this test case
+    // indicates that the scheduler can no longer read a record that it was expected to in the past.
+    // Golden values in `goldens/read-compatible` preserve serialized records that the scheduler is
+    // expected to read.  At the end of a deprecation cycle, these files may need to be updated.
+    // Golden file names are prefixed with an ordering ID (e.g. 2-removeJob) to prescribe a recovery
+    // order.  This is This is necessary to accommodate Ops with relations (e.g. update events
+    // relate to an update).
+
+    // Sanity check that the current read-compatibility values can be replayed.
+    NonVolatileStorage storage = createStorage(new TestPersistence(READ_COMPATIBILITY_OPS));
+    storage.prepare();
+    storage.start(stores -> { });
+    storage.stop();
+
+    File goldensDir = getGoldensDir("read-compatible");
+    List<Op> goldenOps = loadGoldenSchemas(goldensDir).entrySet().stream()
+        .sorted(Ordering.natural().onResultOf(entry ->
+            Integer.parseInt(entry.getKey().split("\\-")[0])))
+        .map(Entry::getValue)
+        .map(DataCompatibilityTest::deserialize)
+        .collect(Collectors.toList());
+
+    // Ensure all currently-known Op types are represented in the goldens.
+    assertEquals(
+        ImmutableSet.copyOf(Op._Fields.values()),
+        goldenOps.stream()
+            .map(TUnion::getSetField)
+            .collect(Collectors.toSet()));
+
+    // Introduce each op one at a time to pinpoint a specific failed op.
+    IntStream.range(1, goldenOps.size())
+        .forEach(i -> {
+          NonVolatileStorage store = createStorage(new TestPersistence(goldenOps.subList(0, i)));
+          store.prepare();
+          try {
+            store.start(stores -> { });
+          } catch (RuntimeException e) {
+            Op failedOp = goldenOps.get(i - 1);
+            Op currentOp = READ_COMPATIBILITY_OPS.stream()
+                .filter(op -> op.getSetField() == failedOp.getSetField())
+                .findFirst()
+                .get();
+            StringBuilder error = new StringBuilder()
+                .append("**** Storage compatibility change detected ****")
+                .append("\nFailed to recover when introducing ")
+                .append(failedOp.getSetField().getFieldName())
+                .append("\n")
+                .append(failedOp)
+                .append("\nIf this is expected, you may delete the associated golden file from ")
+                .append(goldensDir.getPath())
+                .append(",\nor you may replace the file with the latest serialized value:")
+                .append("\n")
+                .append(serialize(currentOp));
+            fail(error.toString());
+          }
+          store.stop();
+        });
+  }
+
+  private static File getGoldensDir(String kind) {
+    ClassLoader loader = Thread.currentThread().getContextClassLoader();
+    URL url = loader.getResource(
+        DataCompatibilityTest.class.getPackage().getName().replaceAll("\\.", "/")
+            + "/goldens/" + kind);
+    return new File(url.getPath());
+  }
+
+  private static Map<String, String> loadGoldenSchemas(File goldensDir) {
+    return Stream.of(goldensDir.listFiles())
+        .collect(Collectors.toMap(
+            File::getName,
+            goldenFile -> {
+              try {
+                return Files.asCharSource(goldenFile, UTF_8).read();
+              } catch (IOException e) {
+                throw new UncheckedIOException(e);
+              }
+            }
+        ));
+  }
+
+  private static Map<String, String> generateOpSchemas() {
+    return Stream.of(Op._Fields.values())
+        .map(field -> {
+          Method factory = Stream.of(Op.class.getDeclaredMethods())
+              .filter(method -> method.getName().equals(field.getFieldName()))
+              .findFirst()
+              .get();
+
+          Class<?> paramType = factory.getParameterTypes()[0];
+          Type genericParamType = factory.getGenericParameterTypes()[0];
+          try {
+            return (Op) factory.invoke(null, Generator.valueFor(paramType, genericParamType));
+          } catch (ReflectiveOperationException e) {
+            throw new RuntimeException(e);
+          }
+        })
+        .collect(Collectors.toMap(
+            op -> op.getSetField().getFieldName(),
+            DataCompatibilityTest::serialize
+        ));
+  }
+
+  @Test
+  public void testWriteFormatUnchanged() {
+    // Attempts to flag any changes in the storage format.  While thorough, this check is not
+    // complete.  It attempts to capture the entire schema by synthesizing a fully-populated
+    // instance of each Op type.  For TUnions, the struct generator picks an arbitrary field to set,
+    // meaning that it will only see one of the multiple possible schemas for any given TUnion.
+    // These generated structs effectively give a view of the struct schema, which is compared to
+    // golden files in `goldens/current`.
+
+    Map<String, String> schemasByName = generateOpSchemas();
+    File goldensDir = getGoldensDir("current");
+    Map<String, String> goldensByName = loadGoldenSchemas(goldensDir);
+
+    MapDifference<String, String> difference = Maps.difference(goldensByName, schemasByName);
+    if (difference.areEqual()) {
+      return;
+    }
+
+    StringBuilder error = new StringBuilder();
+    StringBuilder remedy = new StringBuilder();
+
+    Set<String> removedOps = difference.entriesOnlyOnLeft().keySet();
+    if (!removedOps.isEmpty()) {
+      error.append("Removal of storage Op(s): ").append(removedOps)
+          .append("\nOps may only be removed after a release that")
+          .append("\n  * formally deprecates the Op in release notes")
+          .append("\n  * performs a no-op read of the Op type")
+          .append("\n  * included warning logging when the Op was read")
+          .append("\n  * ensures the Op is removed from storage")
+          .append("\n\nHowever, you should also consider leaving the Op indefinitely and removing")
+          .append("\nall fields as a safer alternative.");
+
+      remedy.append("deleting the files ")
+          .append(removedOps.stream()
+              .map(removed -> new File(goldensDir, removed).getAbsolutePath())
+              .collect(Collectors.joining(", ")));
+    }
+
+    String goldenChangeInstructions = Streams.concat(
+        difference.entriesOnlyOnRight().entrySet().stream(),
+        difference.entriesDiffering().entrySet().stream()
+            .map(entry ->
+                new SimpleImmutableEntry<>(entry.getKey(), entry.getValue().rightValue())))
+        .map(entry -> new StringBuilder()
+            .append("\n").append(new File(goldensDir, entry.getKey()).getPath()).append(":")
+            .append("\n").append(entry.getValue())
+            .toString())
+        .collect(Collectors.joining("\n"));
+
+    Set<String> addedOps = difference.entriesOnlyOnRight().keySet();
+    if (!addedOps.isEmpty()) {
+      error.append("Addition of storage Op(s): ").append(addedOps)
+          .append("\nOps may only be introduced")
+          .append("\n  a.) in a release that supports reading but not writing the Op")
+          .append("\n  b.) in a release that writes the Op only with an operator-controlled flag");
+
+      remedy.append("creating the following files")
+          .append(goldenChangeInstructions);
+    }
+
+    Map<String, ValueDifference<String>> modified = difference.entriesDiffering();
+    if (!modified.isEmpty()) {
+      error.append("Schema changes to Op(s): " + modified.keySet())
+          .append("\nThis check detects that changes occured, not how the schema changed.")
+          .append("\nSome guidelines for evolving schemas:")
+          .append("\n  * Introducing fields: you must handle reading records that do not")
+          .append("\n    yet have the field set.  This can be done with a backfill routine during")
+          .append("\n    storage recovery if a field is required in some parts of the code")
+          .append("\n  * Removing fields: must only be done after a release in which the field")
+          .append("\n    is unused and announced as deprecated")
+          .append("\n  * Changed fields: the type or thrift field ID of a field must never change");
+
+      remedy.append("changing the following files")
+          .append(goldenChangeInstructions);
+    }
+
+    fail(new StringBuilder()
+        .append("**** Storage compatibility change detected ****")
+        .append("\n")
+        .append(error)
+        .append("\n\nIf the necessary compatibility procedures have been performed,")
+        .append("\nyou may clear this check by ")
+        .append(remedy)
+        .toString());
+  }
+
+  private static class TestPersistence implements Persistence {
+    private final List<Op> ops;
+
+    TestPersistence(List<Op> ops) {
+      this.ops = ops;
+    }
+
+    @Override
+    public void prepare() {
+      // No-op.
+    }
+
+    @Override
+    public Stream<Edit> recover() {
+      return ops.stream().map(Edit::op);
+    }
+
+    @Override
+    public void persist(Stream<Op> records) {
+      // no-op.
+    }
+  }
+
+  private static String serialize(Op op) {
+    try {
+      String unformattedJson =
+          new String(new TSerializer(new TJSONProtocol.Factory()).serialize(op), UTF_8);
+
+      // Pretty print the json for easier review of diffs.
+      return new GsonBuilder().setPrettyPrinting().create()
+          .toJson(new JsonParser().parse(unformattedJson)) + "\n";
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static Op deserialize(String serializedOp) {
+    try {
+      Op op = new Op();
+
+      String nonPrettyJson = new GsonBuilder().create()
+          .toJson(new JsonParser().parse(serializedOp));
+
+      new TDeserializer(new TJSONProtocol.Factory())
+          .deserialize(op, nonPrettyJson.getBytes(UTF_8));
+      return op;
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/java/org/apache/aurora/scheduler/storage/durability/Generator.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/durability/Generator.java b/src/test/java/org/apache/aurora/scheduler/storage/durability/Generator.java
new file mode 100644
index 0000000..edd46a8
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/storage/durability/Generator.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.aurora.scheduler.storage.durability;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Ordering;
+
+import org.apache.thrift.TBase;
+import org.apache.thrift.TEnum;
+import org.apache.thrift.TUnion;
+
+final class Generator {
+  private Generator() {
+    // utility class.
+  }
+
+  private static Map<Class<?>, Object> simple = ImmutableMap.<Class<?>, Object>builder()
+      .put(boolean.class, true)
+      .put(Boolean.class, true)
+      .put(int.class, 2)
+      .put(Integer.class, 2)
+      .put(long.class, 4L)
+      .put(Long.class, 4L)
+      .put(double.class, 8.0)
+      .put(Double.class, 8.0)
+      .put(String.class, "string-value")
+      .build();
+
+  /**
+   * Generates a value of the given type.
+   *
+   * @param clazz The non-generic type to generate.
+   * @param parameterizedType The possibly-generic type (with type parameter information).
+   * @return A synthetically-generated and fully-populated instance.
+   * @throws ReflectiveOperationException If generation failed due to a reflection issue.
+   */
+  static Object valueFor(Class<?> clazz, Type parameterizedType)
+      throws ReflectiveOperationException {
+
+    if (simple.containsKey(clazz)) {
+      return simple.get(clazz);
+    }
+
+    if (TBase.class.isAssignableFrom(clazz)) {
+      @SuppressWarnings("unchecked")
+      Class<TBase<?, ?>> tbase = (Class<TBase<?, ?>>) parameterizedType;
+      return newStruct(tbase);
+    } else if (TEnum.class.isAssignableFrom(clazz)) {
+      Method values = clazz.getDeclaredMethod("values");
+      Object[] result = (Object[]) values.invoke(null);
+      return result[0];
+    } else if (Collection.class.isAssignableFrom(clazz)) {
+      ParameterizedType parameterized = (ParameterizedType) parameterizedType;
+      Type[] arguments = parameterized.getActualTypeArguments();
+      if (clazz == List.class) {
+        return ImmutableList.of(valueFor((Class) arguments[0], arguments[0]));
+      } else if (clazz == Set.class) {
+        return ImmutableSet.of(valueFor((Class) arguments[0], arguments[0]));
+      } else {
+        throw new IllegalArgumentException();
+      }
+    } else if (Map.class.isAssignableFrom(clazz)) {
+      ParameterizedType parameterized = (ParameterizedType) parameterizedType;
+      Type[] arguments = parameterized.getActualTypeArguments();
+      return ImmutableMap.of(
+          valueFor((Class) arguments[0], arguments[0]),
+          valueFor((Class) arguments[1], arguments[1]));
+    } else {
+      throw new IllegalArgumentException("Unsupported type " + parameterizedType);
+    }
+  }
+
+  private static <T extends TBase<?, ?>> void setValue(T struct, Method setter) {
+    try {
+      // All setters are expected to have exactly one parameter.
+      Class<?> paramType = setter.getParameterTypes()[0];
+      Type genericParamType = setter.getGenericParameterTypes()[0];
+
+      setter.invoke(struct, valueFor(paramType, genericParamType));
+    } catch (ReflectiveOperationException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Generates a thrift struct of the given tuype.
+   *
+   * @param structClass Struct type.
+   * @param <T> Struct type.
+   * @return A populated instance.
+   */
+  static <T extends TBase<?, ?>> T newStruct(Class<T> structClass) {
+    T struct;
+    try {
+      struct = structClass.newInstance();
+    } catch (ReflectiveOperationException e) {
+      throw new RuntimeException(e);
+    }
+
+    Stream<Method> setterMethods = Stream.of(structClass.getDeclaredMethods())
+        // Order methods by name for predictable behavior.  This is particularly useful for
+        // deterministic behavior of picking a TUnion field.
+        .sorted(Ordering.natural().onResultOf(Method::getName))
+        .filter(method -> !method.getName().equals("setFieldValue"))
+        .filter(method -> method.getName().startsWith("set"))
+        .filter(method -> !method.getName().endsWith("IsSet"));
+
+    if (TUnion.class.isAssignableFrom(structClass)) {
+      // A TUnion can represent one of many types.  Choose one arbitrarily.
+      setterMethods = setterMethods
+          .limit(1);
+    }
+
+    setterMethods.forEach(setter -> {
+      setValue(struct, setter);
+    });
+
+    return struct;
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/pruneJobUpdateHistory
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/pruneJobUpdateHistory b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/pruneJobUpdateHistory
new file mode 100644
index 0000000..c1eecd1
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/pruneJobUpdateHistory
@@ -0,0 +1,12 @@
+{
+  "17": {
+    "rec": {
+      "1": {
+        "i32": 2
+      },
+      "2": {
+        "i64": 4
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeJob
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeJob b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeJob
new file mode 100644
index 0000000..57acc42
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeJob
@@ -0,0 +1,19 @@
+{
+  "5": {
+    "rec": {
+      "2": {
+        "rec": {
+          "1": {
+            "str": "string-value"
+          },
+          "2": {
+            "str": "string-value"
+          },
+          "3": {
+            "str": "string-value"
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeJobUpdate
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeJobUpdate b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeJobUpdate
new file mode 100644
index 0000000..8377943
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeJobUpdate
@@ -0,0 +1,30 @@
+{
+  "18": {
+    "rec": {
+      "1": {
+        "set": [
+          "rec",
+          1,
+          {
+            "1": {
+              "rec": {
+                "1": {
+                  "str": "string-value"
+                },
+                "2": {
+                  "str": "string-value"
+                },
+                "3": {
+                  "str": "string-value"
+                }
+              }
+            },
+            "2": {
+              "str": "string-value"
+            }
+          }
+        ]
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeLock
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeLock b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeLock
new file mode 100644
index 0000000..2d4a100
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeLock
@@ -0,0 +1,23 @@
+{
+  "13": {
+    "rec": {
+      "1": {
+        "rec": {
+          "1": {
+            "rec": {
+              "1": {
+                "str": "string-value"
+              },
+              "2": {
+                "str": "string-value"
+              },
+              "3": {
+                "str": "string-value"
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeQuota
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeQuota b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeQuota
new file mode 100644
index 0000000..cd872c7
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeQuota
@@ -0,0 +1,9 @@
+{
+  "9": {
+    "rec": {
+      "1": {
+        "str": "string-value"
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeTasks
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeTasks b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeTasks
new file mode 100644
index 0000000..9744a7a
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/removeTasks
@@ -0,0 +1,13 @@
+{
+  "7": {
+    "rec": {
+      "1": {
+        "set": [
+          "str",
+          1,
+          "string-value"
+        ]
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveCronJob
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveCronJob b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveCronJob
new file mode 100644
index 0000000..88e1c36
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveCronJob
@@ -0,0 +1,200 @@
+{
+  "2": {
+    "rec": {
+      "2": {
+        "rec": {
+          "4": {
+            "str": "string-value"
+          },
+          "5": {
+            "i32": 0
+          },
+          "6": {
+            "rec": {
+              "7": {
+                "tf": 1
+              },
+              "11": {
+                "i32": 2
+              },
+              "13": {
+                "i32": 2
+              },
+              "17": {
+                "rec": {
+                  "2": {
+                    "str": "string-value"
+                  }
+                }
+              },
+              "18": {
+                "tf": 1
+              },
+              "20": {
+                "set": [
+                  "rec",
+                  1,
+                  {
+                    "1": {
+                      "str": "string-value"
+                    },
+                    "2": {
+                      "rec": {
+                        "2": {
+                          "rec": {
+                            "1": {
+                              "i32": 2
+                            }
+                          }
+                        }
+                      }
+                    }
+                  }
+                ]
+              },
+              "22": {
+                "map": [
+                  "str",
+                  "str",
+                  1,
+                  {
+                    "string-value": "string-value"
+                  }
+                ]
+              },
+              "23": {
+                "str": "string-value"
+              },
+              "25": {
+                "rec": {
+                  "1": {
+                    "str": "string-value"
+                  },
+                  "2": {
+                    "str": "string-value"
+                  }
+                }
+              },
+              "27": {
+                "set": [
+                  "rec",
+                  1,
+                  {
+                    "1": {
+                      "str": "string-value"
+                    },
+                    "2": {
+                      "str": "string-value"
+                    }
+                  }
+                ]
+              },
+              "28": {
+                "rec": {
+                  "1": {
+                    "str": "string-value"
+                  },
+                  "2": {
+                    "str": "string-value"
+                  },
+                  "3": {
+                    "str": "string-value"
+                  }
+                }
+              },
+              "29": {
+                "rec": {
+                  "2": {
+                    "rec": {
+                      "1": {
+                        "str": "string-value"
+                      },
+                      "2": {
+                        "lst": [
+                          "rec",
+                          1,
+                          {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "str": "string-value"
+                            }
+                          }
+                        ]
+                      }
+                    }
+                  }
+                }
+              },
+              "30": {
+                "str": "string-value"
+              },
+              "32": {
+                "set": [
+                  "rec",
+                  1,
+                  {
+                    "3": {
+                      "i64": 4
+                    }
+                  }
+                ]
+              },
+              "33": {
+                "set": [
+                  "rec",
+                  1,
+                  {
+                    "1": {
+                      "str": "string-value"
+                    },
+                    "2": {
+                      "tf": 1
+                    },
+                    "3": {
+                      "tf": 1
+                    }
+                  }
+                ]
+              },
+              "34": {
+                "rec": {
+                  "1": {
+                    "tf": 1
+                  },
+                  "2": {
+                    "i64": 4
+                  }
+                }
+              }
+            }
+          },
+          "7": {
+            "rec": {
+              "2": {
+                "str": "string-value"
+              }
+            }
+          },
+          "8": {
+            "i32": 2
+          },
+          "9": {
+            "rec": {
+              "1": {
+                "str": "string-value"
+              },
+              "2": {
+                "str": "string-value"
+              },
+              "3": {
+                "str": "string-value"
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveFrameworkId
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveFrameworkId b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveFrameworkId
new file mode 100644
index 0000000..2374b8d
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveFrameworkId
@@ -0,0 +1,9 @@
+{
+  "1": {
+    "rec": {
+      "1": {
+        "str": "string-value"
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveHostAttributes
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveHostAttributes b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveHostAttributes
new file mode 100644
index 0000000..c50ca44
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveHostAttributes
@@ -0,0 +1,37 @@
+{
+  "10": {
+    "rec": {
+      "1": {
+        "rec": {
+          "1": {
+            "str": "string-value"
+          },
+          "2": {
+            "set": [
+              "rec",
+              1,
+              {
+                "1": {
+                  "str": "string-value"
+                },
+                "2": {
+                  "set": [
+                    "str",
+                    1,
+                    "string-value"
+                  ]
+                }
+              }
+            ]
+          },
+          "3": {
+            "i32": 1
+          },
+          "4": {
+            "str": "string-value"
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobInstanceUpdateEvent
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobInstanceUpdateEvent b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobInstanceUpdateEvent
new file mode 100644
index 0000000..48902d3
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobInstanceUpdateEvent
@@ -0,0 +1,39 @@
+{
+  "16": {
+    "rec": {
+      "1": {
+        "rec": {
+          "1": {
+            "i32": 2
+          },
+          "2": {
+            "i64": 4
+          },
+          "3": {
+            "i32": 1
+          }
+        }
+      },
+      "3": {
+        "rec": {
+          "1": {
+            "rec": {
+              "1": {
+                "str": "string-value"
+              },
+              "2": {
+                "str": "string-value"
+              },
+              "3": {
+                "str": "string-value"
+              }
+            }
+          },
+          "2": {
+            "str": "string-value"
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobUpdate
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobUpdate b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobUpdate
new file mode 100644
index 0000000..32fdcda
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobUpdate
@@ -0,0 +1,469 @@
+{
+  "14": {
+    "rec": {
+      "1": {
+        "rec": {
+          "1": {
+            "rec": {
+              "3": {
+                "str": "string-value"
+              },
+              "4": {
+                "rec": {
+                  "1": {
+                    "i32": 0
+                  },
+                  "2": {
+                    "i64": 4
+                  },
+                  "3": {
+                    "i64": 4
+                  }
+                }
+              },
+              "5": {
+                "rec": {
+                  "1": {
+                    "rec": {
+                      "1": {
+                        "str": "string-value"
+                      },
+                      "2": {
+                        "str": "string-value"
+                      },
+                      "3": {
+                        "str": "string-value"
+                      }
+                    }
+                  },
+                  "2": {
+                    "str": "string-value"
+                  }
+                }
+              },
+              "6": {
+                "set": [
+                  "rec",
+                  1,
+                  {
+                    "1": {
+                      "str": "string-value"
+                    },
+                    "2": {
+                      "str": "string-value"
+                    }
+                  }
+                ]
+              }
+            }
+          },
+          "2": {
+            "rec": {
+              "1": {
+                "set": [
+                  "rec",
+                  1,
+                  {
+                    "1": {
+                      "rec": {
+                        "7": {
+                          "tf": 1
+                        },
+                        "11": {
+                          "i32": 2
+                        },
+                        "13": {
+                          "i32": 2
+                        },
+                        "17": {
+                          "rec": {
+                            "2": {
+                              "str": "string-value"
+                            }
+                          }
+                        },
+                        "18": {
+                          "tf": 1
+                        },
+                        "20": {
+                          "set": [
+                            "rec",
+                            1,
+                            {
+                              "1": {
+                                "str": "string-value"
+                              },
+                              "2": {
+                                "rec": {
+                                  "2": {
+                                    "rec": {
+                                      "1": {
+                                        "i32": 2
+                                      }
+                                    }
+                                  }
+                                }
+                              }
+                            }
+                          ]
+                        },
+                        "22": {
+                          "map": [
+                            "str",
+                            "str",
+                            1,
+                            {
+                              "string-value": "string-value"
+                            }
+                          ]
+                        },
+                        "23": {
+                          "str": "string-value"
+                        },
+                        "25": {
+                          "rec": {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "str": "string-value"
+                            }
+                          }
+                        },
+                        "27": {
+                          "set": [
+                            "rec",
+                            1,
+                            {
+                              "1": {
+                                "str": "string-value"
+                              },
+                              "2": {
+                                "str": "string-value"
+                              }
+                            }
+                          ]
+                        },
+                        "28": {
+                          "rec": {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "str": "string-value"
+                            },
+                            "3": {
+                              "str": "string-value"
+                            }
+                          }
+                        },
+                        "29": {
+                          "rec": {
+                            "2": {
+                              "rec": {
+                                "1": {
+                                  "str": "string-value"
+                                },
+                                "2": {
+                                  "lst": [
+                                    "rec",
+                                    1,
+                                    {
+                                      "1": {
+                                        "str": "string-value"
+                                      },
+                                      "2": {
+                                        "str": "string-value"
+                                      }
+                                    }
+                                  ]
+                                }
+                              }
+                            }
+                          }
+                        },
+                        "30": {
+                          "str": "string-value"
+                        },
+                        "32": {
+                          "set": [
+                            "rec",
+                            1,
+                            {
+                              "3": {
+                                "i64": 4
+                              }
+                            }
+                          ]
+                        },
+                        "33": {
+                          "set": [
+                            "rec",
+                            1,
+                            {
+                              "1": {
+                                "str": "string-value"
+                              },
+                              "2": {
+                                "tf": 1
+                              },
+                              "3": {
+                                "tf": 1
+                              }
+                            }
+                          ]
+                        },
+                        "34": {
+                          "rec": {
+                            "1": {
+                              "tf": 1
+                            },
+                            "2": {
+                              "i64": 4
+                            }
+                          }
+                        }
+                      }
+                    },
+                    "2": {
+                      "set": [
+                        "rec",
+                        1,
+                        {
+                          "1": {
+                            "i32": 2
+                          },
+                          "2": {
+                            "i32": 2
+                          }
+                        }
+                      ]
+                    }
+                  }
+                ]
+              },
+              "2": {
+                "rec": {
+                  "1": {
+                    "rec": {
+                      "7": {
+                        "tf": 1
+                      },
+                      "11": {
+                        "i32": 2
+                      },
+                      "13": {
+                        "i32": 2
+                      },
+                      "17": {
+                        "rec": {
+                          "2": {
+                            "str": "string-value"
+                          }
+                        }
+                      },
+                      "18": {
+                        "tf": 1
+                      },
+                      "20": {
+                        "set": [
+                          "rec",
+                          1,
+                          {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "rec": {
+                                "2": {
+                                  "rec": {
+                                    "1": {
+                                      "i32": 2
+                                    }
+                                  }
+                                }
+                              }
+                            }
+                          }
+                        ]
+                      },
+                      "22": {
+                        "map": [
+                          "str",
+                          "str",
+                          1,
+                          {
+                            "string-value": "string-value"
+                          }
+                        ]
+                      },
+                      "23": {
+                        "str": "string-value"
+                      },
+                      "25": {
+                        "rec": {
+                          "1": {
+                            "str": "string-value"
+                          },
+                          "2": {
+                            "str": "string-value"
+                          }
+                        }
+                      },
+                      "27": {
+                        "set": [
+                          "rec",
+                          1,
+                          {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "str": "string-value"
+                            }
+                          }
+                        ]
+                      },
+                      "28": {
+                        "rec": {
+                          "1": {
+                            "str": "string-value"
+                          },
+                          "2": {
+                            "str": "string-value"
+                          },
+                          "3": {
+                            "str": "string-value"
+                          }
+                        }
+                      },
+                      "29": {
+                        "rec": {
+                          "2": {
+                            "rec": {
+                              "1": {
+                                "str": "string-value"
+                              },
+                              "2": {
+                                "lst": [
+                                  "rec",
+                                  1,
+                                  {
+                                    "1": {
+                                      "str": "string-value"
+                                    },
+                                    "2": {
+                                      "str": "string-value"
+                                    }
+                                  }
+                                ]
+                              }
+                            }
+                          }
+                        }
+                      },
+                      "30": {
+                        "str": "string-value"
+                      },
+                      "32": {
+                        "set": [
+                          "rec",
+                          1,
+                          {
+                            "3": {
+                              "i64": 4
+                            }
+                          }
+                        ]
+                      },
+                      "33": {
+                        "set": [
+                          "rec",
+                          1,
+                          {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "tf": 1
+                            },
+                            "3": {
+                              "tf": 1
+                            }
+                          }
+                        ]
+                      },
+                      "34": {
+                        "rec": {
+                          "1": {
+                            "tf": 1
+                          },
+                          "2": {
+                            "i64": 4
+                          }
+                        }
+                      }
+                    }
+                  },
+                  "2": {
+                    "set": [
+                      "rec",
+                      1,
+                      {
+                        "1": {
+                          "i32": 2
+                        },
+                        "2": {
+                          "i32": 2
+                        }
+                      }
+                    ]
+                  }
+                }
+              },
+              "3": {
+                "rec": {
+                  "1": {
+                    "i32": 2
+                  },
+                  "2": {
+                    "i32": 2
+                  },
+                  "3": {
+                    "i32": 2
+                  },
+                  "5": {
+                    "i32": 2
+                  },
+                  "6": {
+                    "tf": 1
+                  },
+                  "7": {
+                    "set": [
+                      "rec",
+                      1,
+                      {
+                        "1": {
+                          "i32": 2
+                        },
+                        "2": {
+                          "i32": 2
+                        }
+                      }
+                    ]
+                  },
+                  "8": {
+                    "tf": 1
+                  },
+                  "9": {
+                    "i32": 2
+                  }
+                }
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobUpdateEvent
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobUpdateEvent b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobUpdateEvent
new file mode 100644
index 0000000..ca0b1dc
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveJobUpdateEvent
@@ -0,0 +1,42 @@
+{
+  "15": {
+    "rec": {
+      "1": {
+        "rec": {
+          "1": {
+            "i32": 0
+          },
+          "2": {
+            "i64": 4
+          },
+          "3": {
+            "str": "string-value"
+          },
+          "4": {
+            "str": "string-value"
+          }
+        }
+      },
+      "3": {
+        "rec": {
+          "1": {
+            "rec": {
+              "1": {
+                "str": "string-value"
+              },
+              "2": {
+                "str": "string-value"
+              },
+              "3": {
+                "str": "string-value"
+              }
+            }
+          },
+          "2": {
+            "str": "string-value"
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveLock
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveLock b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveLock
new file mode 100644
index 0000000..6f2cb02
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveLock
@@ -0,0 +1,39 @@
+{
+  "12": {
+    "rec": {
+      "1": {
+        "rec": {
+          "1": {
+            "rec": {
+              "1": {
+                "rec": {
+                  "1": {
+                    "str": "string-value"
+                  },
+                  "2": {
+                    "str": "string-value"
+                  },
+                  "3": {
+                    "str": "string-value"
+                  }
+                }
+              }
+            }
+          },
+          "2": {
+            "str": "string-value"
+          },
+          "3": {
+            "str": "string-value"
+          },
+          "4": {
+            "i64": 4
+          },
+          "5": {
+            "str": "string-value"
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveQuota
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveQuota b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveQuota
new file mode 100644
index 0000000..6b0d800
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveQuota
@@ -0,0 +1,33 @@
+{
+  "8": {
+    "rec": {
+      "1": {
+        "str": "string-value"
+      },
+      "2": {
+        "rec": {
+          "1": {
+            "dbl": 8.0
+          },
+          "2": {
+            "i64": 4
+          },
+          "3": {
+            "i64": 4
+          },
+          "4": {
+            "set": [
+              "rec",
+              1,
+              {
+                "3": {
+                  "i64": 4
+                }
+              }
+            ]
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveTasks
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveTasks b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveTasks
new file mode 100644
index 0000000..4323031
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/current/saveTasks
@@ -0,0 +1,233 @@
+{
+  "6": {
+    "rec": {
+      "1": {
+        "set": [
+          "rec",
+          1,
+          {
+            "1": {
+              "rec": {
+                "1": {
+                  "str": "string-value"
+                },
+                "2": {
+                  "str": "string-value"
+                },
+                "3": {
+                  "str": "string-value"
+                },
+                "4": {
+                  "rec": {
+                    "7": {
+                      "tf": 1
+                    },
+                    "11": {
+                      "i32": 2
+                    },
+                    "13": {
+                      "i32": 2
+                    },
+                    "17": {
+                      "rec": {
+                        "2": {
+                          "str": "string-value"
+                        }
+                      }
+                    },
+                    "18": {
+                      "tf": 1
+                    },
+                    "20": {
+                      "set": [
+                        "rec",
+                        1,
+                        {
+                          "1": {
+                            "str": "string-value"
+                          },
+                          "2": {
+                            "rec": {
+                              "2": {
+                                "rec": {
+                                  "1": {
+                                    "i32": 2
+                                  }
+                                }
+                              }
+                            }
+                          }
+                        }
+                      ]
+                    },
+                    "22": {
+                      "map": [
+                        "str",
+                        "str",
+                        1,
+                        {
+                          "string-value": "string-value"
+                        }
+                      ]
+                    },
+                    "23": {
+                      "str": "string-value"
+                    },
+                    "25": {
+                      "rec": {
+                        "1": {
+                          "str": "string-value"
+                        },
+                        "2": {
+                          "str": "string-value"
+                        }
+                      }
+                    },
+                    "27": {
+                      "set": [
+                        "rec",
+                        1,
+                        {
+                          "1": {
+                            "str": "string-value"
+                          },
+                          "2": {
+                            "str": "string-value"
+                          }
+                        }
+                      ]
+                    },
+                    "28": {
+                      "rec": {
+                        "1": {
+                          "str": "string-value"
+                        },
+                        "2": {
+                          "str": "string-value"
+                        },
+                        "3": {
+                          "str": "string-value"
+                        }
+                      }
+                    },
+                    "29": {
+                      "rec": {
+                        "2": {
+                          "rec": {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "lst": [
+                                "rec",
+                                1,
+                                {
+                                  "1": {
+                                    "str": "string-value"
+                                  },
+                                  "2": {
+                                    "str": "string-value"
+                                  }
+                                }
+                              ]
+                            }
+                          }
+                        }
+                      }
+                    },
+                    "30": {
+                      "str": "string-value"
+                    },
+                    "32": {
+                      "set": [
+                        "rec",
+                        1,
+                        {
+                          "3": {
+                            "i64": 4
+                          }
+                        }
+                      ]
+                    },
+                    "33": {
+                      "set": [
+                        "rec",
+                        1,
+                        {
+                          "1": {
+                            "str": "string-value"
+                          },
+                          "2": {
+                            "tf": 1
+                          },
+                          "3": {
+                            "tf": 1
+                          }
+                        }
+                      ]
+                    },
+                    "34": {
+                      "rec": {
+                        "1": {
+                          "tf": 1
+                        },
+                        "2": {
+                          "i64": 4
+                        }
+                      }
+                    }
+                  }
+                },
+                "5": {
+                  "map": [
+                    "str",
+                    "i32",
+                    1,
+                    {
+                      "string-value": 2
+                    }
+                  ]
+                },
+                "6": {
+                  "i32": 2
+                }
+              }
+            },
+            "2": {
+              "i32": 11
+            },
+            "3": {
+              "i32": 2
+            },
+            "4": {
+              "lst": [
+                "rec",
+                1,
+                {
+                  "1": {
+                    "i64": 4
+                  },
+                  "2": {
+                    "i32": 11
+                  },
+                  "3": {
+                    "str": "string-value"
+                  },
+                  "4": {
+                    "str": "string-value"
+                  }
+                }
+              ]
+            },
+            "5": {
+              "str": "string-value"
+            },
+            "6": {
+              "i32": 2
+            }
+          }
+        ]
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/1-pruneJobUpdateHistory
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/1-pruneJobUpdateHistory b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/1-pruneJobUpdateHistory
new file mode 100644
index 0000000..c1eecd1
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/1-pruneJobUpdateHistory
@@ -0,0 +1,12 @@
+{
+  "17": {
+    "rec": {
+      "1": {
+        "i32": 2
+      },
+      "2": {
+        "i64": 4
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/10-saveJobUpdate
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/10-saveJobUpdate b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/10-saveJobUpdate
new file mode 100644
index 0000000..32fdcda
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/10-saveJobUpdate
@@ -0,0 +1,469 @@
+{
+  "14": {
+    "rec": {
+      "1": {
+        "rec": {
+          "1": {
+            "rec": {
+              "3": {
+                "str": "string-value"
+              },
+              "4": {
+                "rec": {
+                  "1": {
+                    "i32": 0
+                  },
+                  "2": {
+                    "i64": 4
+                  },
+                  "3": {
+                    "i64": 4
+                  }
+                }
+              },
+              "5": {
+                "rec": {
+                  "1": {
+                    "rec": {
+                      "1": {
+                        "str": "string-value"
+                      },
+                      "2": {
+                        "str": "string-value"
+                      },
+                      "3": {
+                        "str": "string-value"
+                      }
+                    }
+                  },
+                  "2": {
+                    "str": "string-value"
+                  }
+                }
+              },
+              "6": {
+                "set": [
+                  "rec",
+                  1,
+                  {
+                    "1": {
+                      "str": "string-value"
+                    },
+                    "2": {
+                      "str": "string-value"
+                    }
+                  }
+                ]
+              }
+            }
+          },
+          "2": {
+            "rec": {
+              "1": {
+                "set": [
+                  "rec",
+                  1,
+                  {
+                    "1": {
+                      "rec": {
+                        "7": {
+                          "tf": 1
+                        },
+                        "11": {
+                          "i32": 2
+                        },
+                        "13": {
+                          "i32": 2
+                        },
+                        "17": {
+                          "rec": {
+                            "2": {
+                              "str": "string-value"
+                            }
+                          }
+                        },
+                        "18": {
+                          "tf": 1
+                        },
+                        "20": {
+                          "set": [
+                            "rec",
+                            1,
+                            {
+                              "1": {
+                                "str": "string-value"
+                              },
+                              "2": {
+                                "rec": {
+                                  "2": {
+                                    "rec": {
+                                      "1": {
+                                        "i32": 2
+                                      }
+                                    }
+                                  }
+                                }
+                              }
+                            }
+                          ]
+                        },
+                        "22": {
+                          "map": [
+                            "str",
+                            "str",
+                            1,
+                            {
+                              "string-value": "string-value"
+                            }
+                          ]
+                        },
+                        "23": {
+                          "str": "string-value"
+                        },
+                        "25": {
+                          "rec": {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "str": "string-value"
+                            }
+                          }
+                        },
+                        "27": {
+                          "set": [
+                            "rec",
+                            1,
+                            {
+                              "1": {
+                                "str": "string-value"
+                              },
+                              "2": {
+                                "str": "string-value"
+                              }
+                            }
+                          ]
+                        },
+                        "28": {
+                          "rec": {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "str": "string-value"
+                            },
+                            "3": {
+                              "str": "string-value"
+                            }
+                          }
+                        },
+                        "29": {
+                          "rec": {
+                            "2": {
+                              "rec": {
+                                "1": {
+                                  "str": "string-value"
+                                },
+                                "2": {
+                                  "lst": [
+                                    "rec",
+                                    1,
+                                    {
+                                      "1": {
+                                        "str": "string-value"
+                                      },
+                                      "2": {
+                                        "str": "string-value"
+                                      }
+                                    }
+                                  ]
+                                }
+                              }
+                            }
+                          }
+                        },
+                        "30": {
+                          "str": "string-value"
+                        },
+                        "32": {
+                          "set": [
+                            "rec",
+                            1,
+                            {
+                              "3": {
+                                "i64": 4
+                              }
+                            }
+                          ]
+                        },
+                        "33": {
+                          "set": [
+                            "rec",
+                            1,
+                            {
+                              "1": {
+                                "str": "string-value"
+                              },
+                              "2": {
+                                "tf": 1
+                              },
+                              "3": {
+                                "tf": 1
+                              }
+                            }
+                          ]
+                        },
+                        "34": {
+                          "rec": {
+                            "1": {
+                              "tf": 1
+                            },
+                            "2": {
+                              "i64": 4
+                            }
+                          }
+                        }
+                      }
+                    },
+                    "2": {
+                      "set": [
+                        "rec",
+                        1,
+                        {
+                          "1": {
+                            "i32": 2
+                          },
+                          "2": {
+                            "i32": 2
+                          }
+                        }
+                      ]
+                    }
+                  }
+                ]
+              },
+              "2": {
+                "rec": {
+                  "1": {
+                    "rec": {
+                      "7": {
+                        "tf": 1
+                      },
+                      "11": {
+                        "i32": 2
+                      },
+                      "13": {
+                        "i32": 2
+                      },
+                      "17": {
+                        "rec": {
+                          "2": {
+                            "str": "string-value"
+                          }
+                        }
+                      },
+                      "18": {
+                        "tf": 1
+                      },
+                      "20": {
+                        "set": [
+                          "rec",
+                          1,
+                          {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "rec": {
+                                "2": {
+                                  "rec": {
+                                    "1": {
+                                      "i32": 2
+                                    }
+                                  }
+                                }
+                              }
+                            }
+                          }
+                        ]
+                      },
+                      "22": {
+                        "map": [
+                          "str",
+                          "str",
+                          1,
+                          {
+                            "string-value": "string-value"
+                          }
+                        ]
+                      },
+                      "23": {
+                        "str": "string-value"
+                      },
+                      "25": {
+                        "rec": {
+                          "1": {
+                            "str": "string-value"
+                          },
+                          "2": {
+                            "str": "string-value"
+                          }
+                        }
+                      },
+                      "27": {
+                        "set": [
+                          "rec",
+                          1,
+                          {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "str": "string-value"
+                            }
+                          }
+                        ]
+                      },
+                      "28": {
+                        "rec": {
+                          "1": {
+                            "str": "string-value"
+                          },
+                          "2": {
+                            "str": "string-value"
+                          },
+                          "3": {
+                            "str": "string-value"
+                          }
+                        }
+                      },
+                      "29": {
+                        "rec": {
+                          "2": {
+                            "rec": {
+                              "1": {
+                                "str": "string-value"
+                              },
+                              "2": {
+                                "lst": [
+                                  "rec",
+                                  1,
+                                  {
+                                    "1": {
+                                      "str": "string-value"
+                                    },
+                                    "2": {
+                                      "str": "string-value"
+                                    }
+                                  }
+                                ]
+                              }
+                            }
+                          }
+                        }
+                      },
+                      "30": {
+                        "str": "string-value"
+                      },
+                      "32": {
+                        "set": [
+                          "rec",
+                          1,
+                          {
+                            "3": {
+                              "i64": 4
+                            }
+                          }
+                        ]
+                      },
+                      "33": {
+                        "set": [
+                          "rec",
+                          1,
+                          {
+                            "1": {
+                              "str": "string-value"
+                            },
+                            "2": {
+                              "tf": 1
+                            },
+                            "3": {
+                              "tf": 1
+                            }
+                          }
+                        ]
+                      },
+                      "34": {
+                        "rec": {
+                          "1": {
+                            "tf": 1
+                          },
+                          "2": {
+                            "i64": 4
+                          }
+                        }
+                      }
+                    }
+                  },
+                  "2": {
+                    "set": [
+                      "rec",
+                      1,
+                      {
+                        "1": {
+                          "i32": 2
+                        },
+                        "2": {
+                          "i32": 2
+                        }
+                      }
+                    ]
+                  }
+                }
+              },
+              "3": {
+                "rec": {
+                  "1": {
+                    "i32": 2
+                  },
+                  "2": {
+                    "i32": 2
+                  },
+                  "3": {
+                    "i32": 2
+                  },
+                  "5": {
+                    "i32": 2
+                  },
+                  "6": {
+                    "tf": 1
+                  },
+                  "7": {
+                    "set": [
+                      "rec",
+                      1,
+                      {
+                        "1": {
+                          "i32": 2
+                        },
+                        "2": {
+                          "i32": 2
+                        }
+                      }
+                    ]
+                  },
+                  "8": {
+                    "tf": 1
+                  },
+                  "9": {
+                    "i32": 2
+                  }
+                }
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/11-saveJobUpdateEvent
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/11-saveJobUpdateEvent b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/11-saveJobUpdateEvent
new file mode 100644
index 0000000..ca0b1dc
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/11-saveJobUpdateEvent
@@ -0,0 +1,42 @@
+{
+  "15": {
+    "rec": {
+      "1": {
+        "rec": {
+          "1": {
+            "i32": 0
+          },
+          "2": {
+            "i64": 4
+          },
+          "3": {
+            "str": "string-value"
+          },
+          "4": {
+            "str": "string-value"
+          }
+        }
+      },
+      "3": {
+        "rec": {
+          "1": {
+            "rec": {
+              "1": {
+                "str": "string-value"
+              },
+              "2": {
+                "str": "string-value"
+              },
+              "3": {
+                "str": "string-value"
+              }
+            }
+          },
+          "2": {
+            "str": "string-value"
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/12-saveJobInstanceUpdateEvent
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/12-saveJobInstanceUpdateEvent b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/12-saveJobInstanceUpdateEvent
new file mode 100644
index 0000000..48902d3
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/12-saveJobInstanceUpdateEvent
@@ -0,0 +1,39 @@
+{
+  "16": {
+    "rec": {
+      "1": {
+        "rec": {
+          "1": {
+            "i32": 2
+          },
+          "2": {
+            "i64": 4
+          },
+          "3": {
+            "i32": 1
+          }
+        }
+      },
+      "3": {
+        "rec": {
+          "1": {
+            "rec": {
+              "1": {
+                "str": "string-value"
+              },
+              "2": {
+                "str": "string-value"
+              },
+              "3": {
+                "str": "string-value"
+              }
+            }
+          },
+          "2": {
+            "str": "string-value"
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/13-saveLock
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/13-saveLock b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/13-saveLock
new file mode 100644
index 0000000..6f2cb02
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/13-saveLock
@@ -0,0 +1,39 @@
+{
+  "12": {
+    "rec": {
+      "1": {
+        "rec": {
+          "1": {
+            "rec": {
+              "1": {
+                "rec": {
+                  "1": {
+                    "str": "string-value"
+                  },
+                  "2": {
+                    "str": "string-value"
+                  },
+                  "3": {
+                    "str": "string-value"
+                  }
+                }
+              }
+            }
+          },
+          "2": {
+            "str": "string-value"
+          },
+          "3": {
+            "str": "string-value"
+          },
+          "4": {
+            "i64": 4
+          },
+          "5": {
+            "str": "string-value"
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/8e5e08eb/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/14-saveQuota
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/14-saveQuota b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/14-saveQuota
new file mode 100644
index 0000000..eed5ea5
--- /dev/null
+++ b/src/test/resources/org/apache/aurora/scheduler/storage/durability/goldens/read-compatible/14-saveQuota
@@ -0,0 +1,43 @@
+{
+  "8": {
+    "rec": {
+      "1": {
+        "str": "role"
+      },
+      "2": {
+        "rec": {
+          "1": {
+            "dbl": 2.0
+          },
+          "2": {
+            "i64": 1
+          },
+          "3": {
+            "i64": 1
+          },
+          "4": {
+            "set": [
+              "rec",
+              3,
+              {
+                "1": {
+                  "dbl": 2.0
+                }
+              },
+              {
+                "3": {
+                  "i64": 1
+                }
+              },
+              {
+                "2": {
+                  "i64": 1
+                }
+              }
+            ]
+          }
+        }
+      }
+    }
+  }
+}