You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by jc...@apache.org on 2016/04/08 02:33:44 UTC

aurora git commit: Add support for automated DB migrations and rollbacks when restoring from dbScript stored in a snapshot.

Repository: aurora
Updated Branches:
  refs/heads/master c0cb631de -> 5ecded788


Add support for automated DB migrations and rollbacks when restoring from dbScript stored in a
snapshot.

Bugs closed: AURORA-1648

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


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

Branch: refs/heads/master
Commit: 5ecded788251bcaece0d972c963364f700aa9f3b
Parents: c0cb631
Author: Joshua Cohen <jc...@apache.org>
Authored: Thu Apr 7 19:33:24 2016 -0500
Committer: Joshua Cohen <jc...@apache.org>
Committed: Thu Apr 7 19:33:24 2016 -0500

----------------------------------------------------------------------
 RELEASE-NOTES.md                                |   2 +
 build.gradle                                    |   5 +-
 config/checkstyle/suppressions.xml              |   2 +
 docs/development/db-migration.md                |  33 +++++
 docs/development/thrift.md                      |   9 +-
 .../aurora/scheduler/app/SchedulerMain.java     |   1 +
 .../storage/backup/TemporaryStorage.java        |   5 +-
 .../aurora/scheduler/storage/db/DbModule.java   |  26 ++++
 .../aurora/scheduler/storage/db/DbUtil.java     |   7 +-
 .../scheduler/storage/db/MigrationManager.java  |  29 ++++
 .../storage/db/MigrationManagerImpl.java        | 135 +++++++++++++++++++
 .../scheduler/storage/db/MigrationMapper.java   |  51 +++++++
 .../db/views/MigrationChangelogEntry.java       |  48 +++++++
 .../storage/log/SnapshotStoreImpl.java          |  13 +-
 .../scheduler/storage/db/MigrationMapper.xml    |  55 ++++++++
 .../storage/db/MigrationManagerImplIT.java      | 114 ++++++++++++++++
 .../db/testmigration/V001_TestMigration.java    |  40 ++++++
 .../storage/log/SnapshotStoreImplIT.java        |   8 +-
 18 files changed, 569 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index ebc252f..0ca6486 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -29,6 +29,8 @@
 - Added a new scheduler argument `--populate_discovery_info`. If set to true, Aurora will start
   to populate DiscoveryInfo field on TaskInfo of Mesos. This could be used for alternative
   service discovery solution like Mesos-DNS.
+- Added support for automatic schema upgrades and downgrades when restoring a snapshot that contains
+  a DB dump.
 
 ### Deprecations and removals:
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index ad5ec5c..fc61adf 100644
--- a/build.gradle
+++ b/build.gradle
@@ -87,6 +87,7 @@ For more details, please see https://issues.apache.org/jira/browse/AURORA-1169
   ext.jerseyRev = '1.19'
   ext.jsrRev = '3.0.1'
   ext.junitRev = '4.12'
+  ext.mybatisRev = '3.3.1'
   ext.protobufRev = '2.6.1'
   ext.servletRev = '3.1.0'
   ext.slf4jRev = '1.7.12'
@@ -116,6 +117,7 @@ For more details, please see https://issues.apache.org/jira/browse/AURORA-1169
         force "org.apache.thrift:libthrift:${thriftRev}"
         force "org.hamcrest:hamcrest-core:1.3"
         force "org.slf4j:slf4j-api:${slf4jRev}"
+        force "org.mybatis:mybatis:${mybatisRev}"
       }
     }
   }
@@ -364,8 +366,9 @@ dependencies {
   compile "org.eclipse.jetty:jetty-server:${jettyDep}"
   compile "org.eclipse.jetty:jetty-servlet:${jettyDep}"
   compile "org.eclipse.jetty:jetty-servlets:${jettyDep}"
-  compile 'org.mybatis:mybatis:3.3.1'
+  compile "org.mybatis:mybatis:${mybatisRev}"
   compile 'org.mybatis:mybatis-guice:3.7'
+  compile 'org.mybatis:mybatis-migrations:3.2.0'
   compile 'org.quartz-scheduler:quartz:2.2.2'
   compile "uno.perk:forward:1.0.0"
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/config/checkstyle/suppressions.xml
----------------------------------------------------------------------
diff --git a/config/checkstyle/suppressions.xml b/config/checkstyle/suppressions.xml
index 50a53e0..30a32ce 100644
--- a/config/checkstyle/suppressions.xml
+++ b/config/checkstyle/suppressions.xml
@@ -21,4 +21,6 @@ limitations under the License.
   <!-- Allow use of System.exit() in main. -->
   <suppress files="org/apache/aurora/scheduler/app/SchedulerMain.java"
             checks="RegexpSinglelineJava"/>
+  <suppress files="org/apache/aurora/scheduler/storage/db/testmigration/V001_TestMigration.java"
+            checks="TypeName" />
 </suppressions>

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/docs/development/db-migration.md
----------------------------------------------------------------------
diff --git a/docs/development/db-migration.md b/docs/development/db-migration.md
new file mode 100644
index 0000000..bddfee1
--- /dev/null
+++ b/docs/development/db-migration.md
@@ -0,0 +1,33 @@
+DB Migrations
+=============
+
+Changes to the DB schema should be made in the form of migrations. This ensures that all changes
+are applied correctly after a DB dump from a previous version is restored.
+
+DB migrations are managed through a system built on top of
+[MyBatis Migrations](http://www.mybatis.org/migrations/). The migrations are run automatically when
+a snapshot is restored, no manual interaction is required by cluster operators.
+
+Upgrades
+--------
+When adding or altering tables or changing data, a new migration class should be created under the
+org.apache.aurora.scheduler.storage.db.migration package. The class should implement the
+[MigrationScript](https://github.com/mybatis/migrations/blob/master/src/main/java/org/apache/ibatis/migration/MigrationScript.java)
+interface (see [V001_TestMigration](../../src/test/java/org/apache/aurora/scheduler/storage/db/testmigration/V001_TestMigration.java)
+as an example). The upgrade and downgrade scripts are defined in this class. When restoring a
+snapshot the list of migrations on the classpath is compared to the list of applied changes in the
+DB. Any changes that have not yet been applied are executed and their downgrade script is stored
+alongside the changelog entry in the database to faciliate downgrades in the event of a rollback.
+
+Downgrades
+----------
+If, while running migrations, a rollback is detected, i.e. a change exists in the DB changelog that
+does not exist on the classpath, the downgrade script associated with each affected change is
+applied.
+
+Baselines
+---------
+After enough time has passed (at least 1 official release), it should be safe to baseline migrations
+if desired. This can be accomplished by adding the changes from migrations directly to
+[schema.sql](../../src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql), removing
+the corresponding migration classes and adding a migration to remove the changelog entries.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/docs/development/thrift.md
----------------------------------------------------------------------
diff --git a/docs/development/thrift.md b/docs/development/thrift.md
index 7f098c2..d0426f5 100644
--- a/docs/development/thrift.md
+++ b/docs/development/thrift.md
@@ -33,12 +33,9 @@ communicate with scheduler/client from vCurrent-1.
 * Add a new field as an eventual replacement of the old one and implement a dual read/write
 anywhere the old field is used. If a thrift struct is mapped in the DB store make sure both columns
 are marked as `NOT NULL`
-* Check [storage.thrift](../../api/src/main/thrift/org/apache/aurora/gen/storage.thrift) to see if the
-affected struct is stored in Aurora scheduler storage. If so, you most likely need to backfill
-existing data to ensure both fields are populated eagerly on startup. See
-[this patch](https://reviews.apache.org/r/43172) as a real-life example of thrift-struct
-backfilling. IMPORTANT: backfilling implementation needs to ensure both fields are populated. This
-is critical to enable graceful scheduler upgrade as well as rollback to the old version if needed.
+* Check [storage.thrift](../../api/src/main/thrift/org/apache/aurora/gen/storage.thrift) to see if
+the affected struct is stored in Aurora scheduler storage. If so, it's almost certainly also
+necessary to perform a [DB migration](db-migration.md).
 * Add a deprecation jira ticket into the vCurrent+1 release candidate
 * Add a TODO for the deprecated field mentioning the jira ticket
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java b/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
index ecdaa7e..6c7ae7e 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/SchedulerMain.java
@@ -137,6 +137,7 @@ public class SchedulerMain {
         new StatsModule(),
         new AppModule(),
         new CronModule(),
+        new DbModule.MigrationManagerModule(),
         DbModule.productionModule(Bindings.annotatedKeyFactory(Storage.Volatile.class)),
         new DbModule.GarbageCollectorModule());
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/main/java/org/apache/aurora/scheduler/storage/backup/TemporaryStorage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/backup/TemporaryStorage.java b/src/main/java/org/apache/aurora/scheduler/storage/backup/TemporaryStorage.java
index 5c7d92f..d08873c 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/backup/TemporaryStorage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/backup/TemporaryStorage.java
@@ -76,7 +76,10 @@ interface TemporaryStorage {
           storage,
           // Safe to pass false here to default to the non-experimental task store
           // during restore from backup procedure.
-          false /** useDbSnapshotForTaskStore */);
+          false /** useDbSnapshotForTaskStore */,
+          // We can just pass an empty lambda for the MigrationManager as migration is a no-op
+          // when restoring from backup.
+          () -> { } /** migrationManager */);
       snapshotStore.applySnapshot(snapshot);
 
       return new TemporaryStorage() {

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
index baf460e..743993c 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
@@ -50,6 +50,8 @@ import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.TaskStore;
 import org.apache.aurora.scheduler.storage.db.typehandlers.TypeHandlers;
 import org.apache.aurora.scheduler.storage.mem.InMemStoresModule;
+import org.apache.ibatis.migration.JavaMigrationLoader;
+import org.apache.ibatis.migration.MigrationLoader;
 import org.apache.ibatis.session.AutoMappingBehavior;
 import org.apache.ibatis.session.SqlSessionFactory;
 import org.apache.ibatis.transaction.jdbc.JdbcTransactionFactory;
@@ -95,6 +97,7 @@ public final class DbModule extends PrivateModule {
       .add(JobUpdateEventMapper.class)
       .add(JobUpdateDetailsMapper.class)
       .add(LockMapper.class)
+      .add(MigrationMapper.class)
       .add(QuotaMapper.class)
       .add(TaskConfigMapper.class)
       .add(TaskMapper.class)
@@ -340,4 +343,27 @@ public final class DbModule extends PrivateModule {
           .to(RowGarbageCollector.class);
     }
   }
+
+  public static class MigrationManagerModule extends PrivateModule {
+    private static final String MIGRATION_PACKAGE =
+        "org.apache.aurora.scheduler.storage.db.migration";
+
+    private final MigrationLoader migrationLoader;
+
+    public MigrationManagerModule() {
+      this.migrationLoader = new JavaMigrationLoader(MIGRATION_PACKAGE);
+    }
+
+    public MigrationManagerModule(MigrationLoader migrationLoader) {
+      this.migrationLoader = requireNonNull(migrationLoader);
+    }
+
+    @Override
+    protected void configure() {
+      bind(MigrationLoader.class).toInstance(migrationLoader);
+
+      bind(MigrationManager.class).to(MigrationManagerImpl.class);
+      expose(MigrationManager.class);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/main/java/org/apache/aurora/scheduler/storage/db/DbUtil.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbUtil.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbUtil.java
index 6f4e671..7c1127e 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbUtil.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbUtil.java
@@ -44,7 +44,12 @@ public final class DbUtil {
    * @return An injector with bindings necessary for a storage system.
    */
   public static Injector createStorageInjector(Module dbModule) {
+    return createStorageInjector(dbModule, new DbModule.MigrationManagerModule());
+  }
+
+  public static Injector createStorageInjector(Module dbModule, Module migrationModule) {
     Injector injector = Guice.createInjector(
+        migrationModule,
         dbModule,
         new AbstractModule() {
           @Override
@@ -74,7 +79,7 @@ public final class DbUtil {
   /**
    * Creates a new, empty storage system with a task store defined by the command line flag.
    *
-   * @return A new storage instance.
+   * @return An new storage instance.
    */
   public static Storage createFlaggedStorage() {
     return createStorageInjector(testModuleWithWorkQueue(PLAIN, Optional.absent()))

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationManager.java b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationManager.java
new file mode 100644
index 0000000..fa986da
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationManager.java
@@ -0,0 +1,29 @@
+/**
+ * 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.db;
+
+import java.sql.SQLException;
+
+/**
+ * Manage schema migrations.
+ */
+public interface MigrationManager {
+  /**
+   * Perform a migration, upgrading the schema if there are unapplied changes or downgrading it if
+   * there are applies changes which do not exist in the current version.
+   *
+   * @throws SQLException In the event of a problem performing the migration.
+   */
+  void migrate() throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationManagerImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationManagerImpl.java b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationManagerImpl.java
new file mode 100644
index 0000000..e2b8416
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationManagerImpl.java
@@ -0,0 +1,135 @@
+/**
+ * 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.db;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import javax.inject.Inject;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Throwables;
+import com.google.common.io.CharStreams;
+
+import org.apache.aurora.scheduler.storage.db.views.MigrationChangelogEntry;
+import org.apache.ibatis.migration.Change;
+import org.apache.ibatis.migration.DataSourceConnectionProvider;
+import org.apache.ibatis.migration.MigrationLoader;
+import org.apache.ibatis.migration.operations.UpOperation;
+import org.apache.ibatis.migration.options.DatabaseOperationOption;
+import org.apache.ibatis.session.SqlSession;
+import org.apache.ibatis.session.SqlSessionFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.util.Objects.requireNonNull;
+
+public class MigrationManagerImpl implements MigrationManager {
+  private static final Logger LOG = LoggerFactory.getLogger(MigrationManagerImpl.class);
+
+  private final SqlSessionFactory sqlSessionFactory;
+  private final MigrationLoader migrationLoader;
+
+  @Inject
+  MigrationManagerImpl(SqlSessionFactory sqlSessionFactory, MigrationLoader migrationLoader) {
+    this.sqlSessionFactory = requireNonNull(sqlSessionFactory);
+    this.migrationLoader = requireNonNull(migrationLoader);
+  }
+
+  @Override
+  public void migrate() throws SQLException {
+    LOG.info("Running db migrations.");
+
+    try (SqlSession sqlSession = sqlSessionFactory.openSession(true /* auto commit */)) {
+      MigrationMapper mapper = sqlSession.getMapper(MigrationMapper.class);
+
+      LOG.info("Bootstrapping changelog table (if necessary).");
+      mapper.bootstrapChangelog();
+
+      if (!checkRollback(mapper, migrationLoader.getMigrations())) {
+        DatabaseOperationOption options = new DatabaseOperationOption();
+        options.setAutoCommit(true);
+
+        new UpOperation().operate(
+            new DataSourceConnectionProvider(
+                sqlSessionFactory.getConfiguration().getEnvironment().getDataSource()),
+            migrationLoader,
+            options,
+            null);
+
+        saveDowngradeScript(mapper);
+      }
+    }
+  }
+
+  /**
+   * Iterates applied changes to ensure they all exist on the classpath. For any changes that do not
+   * exist on the classpath, their downgrade script is run.
+   *
+   * @param mapper A {@link MigrationMapper} instance used to modify the changelog.
+   * @param changes The list of {@link Change}s found on the classpath.
+   * @return true if a rollback was detected, false otherwise.
+   * @throws SQLException in the event a SQL failure.
+   */
+  private boolean checkRollback(MigrationMapper mapper, List<Change> changes) throws SQLException {
+    boolean rollback = false;
+
+    List<MigrationChangelogEntry> appliedChanges = mapper.selectAll();
+
+    for (MigrationChangelogEntry change : appliedChanges) {
+      // We cannot directly call changes.contains(...) since contains relies on Change#equals
+      // which includes class in its equality check rather than checking instanceof. Instead we just
+      // find the first element in changes whose id matches our applied change. If it does not exist
+      // then this must be a rollback.
+      if (changes.stream().findFirst().filter(c -> c.getId().equals(change.getId())).isPresent()) {
+        LOG.info("Change " + change.getId() + " has been applied, no other downgrades are "
+            + "necessary");
+        break;
+      }
+
+      LOG.info("No migration corresponding to change id " + change.getId() + " found. Assuming "
+          + "this is a rollback.");
+      LOG.info("Downgrade SQL for " + change.getId() + " is: " + change.getDowngradeScript());
+
+      try (SqlSession session = sqlSessionFactory.openSession(true)) {
+        try (Connection c = session.getConnection()) {
+          try (PreparedStatement downgrade = c.prepareStatement(change.getDowngradeScript())) {
+            downgrade.execute();
+            rollback = true;
+          }
+        }
+      }
+
+      LOG.info("Deleting applied change: " + change.getId());
+      mapper.delete(change.getId());
+    }
+
+    return rollback;
+  }
+
+  private void saveDowngradeScript(MigrationMapper mapper) {
+    for (Change c : migrationLoader.getMigrations()) {
+      try {
+        String downgradeScript = CharStreams.toString(migrationLoader.getScriptReader(c, true));
+        LOG.info("Saving downgrade script for change id " + c.getId() + ": " + downgradeScript);
+
+        mapper.saveDowngradeScript(c.getId(), downgradeScript.getBytes(Charsets.UTF_8));
+      } catch (IOException e) {
+        Throwables.propagate(e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationMapper.java b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationMapper.java
new file mode 100644
index 0000000..0c17aa7
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationMapper.java
@@ -0,0 +1,51 @@
+/**
+ * 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.db;
+
+import java.math.BigDecimal;
+import java.util.List;
+
+import org.apache.aurora.scheduler.storage.db.views.MigrationChangelogEntry;
+import org.apache.ibatis.annotations.Param;
+
+interface MigrationMapper {
+  /**
+   * Creates the changelog table if it does not already exist.
+   */
+  void bootstrapChangelog();
+
+  /**
+   * Saves the downgrade script for the supplied change id into the changelog.
+   *
+   * @param changeId The id of the change.
+   * @param downgradeScript The script to be run when a change is rolled back.
+   */
+  void saveDowngradeScript(
+      @Param("changeId") BigDecimal changeId,
+      @Param("downgradeScript") byte[] downgradeScript);
+
+  /**
+   * Select all applied changes from the changelog.
+   *
+   * @return A list of changelog entries mapping only their ids and downgrade scripts.
+   */
+  List<MigrationChangelogEntry> selectAll();
+
+  /**
+   * Deletes the specified change from the changelog.
+   *
+   * @param changeId The id of the change to delete.
+   */
+  void delete(@Param("changeId") BigDecimal changeId);
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/main/java/org/apache/aurora/scheduler/storage/db/views/MigrationChangelogEntry.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/views/MigrationChangelogEntry.java b/src/main/java/org/apache/aurora/scheduler/storage/db/views/MigrationChangelogEntry.java
new file mode 100644
index 0000000..6159af6
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/views/MigrationChangelogEntry.java
@@ -0,0 +1,48 @@
+/**
+ * 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.db.views;
+
+import java.util.Arrays;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Objects;
+
+import org.apache.ibatis.migration.Change;
+
+public class MigrationChangelogEntry extends Change {
+  private byte[] downgradeScript;
+
+  public String getDowngradeScript() {
+    return new String(downgradeScript, Charsets.UTF_8);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof MigrationChangelogEntry)) {
+      return false;
+    }
+
+    if (!super.equals(o)) {
+      return false;
+    }
+
+    MigrationChangelogEntry other = (MigrationChangelogEntry) o;
+    return Arrays.equals(downgradeScript, other.downgradeScript);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(super.hashCode(), downgradeScript);
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/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 6fee251..b6922e1 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
@@ -56,6 +56,7 @@ import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.MutateWork.NoResult;
 import org.apache.aurora.scheduler.storage.Storage.Volatile;
+import org.apache.aurora.scheduler.storage.db.MigrationManager;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobInstanceUpdateEvent;
@@ -128,7 +129,6 @@ public class SnapshotStoreImpl implements SnapshotStore<Snapshot> {
 
         @Override
         public void restoreFromSnapshot(MutableStoreProvider store, Snapshot snapshot) {
-
           if (snapshot.isSetDbScript()) {
             try (Connection c = ((DataSource) store.getUnsafeStoreAccess()).getConnection()) {
               LOG.info("Dropping all tables");
@@ -148,6 +148,12 @@ public class SnapshotStoreImpl implements SnapshotStore<Snapshot> {
             } catch (SQLException e) {
               Throwables.propagate(e);
             }
+
+            try {
+              migrationManager.migrate();
+            } catch (SQLException e) {
+              Throwables.propagate(e);
+            }
           }
         }
       },
@@ -350,6 +356,7 @@ public class SnapshotStoreImpl implements SnapshotStore<Snapshot> {
   private final Clock clock;
   private final Storage storage;
   private final boolean useDbSnapshotForTaskStore;
+  private final MigrationManager migrationManager;
 
   /**
    * Identifies if experimental task store is in use.
@@ -364,12 +371,14 @@ public class SnapshotStoreImpl implements SnapshotStore<Snapshot> {
       BuildInfo buildInfo,
       Clock clock,
       @Volatile Storage storage,
-      @ExperimentalTaskStore boolean useDbSnapshotForTaskStore) {
+      @ExperimentalTaskStore boolean useDbSnapshotForTaskStore,
+      MigrationManager migrationManager) {
 
     this.buildInfo = requireNonNull(buildInfo);
     this.clock = requireNonNull(clock);
     this.storage = requireNonNull(storage);
     this.useDbSnapshotForTaskStore = useDbSnapshotForTaskStore;
+    this.migrationManager = requireNonNull(migrationManager);
   }
 
   @Timed("snapshot_create")

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/main/resources/org/apache/aurora/scheduler/storage/db/MigrationMapper.xml
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/MigrationMapper.xml b/src/main/resources/org/apache/aurora/scheduler/storage/db/MigrationMapper.xml
new file mode 100644
index 0000000..f6881ac
--- /dev/null
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/MigrationMapper.xml
@@ -0,0 +1,55 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+ -->
+
+<!DOCTYPE mapper
+    PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN"
+    "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.aurora.scheduler.storage.db.MigrationMapper">
+  <update id="bootstrapChangelog">
+    CREATE TABLE IF NOT EXISTS changelog (
+      id BIGINT NOT NULL PRIMARY KEY,
+      applied_at VARCHAR(25) NOT NULL,
+      description VARCHAR(255) NOT NULL,
+      downgrade_script BLOB NULL,
+
+      UNIQUE(id)
+    );
+  </update>
+
+  <update id="saveDowngradeScript">
+    UPDATE changelog
+    SET downgrade_script = #{downgradeScript}
+    WHERE id = #{changeId}
+  </update>
+
+  <resultMap
+      id="changelogResultMap"
+      type="org.apache.aurora.scheduler.storage.db.views.MigrationChangelogEntry">
+
+  </resultMap>
+
+  <select id="selectAll" resultMap="changelogResultMap">
+    SELECT
+      id,
+      downgrade_script
+    FROM changelog
+    ORDER BY id DESC
+  </select>
+
+  <delete id="delete">
+    DELETE FROM changelog
+    WHERE id = #{changeId}
+  </delete>
+</mapper>

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/test/java/org/apache/aurora/scheduler/storage/db/MigrationManagerImplIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/db/MigrationManagerImplIT.java b/src/test/java/org/apache/aurora/scheduler/storage/db/MigrationManagerImplIT.java
new file mode 100644
index 0000000..5edc2ad
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/storage/db/MigrationManagerImplIT.java
@@ -0,0 +1,114 @@
+/**
+ * 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.db;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Optional;
+
+import com.google.common.io.CharStreams;
+import com.google.inject.Injector;
+
+import org.apache.aurora.scheduler.storage.db.views.MigrationChangelogEntry;
+import org.apache.ibatis.migration.Change;
+import org.apache.ibatis.migration.JavaMigrationLoader;
+import org.apache.ibatis.migration.MigrationLoader;
+import org.apache.ibatis.session.SqlSession;
+import org.apache.ibatis.session.SqlSessionFactory;
+import org.junit.Test;
+
+import static org.apache.aurora.scheduler.storage.db.DbModule.testModuleWithWorkQueue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class MigrationManagerImplIT {
+  private Injector createMigrationInjector(MigrationLoader migrationLoader) {
+    return DbUtil.createStorageInjector(
+        testModuleWithWorkQueue(),
+        new DbModule.MigrationManagerModule(migrationLoader));
+  }
+
+  @Test
+  public void testMigrate() throws Exception {
+    MigrationLoader loader = new JavaMigrationLoader(
+        "org.apache.aurora.scheduler.storage.db.testmigration");
+    Injector injector = createMigrationInjector(loader);
+
+    MigrationManager migrationManager = injector.getInstance(MigrationManager.class);
+
+    migrationManager.migrate();
+
+    SqlSessionFactory sqlSessionFactory = injector.getInstance(SqlSessionFactory.class);
+
+    try (SqlSession session = sqlSessionFactory.openSession()) {
+      MigrationMapper mapper = session.getMapper(MigrationMapper.class);
+      List<MigrationChangelogEntry> appliedChanges = mapper.selectAll();
+
+      // Ensure all changes have been applied and their downgrade scripts stored appropriately.
+      for (Change change : loader.getMigrations()) {
+        Optional<MigrationChangelogEntry> appliedChange = appliedChanges
+            .stream()
+            .findFirst()
+            .filter(c -> c.getId().equals(change.getId()));
+
+        assertTrue(appliedChange.isPresent());
+        assertEquals(
+            CharStreams.toString(loader.getScriptReader(change, true /* undo */)),
+            appliedChange.get().getDowngradeScript());
+      }
+    }
+  }
+
+  @Test
+  public void testRollback() throws Exception {
+    // Run a normal migration which will apply one the change found in the testmigration package.
+    MigrationLoader loader = new JavaMigrationLoader(
+        "org.apache.aurora.scheduler.storage.db.testmigration");
+    Injector injector = createMigrationInjector(loader);
+
+    MigrationManager migrationManager = injector.getInstance(MigrationManager.class);
+
+    migrationManager.migrate();
+
+    // Now we intentionally pass a reference to a non-existent package to ensure that no migrations
+    // are found. As such a rollback is expected to be detected and a downgrade be performed.
+    MigrationLoader rollbackLoader = new JavaMigrationLoader(
+        "org.apache.aurora.scheduler.storage.db.nomigrations");
+    Injector rollbackInjector = createMigrationInjector(rollbackLoader);
+    MigrationManager rollbackManager = rollbackInjector.getInstance(MigrationManager.class);
+
+    rollbackManager.migrate();
+
+    SqlSessionFactory sqlSessionFactory = rollbackInjector.getInstance(SqlSessionFactory.class);
+
+    try (SqlSession session = sqlSessionFactory.openSession()) {
+      MigrationMapper mapper = session.getMapper(MigrationMapper.class);
+
+      assertTrue(mapper.selectAll().isEmpty());
+      try (Connection c = session.getConnection()) {
+        try (PreparedStatement select = c.prepareStatement("SELECT * FROM V001_test_table")) {
+          select.execute();
+          fail("Select from V001_test_table should have failed, the table should have been "
+              + "dropped.");
+        } catch (SQLException e) {
+          // This exception is expected.
+          assertTrue(e.getMessage().startsWith("Table \"V001_TEST_TABLE\" not found"));
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/test/java/org/apache/aurora/scheduler/storage/db/testmigration/V001_TestMigration.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/db/testmigration/V001_TestMigration.java b/src/test/java/org/apache/aurora/scheduler/storage/db/testmigration/V001_TestMigration.java
new file mode 100644
index 0000000..45a5b5e
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/storage/db/testmigration/V001_TestMigration.java
@@ -0,0 +1,40 @@
+/**
+ * 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.db.testmigration;
+
+import java.math.BigDecimal;
+
+import org.apache.ibatis.migration.MigrationScript;
+
+public class V001_TestMigration implements MigrationScript {
+  @Override
+  public BigDecimal getId() {
+    return BigDecimal.valueOf(1L);
+  }
+
+  @Override
+  public String getDescription() {
+    return "A test migration";
+  }
+
+  @Override
+  public String getUpScript() {
+    return "CREATE TABLE V001_test_table(id int);";
+  }
+
+  @Override
+  public String getDownScript() {
+    return "DROP TABLE V001_test_table;";
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/5ecded78/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
index d93eed9..b5fd204 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
@@ -54,7 +54,7 @@ import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.TaskTestUtil;
 import org.apache.aurora.scheduler.storage.SnapshotStore;
 import org.apache.aurora.scheduler.storage.Storage;
-import org.apache.aurora.scheduler.storage.db.DbModule;
+import org.apache.aurora.scheduler.storage.db.MigrationManager;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
@@ -70,6 +70,7 @@ import org.junit.Test;
 import static org.apache.aurora.common.inject.Bindings.KeyFactory.PLAIN;
 import static org.apache.aurora.common.util.testing.FakeBuildInfo.generateBuildInfo;
 import static org.apache.aurora.scheduler.storage.Storage.MutateWork.NoResult;
+import static org.apache.aurora.scheduler.storage.db.DbModule.testModuleWithWorkQueue;
 import static org.apache.aurora.scheduler.storage.db.DbUtil.createStorage;
 import static org.apache.aurora.scheduler.storage.db.DbUtil.createStorageInjector;
 import static org.junit.Assert.assertEquals;
@@ -88,7 +89,7 @@ public class SnapshotStoreImplIT {
     storage = dbTaskStore
         ? createStorage()
         : createStorageInjector(
-        DbModule.testModuleWithWorkQueue(PLAIN, Optional.of(new InMemStoresModule(PLAIN))))
+        testModuleWithWorkQueue(PLAIN, Optional.of(new InMemStoresModule(PLAIN))))
         .getInstance(Storage.class);
 
     FakeClock clock = new FakeClock();
@@ -97,7 +98,8 @@ public class SnapshotStoreImplIT {
         generateBuildInfo(),
         clock,
         storage,
-        dbTaskStore);
+        dbTaskStore,
+        createStorageInjector(testModuleWithWorkQueue()).getInstance(MigrationManager.class));
   }
 
   private static Snapshot makeComparable(Snapshot snapshot) {