[ASTERIXDB-1992][ING] Suspend/Resume active entities

- user model changes: wait for completion only returns
  when the entity becomes inactive.
- storage format changes: no
- interface changes:
  01) Introduce:
      IMetadataLockManager for entity locking.
      IActiveNotificationHandler for handling active events.
      IRetryPolicy for recovery of failed active jobs.
      IActiveEntityController for controlling active entities.
  02) IJobLifecycleListener.notifyJobFinish now passes the
      JobStatus and Exceptions.
  03) IActiveEntityEventsListener.isActive() returns true,
      if entity is active, false otherwise.
  04) IActiveEntityEventsListener.unregister() removes the
      listener upon entity deletion.
  05) IActiveEntityEventsListener.getJobFailure() returns
      the job failure if the entity is in a failed state.
  06) IStatementExecutor.getComponentProvider() returns
      the storage component provider.
  07) IStatementExecutor.getApplicationContext() returns
      the application context.
  08) IMetadataManager.upsertEntity to perfrom metadata
      entities' upsert operations.
  09) IMetadataNode.upsertEntity to perfrom metadata
      entities' upsert operations
  10) ICcApplicationContext.getMetadataLockManager() returns
      the lock manger.
  11) ICcApplicationContext.getClusterStateManager() returns
      the cluster state manager.

details:
- Starting and stopping of active entities now go through
  their listeners rather than having the listener indicating
  active entity and not having it indicating inactive entity.
  This facilitates suspend/resume operations of long running
  active jobs for the sake of DDL operations or topology
  changes.
- Unit tests for the vast majority of code paths and
  different possible scenarios have been added.

Change-Id: Ifeac8c73e6bad39a13663b84a52121356e3c6b40
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1875
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
BAD: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <mhubail@apache.org>
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
index 1141912..a810576 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
@@ -30,7 +30,9 @@
         JOB_FINISHED,
         PARTITION_EVENT,
         EXTENSION_EVENT,
-        STATS_UPDATED
+        STATS_UPDATED,
+        STATE_CHANGED,
+        FAILURE
     }
 
     private final JobId jobId;
@@ -45,10 +47,6 @@
         this.eventObject = eventObject;
     }
 
-    public ActiveEvent(JobId jobId, Kind eventKind, EntityId entityId) {
-        this(jobId, eventKind, entityId, null);
-    }
-
     public JobId getJobId() {
         return jobId;
     }
@@ -79,8 +77,8 @@
             return true;
         }
         ActiveEvent other = (ActiveEvent) o;
-        return Objects.equals(entityId, other.entityId) && Objects.equals(eventKind, other.eventKind) && Objects
-                .equals(eventObject, other.eventObject);
+        return Objects.equals(entityId, other.entityId) && Objects.equals(eventKind, other.eventKind)
+                && Objects.equals(eventObject, other.eventObject);
     }
 
     @Override
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJobNotificationHandler.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJobNotificationHandler.java
deleted file mode 100644
index d2b8a89..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJobNotificationHandler.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.asterix.active;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.active.ActiveEvent.Kind;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class ActiveJobNotificationHandler implements Runnable {
-    public static final String ACTIVE_ENTITY_PROPERTY_NAME = "ActiveJob";
-    private static final Logger LOGGER = Logger.getLogger(ActiveJobNotificationHandler.class.getName());
-    private static final boolean DEBUG = false;
-    private final LinkedBlockingQueue<ActiveEvent> eventInbox;
-    private final Map<EntityId, IActiveEntityEventsListener> entityEventListeners;
-    private final Map<JobId, EntityId> jobId2ActiveJobInfos;
-
-    public ActiveJobNotificationHandler() {
-        this.eventInbox = new LinkedBlockingQueue<>();
-        this.jobId2ActiveJobInfos = new HashMap<>();
-        this.entityEventListeners = new HashMap<>();
-    }
-
-    @Override
-    public void run() {
-        Thread.currentThread().setName(ActiveJobNotificationHandler.class.getSimpleName());
-        LOGGER.log(Level.INFO, "Started " + ActiveJobNotificationHandler.class.getSimpleName());
-        while (!Thread.interrupted()) {
-            try {
-                ActiveEvent event = getEventInbox().take();
-                EntityId entityId = jobId2ActiveJobInfos.get(event.getJobId());
-                if (entityId != null) {
-                    IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-                    LOGGER.log(Level.FINER, "Next event is of type " + event.getEventKind());
-                    if (event.getEventKind() == Kind.JOB_FINISHED) {
-                        LOGGER.log(Level.FINER, "Removing the job");
-                        jobId2ActiveJobInfos.remove(event.getJobId());
-                    }
-                    if (listener != null) {
-                        LOGGER.log(Level.FINER, "Notifying the listener");
-                        listener.notify(event);
-                    }
-
-                } else {
-                    LOGGER.log(Level.SEVERE, "Entity not found for received message for job " + event.getJobId());
-                }
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-            } catch (Exception e) {
-                LOGGER.log(Level.SEVERE, "Error handling an active job event", e);
-            }
-        }
-        LOGGER.log(Level.INFO, "Stopped " + ActiveJobNotificationHandler.class.getSimpleName());
-    }
-
-    public synchronized void removeListener(IActiveEntityEventsListener listener) throws HyracksDataException {
-        LOGGER.log(Level.FINER, "Removing the listener since it is not active anymore");
-        unregisterListener(listener);
-    }
-
-    public IActiveEntityEventsListener getActiveEntityListener(EntityId entityId) {
-        if (DEBUG) {
-            LOGGER.log(Level.WARNING, "getActiveEntityListener(EntityId entityId) was called with entity " + entityId);
-            IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-            LOGGER.log(Level.WARNING, "Listener found: " + listener);
-        }
-        return entityEventListeners.get(entityId);
-    }
-
-    public EntityId getEntity(JobId jobId) {
-        return jobId2ActiveJobInfos.get(jobId);
-    }
-
-    public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification) {
-        LOGGER.log(Level.FINER,
-                "notifyJobCreation(JobId jobId, JobSpecification jobSpecification) was called with jobId = " + jobId);
-        Object property = jobSpecification.getProperty(ACTIVE_ENTITY_PROPERTY_NAME);
-        if (property == null || !(property instanceof EntityId)) {
-            LOGGER.log(Level.FINER, "Job was is not active. property found to be: " + property);
-            return;
-        }
-        EntityId entityId = (EntityId) property;
-        monitorJob(jobId, entityId);
-        boolean found = jobId2ActiveJobInfos.get(jobId) != null;
-        LOGGER.log(Level.FINER, "Job was found to be: " + (found ? "Active" : "Inactive"));
-        IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-        if (listener != null) {
-            // It is okay to bypass the event inbox in this case because we know this is the first event for this entity
-            listener.notify(new ActiveEvent(jobId, Kind.JOB_CREATED, entityId, jobSpecification));
-        }
-        LOGGER.log(Level.FINER, "Listener was notified" + jobId);
-    }
-
-    public LinkedBlockingQueue<ActiveEvent> getEventInbox() {
-        return eventInbox;
-    }
-
-    public synchronized IActiveEntityEventsListener[] getEventListeners() {
-        if (DEBUG) {
-            LOGGER.log(Level.WARNING, "getEventListeners() was called");
-            LOGGER.log(Level.WARNING, "returning " + entityEventListeners.size() + " Listeners");
-        }
-        return entityEventListeners.values().toArray(new IActiveEntityEventsListener[entityEventListeners.size()]);
-    }
-
-    public synchronized void registerListener(IActiveEntityEventsListener listener) throws HyracksDataException {
-        if (DEBUG) {
-            LOGGER.log(Level.FINER, "registerListener(IActiveEntityEventsListener listener) was called for the entity "
-                    + listener.getEntityId());
-        }
-        if (entityEventListeners.containsKey(listener.getEntityId())) {
-            throw new HyracksDataException(
-                    "Active Entity Listener " + listener.getEntityId() + " is already registered");
-        }
-        entityEventListeners.put(listener.getEntityId(), listener);
-    }
-
-    public synchronized void unregisterListener(IActiveEntityEventsListener listener) throws HyracksDataException {
-        LOGGER.log(Level.FINER, "unregisterListener(IActiveEntityEventsListener listener) was called for the entity "
-                + listener.getEntityId());
-        IActiveEntityEventsListener registeredListener = entityEventListeners.remove(listener.getEntityId());
-        if (registeredListener == null) {
-            throw new HyracksDataException(
-                    "Active Entity Listener " + listener.getEntityId() + " hasn't been registered");
-        }
-    }
-
-    public synchronized void monitorJob(JobId jobId, EntityId activeJob) {
-        if (DEBUG) {
-            LOGGER.log(Level.WARNING, "monitorJob(JobId jobId, ActiveJob activeJob) called with job id: " + jobId);
-            boolean found = jobId2ActiveJobInfos.get(jobId) != null;
-            LOGGER.log(Level.WARNING, "Job was found to be: " + (found ? "Active" : "Inactive"));
-        }
-        if (entityEventListeners.containsKey(activeJob)) {
-            if (jobId2ActiveJobInfos.containsKey(jobId)) {
-                LOGGER.severe("Job is already being monitored for job: " + jobId);
-                return;
-            }
-            if (DEBUG) {
-                LOGGER.log(Level.WARNING, "monitoring started for job id: " + jobId);
-            }
-        } else {
-            LOGGER.severe("No listener was found for the entity: " + activeJob);
-        }
-        jobId2ActiveJobInfos.put(jobId, activeJob);
-    }
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveLifecycleListener.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveLifecycleListener.java
deleted file mode 100644
index 86c3e7d..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveLifecycleListener.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.asterix.active;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.active.ActiveEvent.Kind;
-import org.apache.asterix.active.message.ActivePartitionMessage;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.api.job.IJobLifecycleListener;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class ActiveLifecycleListener implements IJobLifecycleListener {
-
-    private static final Logger LOGGER = Logger.getLogger(ActiveLifecycleListener.class.getName());
-
-    private final ActiveJobNotificationHandler notificationHandler;
-    private final LinkedBlockingQueue<ActiveEvent> jobEventInbox;
-    private final ExecutorService executorService;
-
-    public ActiveLifecycleListener() {
-        notificationHandler = new ActiveJobNotificationHandler();
-        jobEventInbox = notificationHandler.getEventInbox();
-        executorService = Executors.newSingleThreadExecutor();
-        executorService.execute(notificationHandler);
-    }
-
-    @Override
-    public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
-        EntityId entityId = notificationHandler.getEntity(jobId);
-        if (entityId != null) {
-            jobEventInbox.add(new ActiveEvent(jobId, Kind.JOB_STARTED, entityId));
-        }
-    }
-
-    @Override
-    public synchronized void notifyJobFinish(JobId jobId) throws HyracksException {
-        EntityId entityId = notificationHandler.getEntity(jobId);
-        if (entityId != null) {
-            jobEventInbox.add(new ActiveEvent(jobId, Kind.JOB_FINISHED, entityId));
-        } else {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("NO NEED TO NOTIFY JOB FINISH!");
-            }
-        }
-    }
-
-    @Override
-    public void notifyJobCreation(JobId jobId, JobSpecification spec) throws HyracksException {
-        notificationHandler.notifyJobCreation(jobId, spec);
-    }
-
-    public void receive(ActivePartitionMessage message) {
-        jobEventInbox.add(new ActiveEvent(message.getJobId(), Kind.PARTITION_EVENT,
-                message.getActiveRuntimeId().getEntityId(), message));
-    }
-
-    public void stop() {
-        executorService.shutdown();
-    }
-
-    public ActiveJobNotificationHandler getNotificationHandler() {
-        return notificationHandler;
-    }
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeManager.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeManager.java
index e71367a..18368ae 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeManager.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveRuntimeManager.java
@@ -28,6 +28,10 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
 public class ActiveRuntimeManager {
 
     private static final Logger LOGGER = Logger.getLogger(ActiveRuntimeManager.class.getName());
@@ -65,11 +69,18 @@
         return activeRuntimes.get(runtimeId);
     }
 
-    public void registerRuntime(ActiveRuntimeId runtimeId, ActiveSourceOperatorNodePushable feedRuntime) {
+    public void registerRuntime(ActiveRuntimeId runtimeId, ActiveSourceOperatorNodePushable feedRuntime)
+            throws HyracksDataException {
+        if (activeRuntimes.containsKey(runtimeId)) {
+            throw new RuntimeDataException(ErrorCode.ACTIVE_RUNTIME_IS_ALREADY_REGISTERED, runtimeId);
+        }
         activeRuntimes.put(runtimeId, feedRuntime);
     }
 
-    public synchronized void deregisterRuntime(ActiveRuntimeId runtimeId) {
+    public void deregisterRuntime(ActiveRuntimeId runtimeId) throws HyracksDataException {
+        if (!activeRuntimes.containsKey(runtimeId)) {
+            throw new RuntimeDataException(ErrorCode.ACTIVE_RUNTIME_IS_NOT_REGISTERED, runtimeId);
+        }
         activeRuntimes.remove(runtimeId);
     }
 
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java
index af8f5ca..eb43d10 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java
@@ -20,27 +20,44 @@
 
 public enum ActivityState {
     /**
-     * The initial state of an activity.
-     */
-    CREATED,
-    /**
-     * The starting state and a possible terminal state. Next state can only be {@code ActivityState.STARTING}
+     * The starting state and a possible terminal state.
      */
     STOPPED,
     /**
-     * A terminal state
+     * Failure to recover from a temporary faliure caused the activity to fail permanantly.
+     * No further recovery attempts will be made.
      */
-    FAILED,
+    PERMANENTLY_FAILED,
     /**
-     * An intermediate state. Next state can only be {@code ActivityState.STARTED} or {@code ActivityState.FAILED}
+     * An unexpected failure caused the activity to fail but recovery attempts will start taking place
+     */
+    TEMPORARILY_FAILED,
+    /**
+     * Attempting to recover from temporary failure.
+     */
+    RECOVERING,
+    /**
+     * During an attempt to start the activity
      */
     STARTING,
     /**
-     * An intermediate state. Next state can only be {@code ActivityState.STOPPING} or {@code ActivityState.FAILED}
+     * The activity has been started successfully and is running
      */
-    STARTED,
+    RUNNING,
     /**
-     * An intermediate state. Next state can only be {@code ActivityState.STOPPED} or {@code ActivityState.FAILED}
+     * During an attempt to gracefully stop the activity
      */
-    STOPPING
+    STOPPING,
+    /**
+     * During an attempt to gracefully suspend the activity
+     */
+    SUSPENDING,
+    /**
+     * The activitiy has been suspended successfully. Next state must be resuming
+     */
+    SUSPENDED,
+    /**
+     * During an attempt to restart the activity after suspension
+     */
+    RESUMING
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicy.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicy.java
new file mode 100644
index 0000000..6633810
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicy.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.active;
+
+public class CountRetryPolicy implements IRetryPolicy {
+
+    private final int count;
+    private int attempted = 0;
+
+    public CountRetryPolicy(int count) {
+        this.count = count;
+    }
+
+    @Override
+    public boolean retry() {
+        if (attempted < count) {
+            attempted++;
+            return true;
+        }
+        return false;
+    }
+
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java
new file mode 100644
index 0000000..5e26ae4
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.active;
+
+public class CountRetryPolicyFactory implements IRetryPolicyFactory {
+
+    private final int count;
+
+    public CountRetryPolicyFactory(int count) {
+        this.count = count;
+    }
+
+    @Override
+    public IRetryPolicy create(IActiveEntityEventsListener listener) {
+        return new CountRetryPolicy(count);
+    }
+
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEventSubscriber.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java
similarity index 82%
rename from asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEventSubscriber.java
rename to asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java
index 69f7f1c..f9357b4 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEventSubscriber.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java
@@ -21,12 +21,12 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * An active event subscriber that subscribe to events related to active entity
+ * An active event subscriber that subscribes to events related to active entity
  */
-public interface IActiveEventSubscriber {
+public interface IActiveEntityEventSubscriber {
 
     /**
-     * Notify the subscriber of a new event
+     * Notifies the subscriber of a new event
      *
      * @param event
      * @throws HyracksDataException
@@ -43,14 +43,9 @@
     /**
      * Wait until the terminal event has been received
      *
-     * @throws InterruptedException
+     * @throws Exception
      */
-    void sync() throws InterruptedException;
-
-    /**
-     * Stop watching events
-     */
-    void unsubscribe();
+    void sync() throws HyracksDataException, InterruptedException;
 
     /**
      * callback upon successful subscription
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
index 4bc02f3..03b0cfc 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
@@ -58,7 +58,7 @@
      * @param subscriber
      * @throws HyracksDataException
      */
-    void subscribe(IActiveEventSubscriber subscriber) throws HyracksDataException;
+    void subscribe(IActiveEntityEventSubscriber subscriber) throws HyracksDataException;
 
     /**
      * The most recent acquired stats for the active entity
@@ -79,4 +79,21 @@
      * @throws HyracksDataException
      */
     void refreshStats(long timeout) throws HyracksDataException;
+
+    /**
+     * @return true, if entity is active, false otherwise
+     */
+    boolean isActive();
+
+    /**
+     * unregister the listener upon deletion of entity
+     *
+     * @throws HyracksDataException
+     */
+    void unregister() throws HyracksDataException;
+
+    /**
+     * Get the job failure for the last failed run
+     */
+    Exception getJobFailure();
 }
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java
new file mode 100644
index 0000000..8b9f232
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java
@@ -0,0 +1,93 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.active;
+
+import org.apache.asterix.active.message.ActivePartitionMessage;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IJobLifecycleListener;
+
+/**
+ * Represents the notification handler for events of active entity jobs
+ */
+public interface IActiveNotificationHandler {
+
+    /**
+     * Recover all active jobs that failed
+     *
+     * @throws HyracksDataException
+     */
+    void recover() throws HyracksDataException;
+
+    /**
+     * Set whether handler initialization has completed or not
+     *
+     * @param initialized
+     * @throws HyracksDataException
+     */
+    void setInitialized(boolean initialized) throws HyracksDataException;
+
+    /**
+     * @return true if initialization has completed, false otherwise
+     */
+    boolean isInitialized();
+
+    /**
+     * Register a listener for events of an active entity
+     *
+     * @param listener
+     *            the listener to register
+     * @throws HyracksDataException
+     *             if the active entity already has a listener associated with it
+     */
+    void registerListener(IActiveEntityEventsListener listener) throws HyracksDataException;
+
+    /**
+     * Unregister a listener for events of an active entity
+     *
+     * @param listener
+     *            the listener to unregister
+     * @throws HyracksDataException
+     *             if the entity is still active or if the listener was not registered
+     */
+    void unregisterListener(IActiveEntityEventsListener listener) throws HyracksDataException;
+
+    /**
+     * @return all the registered event listeners
+     */
+    IActiveEntityEventsListener[] getEventListeners();
+
+    /**
+     * Lookup an event listener using the entity id
+     *
+     * @param entityId
+     *            the lookup key
+     * @return the registered listener if found, null otherwise
+     */
+    IActiveEntityEventsListener getListener(EntityId entityId);
+
+    /**
+     * Recieves an active job message from an nc
+     *
+     * @param message
+     *            the message
+     */
+    void receive(ActivePartitionMessage message);
+
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicy.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicy.java
new file mode 100644
index 0000000..a010984
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicy.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.active;
+
+@FunctionalInterface
+public interface IRetryPolicy {
+    /**
+     * @return true if one more attempt should be done
+     */
+    boolean retry();
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java
new file mode 100644
index 0000000..a946337
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.active;
+
+@FunctionalInterface
+public interface IRetryPolicyFactory {
+    /**
+     * @return an instance of retry policy
+     */
+    IRetryPolicy create(IActiveEntityEventsListener listener);
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java
new file mode 100644
index 0000000..074f8f4
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.active;
+
+public class InfiniteRetryPolicy implements IRetryPolicy {
+
+    private final IActiveEntityEventsListener listener;
+
+    public InfiniteRetryPolicy(IActiveEntityEventsListener listener) {
+        this.listener = listener;
+    }
+
+    @Override
+    public boolean retry() {
+        synchronized (listener) {
+            try {
+                listener.wait(5000); //NOSONAR this method is being called in a while loop
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                return false;
+            }
+        }
+        return true;
+    }
+
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java
new file mode 100644
index 0000000..b31d245
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.active;
+
+public class InfiniteRetryPolicyFactory implements IRetryPolicyFactory {
+
+    @Override
+    public IRetryPolicy create(IActiveEntityEventsListener listener) {
+        return new InfiniteRetryPolicy(listener);
+    }
+
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java
new file mode 100644
index 0000000..1596c17
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.active;
+
+public class NoRetryPolicyFactory implements IRetryPolicyFactory {
+    public static final NoRetryPolicyFactory INSTANCE = new NoRetryPolicyFactory();
+    private static final IRetryPolicy policy = () -> false;
+
+    private NoRetryPolicyFactory() {
+    }
+
+    @Override
+    public IRetryPolicy create(IActiveEntityEventsListener listener) {
+        return policy;
+    }
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/SingleThreadEventProcessor.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/SingleThreadEventProcessor.java
new file mode 100644
index 0000000..0a36216
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/SingleThreadEventProcessor.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.active;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class SingleThreadEventProcessor<T> implements Runnable {
+
+    private static final Logger LOGGER = Logger.getLogger(SingleThreadEventProcessor.class.getName());
+    private final String name;
+    private final LinkedBlockingQueue<T> eventInbox;
+    private final ExecutorService executorService;
+    private final Future<?> future;
+
+    public SingleThreadEventProcessor(String threadName) {
+        this.name = threadName;
+        eventInbox = new LinkedBlockingQueue<>();
+        executorService = Executors.newSingleThreadExecutor(r -> new Thread(r, threadName));
+        future = executorService.submit(this);
+    }
+
+    @Override
+    public final void run() {
+        LOGGER.log(Level.INFO, "Started " + Thread.currentThread().getName());
+        while (!Thread.currentThread().isInterrupted()) {
+            try {
+                T event = eventInbox.take();
+                handle(event);
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            } catch (Exception e) {
+                LOGGER.log(Level.SEVERE, "Error handling an event", e);
+            }
+        }
+        LOGGER.log(Level.WARNING, "Stopped " + Thread.currentThread().getName());
+    }
+
+    protected abstract void handle(T event) throws Exception; //NOSONAR
+
+    public void add(T event) {
+        if (!eventInbox.add(event)) {
+            throw new IllegalStateException();
+        }
+    }
+
+    public void stop() throws HyracksDataException, InterruptedException {
+        future.cancel(true);
+        executorService.shutdown();
+        if (!executorService.awaitTermination(10, TimeUnit.SECONDS)) {
+            throw HyracksDataException.create(ErrorCode.FAILED_TO_SHUTDOWN_EVENT_PROCESSOR, name);
+        }
+    }
+}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
index 9391044..a47d5a5 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
@@ -21,8 +21,8 @@
 import java.io.Serializable;
 import java.util.Objects;
 
-import org.apache.asterix.active.ActiveLifecycleListener;
 import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.IActiveNotificationHandler;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -64,7 +64,7 @@
 
     @Override
     public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        ActiveLifecycleListener activeListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
+        IActiveNotificationHandler activeListener = (IActiveNotificationHandler) appCtx.getActiveNotificationHandler();
         activeListener.receive(this);
     }
 
@@ -87,7 +87,7 @@
             return true;
         }
         ActivePartitionMessage other = (ActivePartitionMessage) o;
-        return Objects.equals(other.activeRuntimeId, activeRuntimeId) && Objects.equals(other.jobId, jobId) && Objects
-                .equals(other.payload, payload);
+        return Objects.equals(other.activeRuntimeId, activeRuntimeId) && Objects.equals(other.jobId, jobId)
+                && Objects.equals(other.payload, payload);
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 213c60b..cd0a63c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.algebra.operators.physical;
 
-import org.apache.asterix.metadata.MetadataException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
index eee6bdc..1d47095 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
@@ -25,6 +25,8 @@
 import java.util.logging.Logger;
 
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
+import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.lang.common.base.Statement;
@@ -51,11 +53,13 @@
     public void validateOperation(ICcApplicationContext appCtx, Dataverse defaultDataverse, Statement stmt)
             throws AsterixException {
 
-        if (!(ClusterStateManager.INSTANCE.getState().equals(ClusterState.ACTIVE)
-                && ClusterStateManager.INSTANCE.isGlobalRecoveryCompleted())) {
+        final IClusterStateManager clusterStateManager = ClusterStateManager.INSTANCE;
+        final IGlobalRecoveryManager globalRecoveryManager = appCtx.getGlobalRecoveryManager();
+        if (!(clusterStateManager.getState().equals(ClusterState.ACTIVE)
+                && globalRecoveryManager.isRecoveryCompleted())) {
             int maxWaitCycles = appCtx.getExternalProperties().getMaxWaitClusterActive();
             try {
-                ClusterStateManager.INSTANCE.waitForState(ClusterState.ACTIVE, maxWaitCycles, TimeUnit.SECONDS);
+                clusterStateManager.waitForState(ClusterState.ACTIVE, maxWaitCycles, TimeUnit.SECONDS);
             } catch (HyracksDataException e) {
                 throw new AsterixException(e);
             } catch (InterruptedException e) {
@@ -64,7 +68,7 @@
                 }
                 Thread.currentThread().interrupt();
             }
-            if (!ClusterStateManager.INSTANCE.getState().equals(ClusterState.ACTIVE)) {
+            if (!clusterStateManager.getState().equals(ClusterState.ACTIVE)) {
                 throw new AsterixException("Cluster is in " + ClusterState.UNUSABLE + " state."
                         + "\n One or more Node Controllers have left or haven't joined yet.\n");
             } else {
@@ -74,16 +78,16 @@
             }
         }
 
-        if (ClusterStateManager.INSTANCE.getState().equals(ClusterState.UNUSABLE)) {
+        if (clusterStateManager.getState().equals(ClusterState.UNUSABLE)) {
             throw new AsterixException("Cluster is in " + ClusterState.UNUSABLE + " state."
                     + "\n One or more Node Controllers have left.\n");
         }
 
-        if (!ClusterStateManager.INSTANCE.isGlobalRecoveryCompleted()) {
+        if (!globalRecoveryManager.isRecoveryCompleted()) {
             int maxWaitCycles = appCtx.getExternalProperties().getMaxWaitClusterActive();
             int waitCycleCount = 0;
             try {
-                while (!ClusterStateManager.INSTANCE.isGlobalRecoveryCompleted() && waitCycleCount < maxWaitCycles) {
+                while (!globalRecoveryManager.isRecoveryCompleted() && waitCycleCount < maxWaitCycles) {
                     Thread.sleep(1000);
                     waitCycleCount++;
                 }
@@ -93,7 +97,7 @@
                 }
                 Thread.currentThread().interrupt();
             }
-            if (!ClusterStateManager.INSTANCE.isGlobalRecoveryCompleted()) {
+            if (!globalRecoveryManager.isRecoveryCompleted()) {
                 throw new AsterixException("Cluster Global recovery is not yet complete and the system is in "
                         + ClusterState.ACTIVE + " state");
             }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index b8a6d8a..a1c5cf43 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -36,6 +36,7 @@
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.external.util.ExternalDataUtils;
@@ -72,7 +73,6 @@
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceId;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ActiveStatsApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ActiveStatsApiServlet.java
index 593faa6..15f0ace 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ActiveStatsApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ActiveStatsApiServlet.java
@@ -18,15 +18,13 @@
  */
 package org.apache.asterix.api.http.server;
 
-import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.concurrent.ConcurrentMap;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import io.netty.handler.codec.http.HttpResponseStatus;
-import org.apache.asterix.active.ActiveLifecycleListener;
 import org.apache.asterix.active.IActiveEntityEventsListener;
+import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.external.feed.watch.StatsSubscriber;
 import org.apache.hyracks.http.api.IServletRequest;
@@ -38,19 +36,21 @@
 import com.fasterxml.jackson.databind.SerializationFeature;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
+import io.netty.handler.codec.http.HttpResponseStatus;
+
 public class ActiveStatsApiServlet extends AbstractServlet {
 
     private static final Logger LOGGER = Logger.getLogger(ActiveStatsApiServlet.class.getName());
     private static final int DEFAULT_EXPIRE_TIME = 2000;
-    private final ActiveLifecycleListener activeLifecycleListener;
+    private final ActiveNotificationHandler activeNotificationHandler;
 
     public ActiveStatsApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx) {
         super(ctx, paths);
-        this.activeLifecycleListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
+        this.activeNotificationHandler = (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
     }
 
     private JsonNode constructNode(ObjectMapper om, IActiveEntityEventsListener eventListener, long currentTime,
-            long ttl) throws InterruptedException, IOException {
+            long ttl) throws Exception {
         long statsTimeStamp = eventListener.getStatsTimeStamp();
         if (currentTime - statsTimeStamp > ttl) {
             StatsSubscriber subscriber = new StatsSubscriber(eventListener);
@@ -66,7 +66,7 @@
         // Obtain all feed status
         String localPath = localPath(request);
         int expireTime;
-        IActiveEntityEventsListener[] listeners = activeLifecycleListener.getNotificationHandler().getEventListeners();
+        IActiveEntityEventsListener[] listeners = activeNotificationHandler.getEventListeners();
         ObjectMapper om = new ObjectMapper();
         om.enable(SerializationFeature.INDENT_OUTPUT);
         ObjectNode resNode = om.createObjectNode();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
index 03958ed..d9a63f7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
@@ -29,7 +29,6 @@
 import java.util.logging.Logger;
 
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -93,7 +92,7 @@
             MetadataManager.INSTANCE.init();
             MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
             // Retrieves file splits of the dataset.
-            MetadataProvider metadataProvider = new MetadataProvider(appCtx, null, new StorageComponentProvider());
+            MetadataProvider metadataProvider = new MetadataProvider(appCtx, null);
             try {
                 metadataProvider.setMetadataTxnContext(mdTxnCtx);
                 Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
index e9d231a..27e2806 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
@@ -36,8 +36,9 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -72,7 +73,7 @@
     private final ExecutorService executor = Executors.newSingleThreadExecutor();
 
     // A queue that maintains submitted rebalance requests.
-    private final Queue<Future> rebalanceTasks = new ArrayDeque<>();
+    private final Queue<Future<Void>> rebalanceTasks = new ArrayDeque<>();
 
     // A queue that tracks the termination of rebalance threads.
     private final Queue<CountDownLatch> rebalanceFutureTerminated = new ArrayDeque<>();
@@ -141,7 +142,7 @@
 
     // Cancels all rebalance tasks.
     private synchronized void cancelRebalance() throws InterruptedException {
-        for (Future rebalanceTask : rebalanceTasks) {
+        for (Future<Void> rebalanceTask : rebalanceTasks) {
             rebalanceTask.cancel(true);
         }
     }
@@ -156,14 +157,15 @@
     private synchronized CountDownLatch scheduleRebalance(String dataverseName, String datasetName,
             String[] targetNodes, IServletResponse response) {
         CountDownLatch terminated = new CountDownLatch(1);
-        Future task = executor.submit(() -> doRebalance(dataverseName, datasetName, targetNodes, response, terminated));
+        Future<Void> task =
+                executor.submit(() -> doRebalance(dataverseName, datasetName, targetNodes, response, terminated));
         rebalanceTasks.add(task);
         rebalanceFutureTerminated.add(terminated);
         return terminated;
     }
 
     // Performs the actual rebalance.
-    private void doRebalance(String dataverseName, String datasetName, String[] targetNodes, IServletResponse response,
+    private Void doRebalance(String dataverseName, String datasetName, String[] targetNodes, IServletResponse response,
             CountDownLatch terminated) {
         try {
             // Sets the content type.
@@ -198,6 +200,7 @@
             // Notify that the rebalance task is terminated.
             terminated.countDown();
         }
+        return null;
     }
 
     // Lists all datasets that should be rebalanced in a given datavserse.
@@ -241,9 +244,23 @@
     // Rebalances a given dataset.
     private void rebalanceDataset(String dataverseName, String datasetName, String[] targetNodes) throws Exception {
         IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
-        MetadataProvider metadataProvider = new MetadataProvider(appCtx, null, new StorageComponentProvider());
-        RebalanceUtil.rebalance(dataverseName, datasetName, new LinkedHashSet<>(Arrays.asList(targetNodes)),
-                metadataProvider, hcc, NoOpDatasetRebalanceCallback.INSTANCE);
+        MetadataProvider metadataProvider = new MetadataProvider(appCtx, null);
+        try {
+            ActiveNotificationHandler activeNotificationHandler =
+                    (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
+            activeNotificationHandler.suspend(metadataProvider);
+            try {
+                IMetadataLockManager lockManager = appCtx.getMetadataLockManager();
+                lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(),
+                        dataverseName + '.' + datasetName);
+                RebalanceUtil.rebalance(dataverseName, datasetName, new LinkedHashSet<>(Arrays.asList(targetNodes)),
+                        metadataProvider, hcc, NoOpDatasetRebalanceCallback.INSTANCE);
+            } finally {
+                activeNotificationHandler.resume(metadataProvider);
+            }
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
     }
 
     // Sends HTTP response to the request client.
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
new file mode 100644
index 0000000..caf4bec
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
@@ -0,0 +1,659 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.app.active;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.active.ActiveEvent;
+import org.apache.asterix.active.ActiveEvent.Kind;
+import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.IActiveEntityEventSubscriber;
+import org.apache.asterix.active.IRetryPolicy;
+import org.apache.asterix.active.IRetryPolicyFactory;
+import org.apache.asterix.active.NoRetryPolicyFactory;
+import org.apache.asterix.active.message.ActiveManagerMessage;
+import org.apache.asterix.active.message.ActivePartitionMessage;
+import org.apache.asterix.active.message.StatsRequestMessage;
+import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.messaging.api.ICCMessageBroker;
+import org.apache.asterix.common.messaging.api.INcAddressedMessage;
+import org.apache.asterix.common.metadata.IDataset;
+import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
+import org.apache.asterix.metadata.api.IActiveEntityController;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.MetadataLockUtil;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobStatus;
+
+public abstract class ActiveEntityEventsListener implements IActiveEntityController {
+
+    private static final Logger LOGGER = Logger.getLogger(ActiveEntityEventsListener.class.getName());
+    private static final ActiveEvent STATE_CHANGED = new ActiveEvent(null, Kind.STATE_CHANGED, null, null);
+    private static final EnumSet<ActivityState> TRANSITION_STATES = EnumSet.of(ActivityState.RESUMING,
+            ActivityState.STARTING, ActivityState.STOPPING, ActivityState.RECOVERING);
+    // finals
+    protected final IClusterStateManager clusterStateManager;
+    protected final ActiveNotificationHandler handler;
+    protected final List<IActiveEntityEventSubscriber> subscribers = new ArrayList<>();
+    protected final IStatementExecutor statementExecutor;
+    protected final ICcApplicationContext appCtx;
+    protected final MetadataProvider metadataProvider;
+    protected final IHyracksClientConnection hcc;
+    protected final EntityId entityId;
+    private final List<Dataset> datasets;
+    protected final ActiveEvent statsUpdatedEvent;
+    protected final String runtimeName;
+    protected final IRetryPolicyFactory retryPolicyFactory;
+    // mutables
+    protected volatile ActivityState state;
+    private AlgebricksAbsolutePartitionConstraint locations;
+    protected ActivityState prevState;
+    protected JobId jobId;
+    protected long statsTimestamp;
+    protected String stats;
+    protected boolean isFetchingStats;
+    protected int numRegistered;
+    protected volatile Future<Void> recoveryTask;
+    protected volatile boolean cancelRecovery;
+    protected volatile boolean suspended = false;
+    // failures
+    protected Exception jobFailure;
+    protected Exception resumeFailure;
+    protected Exception startFailure;
+    protected Exception stopFailure;
+    protected Exception recoverFailure;
+
+    public ActiveEntityEventsListener(IStatementExecutor statementExecutor, ICcApplicationContext appCtx,
+            IHyracksClientConnection hcc, EntityId entityId, List<Dataset> datasets,
+            AlgebricksAbsolutePartitionConstraint locations, String runtimeName, IRetryPolicyFactory retryPolicyFactory)
+            throws HyracksDataException {
+        this.statementExecutor = statementExecutor;
+        this.appCtx = appCtx;
+        this.clusterStateManager = appCtx.getClusterStateManager();
+        this.metadataProvider = new MetadataProvider(appCtx, null);
+        metadataProvider.setConfig(new HashMap<>());
+        this.hcc = hcc;
+        this.entityId = entityId;
+        this.datasets = datasets;
+        this.retryPolicyFactory = retryPolicyFactory;
+        this.state = ActivityState.STOPPED;
+        this.statsTimestamp = -1;
+        this.isFetchingStats = false;
+        this.statsUpdatedEvent = new ActiveEvent(null, Kind.STATS_UPDATED, entityId, null);
+        this.stats = "{\"Stats\":\"N/A\"}";
+        this.runtimeName = runtimeName;
+        this.locations = locations;
+        this.numRegistered = 0;
+        this.handler =
+                (ActiveNotificationHandler) metadataProvider.getApplicationContext().getActiveNotificationHandler();
+        handler.registerListener(this);
+    }
+
+    protected synchronized void setState(ActivityState newState) {
+        LOGGER.log(Level.WARNING, "State is being set to " + newState + " from " + state);
+        this.prevState = state;
+        this.state = newState;
+        if (newState == ActivityState.SUSPENDED) {
+            suspended = true;
+        }
+        notifySubscribers(STATE_CHANGED);
+    }
+
+    @Override
+    public synchronized void notify(ActiveEvent event) {
+        try {
+            LOGGER.warning("EventListener is notified.");
+            ActiveEvent.Kind eventKind = event.getEventKind();
+            switch (eventKind) {
+                case JOB_CREATED:
+                    jobCreated(event);
+                    break;
+                case JOB_STARTED:
+                    start(event);
+                    break;
+                case JOB_FINISHED:
+                    finish(event);
+                    break;
+                case PARTITION_EVENT:
+                    handle((ActivePartitionMessage) event.getEventObject());
+                    break;
+                default:
+                    LOGGER.log(Level.WARNING, "Unhandled feed event notification: " + event);
+                    break;
+            }
+            notifySubscribers(event);
+        } catch (Exception e) {
+            LOGGER.log(Level.SEVERE, "Unhandled Exception", e);
+        }
+    }
+
+    protected void jobCreated(ActiveEvent event) {
+        // Do nothing
+    }
+
+    protected synchronized void handle(ActivePartitionMessage message) {
+        if (message.getEvent() == ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED) {
+            numRegistered++;
+            if (numRegistered == locations.getLocations().length) {
+                setState(ActivityState.RUNNING);
+            }
+        } else if (message.getEvent() == ActivePartitionMessage.ACTIVE_RUNTIME_DEREGISTERED) {
+            numRegistered--;
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    protected void finish(ActiveEvent event) throws HyracksDataException {
+        jobId = null;
+        Pair<JobStatus, List<Exception>> status = (Pair<JobStatus, List<Exception>>) event.getEventObject();
+        JobStatus jobStatus = status.getLeft();
+        List<Exception> exceptions = status.getRight();
+        if (jobStatus.equals(JobStatus.FAILURE)) {
+            jobFailure = exceptions.isEmpty() ? new RuntimeDataException(ErrorCode.UNREPORTED_TASK_FAILURE_EXCEPTION)
+                    : exceptions.get(0);
+            setState(ActivityState.TEMPORARILY_FAILED);
+            if (prevState != ActivityState.SUSPENDING && prevState != ActivityState.RECOVERING) {
+                recover();
+            }
+        } else {
+            setState(state == ActivityState.SUSPENDING ? ActivityState.SUSPENDED : ActivityState.STOPPED);
+        }
+    }
+
+    protected void start(ActiveEvent event) {
+        this.jobId = event.getJobId();
+        numRegistered = 0;
+    }
+
+    @Override
+    public synchronized void subscribe(IActiveEntityEventSubscriber subscriber) throws HyracksDataException {
+        subscriber.subscribed(this);
+        if (!subscriber.isDone()) {
+            subscribers.add(subscriber);
+        }
+    }
+
+    @Override
+    public EntityId getEntityId() {
+        return entityId;
+    }
+
+    @Override
+    public ActivityState getState() {
+        return state;
+    }
+
+    @Override
+    public synchronized boolean isEntityUsingDataset(IDataset dataset) {
+        return isActive() && getDatasets().contains(dataset);
+    }
+
+    @Override
+    public synchronized void remove(Dataset dataset) throws HyracksDataException {
+        if (isActive()) {
+            throw new RuntimeDataException(ErrorCode.CANNOT_REMOVE_DATASET_FROM_ACTIVE_ENTITY, entityId, state);
+        }
+        getDatasets().remove(dataset);
+    }
+
+    @Override
+    public synchronized void add(Dataset dataset) throws HyracksDataException {
+        if (isActive()) {
+            throw new RuntimeDataException(ErrorCode.CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY, entityId, state);
+        }
+        getDatasets().add(dataset);
+    }
+
+    public JobId getJobId() {
+        return jobId;
+    }
+
+    @Override
+    public String getStats() {
+        return stats;
+    }
+
+    @Override
+    public long getStatsTimeStamp() {
+        return statsTimestamp;
+    }
+
+    public String formatStats(List<String> responses) {
+        StringBuilder strBuilder = new StringBuilder();
+        strBuilder.append("{\"Stats\": [").append(responses.get(0));
+        for (int i = 1; i < responses.size(); i++) {
+            strBuilder.append(", ").append(responses.get(i));
+        }
+        strBuilder.append("]}");
+        return strBuilder.toString();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void refreshStats(long timeout) throws HyracksDataException {
+        LOGGER.log(Level.WARNING, "refreshStats called");
+        synchronized (this) {
+            if (state != ActivityState.RUNNING || isFetchingStats) {
+                LOGGER.log(Level.WARNING,
+                        "returning immediately since state = " + state + " and fetchingStats = " + isFetchingStats);
+                return;
+            } else {
+                isFetchingStats = true;
+            }
+        }
+        ICCMessageBroker messageBroker =
+                (ICCMessageBroker) metadataProvider.getApplicationContext().getServiceContext().getMessageBroker();
+        long reqId = messageBroker.newRequestId();
+        List<INcAddressedMessage> requests = new ArrayList<>();
+        List<String> ncs = Arrays.asList(locations.getLocations());
+        for (int i = 0; i < ncs.size(); i++) {
+            requests.add(new StatsRequestMessage(ActiveManagerMessage.REQUEST_STATS,
+                    new ActiveRuntimeId(entityId, runtimeName, i), reqId));
+        }
+        try {
+            List<String> responses = (List<String>) messageBroker.sendSyncRequestToNCs(reqId, ncs, requests, timeout);
+            stats = formatStats(responses);
+            statsTimestamp = System.currentTimeMillis();
+            notifySubscribers(statsUpdatedEvent);
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+        isFetchingStats = false;
+    }
+
+    protected synchronized void notifySubscribers(ActiveEvent event) {
+        notifyAll();
+        Iterator<IActiveEntityEventSubscriber> it = subscribers.iterator();
+        while (it.hasNext()) {
+            IActiveEntityEventSubscriber subscriber = it.next();
+            if (subscriber.isDone()) {
+                it.remove();
+            } else {
+                try {
+                    subscriber.notify(event);
+                } catch (HyracksDataException e) {
+                    LOGGER.log(Level.WARNING, "Failed to notify subscriber", e);
+                }
+                if (subscriber.isDone()) {
+                    it.remove();
+                }
+            }
+        }
+    }
+
+    public AlgebricksAbsolutePartitionConstraint getLocations() {
+        return locations;
+    }
+
+    /**
+     * this method is called whenever an action is requested. It ensures no interleaved requests
+     *
+     * @throws InterruptedException
+     */
+    protected synchronized void waitForNonTransitionState() throws InterruptedException {
+        while (TRANSITION_STATES.contains(state) || suspended) {
+            this.wait();
+        }
+    }
+
+    /**
+     * this method is called before an action call is returned. It ensures that the request didn't fail
+     *
+     */
+    protected synchronized void checkNoFailure() throws HyracksDataException {
+        if (state == ActivityState.PERMANENTLY_FAILED) {
+            throw HyracksDataException.create(jobFailure);
+        }
+    }
+
+    @Override
+    public synchronized void recover() throws HyracksDataException {
+        LOGGER.log(Level.WARNING, "Recover is called on " + entityId);
+        if (recoveryTask != null) {
+            LOGGER.log(Level.WARNING,
+                    "But recovery task for " + entityId + " is already there!! throwing an exception");
+            throw new RuntimeDataException(ErrorCode.DOUBLE_RECOVERY_ATTEMPTS);
+        }
+        if (retryPolicyFactory == NoRetryPolicyFactory.INSTANCE) {
+            LOGGER.log(Level.WARNING, "But it has no recovery policy, so it is set to permanent failure");
+            setState(ActivityState.PERMANENTLY_FAILED);
+        } else {
+            ExecutorService executor = appCtx.getServiceContext().getControllerService().getExecutor();
+            IRetryPolicy policy = retryPolicyFactory.create(this);
+            cancelRecovery = false;
+            setState(ActivityState.TEMPORARILY_FAILED);
+            LOGGER.log(Level.WARNING, "Recovery task has been submitted");
+            recoveryTask = executor.submit(() -> doRecover(policy));
+        }
+    }
+
+    protected Void doRecover(IRetryPolicy policy)
+            throws AlgebricksException, HyracksDataException, InterruptedException {
+        LOGGER.log(Level.WARNING, "Actual Recovery task has started");
+        if (getState() != ActivityState.TEMPORARILY_FAILED) {
+            LOGGER.log(Level.WARNING, "but its state is not temp failure and so we're just returning");
+            return null;
+        }
+        LOGGER.log(Level.WARNING, "calling the policy");
+        while (policy.retry()) {
+            synchronized (this) {
+                if (cancelRecovery) {
+                    recoveryTask = null;
+                    return null;
+                }
+                while (clusterStateManager.getState() != ClusterState.ACTIVE) {
+                    if (cancelRecovery) {
+                        recoveryTask = null;
+                        return null;
+                    }
+                    wait();
+                }
+            }
+            waitForNonTransitionState();
+            IMetadataLockManager lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
+            lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(),
+                    entityId.getDataverse() + '.' + entityId.getEntityName());
+            for (Dataset dataset : getDatasets()) {
+                MetadataLockUtil.modifyDatasetBegin(lockManager, metadataProvider.getLocks(),
+                        dataset.getDataverseName(), DatasetUtil.getFullyQualifiedName(dataset));
+            }
+            synchronized (this) {
+                try {
+                    setState(ActivityState.RECOVERING);
+                    doStart(metadataProvider);
+                    return null;
+                } catch (Exception e) {
+                    LOGGER.log(Level.WARNING, "Attempt to revive " + entityId + " failed", e);
+                    setState(ActivityState.TEMPORARILY_FAILED);
+                    recoverFailure = e;
+                } finally {
+                    metadataProvider.getLocks().reset();
+                }
+                notifyAll();
+            }
+        }
+        IMetadataLockManager lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
+        try {
+            lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(),
+                    entityId.getDataverse() + '.' + entityId.getEntityName());
+            for (Dataset dataset : getDatasets()) {
+                MetadataLockUtil.modifyDatasetBegin(lockManager, metadataProvider.getLocks(), dataset.getDatasetName(),
+                        DatasetUtil.getFullyQualifiedName(dataset));
+            }
+            synchronized (this) {
+                if (state == ActivityState.TEMPORARILY_FAILED) {
+                    setState(ActivityState.PERMANENTLY_FAILED);
+                    recoveryTask = null;
+                }
+                notifyAll();
+            }
+        } finally {
+            metadataProvider.getLocks().reset();
+        }
+        return null;
+    }
+
+    @Override
+    public synchronized void start(MetadataProvider metadataProvider)
+            throws HyracksDataException, InterruptedException {
+        waitForNonTransitionState();
+        if (state != ActivityState.PERMANENTLY_FAILED && state != ActivityState.STOPPED) {
+            throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_ALREADY_STARTED, entityId, state);
+        }
+        try {
+            setState(ActivityState.STARTING);
+            doStart(metadataProvider);
+            setRunning(metadataProvider, true);
+        } catch (Exception e) {
+            setState(ActivityState.PERMANENTLY_FAILED);
+            LOGGER.log(Level.SEVERE, "Failed to start the entity " + entityId, e);
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    protected abstract void doStart(MetadataProvider metadataProvider) throws HyracksDataException, AlgebricksException;
+
+    protected abstract Void doStop(MetadataProvider metadataProvider) throws HyracksDataException, AlgebricksException;
+
+    protected abstract Void doSuspend(MetadataProvider metadataProvider)
+            throws HyracksDataException, AlgebricksException;
+
+    protected abstract void doResume(MetadataProvider metadataProvider)
+            throws HyracksDataException, AlgebricksException;
+
+    protected abstract void setRunning(MetadataProvider metadataProvider, boolean running)
+            throws HyracksDataException, AlgebricksException;
+
+    @Override
+    public void stop(MetadataProvider metadataProvider) throws HyracksDataException, InterruptedException {
+        Future<Void> aRecoveryTask = null;
+        synchronized (this) {
+            waitForNonTransitionState();
+            if (state != ActivityState.RUNNING && state != ActivityState.PERMANENTLY_FAILED
+                    && state != ActivityState.TEMPORARILY_FAILED) {
+                throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_CANNOT_BE_STOPPED, entityId, state);
+            }
+            if (state == ActivityState.TEMPORARILY_FAILED || state == ActivityState.PERMANENTLY_FAILED) {
+                if (recoveryTask != null) {
+                    aRecoveryTask = recoveryTask;
+                    cancelRecovery = true;
+                    recoveryTask.cancel(true);
+                }
+                setState(ActivityState.STOPPED);
+                try {
+                    setRunning(metadataProvider, false);
+                } catch (Exception e) {
+                    LOGGER.log(Level.SEVERE, "Failed to set the entity state as not running " + entityId, e);
+                    throw HyracksDataException.create(e);
+                }
+            } else if (state == ActivityState.RUNNING) {
+                setState(ActivityState.STOPPING);
+                try {
+                    doStop(metadataProvider);
+                    setRunning(metadataProvider, false);
+                } catch (Exception e) {
+                    setState(ActivityState.PERMANENTLY_FAILED);
+                    LOGGER.log(Level.SEVERE, "Failed to stop the entity " + entityId, e);
+                    throw HyracksDataException.create(e);
+                }
+            } else {
+                throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_CANNOT_BE_STOPPED, entityId, state);
+            }
+        }
+        try {
+            if (aRecoveryTask != null) {
+                aRecoveryTask.get();
+            }
+        } catch (InterruptedException e) {
+            throw e;
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public void suspend(MetadataProvider metadataProvider) throws HyracksDataException, InterruptedException {
+        WaitForStateSubscriber subscriber;
+        Future<Void> suspendTask;
+        synchronized (this) {
+            LOGGER.log(Level.WARNING, "suspending entity " + entityId);
+            LOGGER.log(Level.WARNING, "Waiting for ongoing activities");
+            waitForNonTransitionState();
+            LOGGER.log(Level.WARNING, "Proceeding with suspension. Current state is " + state);
+            if (state == ActivityState.STOPPED || state == ActivityState.PERMANENTLY_FAILED) {
+                suspended = true;
+                return;
+            }
+            if (state == ActivityState.SUSPENDED) {
+                throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_ALREADY_SUSPENDED, entityId, state);
+            }
+            if (state == ActivityState.TEMPORARILY_FAILED) {
+                suspended = true;
+                setState(ActivityState.SUSPENDED);
+                return;
+            }
+            setState(ActivityState.SUSPENDING);
+            subscriber = new WaitForStateSubscriber(this,
+                    EnumSet.of(ActivityState.SUSPENDED, ActivityState.TEMPORARILY_FAILED));
+            suspendTask = metadataProvider.getApplicationContext().getServiceContext().getControllerService()
+                    .getExecutor().submit(() -> doSuspend(metadataProvider));
+            LOGGER.log(Level.WARNING, "Suspension task has been submitted");
+        }
+        try {
+            LOGGER.log(Level.WARNING, "Waiting for suspension task to complete");
+            suspendTask.get();
+            LOGGER.log(Level.WARNING, "waiting for state to become SUSPENDED or TEMPORARILY_FAILED");
+            subscriber.sync();
+        } catch (Exception e) {
+            synchronized (this) {
+                LOGGER.log(Level.SEVERE, "Failure while waiting for " + entityId + " to become suspended", e);
+                // failed to suspend
+                if (state == ActivityState.SUSPENDING) {
+                    if (jobId != null) {
+                        // job is still running
+                        // restore state
+                        setState(prevState);
+                    } else {
+                        setState(ActivityState.PERMANENTLY_FAILED);
+                    }
+                }
+                throw HyracksDataException.create(e);
+            }
+        }
+    }
+
+    @Override
+    public synchronized void resume(MetadataProvider metadataProvider) throws HyracksDataException {
+        if (state == ActivityState.STOPPED || state == ActivityState.PERMANENTLY_FAILED) {
+            suspended = false;
+            notifyAll();
+            return;
+        }
+        if (state != ActivityState.SUSPENDED && state != ActivityState.TEMPORARILY_FAILED) {
+            throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_CANNOT_RESUME_FROM_STATE, entityId, state);
+        }
+        try {
+            if (prevState == ActivityState.TEMPORARILY_FAILED) {
+                setState(ActivityState.TEMPORARILY_FAILED);
+                return;
+            }
+            setState(ActivityState.RESUMING);
+            WaitForStateSubscriber subscriber = new WaitForStateSubscriber(this,
+                    EnumSet.of(ActivityState.RUNNING, ActivityState.TEMPORARILY_FAILED));
+            recoveryTask = metadataProvider.getApplicationContext().getServiceContext().getControllerService()
+                    .getExecutor().submit(() -> resumeOrRecover(metadataProvider));
+            try {
+                subscriber.sync();
+            } catch (Exception e) {
+                LOGGER.log(Level.WARNING, "Failure while attempting to resume " + entityId, e);
+                throw HyracksDataException.create(e);
+            }
+        } finally {
+            suspended = false;
+            notifyAll();
+        }
+    }
+
+    protected Void resumeOrRecover(MetadataProvider metadataProvider)
+            throws HyracksDataException, AlgebricksException, InterruptedException {
+        try {
+            doResume(metadataProvider);
+            synchronized (this) {
+                setState(ActivityState.RUNNING);
+                recoveryTask = null;
+            }
+        } catch (Exception e) {
+            LOGGER.log(Level.WARNING, "First attempt to resume " + entityId + " Failed", e);
+            setState(ActivityState.TEMPORARILY_FAILED);
+            if (retryPolicyFactory == NoRetryPolicyFactory.INSTANCE) {
+                setState(ActivityState.PERMANENTLY_FAILED);
+            } else {
+                IRetryPolicy policy = retryPolicyFactory.create(this);
+                cancelRecovery = false;
+                doRecover(policy);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public boolean isActive() {
+        return state != ActivityState.STOPPED && state != ActivityState.PERMANENTLY_FAILED;
+    }
+
+    @Override
+    public void unregister() throws HyracksDataException {
+        handler.unregisterListener(this);
+    }
+
+    public void setLocations(AlgebricksAbsolutePartitionConstraint locations) {
+        this.locations = locations;
+    }
+
+    public Future<Void> getRecoveryTask() {
+        return recoveryTask;
+    }
+
+    public synchronized void cancelRecovery() {
+        cancelRecovery = true;
+        notifyAll();
+    }
+
+    @Override
+    public Exception getJobFailure() {
+        return jobFailure;
+    }
+
+    @Override
+    public List<Dataset> getDatasets() {
+        return datasets;
+    }
+
+    @Override
+    public synchronized void replace(Dataset dataset) {
+        if (getDatasets().contains(dataset)) {
+            getDatasets().remove(dataset);
+            getDatasets().add(dataset);
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
new file mode 100644
index 0000000..b34d011
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.app.active;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.active.ActiveEvent;
+import org.apache.asterix.active.ActiveEvent.Kind;
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.IActiveEntityEventsListener;
+import org.apache.asterix.active.IActiveNotificationHandler;
+import org.apache.asterix.active.SingleThreadEventProcessor;
+import org.apache.asterix.active.message.ActivePartitionMessage;
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.metadata.api.IActiveEntityController;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.IJobLifecycleListener;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.JobStatus;
+
+public class ActiveNotificationHandler extends SingleThreadEventProcessor<ActiveEvent>
+        implements IActiveNotificationHandler, IJobLifecycleListener {
+
+    private static final Logger LOGGER = Logger.getLogger(ActiveNotificationHandler.class.getName());
+    private static final boolean DEBUG = false;
+    public static final String ACTIVE_ENTITY_PROPERTY_NAME = "ActiveJob";
+    private final Map<EntityId, IActiveEntityEventsListener> entityEventListeners;
+    private final Map<JobId, EntityId> jobId2EntityId;
+    private boolean initialized = false;
+    private boolean suspended = false;
+
+    public ActiveNotificationHandler() {
+        super(ActiveNotificationHandler.class.getSimpleName());
+        jobId2EntityId = new HashMap<>();
+        entityEventListeners = new HashMap<>();
+    }
+
+    // *** SingleThreadEventProcessor<ActiveEvent>
+
+    @Override
+    protected void handle(ActiveEvent event) {
+        EntityId entityId = jobId2EntityId.get(event.getJobId());
+        if (entityId != null) {
+            IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
+            LOGGER.log(Level.WARNING, "Next event is of type " + event.getEventKind());
+            if (event.getEventKind() == Kind.JOB_FINISHED) {
+                LOGGER.log(Level.WARNING, "Removing the job");
+                jobId2EntityId.remove(event.getJobId());
+            }
+            if (listener != null) {
+                LOGGER.log(Level.WARNING, "Notifying the listener");
+                listener.notify(event);
+            }
+        } else {
+            LOGGER.log(Level.SEVERE, "Entity not found for received message for job " + event.getJobId());
+        }
+    }
+
+    // *** IJobLifecycleListener
+
+    @Override
+    public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification) throws HyracksDataException {
+        LOGGER.log(Level.WARNING,
+                "notifyJobCreation(JobId jobId, JobSpecification jobSpecification) was called with jobId = " + jobId);
+        Object property = jobSpecification.getProperty(ACTIVE_ENTITY_PROPERTY_NAME);
+        if (property == null || !(property instanceof EntityId)) {
+            LOGGER.log(Level.WARNING, "Job is not of type active job. property found to be: " + property);
+            return;
+        }
+        EntityId entityId = (EntityId) property;
+        monitorJob(jobId, entityId);
+        boolean found = jobId2EntityId.get(jobId) != null;
+        LOGGER.log(Level.WARNING, "Job was found to be: " + (found ? "Active" : "Inactive"));
+        add(new ActiveEvent(jobId, Kind.JOB_CREATED, entityId, jobSpecification));
+    }
+
+    private synchronized void monitorJob(JobId jobId, EntityId entityId) {
+        if (DEBUG) {
+            LOGGER.log(Level.WARNING, "monitorJob(JobId jobId, ActiveJob activeJob) called with job id: " + jobId);
+            boolean found = jobId2EntityId.get(jobId) != null;
+            LOGGER.log(Level.WARNING, "Job was found to be: " + (found ? "Active" : "Inactive"));
+        }
+        if (entityEventListeners.containsKey(entityId)) {
+            if (jobId2EntityId.containsKey(jobId)) {
+                LOGGER.severe("Job is already being monitored for job: " + jobId);
+                return;
+            }
+            if (DEBUG) {
+                LOGGER.log(Level.WARNING, "monitoring started for job id: " + jobId);
+            }
+        } else {
+            LOGGER.severe("No listener was found for the entity: " + entityId);
+        }
+        jobId2EntityId.put(jobId, entityId);
+    }
+
+    @Override
+    public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
+        EntityId entityId = jobId2EntityId.get(jobId);
+        if (entityId != null) {
+            add(new ActiveEvent(jobId, Kind.JOB_STARTED, entityId, null));
+        }
+    }
+
+    @Override
+    public synchronized void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions)
+            throws HyracksException {
+        EntityId entityId = jobId2EntityId.get(jobId);
+        if (entityId != null) {
+            add(new ActiveEvent(jobId, Kind.JOB_FINISHED, entityId, Pair.of(jobStatus, exceptions)));
+        } else {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("NO NEED TO NOTIFY JOB FINISH!");
+            }
+        }
+    }
+
+    // *** IActiveNotificationHandler
+
+    @Override
+    public void receive(ActivePartitionMessage message) {
+        add(new ActiveEvent(message.getJobId(), Kind.PARTITION_EVENT, message.getActiveRuntimeId().getEntityId(),
+                message));
+    }
+
+    @Override
+    public IActiveEntityEventsListener getListener(EntityId entityId) {
+        if (DEBUG) {
+            LOGGER.log(Level.WARNING, "getActiveEntityListener(EntityId entityId) was called with entity " + entityId);
+            IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
+            LOGGER.log(Level.WARNING, "Listener found: " + listener);
+        }
+        return entityEventListeners.get(entityId);
+    }
+
+    @Override
+    public synchronized IActiveEntityEventsListener[] getEventListeners() {
+        if (DEBUG) {
+            LOGGER.log(Level.WARNING, "getEventListeners() was called");
+            LOGGER.log(Level.WARNING, "returning " + entityEventListeners.size() + " Listeners");
+        }
+        return entityEventListeners.values().toArray(new IActiveEntityEventsListener[entityEventListeners.size()]);
+    }
+
+    @Override
+    public synchronized void registerListener(IActiveEntityEventsListener listener) throws HyracksDataException {
+        if (suspended) {
+            throw new RuntimeDataException(ErrorCode.ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED);
+        }
+        if (DEBUG) {
+            LOGGER.log(Level.WARNING,
+                    "registerListener(IActiveEntityEventsListener listener) was called for the entity "
+                            + listener.getEntityId());
+        }
+        if (entityEventListeners.containsKey(listener.getEntityId())) {
+            throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_IS_ALREADY_REGISTERED, listener.getEntityId());
+        }
+        entityEventListeners.put(listener.getEntityId(), listener);
+    }
+
+    @Override
+    public synchronized void unregisterListener(IActiveEntityEventsListener listener) throws HyracksDataException {
+        if (suspended) {
+            throw new RuntimeDataException(ErrorCode.ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED);
+        }
+        LOGGER.log(Level.WARNING, "unregisterListener(IActiveEntityEventsListener listener) was called for the entity "
+                + listener.getEntityId());
+        IActiveEntityEventsListener registeredListener = entityEventListeners.remove(listener.getEntityId());
+        if (registeredListener == null) {
+            throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_LISTENER_IS_NOT_REGISTERED, listener.getEntityId());
+        }
+        if (registeredListener.isActive()) {
+            entityEventListeners.put(registeredListener.getEntityId(), registeredListener);
+            throw new RuntimeDataException(ErrorCode.CANNOT_DERIGESTER_ACTIVE_ENTITY_LISTENER, listener.getEntityId());
+        }
+    }
+
+    @Override
+    public boolean isInitialized() {
+        return initialized;
+    }
+
+    @Override
+    public void setInitialized(boolean initialized) throws HyracksDataException {
+        if (this.initialized) {
+            throw new RuntimeDataException(ErrorCode.DOUBLE_INITIALIZATION_OF_ACTIVE_NOTIFICATION_HANDLER);
+        }
+        this.initialized = initialized;
+    }
+
+    @Override
+    public synchronized void recover() throws HyracksDataException {
+        LOGGER.log(Level.WARNING, "Starting active recovery");
+        for (IActiveEntityEventsListener listener : entityEventListeners.values()) {
+            synchronized (listener) {
+                LOGGER.log(Level.WARNING, "Entity " + listener.getEntityId() + " is " + listener.getStats());
+                if (listener.getState() == ActivityState.PERMANENTLY_FAILED
+                        && listener instanceof IActiveEntityController) {
+                    LOGGER.log(Level.WARNING, "Recovering");
+                    ((IActiveEntityController) listener).recover();
+                } else {
+                    LOGGER.log(Level.WARNING, "Only notifying");
+                    listener.notifyAll();
+                }
+            }
+        }
+    }
+
+    public void suspend(MetadataProvider mdProvider)
+            throws AsterixException, HyracksDataException, InterruptedException {
+        synchronized (this) {
+            if (suspended) {
+                throw new RuntimeDataException(ErrorCode.ACTIVE_EVENT_HANDLER_ALREADY_SUSPENDED);
+            }
+            LOGGER.log(Level.WARNING, "Suspending active events handler");
+            suspended = true;
+        }
+        IMetadataLockManager lockManager = mdProvider.getApplicationContext().getMetadataLockManager();
+        Collection<IActiveEntityEventsListener> registeredListeners = entityEventListeners.values();
+        for (IActiveEntityEventsListener listener : registeredListeners) {
+            // write lock the listener
+            // exclusive lock all the datasets
+            String dataverseName = listener.getEntityId().getDataverse();
+            String entityName = listener.getEntityId().getEntityName();
+            LOGGER.log(Level.WARNING, "Suspending " + listener.getEntityId());
+            LOGGER.log(Level.WARNING, "Acquiring locks");
+            lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
+            List<Dataset> datasets = ((ActiveEntityEventsListener) listener).getDatasets();
+            for (Dataset dataset : datasets) {
+                lockManager.acquireDatasetExclusiveModificationLock(mdProvider.getLocks(),
+                        DatasetUtil.getFullyQualifiedName(dataset));
+            }
+            LOGGER.log(Level.WARNING, "locks acquired");
+            ((ActiveEntityEventsListener) listener).suspend(mdProvider);
+            LOGGER.log(Level.WARNING, listener.getEntityId() + " suspended");
+        }
+    }
+
+    public void resume(MetadataProvider mdProvider)
+            throws AsterixException, HyracksDataException, InterruptedException {
+        LOGGER.log(Level.WARNING, "Resuming active events handler");
+        for (IActiveEntityEventsListener listener : entityEventListeners.values()) {
+            LOGGER.log(Level.WARNING, "Resuming " + listener.getEntityId());
+            ((ActiveEntityEventsListener) listener).resume(mdProvider);
+            LOGGER.log(Level.WARNING, listener.getEntityId() + " resumed");
+        }
+        synchronized (this) {
+            suspended = false;
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java
new file mode 100644
index 0000000..45c79a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.app.active;
+
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.IActiveEntityEventSubscriber;
+import org.apache.asterix.active.IRetryPolicyFactory;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
+import org.apache.asterix.app.translator.QueryTranslator;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.utils.JobUtils;
+import org.apache.asterix.compiler.provider.AqlCompilationProvider;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
+import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.utils.FeedOperations;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobSpecification;
+
+public class FeedEventsListener extends ActiveEntityEventsListener {
+
+    private final Feed feed;
+    private final List<FeedConnection> feedConnections;
+
+    public FeedEventsListener(IStatementExecutor statementExecutor, ICcApplicationContext appCtx,
+            IHyracksClientConnection hcc, EntityId entityId, List<Dataset> datasets,
+            AlgebricksAbsolutePartitionConstraint locations, String runtimeName, IRetryPolicyFactory retryPolicyFactory,
+            Feed feed, final List<FeedConnection> feedConnections) throws HyracksDataException {
+        super(statementExecutor, appCtx, hcc, entityId, datasets, locations, runtimeName, retryPolicyFactory);
+        this.feed = feed;
+        this.feedConnections = feedConnections;
+    }
+
+    @Override
+    public synchronized void remove(Dataset dataset) throws HyracksDataException {
+        super.remove(dataset);
+        feedConnections.removeIf(o -> o.getDataverseName().equals(dataset.getDataverseName())
+                && o.getDatasetName().equals(dataset.getDatasetName()));
+    }
+
+    public synchronized void addFeedConnection(FeedConnection feedConnection) {
+        feedConnections.add(feedConnection);
+    }
+
+    public Feed getFeed() {
+        return feed;
+    }
+
+    @Override
+    protected void doStart(MetadataProvider mdProvider) throws HyracksDataException, AlgebricksException {
+        try {
+            ILangCompilationProvider compilationProvider = new AqlCompilationProvider();
+            IStorageComponentProvider storageComponentProvider = new StorageComponentProvider();
+            DefaultStatementExecutorFactory statementExecutorFactory = new DefaultStatementExecutorFactory();
+            Pair<JobSpecification, AlgebricksAbsolutePartitionConstraint> jobInfo = FeedOperations.buildStartFeedJob(
+                    ((QueryTranslator) statementExecutor).getSessionOutput(), mdProvider, feed, feedConnections,
+                    compilationProvider, storageComponentProvider, statementExecutorFactory, hcc);
+            JobSpecification feedJob = jobInfo.getLeft();
+            IActiveEntityEventSubscriber eventSubscriber =
+                    new WaitForStateSubscriber(this, Collections.singleton(ActivityState.RUNNING));
+            feedJob.setProperty(ActiveNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME, entityId);
+            // TODO(Yingyi): currently we do not check IFrameWriter protocol violations for Feed jobs.
+            // We will need to design general exception handling mechanism for feeds.
+            setLocations(jobInfo.getRight());
+            boolean wait = Boolean.parseBoolean(mdProvider.getConfig().get(StartFeedStatement.WAIT_FOR_COMPLETION));
+            JobUtils.runJob(hcc, feedJob, false);
+            eventSubscriber.sync();
+            if (wait) {
+                IActiveEntityEventSubscriber stoppedSubscriber = new WaitForStateSubscriber(this,
+                        EnumSet.of(ActivityState.STOPPED, ActivityState.PERMANENTLY_FAILED));
+                stoppedSubscriber.sync();
+            }
+        } catch (AlgebricksException e) {
+            throw e;
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    protected Void doStop(MetadataProvider metadataProvider) throws HyracksDataException, AlgebricksException {
+        IActiveEntityEventSubscriber eventSubscriber =
+                new WaitForStateSubscriber(this, Collections.singleton(ActivityState.STOPPED));
+        try {
+            // Construct ActiveMessage
+            for (int i = 0; i < getLocations().getLocations().length; i++) {
+                String intakeLocation = getLocations().getLocations()[i];
+                FeedOperations.SendStopMessageToNode(metadataProvider.getApplicationContext(), entityId, intakeLocation,
+                        i);
+            }
+            eventSubscriber.sync();
+        } catch (AlgebricksException e) {
+            throw e;
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+        return null;
+    }
+
+    @Override
+    protected void setRunning(MetadataProvider metadataProvider, boolean running)
+            throws HyracksDataException, AlgebricksException {
+        // No op
+    }
+
+    @Override
+    protected Void doSuspend(MetadataProvider metadataProvider) throws HyracksDataException, AlgebricksException {
+        throw new RuntimeDataException(ErrorCode.OPERATION_NOT_SUPPORTED);
+    }
+
+    @Override
+    protected void doResume(MetadataProvider metadataProvider) throws HyracksDataException, AlgebricksException {
+        throw new RuntimeDataException(ErrorCode.OPERATION_NOT_SUPPORTED);
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
index 00fb1b0..f6bd708 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
@@ -50,7 +50,6 @@
     @Override
     public IStatementExecutor create(ICcApplicationContext appCtx, List<Statement> statements, SessionOutput output,
             ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider) {
-        return new QueryTranslator(appCtx, statements, output, compilationProvider, storageComponentProvider,
-                executorService);
+        return new QueryTranslator(appCtx, statements, output, compilationProvider, executorService);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index bd5c024..7725936 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -34,53 +34,50 @@
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
-import java.util.Map.Entry;
 import java.util.concurrent.ExecutorService;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.active.ActiveJobNotificationHandler;
-import org.apache.asterix.active.ActiveLifecycleListener;
 import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.IActiveEntityEventsListener;
-import org.apache.asterix.active.IActiveEventSubscriber;
+import org.apache.asterix.active.NoRetryPolicyFactory;
 import org.apache.asterix.algebra.extension.IExtensionStatement;
 import org.apache.asterix.api.common.APIFramework;
 import org.apache.asterix.api.http.server.AbstractQueryApiServlet;
 import org.apache.asterix.api.http.server.ApiServlet;
 import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.active.ActiveEntityEventsListener;
+import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.app.active.FeedEventsListener;
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.config.ClusterProperties;
-import org.apache.asterix.common.config.ExternalProperties;
-import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.config.DatasetConfig.TransactionState;
-import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.config.ExternalProperties;
+import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.common.metadata.IDataset;
 import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.common.utils.JobUtils.ProgressState;
-import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.external.feed.management.ActiveEntityEventsListener;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.base.IRewriterFactory;
@@ -123,7 +120,6 @@
 import org.apache.asterix.lang.common.statement.WriteStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.metadata.IDatasetDetails;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
@@ -145,12 +141,12 @@
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
 import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
-import org.apache.asterix.metadata.lock.MetadataLockManager;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.ExternalIndexingOperations;
 import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.metadata.utils.MetadataConstants;
+import org.apache.asterix.metadata.utils.MetadataLockUtil;
 import org.apache.asterix.metadata.utils.MetadataUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -159,16 +155,16 @@
 import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.asterix.translator.AbstractLangTranslator;
-import org.apache.asterix.translator.IStatementExecutor;
-import org.apache.asterix.translator.IStatementExecutorContext;
-import org.apache.asterix.translator.SessionConfig;
-import org.apache.asterix.translator.SessionOutput;
-import org.apache.asterix.translator.TypeTranslator;
 import org.apache.asterix.translator.CompiledStatements.CompiledDeleteStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledInsertStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledUpsertStatement;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutorContext;
+import org.apache.asterix.translator.SessionConfig;
+import org.apache.asterix.translator.SessionOutput;
+import org.apache.asterix.translator.TypeTranslator;
 import org.apache.asterix.translator.util.ValidateUtil;
 import org.apache.asterix.utils.DataverseUtil;
 import org.apache.asterix.utils.FeedOperations;
@@ -178,7 +174,6 @@
 import org.apache.commons.lang3.mutable.MutableBoolean;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.commons.lang3.tuple.Triple;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
@@ -217,18 +212,17 @@
     protected final List<FunctionDecl> declaredFunctions;
     protected final APIFramework apiFramework;
     protected final IRewriterFactory rewriterFactory;
-    protected final IStorageComponentProvider componentProvider;
     protected final ExecutorService executorService;
     protected final EnumSet<JobFlag> jobFlags = EnumSet.noneOf(JobFlag.class);
+    protected final IMetadataLockManager lockManager;
 
     public QueryTranslator(ICcApplicationContext appCtx, List<Statement> statements, SessionOutput output,
-            ILangCompilationProvider compliationProvider, IStorageComponentProvider componentProvider,
-            ExecutorService executorService) {
+            ILangCompilationProvider compliationProvider, ExecutorService executorService) {
         this.appCtx = appCtx;
+        this.lockManager = appCtx.getMetadataLockManager();
         this.statements = statements;
         this.sessionOutput = output;
         this.sessionConfig = output.config();
-        this.componentProvider = componentProvider;
         declaredFunctions = getDeclaredFunctions(statements);
         apiFramework = new APIFramework(compliationProvider);
         rewriterFactory = compliationProvider.getRewriterFactory();
@@ -281,7 +275,7 @@
                 }
                 validateOperation(appCtx, activeDataverse, stmt);
                 rewriteStatement(stmt); // Rewrite the statement's AST.
-                MetadataProvider metadataProvider = new MetadataProvider(appCtx, activeDataverse, componentProvider);
+                MetadataProvider metadataProvider = new MetadataProvider(appCtx, activeDataverse);
                 metadataProvider.setWriterFactory(writerFactory);
                 metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
                 metadataProvider.setOutputFile(outputFile);
@@ -431,7 +425,7 @@
         String dvName = dvd.getDataverseName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.acquireDataverseReadLock(metadataProvider.getLocks(), dvName);
+        lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dvName);
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
             if (dv == null) {
@@ -454,7 +448,7 @@
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
 
-        MetadataLockManager.INSTANCE.acquireDataverseReadLock(metadataProvider.getLocks(), dvName);
+        lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dvName);
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
             if (dv != null) {
@@ -529,7 +523,7 @@
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.createDatasetBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 itemTypeDataverseName, itemTypeDataverseName + "." + itemTypeName, metaItemTypeDataverseName,
                 metaItemTypeDataverseName + "." + metaItemTypeName, nodegroupName, compactionPolicy,
                 dataverseName + "." + datasetName, defaultCompactionPolicy);
@@ -693,11 +687,11 @@
     protected static void validateIfResourceIsActiveInFeed(ICcApplicationContext appCtx, Dataset dataset)
             throws CompilationException {
         StringBuilder builder = null;
-        ActiveLifecycleListener activeListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
-        ActiveJobNotificationHandler activeEventHandler = activeListener.getNotificationHandler();
+        ActiveNotificationHandler activeEventHandler =
+                (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
         IActiveEntityEventsListener[] listeners = activeEventHandler.getEventListeners();
         for (IActiveEntityEventsListener listener : listeners) {
-            if (listener.isEntityUsingDataset(dataset)) {
+            if (listener.isEntityUsingDataset(dataset) && listener.isActive()) {
                 if (builder == null) {
                     builder = new StringBuilder();
                 }
@@ -741,15 +735,15 @@
         CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
         String dataverseName = getActiveDataverse(stmtCreateIndex.getDataverseName());
         String datasetName = stmtCreateIndex.getDatasetName().getValue();
+        String indexName = stmtCreateIndex.getIndexName().getValue();
         List<Integer> keySourceIndicators = stmtCreateIndex.getFieldSourceIndicators();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.createIndexBegin(metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + datasetName);
-        String indexName = null;
+        String datasetFullyQualifiedName = dataverseName + "." + datasetName;
         Dataset ds = null;
-        // For external datasets
         Index index = null;
+        MetadataLockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName,
+                datasetFullyQualifiedName);
         try {
             ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
@@ -757,7 +751,6 @@
                         "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName);
             }
 
-            indexName = stmtCreateIndex.getIndexName().getValue();
             index = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
                     datasetName, indexName);
             if (index != null) {
@@ -1111,7 +1104,7 @@
         String typeName = stmtCreateType.getIdent().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.createTypeBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.createTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + typeName);
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
@@ -1157,7 +1150,7 @@
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         List<JobSpecification> jobsToExecute = new ArrayList<>();
-        MetadataLockManager.INSTANCE.acquireDataverseWriteLock(metadataProvider.getLocks(), dataverseName);
+        lockManager.acquireDataverseWriteLock(metadataProvider.getLocks(), dataverseName);
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
             if (dv == null) {
@@ -1168,26 +1161,31 @@
                     throw new AlgebricksException("There is no dataverse with this name " + dataverseName + ".");
                 }
             }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
             // # disconnect all feeds from any datasets in the dataverse.
-            ActiveLifecycleListener activeListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
-            ActiveJobNotificationHandler activeEventHandler = activeListener.getNotificationHandler();
+            ActiveNotificationHandler activeEventHandler =
+                    (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
             IActiveEntityEventsListener[] activeListeners = activeEventHandler.getEventListeners();
-            Identifier dvId = new Identifier(dataverseName);
-            MetadataProvider tempMdProvider = new MetadataProvider(appCtx, metadataProvider.getDefaultDataverse(),
-                    metadataProvider.getStorageComponentProvider());
-            tempMdProvider.setConfig(metadataProvider.getConfig());
             for (IActiveEntityEventsListener listener : activeListeners) {
                 EntityId activeEntityId = listener.getEntityId();
                 if (activeEntityId.getExtensionName().equals(Feed.EXTENSION_NAME)
                         && activeEntityId.getDataverse().equals(dataverseName)) {
-                    tempMdProvider.getLocks().reset();
-                    stopFeedBeforeDelete(new Pair<>(dvId, new Identifier(activeEntityId.getEntityName())),
-                            tempMdProvider);
-                    // prepare job to remove feed log storage
-                    jobsToExecute.add(FeedOperations.buildRemoveFeedStorageJob(metadataProvider,
-                            MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverseName, activeEntityId.getEntityName())));
+                    if (listener.getState() != ActivityState.STOPPED) {
+                        ((ActiveEntityEventsListener) listener).stop(metadataProvider);
+                    }
+                    FeedEventsListener feedListener = (FeedEventsListener) listener;
+                    mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                    bActiveTxn = true;
+                    metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                    doDropFeed(hcc, metadataProvider, feedListener.getFeed());
+                    MetadataManager.INSTANCE.commitTransaction(metadataProvider.getMetadataTxnContext());
+                    bActiveTxn = false;
                 }
             }
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
 
             // #. prepare jobs which will drop corresponding datasets with indexes.
             List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dataverseName);
@@ -1243,7 +1241,7 @@
             // Drops all node groups that no longer needed
             for (Dataset dataset : datasets) {
                 String nodeGroup = dataset.getNodeGroupName();
-                MetadataLockManager.INSTANCE.acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodeGroup);
+                lockManager.acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodeGroup);
                 if (MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, nodeGroup) != null) {
                     MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, nodeGroup, true);
                 }
@@ -1294,26 +1292,12 @@
         }
     }
 
-    protected void stopFeedBeforeDelete(Pair<Identifier, Identifier> feedNameComp, MetadataProvider metadataProvider) {
-        StopFeedStatement disStmt = new StopFeedStatement(feedNameComp);
-        try {
-            handleStopFeedStatement(metadataProvider, disStmt);
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Stopped feed " + feedNameComp.second.getValue());
-            }
-        } catch (Exception exception) {
-            if (LOGGER.isLoggable(Level.WARNING)) {
-                LOGGER.warning("Unable to stop feed " + feedNameComp.second.getValue() + exception);
-            }
-        }
-    }
-
     public void handleDatasetDropStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         DropDatasetStatement stmtDelete = (DropDatasetStatement) stmt;
         String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
         String datasetName = stmtDelete.getDatasetName().getValue();
-        MetadataLockManager.INSTANCE.dropDatasetBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName);
         try {
             doDropDataset(dataverseName, datasetName, metadataProvider, stmtDelete.getIfExists(), hcc, true);
@@ -1386,14 +1370,14 @@
         IndexDropStatement stmtIndexDrop = (IndexDropStatement) stmt;
         String datasetName = stmtIndexDrop.getDatasetName().getValue();
         String dataverseName = getActiveDataverse(stmtIndexDrop.getDataverseName());
+        String indexName = stmtIndexDrop.getIndexName().getValue();
         ProgressState progress = ProgressState.NO_PROGRESS;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         List<JobSpecification> jobsToExecute = new ArrayList<>();
-        MetadataLockManager.INSTANCE.dropIndexBegin(metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + datasetName);
-        String indexName = null;
+        String dsFullyQualifiedName = dataverseName + "." + datasetName;
+        MetadataLockUtil.dropIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, dsFullyQualifiedName);
         // For external index
         boolean dropFilesIndex = false;
         try {
@@ -1402,8 +1386,8 @@
                 throw new AlgebricksException(
                         "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName);
             }
-            ActiveLifecycleListener activeListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
-            ActiveJobNotificationHandler activeEventHandler = activeListener.getNotificationHandler();
+            ActiveNotificationHandler activeEventHandler =
+                    (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
             IActiveEntityEventsListener[] listeners = activeEventHandler.getEventListeners();
             StringBuilder builder = null;
             for (IActiveEntityEventsListener listener : listeners) {
@@ -1420,7 +1404,6 @@
             }
 
             if (ds.getDatasetType() == DatasetType.INTERNAL) {
-                indexName = stmtIndexDrop.getIndexName().getValue();
                 Index index = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 if (index == null) {
                     if (stmtIndexDrop.getIfExists()) {
@@ -1581,7 +1564,7 @@
 
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.dropTypeBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.dropTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + typeName);
         try {
             Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
@@ -1606,7 +1589,7 @@
         String nodegroupName = stmtDelete.getNodeGroupName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodegroupName);
+        lockManager.acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodegroupName);
         try {
             NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, nodegroupName);
             if (ng == null) {
@@ -1634,7 +1617,7 @@
 
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.functionStatementBegin(metadataProvider.getLocks(), dataverse,
+        MetadataLockUtil.functionStatementBegin(lockManager, metadataProvider.getLocks(), dataverse,
                 dataverse + "." + functionName);
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
@@ -1662,7 +1645,7 @@
         signature.setNamespace(getActiveDataverseName(signature.getNamespace()));
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.functionStatementBegin(metadataProvider.getLocks(), signature.getNamespace(),
+        MetadataLockUtil.functionStatementBegin(lockManager, metadataProvider.getLocks(), signature.getNamespace(),
                 signature.getNamespace() + "." + signature.getName());
         try {
             Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
@@ -1692,7 +1675,7 @@
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.modifyDatasetBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.modifyDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName);
         try {
             CompiledLoadFromFileStatement cls =
@@ -1723,7 +1706,7 @@
         final IMetadataLocker locker = new IMetadataLocker() {
             @Override
             public void lock() throws AsterixException {
-                MetadataLockManager.INSTANCE.insertDeleteUpsertBegin(metadataProvider.getLocks(),
+                MetadataLockUtil.insertDeleteUpsertBegin(lockManager, metadataProvider.getLocks(),
                         dataverseName + "." + stmtInsertUpsert.getDatasetName());
             }
 
@@ -1783,7 +1766,7 @@
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.insertDeleteUpsertBegin(metadataProvider.getLocks(),
+        MetadataLockUtil.insertDeleteUpsertBegin(lockManager, metadataProvider.getLocks(),
                 dataverseName + "." + stmtDelete.getDatasetName());
         try {
             metadataProvider.setWriteTransaction(true);
@@ -1860,7 +1843,7 @@
         String feedName = cfs.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.createFeedBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.createFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + feedName);
         Feed feed = null;
         try {
@@ -1895,7 +1878,7 @@
         CreateFeedPolicyStatement cfps = (CreateFeedPolicyStatement) stmt;
         dataverse = getActiveDataverse(null);
         policy = cfps.getPolicyName();
-        MetadataLockManager.INSTANCE.createFeedPolicyBegin(metadataProvider.getLocks(), dataverse,
+        MetadataLockUtil.createFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverse,
                 dataverse + "." + policy);
         try {
             mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1956,7 +1939,7 @@
         String feedName = stmtFeedDrop.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.dropFeedBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.dropFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + feedName);
         try {
             Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverseName, feedName);
@@ -1967,27 +1950,8 @@
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 return;
             }
-
-            EntityId feedId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
-            ActiveLifecycleListener activeListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
-            ActiveJobNotificationHandler activeEventHandler = activeListener.getNotificationHandler();
-            ActiveEntityEventsListener listener =
-                    (ActiveEntityEventsListener) activeEventHandler.getActiveEntityListener(feedId);
-            if (listener != null) {
-                throw new AlgebricksException("Feed " + feedId
-                        + " is currently active and connected to the following dataset(s) \n" + listener.toString());
-            } else {
-                JobSpecification spec = FeedOperations.buildRemoveFeedStorageJob(metadataProvider,
-                        MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName()));
-                runJob(hcc, spec);
-                MetadataManager.INSTANCE.dropFeed(mdTxnCtx, dataverseName, feedName);
-            }
-
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Removed feed " + feedId);
-            }
+            doDropFeed(hcc, metadataProvider, feed);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
             throw e;
@@ -1996,13 +1960,36 @@
         }
     }
 
+    protected void doDropFeed(IHyracksClientConnection hcc, MetadataProvider metadataProvider, Feed feed)
+            throws Exception {
+        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
+        EntityId feedId = feed.getFeedId();
+        ActiveNotificationHandler activeNotificationHandler =
+                (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
+        ActiveEntityEventsListener listener =
+                (ActiveEntityEventsListener) activeNotificationHandler.getListener(feedId);
+        if (listener != null && listener.getState() != ActivityState.STOPPED) {
+            throw new AlgebricksException("Feed " + feedId
+                    + " is currently active and connected to the following dataset(s) \n" + listener.toString());
+        } else if (listener != null) {
+            listener.unregister();
+        }
+        JobSpecification spec = FeedOperations.buildRemoveFeedStorageJob(metadataProvider,
+                MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName()));
+        runJob(hcc, spec);
+        MetadataManager.INSTANCE.dropFeed(mdTxnCtx, feed.getDataverseName(), feed.getFeedName());
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Removed feed " + feedId);
+        }
+    }
+
     protected void handleDropFeedPolicyStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         FeedPolicyDropStatement stmtFeedPolicyDrop = (FeedPolicyDropStatement) stmt;
         String dataverseName = getActiveDataverse(stmtFeedPolicyDrop.getDataverseName());
         String policyName = stmtFeedPolicyDrop.getPolicyName().getValue();
-        MetadataLockManager.INSTANCE.dropFeedPolicyBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.dropFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + policyName);
         try {
             FeedPolicyEntity feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverseName, policyName);
@@ -2028,56 +2015,45 @@
         StartFeedStatement sfs = (StartFeedStatement) stmt;
         String dataverseName = getActiveDataverse(sfs.getDataverseName());
         String feedName = sfs.getFeedName().getValue();
-        // Transcation handler
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        // Runtime handler
-        EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
-        // Feed & Feed Connections
-        Feed feed = FeedMetadataUtil.validateIfFeedExists(dataverseName, feedName,
-                metadataProvider.getMetadataTxnContext());
-        List<FeedConnection> feedConnections = MetadataManager.INSTANCE
-                .getFeedConections(metadataProvider.getMetadataTxnContext(), dataverseName, feedName);
-        ILangCompilationProvider compilationProvider = new AqlCompilationProvider();
-        IStorageComponentProvider storageComponentProvider = new StorageComponentProvider();
-        DefaultStatementExecutorFactory qtFactory = new DefaultStatementExecutorFactory();
-        ActiveLifecycleListener activeListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
-        ActiveJobNotificationHandler activeEventHandler = activeListener.getNotificationHandler();
-        ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler
-                .getActiveEntityListener(entityId);
-        if (listener != null) {
-            throw new AlgebricksException("Feed " + feedName + " is started already.");
-        }
-        // Start
-        MetadataLockManager.INSTANCE.startFeedBegin(metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + feedName, feedConnections);
+        boolean committed = false;
+        MetadataLockUtil.startFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
+                dataverseName + "." + feedName);
         try {
-            // Prepare policy
-            List<IDataset> datasets = new ArrayList<>();
-            for (FeedConnection connection : feedConnections) {
-                Dataset ds = metadataProvider.findDataset(connection.getDataverseName(), connection.getDatasetName());
-                datasets.add(ds);
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            // Runtime handler
+            EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
+            // Feed & Feed Connections
+            Feed feed = FeedMetadataUtil.validateIfFeedExists(dataverseName, feedName,
+                    metadataProvider.getMetadataTxnContext());
+            List<FeedConnection> feedConnections = MetadataManager.INSTANCE
+                    .getFeedConections(metadataProvider.getMetadataTxnContext(), dataverseName, feedName);
+            for (FeedConnection feedConnection : feedConnections) {
+                // what if the dataset is in a different dataverse
+                String fqName = feedConnection.getDataverseName() + "." + feedConnection.getDatasetName();
+                lockManager.acquireDatasetReadLock(metadataProvider.getLocks(), fqName);
             }
-            org.apache.commons.lang3.tuple.Pair<JobSpecification, AlgebricksAbsolutePartitionConstraint> jobInfo =
-                    FeedOperations.buildStartFeedJob(sessionOutput, metadataProvider, feed, feedConnections,
-                            compilationProvider, storageComponentProvider, qtFactory, hcc);
-
-            JobSpecification feedJob = jobInfo.getLeft();
-            listener = new ActiveEntityEventsListener(appCtx, entityId, datasets, jobInfo.getRight(),
-                    FeedIntakeOperatorNodePushable.class.getSimpleName());
-            activeEventHandler.registerListener(listener);
-            IActiveEventSubscriber eventSubscriber = new WaitForStateSubscriber(listener, ActivityState.STARTED);
-            feedJob.setProperty(ActiveJobNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME, entityId);
-            // TODO(Yingyi): currently we do not check IFrameWriter protocol violations for Feed jobs.
-            // We will need to design general exception handling mechanism for feeds.
-            JobUtils.runJob(hcc, feedJob,
-                    Boolean.valueOf(metadataProvider.getConfig().get(StartFeedStatement.WAIT_FOR_COMPLETION)));
-            eventSubscriber.sync();
-            LOGGER.log(Level.INFO, "Submitted");
+            ActiveNotificationHandler activeEventHandler =
+                    (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
+            ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler.getListener(entityId);
+            if (listener == null) {
+                // Prepare policy
+                List<Dataset> datasets = new ArrayList<>();
+                for (FeedConnection connection : feedConnections) {
+                    Dataset ds =
+                            metadataProvider.findDataset(connection.getDataverseName(), connection.getDatasetName());
+                    datasets.add(ds);
+                }
+                listener = new FeedEventsListener(this, metadataProvider.getApplicationContext(), hcc, entityId,
+                        datasets, null, FeedIntakeOperatorNodePushable.class.getSimpleName(),
+                        NoRetryPolicyFactory.INSTANCE, feed, feedConnections);
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            committed = true;
+            listener.start(metadataProvider);
         } catch (Exception e) {
-            abort(e, e, mdTxnCtx);
-            if (listener != null) {
-                activeEventHandler.unregisterListener(listener);
+            if (!committed) {
+                abort(e, e, mdTxnCtx);
             }
             throw e;
         } finally {
@@ -2089,32 +2065,18 @@
         StopFeedStatement sfst = (StopFeedStatement) stmt;
         String dataverseName = getActiveDataverse(sfst.getDataverseName());
         String feedName = sfst.getFeedName().getValue();
-        EntityId feedId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
-        ActiveLifecycleListener activeListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
-        ActiveJobNotificationHandler activeEventHandler = activeListener.getNotificationHandler();
+        EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
+        ActiveNotificationHandler activeEventHandler =
+                (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
         // Obtain runtime info from ActiveListener
-        ActiveEntityEventsListener listener =
-                (ActiveEntityEventsListener) activeEventHandler.getActiveEntityListener(feedId);
+        ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler.getListener(entityId);
         if (listener == null) {
             throw new AlgebricksException("Feed " + feedName + " is not started.");
         }
-        IActiveEventSubscriber eventSubscriber = new WaitForStateSubscriber(listener, ActivityState.STOPPED);
-        // Transaction
-        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.stopFeedBegin(metadataProvider.getLocks(), dataverseName, feedName);
+        MetadataLockUtil.stopFeedBegin(lockManager, metadataProvider.getLocks(), entityId.getDataverse(),
+                entityId.getEntityName());
         try {
-            // validate
-            FeedMetadataUtil.validateIfFeedExists(dataverseName, feedName, mdTxnCtx);
-            // Construct ActiveMessage
-            for (int i = 0; i < listener.getLocations().getLocations().length; i++) {
-                String intakeLocation = listener.getLocations().getLocations()[i];
-                FeedOperations.SendStopMessageToNode(appCtx, feedId, intakeLocation, i);
-            }
-            eventSubscriber.sync();
-        } catch (Exception e) {
-            abort(e, e, mdTxnCtx);
-            throw e;
+            listener.stop(metadataProvider);
         } finally {
             metadataProvider.getLocks().unlock();
         }
@@ -2130,20 +2092,20 @@
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         // Check whether feed is alive
-        ActiveLifecycleListener activeListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
-        ActiveJobNotificationHandler activeEventHandler = activeListener.getNotificationHandler();
-        if (activeEventHandler
-                .getActiveEntityListener(new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName)) != null) {
-            throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, feedName);
-        }
+        ActiveNotificationHandler activeEventHandler =
+                (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
         // Transaction handling
-        MetadataLockManager.INSTANCE.connectFeedBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.connectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName, dataverseName + "." + feedName);
         try {
             // validation
-            FeedMetadataUtil.validateIfDatasetExists(metadataProvider, dataverseName, datasetName, mdTxnCtx);
+            Dataset dataset = FeedMetadataUtil.validateIfDatasetExists(metadataProvider, dataverseName, datasetName);
             Feed feed = FeedMetadataUtil.validateIfFeedExists(dataverseName, feedName,
                     metadataProvider.getMetadataTxnContext());
+            FeedEventsListener listener = (FeedEventsListener) activeEventHandler.getListener(feed.getFeedId());
+            if (listener != null && listener.isActive()) {
+                throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, feedName);
+            }
             ARecordType outputType = FeedMetadataUtil.getOutputType(feed, feed.getAdapterConfiguration(),
                     ExternalDataConstants.KEY_TYPE_NAME);
             List<FunctionSignature> appliedFunctions = cfs.getAppliedFunctions();
@@ -2169,6 +2131,10 @@
                 MetadataManager.INSTANCE.updateFunction(mdTxnCtx, func);
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            if (listener != null) {
+                listener.add(dataset);
+                listener.addFeedConnection(fc);
+            }
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
             throw e;
@@ -2184,21 +2150,25 @@
         String feedName = cfs.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        ActiveLifecycleListener activeListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
-        ActiveJobNotificationHandler activeEventHandler = activeListener.getNotificationHandler();
-        // Check whether feed is alive
-        if (activeEventHandler
-                .getActiveEntityListener(new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName)) != null) {
-            throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, feedName);
-        }
-        MetadataLockManager.INSTANCE.disconnectFeedBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.disconnectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName, dataverseName + "." + cfs.getFeedName());
         try {
-            FeedMetadataUtil.validateIfDatasetExists(metadataProvider, dataverseName, cfs.getDatasetName().getValue(),
-                    mdTxnCtx);
+            ActiveNotificationHandler activeEventHandler =
+                    (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
+            // Check whether feed is alive
+            ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler
+                    .getListener(new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName));
+            if (listener != null && listener.isActive()) {
+                throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, feedName);
+            }
+            FeedMetadataUtil.validateIfDatasetExists(metadataProvider, dataverseName, cfs.getDatasetName().getValue());
             FeedMetadataUtil.validateIfFeedExists(dataverseName, cfs.getFeedName().getValue(), mdTxnCtx);
             FeedConnection fc = MetadataManager.INSTANCE.getFeedConnection(metadataProvider.getMetadataTxnContext(),
                     dataverseName, feedName, datasetName);
+            Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
+            if (ds == null) {
+                throw new CompilationException("Dataset " + dataverseName + "." + datasetName + " doesn't exist");
+            }
             if (fc == null) {
                 throw new CompilationException("Feed " + feedName + " is currently not connected to "
                         + cfs.getDatasetName().getValue() + ". Invalid operation!");
@@ -2210,6 +2180,9 @@
                 MetadataManager.INSTANCE.updateFunction(mdTxnCtx, function);
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            if (listener != null) {
+                listener.remove(ds);
+            }
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
             throw e;
@@ -2227,7 +2200,7 @@
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         List<JobSpecification> jobsToExecute = new ArrayList<>();
-        MetadataLockManager.INSTANCE.compactBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.compactBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName);
         try {
             Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
@@ -2447,7 +2420,7 @@
 
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.acquireNodeGroupWriteLock(metadataProvider.getLocks(), ngName);
+        lockManager.acquireNodeGroupWriteLock(metadataProvider.getLocks(), ngName);
         try {
             NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, ngName);
             if (ng != null) {
@@ -2490,7 +2463,7 @@
         Dataset transactionDataset = null;
         boolean lockAquired = false;
         boolean success = false;
-        MetadataLockManager.INSTANCE.refreshDatasetBegin(metadataProvider.getLocks(), dataverseName,
+        MetadataLockUtil.refreshDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName);
         try {
             ds = metadataProvider.findDataset(dataverseName, datasetName);
@@ -2721,7 +2694,7 @@
                 DatasetUtil.isFullyQualifiedName(datasetNameTo) ? datasetNameTo : dataverseNameTo + '.' + datasetNameTo;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.insertDeleteUpsertBegin(metadataProvider.getLocks(), fullyQualifiedDatasetNameTo);
+        MetadataLockUtil.insertDeleteUpsertBegin(lockManager, metadataProvider.getLocks(), fullyQualifiedDatasetNameTo);
         try {
             prepareRunExternalRuntime(metadataProvider, hcc, pregelixStmt, dataverseNameFrom, dataverseNameTo,
                     datasetNameFrom, datasetNameTo, mdTxnCtx);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index cd9138a..dc92f92 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -30,15 +30,14 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.active.ActiveLifecycleListener;
 import org.apache.asterix.api.http.ctx.StatementExecutorContext;
+import org.apache.asterix.api.http.server.ActiveStatsApiServlet;
 import org.apache.asterix.api.http.server.ApiServlet;
 import org.apache.asterix.api.http.server.ClusterApiServlet;
 import org.apache.asterix.api.http.server.ClusterControllerDetailsApiServlet;
 import org.apache.asterix.api.http.server.ConnectorApiServlet;
 import org.apache.asterix.api.http.server.DdlApiServlet;
 import org.apache.asterix.api.http.server.DiagnosticsApiServlet;
-import org.apache.asterix.api.http.server.ActiveStatsApiServlet;
 import org.apache.asterix.api.http.server.FullApiServlet;
 import org.apache.asterix.api.http.server.NodeControllerDetailsApiServlet;
 import org.apache.asterix.api.http.server.QueryApiServlet;
@@ -52,6 +51,7 @@
 import org.apache.asterix.api.http.server.UpdateApiServlet;
 import org.apache.asterix.api.http.server.VersionApiServlet;
 import org.apache.asterix.api.http.servlet.ServletConstants;
+import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
 import org.apache.asterix.app.cc.ResourceIdManager;
 import org.apache.asterix.app.external.ExternalLibraryUtils;
@@ -65,6 +65,7 @@
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.metadata.lock.MetadataLockManager;
 import org.apache.asterix.common.replication.IFaultToleranceStrategy;
 import org.apache.asterix.common.replication.IReplicationStrategy;
 import org.apache.asterix.common.utils.Servlets;
@@ -130,11 +131,11 @@
                 .create(ClusterProperties.INSTANCE.getCluster(), repStrategy, ccServiceCtx);
         ExternalLibraryUtils.setUpExternaLibraries(libraryManager, false);
         componentProvider = new StorageComponentProvider();
-        GlobalRecoveryManager.instantiate(ccServiceCtx, getHcc(), componentProvider);
+        GlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
         statementExecutorCtx = new StatementExecutorContext();
         appCtx = new CcApplicationContext(ccServiceCtx, getHcc(), libraryManager, resourceIdManager,
-                () -> MetadataManager.INSTANCE, GlobalRecoveryManager.instance(), ftStrategy,
-                new ActiveLifecycleListener(), componentProvider);
+                () -> MetadataManager.INSTANCE, globalRecoveryManager, ftStrategy, new ActiveNotificationHandler(),
+                componentProvider, new MetadataLockManager());
         ClusterStateManager.INSTANCE.setCcAppCtx(appCtx);
         ccExtensionManager = new CCExtensionManager(getExtensions());
         appCtx.setExtensionManager(ccExtensionManager);
@@ -147,18 +148,22 @@
         setAsterixStateProxy(AsterixStateProxy.registerRemoteObject(metadataProperties.getMetadataCallbackPort()));
         ccServiceCtx.setDistributedState(proxy);
         MetadataManager.initialize(proxy, metadataProperties);
-        ccServiceCtx.addJobLifecycleListener(appCtx.getActiveLifecycleListener());
+        ccServiceCtx.addJobLifecycleListener(appCtx.getActiveNotificationHandler());
 
         // create event loop groups
         webManager = new WebManager();
         configureServers();
         webManager.start();
-        ClusterManagerProvider.getClusterManager().registerSubscriber(GlobalRecoveryManager.instance());
+        ClusterManagerProvider.getClusterManager().registerSubscriber(globalRecoveryManager);
         ccServiceCtx.addClusterLifecycleListener(new ClusterLifecycleListener(appCtx));
 
         jobCapacityController = new JobCapacityController(controllerService.getResourceManager());
     }
 
+    protected GlobalRecoveryManager createGlobalRecoveryManager() throws Exception {
+        return new GlobalRecoveryManager(ccServiceCtx, getHcc(), componentProvider);
+    }
+
     @Override
     protected void configureLoggingLevel(Level level) {
         super.configureLoggingLevel(level);
@@ -178,7 +183,7 @@
 
     @Override
     public void stop() throws Exception {
-        ((ActiveLifecycleListener) appCtx.getActiveLifecycleListener()).stop();
+        ((ActiveNotificationHandler) appCtx.getActiveNotificationHandler()).stop();
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Stopping Asterix cluster controller");
         }
@@ -288,9 +293,8 @@
                         ccExtensionManager.getCompilationProvider(SQLPP), getStatementExecutorFactory(),
                         componentProvider);
             case Servlets.QUERY_AQL:
-                return new QueryServiceServlet(ctx, paths, appCtx, AQL,
-                        ccExtensionManager.getCompilationProvider(AQL), getStatementExecutorFactory(),
-                        componentProvider);
+                return new QueryServiceServlet(ctx, paths, appCtx, AQL, ccExtensionManager.getCompilationProvider(AQL),
+                        getStatementExecutorFactory(), componentProvider);
             case Servlets.CONNECTOR:
                 return new ConnectorApiServlet(ctx, paths, appCtx);
             case Servlets.REBALANCE:
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
index 2a1fd0b..3209557 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
@@ -43,33 +43,29 @@
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.utils.ExternalIndexingOperations;
 import org.apache.asterix.metadata.utils.MetadataConstants;
-import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 
 public class GlobalRecoveryManager implements IGlobalRecoveryManager {
 
     private static final Logger LOGGER = Logger.getLogger(GlobalRecoveryManager.class.getName());
-    private static GlobalRecoveryManager instance;
-    private static ClusterState state;
-    private final IStorageComponentProvider componentProvider;
-    private final ICCServiceContext ccServiceCtx;
-    private IHyracksClientConnection hcc;
+    protected final IStorageComponentProvider componentProvider;
+    protected final ICCServiceContext serviceCtx;
+    protected IHyracksClientConnection hcc;
+    protected volatile boolean recoveryCompleted;
 
-    private GlobalRecoveryManager(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
-                                  IStorageComponentProvider componentProvider) {
-        setState(ClusterState.UNUSABLE);
-        this.ccServiceCtx = ccServiceCtx;
+    public GlobalRecoveryManager(ICCServiceContext serviceCtx, IHyracksClientConnection hcc,
+            IStorageComponentProvider componentProvider) {
+        this.serviceCtx = serviceCtx;
         this.hcc = hcc;
         this.componentProvider = componentProvider;
     }
 
     @Override
     public Set<IClusterManagementWork> notifyNodeFailure(Collection<String> deadNodeIds) {
-        setState(ClusterStateManager.INSTANCE.getState());
-        ClusterStateManager.INSTANCE.setGlobalRecoveryCompleted(false);
         return Collections.emptySet();
     }
 
@@ -85,54 +81,59 @@
     }
 
     @Override
-    public void startGlobalRecovery(ICcApplicationContext appCtx) {
-        // perform global recovery if state changed to active
-        final ClusterState newState = ClusterStateManager.INSTANCE.getState();
-        boolean needToRecover = !newState.equals(state) && (newState == ClusterState.ACTIVE);
-        if (needToRecover) {
-            setState(newState);
-            ccServiceCtx.getControllerService().getExecutor().submit(() -> {
-                LOGGER.info("Starting Global Recovery");
-                MetadataTransactionContext mdTxnCtx = null;
+    public void startGlobalRecovery(ICcApplicationContext appCtx) throws HyracksDataException {
+        if (!recoveryCompleted) {
+            recover(appCtx);
+        }
+    }
+
+    protected void recover(ICcApplicationContext appCtx) throws HyracksDataException {
+        LOGGER.info("Starting Global Recovery");
+        MetadataTransactionContext mdTxnCtx = null;
+        try {
+            MetadataManager.INSTANCE.init();
+            // Loop over datasets
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            for (Dataverse dataverse : MetadataManager.INSTANCE.getDataverses(mdTxnCtx)) {
+                mdTxnCtx = recoverDataset(appCtx, mdTxnCtx, dataverse);
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            // This needs to be fixed <-- Needs to shutdown the system -->
+            /*
+             * Note: Throwing this illegal state exception will terminate this thread
+             * and feeds listeners will not be notified.
+             */
+            LOGGER.log(Level.SEVERE, "Global recovery was not completed successfully: ", e);
+            if (mdTxnCtx != null) {
                 try {
-                    MetadataManager.INSTANCE.init();
-                    // Loop over datasets
-                    mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-                    for (Dataverse dataverse : MetadataManager.INSTANCE.getDataverses(mdTxnCtx)) {
-                        mdTxnCtx = recoverDataset(appCtx, mdTxnCtx, dataverse);
-                    }
-                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                } catch (Exception e) {
-                    // This needs to be fixed <-- Needs to shutdown the system -->
-                    /*
-                     * Note: Throwing this illegal state exception will terminate this thread
-                     * and feeds listeners will not be notified.
-                     */
-                    LOGGER.log(Level.SEVERE, "Global recovery was not completed successfully: ", e);
-                    if (mdTxnCtx != null) {
-                        try {
-                            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
-                        } catch (Exception e1) {
-                            LOGGER.log(Level.SEVERE, "Exception in aborting", e1);
-                            e1.addSuppressed(e);
-                            throw new IllegalStateException(e1);
-                        }
-                    }
+                    MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+                } catch (Exception e1) {
+                    LOGGER.log(Level.SEVERE, "Exception in aborting", e1);
+                    e1.addSuppressed(e);
+                    throw new IllegalStateException(e);
                 }
-                ClusterStateManager.INSTANCE.setGlobalRecoveryCompleted(true);
-                LOGGER.info("Global Recovery Completed");
-            });
+            }
+            throw HyracksDataException.create(e);
+        }
+        recoveryCompleted = true;
+        LOGGER.info("Global Recovery Completed");
+    }
+
+    @Override
+    public void notifyStateChange(ClusterState newState) {
+        if (newState != ClusterState.ACTIVE) {
+            recoveryCompleted = false;
         }
     }
 
     private MetadataTransactionContext recoverDataset(ICcApplicationContext appCtx, MetadataTransactionContext mdTxnCtx,
-                                                      Dataverse dataverse)
-            throws Exception {
+            Dataverse dataverse) throws Exception {
         if (!dataverse.getDataverseName().equals(MetadataConstants.METADATA_DATAVERSE_NAME)) {
-            MetadataProvider metadataProvider = new MetadataProvider(appCtx, dataverse, componentProvider);
+            MetadataProvider metadataProvider = new MetadataProvider(appCtx, dataverse);
             try {
-                List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx,
-                        dataverse.getDataverseName());
+                List<Dataset> datasets =
+                        MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dataverse.getDataverseName());
                 for (Dataset dataset : datasets) {
                     if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
                         // External dataset
@@ -144,8 +145,8 @@
                         TransactionState datasetState = dsd.getState();
                         if (!indexes.isEmpty()) {
                             if (datasetState == TransactionState.BEGIN) {
-                                List<ExternalFile> files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx,
-                                        dataset);
+                                List<ExternalFile> files =
+                                        MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
                                 // if persumed abort, roll backward
                                 // 1. delete all pending files
                                 for (ExternalFile file : files) {
@@ -156,8 +157,8 @@
                             }
                             // 2. clean artifacts in NCs
                             metadataProvider.setMetadataTxnContext(mdTxnCtx);
-                            JobSpecification jobSpec = ExternalIndexingOperations.buildAbortOp(dataset, indexes,
-                                    metadataProvider);
+                            JobSpecification jobSpec =
+                                    ExternalIndexingOperations.buildAbortOp(dataset, indexes, metadataProvider);
                             executeHyracksJob(jobSpec);
                             // 3. correct the dataset state
                             ((ExternalDatasetDetails) dataset.getDatasetDetails()).setState(TransactionState.COMMIT);
@@ -165,13 +166,13 @@
                             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                             mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
                         } else if (datasetState == TransactionState.READY_TO_COMMIT) {
-                            List<ExternalFile> files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx,
-                                    dataset);
+                            List<ExternalFile> files =
+                                    MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
                             // if ready to commit, roll forward
                             // 1. commit indexes in NCs
                             metadataProvider.setMetadataTxnContext(mdTxnCtx);
-                            JobSpecification jobSpec = ExternalIndexingOperations.buildRecoverOp(dataset, indexes,
-                                    metadataProvider);
+                            JobSpecification jobSpec =
+                                    ExternalIndexingOperations.buildRecoverOp(dataset, indexes, metadataProvider);
                             executeHyracksJob(jobSpec);
                             // 2. add pending files in metadata
                             for (ExternalFile file : files) {
@@ -213,20 +214,11 @@
                 metadataProvider.getLocks().unlock();
             }
         }
-
         return mdTxnCtx;
     }
 
-    public static GlobalRecoveryManager instance() {
-        return instance;
-    }
-
-    public static synchronized void instantiate(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
-                                                IStorageComponentProvider componentProvider) {
-        instance = new GlobalRecoveryManager(ccServiceCtx, hcc, componentProvider);
-    }
-
-    public static synchronized void setState(ClusterState state) {
-        GlobalRecoveryManager.state = state;
+    @Override
+    public boolean isRecoveryCompleted() {
+        return recoveryCompleted;
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
index 1ed37e0..4174685 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
@@ -28,18 +28,19 @@
 import java.util.logging.Logger;
 import java.util.stream.IntStream;
 
-import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.active.IActiveEntityEventsListener;
+import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
-import org.apache.asterix.metadata.declared.MetadataManagerUtil;
+import org.apache.asterix.metadata.api.IActiveEntityController;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.metadata.lock.LockList;
-import org.apache.asterix.metadata.lock.MetadataLockManager;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.rebalance.IDatasetRebalanceCallback;
@@ -119,7 +120,6 @@
             // The target dataset for rebalance.
             targetDataset = sourceDataset.getTargetDatasetForRebalance(nodeGroupName);
 
-
             // Rebalances the source dataset into the target dataset.
             rebalance(sourceDataset, targetDataset, metadataProvider, hcc, datasetRebalanceCallback);
 
@@ -128,8 +128,6 @@
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
             throw e;
-        } finally {
-            metadataProvider.getLocks().reset();
         }
 
         // Up to this point, since the bulk part of a rebalance operation is done,
@@ -143,7 +141,7 @@
             // Executes the 2nd Metadata transaction for switching the metadata entity.
             // It detaches the source dataset and attaches the target dataset to metadata's point of view.
             runMetadataTransaction(metadataProvider,
-                () -> rebalanceSwitch(sourceDataset, targetDataset, metadataProvider, hcc));
+                    () -> rebalanceSwitch(sourceDataset, targetDataset, metadataProvider, hcc));
             // Executes the 3rd Metadata transaction to drop the source dataset files and the node group for
             // the source dataset.
             runMetadataTransaction(metadataProvider, () -> dropSourceDataset(sourceDataset, metadataProvider, hcc));
@@ -188,8 +186,6 @@
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
             throw e;
-        } finally {
-            metadataProvider.getLocks().reset();
         }
     }
 
@@ -219,22 +215,25 @@
     private static void rebalanceSwitch(Dataset source, Dataset target, MetadataProvider metadataProvider,
             IHyracksClientConnection hcc) throws Exception {
         MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-
-        // Acquires the metadata write lock for the source/target dataset.
-        writeLockDataset(metadataProvider.getLocks(), source);
-
-        Dataset sourceDataset = MetadataManagerUtil.findDataset(mdTxnCtx, source.getDataverseName(),
-                source.getDatasetName());
-
-        if (sourceDataset == null) {
-            // The dataset has already been dropped.
-            // In this case, we should drop the generated target dataset files.
-            dropDatasetFiles(target, metadataProvider, hcc);
-            return;
+        // upgrade lock
+        ICcApplicationContext appCtx = metadataProvider.getApplicationContext();
+        ActiveNotificationHandler activeNotificationHandler =
+                (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
+        IMetadataLockManager lockManager = appCtx.getMetadataLockManager();
+        lockManager.upgradeDatasetLockToWrite(metadataProvider.getLocks(), DatasetUtil.getFullyQualifiedName(target));
+        try {
+            // Updates the dataset entry in the metadata storage
+            MetadataManager.INSTANCE.updateDataset(mdTxnCtx, target);
+            for (IActiveEntityEventsListener listener : activeNotificationHandler.getEventListeners()) {
+                if (listener instanceof IActiveEntityController) {
+                    IActiveEntityController controller = (IActiveEntityController) listener;
+                    controller.replace(target);
+                }
+            }
+        } finally {
+            lockManager.downgradeDatasetLockToExclusiveModify(metadataProvider.getLocks(),
+                    DatasetUtil.getFullyQualifiedName(target));
         }
-
-        // Updates the dataset entry in the metadata storage
-        MetadataManager.INSTANCE.updateDataset(mdTxnCtx, target);
     }
 
     // Drops the source dataset.
@@ -245,12 +244,12 @@
         dropDatasetFiles(source, metadataProvider, hcc);
 
         // Drops the metadata entry of source dataset's node group.
+        ICcApplicationContext appCtx = metadataProvider.getApplicationContext();
         String sourceNodeGroup = source.getNodeGroupName();
-        MetadataLockManager.INSTANCE.acquireNodeGroupWriteLock(metadataProvider.getLocks(), sourceNodeGroup);
+        appCtx.getMetadataLockManager().acquireNodeGroupWriteLock(metadataProvider.getLocks(), sourceNodeGroup);
         MetadataManager.INSTANCE.dropNodegroup(metadataProvider.getMetadataTxnContext(), sourceNodeGroup, true);
     }
 
-
     // Creates the files for the rebalance target dataset.
     private static void createRebalanceTarget(Dataset target, MetadataProvider metadataProvider,
             IHyracksClientConnection hcc) throws Exception {
@@ -301,8 +300,8 @@
         int numKeys = source.getPrimaryKeys().size();
         int numValues = source.hasMetaPart() ? 2 : 1;
         int[] fieldPermutation = IntStream.range(0, numKeys + numValues).toArray();
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> upsertOpAndConstraints = DatasetUtil
-                .createPrimaryIndexUpsertOp(spec, metadataProvider, target,
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> upsertOpAndConstraints =
+                DatasetUtil.createPrimaryIndexUpsertOp(spec, metadataProvider, target,
                         source.getPrimaryRecordDescriptor(metadataProvider), fieldPermutation,
                         MissingWriterFactory.INSTANCE);
         IOperatorDescriptor upsertOp = upsertOpAndConstraints.first;
@@ -334,13 +333,6 @@
         }
     }
 
-    // Acquires a read lock for the dataverse and a write lock for the dataset, in order to populate the dataset.
-    private static void writeLockDataset(LockList locks, Dataset dataset) throws AsterixException {
-        MetadataLockManager.INSTANCE.acquireDataverseReadLock(locks, dataset.getDataverseName());
-        MetadataLockManager.INSTANCE.acquireDatasetWriteLock(locks,
-                dataset.getDataverseName() + "." + dataset.getDatasetName());
-    }
-
     // Creates and loads all secondary indexes for the rebalance target dataset.
     private static void createAndLoadSecondaryIndexesForTarget(Dataset source, Dataset target,
             MetadataProvider metadataProvider, IHyracksClientConnection hcc) throws Exception {
@@ -349,13 +341,13 @@
                 continue;
             }
             // Creates the secondary index.
-            JobSpecification indexCreationJobSpec = IndexUtil.buildSecondaryIndexCreationJobSpec(target, index,
-                    metadataProvider);
+            JobSpecification indexCreationJobSpec =
+                    IndexUtil.buildSecondaryIndexCreationJobSpec(target, index, metadataProvider);
             JobUtils.runJob(hcc, indexCreationJobSpec, true);
 
             // Loads the secondary index.
-            JobSpecification indexLoadingJobSpec = IndexUtil.buildSecondaryIndexLoadingJobSpec(target, index,
-                    metadataProvider);
+            JobSpecification indexLoadingJobSpec =
+                    IndexUtil.buildSecondaryIndexLoadingJobSpec(target, index, metadataProvider);
             JobUtils.runJob(hcc, indexLoadingJobSpec, true);
         }
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java
index 3d6543b..5abbe40 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java
@@ -30,7 +30,6 @@
 
 import org.apache.asterix.api.http.server.ConnectorApiServlet;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -48,10 +47,8 @@
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -183,8 +180,7 @@
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         // Retrieves file splits of the dataset.
         MetadataProvider metadataProvider = new MetadataProvider(
-                (ICcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext(), null,
-                new StorageComponentProvider());
+                (ICcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext(), null);
         try {
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
             Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 99892c5..c1421c5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -227,7 +227,7 @@
         Index index = primaryIndexInfo.getIndex();
         CcApplicationContext appCtx =
                 (CcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext();
-        MetadataProvider mdProvider = new MetadataProvider(appCtx, dataverse, storageComponentProvider);
+        MetadataProvider mdProvider = new MetadataProvider(appCtx, dataverse);
         try {
             return dataset.getResourceFactory(mdProvider, index, primaryIndexInfo.recordType, primaryIndexInfo.metaType,
                     primaryIndexInfo.mergePolicyFactory, primaryIndexInfo.mergePolicyProperties);
@@ -246,8 +246,7 @@
         Dataverse dataverse = new Dataverse(dataset.getDataverseName(), NonTaggedDataFormat.class.getName(),
                 MetadataUtil.PENDING_NO_OP);
         MetadataProvider mdProvider = new MetadataProvider(
-                (ICcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext(), dataverse,
-                storageComponentProvider);
+                (ICcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext(), dataverse);
         try {
             IResourceFactory resourceFactory = dataset.getResourceFactory(mdProvider, primaryIndexInfo.index,
                     recordType, metaType, mergePolicyFactory, mergePolicyProperties);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/Action.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/Action.java
new file mode 100644
index 0000000..71cb038
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/Action.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.test.active;
+
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+abstract class Action {
+    boolean done = false;
+    HyracksDataException failure;
+
+    void execute(MetadataProvider actorMdProvider) {
+        try {
+            doExecute(actorMdProvider);
+        } catch (Exception e) {
+            failure = HyracksDataException.create(e);
+        }
+        synchronized (this) {
+            done = true;
+            notifyAll();
+        }
+    }
+
+    protected abstract void doExecute(MetadataProvider mdProvider) throws Exception;
+
+    boolean hasFailed() {
+        return failure != null;
+    }
+
+    HyracksDataException getFailure() {
+        return failure;
+    }
+
+    synchronized void sync() throws InterruptedException {
+        while (!done) {
+            wait();
+        }
+    }
+
+    boolean isDone() {
+        return done;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
new file mode 100644
index 0000000..0a7b444
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -0,0 +1,1432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.test.active;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.CountRetryPolicyFactory;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.InfiniteRetryPolicyFactory;
+import org.apache.asterix.active.NoRetryPolicyFactory;
+import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
+import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.lock.MetadataLockManager;
+import org.apache.asterix.runtime.utils.CcApplicationContext;
+import org.apache.asterix.test.active.TestEventsListener.Behavior;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobIdFactory;
+import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.application.CCServiceContext;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class ActiveEventsListenerTest {
+
+    static TestClusterControllerActor clusterController;
+    static TestNodeControllerActor[] nodeControllers;
+    static TestUserActor[] users;
+    static String[] nodes = { "node1", "node2" };
+    static ActiveNotificationHandler handler;
+    static String dataverseName = "Default";
+    static String entityName = "entityName";
+    static EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, entityName);
+    static Dataset firstDataset;
+    static Dataset secondDataset;
+    static List<Dataset> allDatasets;
+    static TestEventsListener listener;
+    static IClusterStateManager clusterStateManager;
+    static CcApplicationContext appCtx;
+    static IStatementExecutor statementExecutor;
+    static IHyracksClientConnection hcc;
+    static MetadataProvider metadataProvider;
+    static IStorageComponentProvider componentProvider;
+    static JobIdFactory jobIdFactory;
+    static IMetadataLockManager lockManager = new MetadataLockManager();
+    static AlgebricksAbsolutePartitionConstraint locations;
+    static ExecutorService executor;
+
+    @Before
+    public void setUp() throws Exception {
+        jobIdFactory = new JobIdFactory();
+        handler = new ActiveNotificationHandler();
+        allDatasets = new ArrayList<>();
+        firstDataset = new Dataset(dataverseName, "firstDataset", null, null, null, null, null, null, null, null, 0, 0);
+        secondDataset =
+                new Dataset(dataverseName, "secondDataset", null, null, null, null, null, null, null, null, 0, 0);
+        allDatasets.add(firstDataset);
+        allDatasets.add(secondDataset);
+        AtomicInteger threadCounter = new AtomicInteger(0);
+        executor = Executors.newCachedThreadPool(
+                r -> new Thread(r, "ClusterControllerServiceExecutor[" + threadCounter.getAndIncrement() + "]"));
+        clusterStateManager = Mockito.mock(IClusterStateManager.class);
+        Mockito.when(clusterStateManager.getState()).thenReturn(ClusterState.ACTIVE);
+        ClusterControllerService ccService = Mockito.mock(ClusterControllerService.class);
+        CCServiceContext ccServiceCtx = Mockito.mock(CCServiceContext.class);
+        appCtx = Mockito.mock(CcApplicationContext.class);
+        statementExecutor = Mockito.mock(IStatementExecutor.class);
+        hcc = Mockito.mock(IHyracksClientConnection.class);
+        Mockito.when(appCtx.getActiveNotificationHandler()).thenReturn(handler);
+        Mockito.when(appCtx.getMetadataLockManager()).thenReturn(lockManager);
+        Mockito.when(appCtx.getServiceContext()).thenReturn(ccServiceCtx);
+        Mockito.when(appCtx.getClusterStateManager()).thenReturn(clusterStateManager);
+        componentProvider = new StorageComponentProvider();
+        Mockito.when(appCtx.getStorageComponentProvider()).thenReturn(componentProvider);
+        Mockito.when(ccServiceCtx.getControllerService()).thenReturn(ccService);
+        Mockito.when(ccService.getExecutor()).thenReturn(executor);
+        locations = new AlgebricksAbsolutePartitionConstraint(nodes);
+        metadataProvider = new MetadataProvider(appCtx, null);
+        metadataProvider.setConfig(new HashMap<>());
+        clusterController = new TestClusterControllerActor("CC", handler, allDatasets);
+        nodeControllers = new TestNodeControllerActor[2];
+        nodeControllers[0] = new TestNodeControllerActor(nodes[0], clusterController);
+        nodeControllers[1] = new TestNodeControllerActor(nodes[1], clusterController);
+        listener = new TestEventsListener(clusterController, nodeControllers, jobIdFactory, entityId,
+                new ArrayList<>(allDatasets), statementExecutor, appCtx, hcc, locations,
+                new InfiniteRetryPolicyFactory());
+        users = new TestUserActor[3];
+        users[0] = newUser("Till", appCtx);
+        users[1] = newUser("Mike", appCtx);
+        users[2] = newUser("Dmitry", appCtx);
+    }
+
+    TestUserActor newUser(String name, CcApplicationContext appCtx) {
+        MetadataProvider actorMdProvider = new MetadataProvider(appCtx, null);
+        actorMdProvider.setConfig(new HashMap<>());
+        return new TestUserActor("User: " + name, actorMdProvider, clusterController);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        executor.shutdownNow();
+        executor.awaitTermination(5, TimeUnit.SECONDS);
+        handler.stop();
+        for (Actor user : users) {
+            user.stop();
+        }
+        for (Actor nc : nodeControllers) {
+            nc.stop();
+        }
+        clusterController.stop();
+    }
+
+    @Test
+    public void testStartWhenStartSucceed() throws Exception {
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStart(Behavior.SUCCEED);
+        Action action = users[0].startActivity(listener);
+        action.sync();
+        assertSuccess(action);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @Test
+    public void testStopWhenStopSucceed() throws Exception {
+        testStartWhenStartSucceed();
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.SUCCEED);
+        Action action = users[0].stopActivity(listener);
+        action.sync();
+        assertSuccess(action);
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @Test
+    public void testDoubleStopWhenStopSucceed() throws Exception {
+        testStartWhenStartSucceed();
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.SUCCEED);
+        Action firstStop = users[0].stopActivity(listener);
+        Action secondStop = users[1].stopActivity(listener);
+        firstStop.sync();
+        secondStop.sync();
+        if (firstStop.hasFailed()) {
+            assertFailure(firstStop, ErrorCode.ACTIVE_ENTITY_CANNOT_BE_STOPPED);
+            assertSuccess(secondStop);
+        } else {
+            assertSuccess(firstStop);
+            assertFailure(secondStop, ErrorCode.ACTIVE_ENTITY_CANNOT_BE_STOPPED);
+        }
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @Test
+    public void testDoubleStartWhenStartSucceed() throws Exception {
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStart(Behavior.SUCCEED);
+        Action firstStart = users[0].startActivity(listener);
+        Action secondStart = users[1].startActivity(listener);
+        firstStart.sync();
+        secondStart.sync();
+        if (firstStart.hasFailed()) {
+            assertFailure(firstStart, ErrorCode.ACTIVE_ENTITY_ALREADY_STARTED);
+            assertSuccess(secondStart);
+        } else {
+            assertSuccess(firstStart);
+            assertFailure(secondStart, ErrorCode.ACTIVE_ENTITY_ALREADY_STARTED);
+        }
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @Test
+    public void testStopAfterDoubleStartWhenStartSucceedAndStopSucceed() throws Exception {
+        testDoubleStartWhenStartSucceed();
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.SUCCEED);
+        Action action = users[2].stopActivity(listener);
+        action.sync();
+        assertSuccess(action);
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @Test
+    public void testSuspendFromStopped() throws Exception {
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStop(Behavior.SUCCEED);
+        Action action = users[0].suspendActivity(listener);
+        action.sync();
+        assertSuccess(action);
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        action = users[0].resumeActivity(listener);
+        action.sync();
+        assertSuccess(action);
+    }
+
+    @Test
+    public void testStartWhileSuspend() throws Exception {
+        listener.onStart(Behavior.SUCCEED);
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStop(Behavior.SUCCEED);
+        Action action = users[0].suspendActivity(listener);
+        action.sync();
+        assertSuccess(action);
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        // user[0] has the locks
+        Action startAction = users[1].startActivity(listener);
+        for (int i = 0; i < 100; i++) {
+            Assert.assertFalse(startAction.isDone());
+        }
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        Action resumeAction = users[0].resumeActivity(listener);
+        resumeAction.sync();
+        startAction.sync();
+        assertSuccess(resumeAction);
+        assertSuccess(startAction);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @Test
+    public void testSuspendFromRunning() throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.SUCCEED);
+        Action suspension = users[1].suspendActivity(listener);
+        suspension.sync();
+        assertSuccess(suspension);
+        // resume
+        Assert.assertEquals(ActivityState.SUSPENDED, listener.getState());
+        Action resumption = users[1].resumeActivity(listener);
+        resumption.sync();
+        assertSuccess(resumption);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @Test
+    public void testSuspendFromRunningAndStopFail() throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.FAIL_COMPILE);
+        Action suspension = users[1].suspendActivity(listener);
+        suspension.sync();
+        Assert.assertTrue(suspension.hasFailed());
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testRecovery() throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStart(Behavior.STEP_SUCCEED);
+        WaitForStateSubscriber subscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        subscriber.sync();
+        Assert.assertNotNull(listener.getRecoveryTask());
+        listener.allowStep();
+        WaitForStateSubscriber running = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        running.sync();
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.SUCCEED);
+        Action stopAction = users[2].stopActivity(listener);
+        stopAction.sync();
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @Test
+    public void testSuspendFromRunningButJobFailWhileSuspendingThenResumeSucceed() throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.RUNNING_JOB_FAIL);
+        Action suspension = users[1].suspendActivity(listener);
+        suspension.sync();
+        assertSuccess(suspension);
+        Assert.assertEquals(ActivityState.TEMPORARILY_FAILED, listener.getState());
+        Assert.assertNull(listener.getRecoveryTask());
+        listener.onStart(Behavior.SUCCEED);
+        Action resumption = users[1].resumeActivity(listener);
+        resumption.sync();
+        assertSuccess(resumption);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @Test
+    public void testSuspendFromRunningButJobFailWhileSuspendingThenResumeFailsCompileAndRecoveryStarts()
+            throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.RUNNING_JOB_FAIL);
+        Action suspension = users[1].suspendActivity(listener);
+        suspension.sync();
+        assertSuccess(suspension);
+        Assert.assertEquals(ActivityState.TEMPORARILY_FAILED, listener.getState());
+        Assert.assertNull(listener.getRecoveryTask());
+        listener.onStart(Behavior.FAIL_COMPILE);
+        Action resumption = users[1].resumeActivity(listener);
+        resumption.sync();
+        assertSuccess(resumption);
+        ActivityState state = listener.getState();
+        Assert.assertTrue(state == ActivityState.RECOVERING || state == ActivityState.TEMPORARILY_FAILED);
+        Assert.assertNotNull(listener.getRecoveryTask());
+    }
+
+    @Test
+    public void testSuspendFromRunningButJobFailWhileSuspendingThenResumeFailsRuntimeAndRecoveryStarts()
+            throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.RUNNING_JOB_FAIL);
+        Action suspension = users[1].suspendActivity(listener);
+        suspension.sync();
+        assertSuccess(suspension);
+        Assert.assertEquals(ActivityState.TEMPORARILY_FAILED, listener.getState());
+        Assert.assertNull(listener.getRecoveryTask());
+        listener.onStart(Behavior.FAIL_RUNTIME);
+        Action resumption = users[1].resumeActivity(listener);
+        resumption.sync();
+        assertSuccess(resumption);
+        ActivityState state = listener.getState();
+        Assert.assertTrue(state == ActivityState.RECOVERING || state == ActivityState.TEMPORARILY_FAILED);
+        Assert.assertNotNull(listener.getRecoveryTask());
+    }
+
+    @Test
+    public void testStopWhileSuspended() throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.STEP_SUCCEED);
+        Action suspension = users[1].suspendActivity(listener);
+        WaitForStateSubscriber subscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.SUSPENDING, ActivityState.SUSPENDED));
+        subscriber.sync();
+        Action stopping = users[0].stopActivity(listener);
+        listener.allowStep();
+        listener.allowStep();
+        suspension.sync();
+        assertSuccess(suspension);
+        users[1].resumeActivity(listener);
+        stopping.sync();
+        assertSuccess(stopping);
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testRecoveryFailureAfterOneAttemptCompilationFailure() throws Exception {
+        handler.unregisterListener(listener);
+        listener = new TestEventsListener(clusterController, nodeControllers, jobIdFactory, entityId,
+                new ArrayList<>(allDatasets), statementExecutor, appCtx, hcc, locations,
+                new CountRetryPolicyFactory(1));
+        testStartWhenStartSucceed();
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        WaitForStateSubscriber permFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.PERMANENTLY_FAILED));
+        listener.onStart(Behavior.FAIL_COMPILE);
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Compilation Failure")));
+        tempFailSubscriber.sync();
+        permFailSubscriber.sync();
+        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+    }
+
+    @Test
+    public void testStartAfterPermenantFailure() throws Exception {
+        testRecoveryFailureAfterOneAttemptCompilationFailure();
+        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        listener.onStart(Behavior.SUCCEED);
+        WaitForStateSubscriber subscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        users[1].startActivity(listener);
+        subscriber.sync();
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @Test
+    public void testStopAfterStartAfterPermenantFailure() throws Exception {
+        testStartAfterPermenantFailure();
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.SUCCEED);
+        WaitForStateSubscriber subscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.STOPPED));
+        users[1].stopActivity(listener);
+        subscriber.sync();
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testRecoveryFailureAfterOneAttemptRuntimeFailure() throws Exception {
+        handler.unregisterListener(listener);
+        listener = new TestEventsListener(clusterController, nodeControllers, jobIdFactory, entityId,
+                new ArrayList<>(allDatasets), statementExecutor, appCtx, hcc, locations,
+                new CountRetryPolicyFactory(1));
+        testStartWhenStartSucceed();
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        WaitForStateSubscriber permFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.PERMANENTLY_FAILED));
+        listener.onStart(Behavior.FAIL_RUNTIME);
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        tempFailSubscriber.sync();
+        permFailSubscriber.sync();
+        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testRecoveryFailure() throws Exception {
+        handler.unregisterListener(listener);
+        listener = new TestEventsListener(clusterController, nodeControllers, jobIdFactory, entityId,
+                new ArrayList<>(allDatasets), statementExecutor, appCtx, hcc, locations, NoRetryPolicyFactory.INSTANCE);
+        testStartWhenStartSucceed();
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        WaitForStateSubscriber permFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.PERMANENTLY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        tempFailSubscriber.sync();
+        permFailSubscriber.sync();
+        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testStopDuringRecoveryAttemptThatSucceeds() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_SUCCEED);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        WaitForStateSubscriber stopSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.STOPPED));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        listener.onStop(Behavior.SUCCEED);
+        users[0].stopActivity(listener);
+        listener.allowStep();
+        runningSubscriber.sync();
+        stopSubscriber.sync();
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testStopDuringRecoveryAttemptThatFailsCompile() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_COMPILE);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber secondTempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        WaitForStateSubscriber stopSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.STOPPED));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        listener.onStop(Behavior.SUCCEED);
+        users[0].stopActivity(listener);
+        listener.allowStep();
+        secondTempFailSubscriber.sync();
+        stopSubscriber.sync();
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testStopDuringRecoveryAttemptThatFailsRuntime() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_RUNTIME);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber secondTempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        WaitForStateSubscriber stopSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.STOPPED));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        listener.onStop(Behavior.SUCCEED);
+        users[0].stopActivity(listener);
+        listener.allowStep();
+        secondTempFailSubscriber.sync();
+        stopSubscriber.sync();
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testStartDuringRecoveryAttemptThatSucceeds() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_SUCCEED);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        listener.onStop(Behavior.SUCCEED);
+        Action startAction = users[0].startActivity(listener);
+        listener.allowStep();
+        runningSubscriber.sync();
+        startAction.sync();
+        assertFailure(startAction, ErrorCode.ACTIVE_ENTITY_ALREADY_STARTED);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testStartDuringRecoveryAttemptThatFailsCompile() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_COMPILE);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber secondTempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        listener.onStop(Behavior.SUCCEED);
+        Action action = users[0].startActivity(listener);
+        listener.allowStep();
+        secondTempFailSubscriber.sync();
+        action.sync();
+        assertFailure(action, ErrorCode.ACTIVE_ENTITY_ALREADY_STARTED);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testStartDuringRecoveryAttemptThatFailsRuntime() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_RUNTIME);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber secondTempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        Action action = users[0].startActivity(listener);
+        listener.allowStep();
+        secondTempFailSubscriber.sync();
+        action.sync();
+        assertFailure(action, ErrorCode.ACTIVE_ENTITY_ALREADY_STARTED);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testSuspendDuringRecoveryAttemptThatSucceedsThenResumeSucceeds() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_SUCCEED);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        listener.onStop(Behavior.SUCCEED);
+        Action suspend = users[1].suspendActivity(listener);
+        listener.allowStep();
+        runningSubscriber.sync();
+        suspend.sync();
+        assertSuccess(suspend);
+        Assert.assertEquals(ActivityState.SUSPENDED, listener.getState());
+        listener.onStart(Behavior.SUCCEED);
+        Action resume = users[1].resumeActivity(listener);
+        resume.sync();
+        assertSuccess(resume);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testSuspendDuringRecoveryAttemptThatSucceedsThenResumeFailsCompile() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_SUCCEED);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        listener.onStop(Behavior.SUCCEED);
+        Action suspend = users[1].suspendActivity(listener);
+        listener.allowStep();
+        runningSubscriber.sync();
+        suspend.sync();
+        assertSuccess(suspend);
+        Assert.assertEquals(ActivityState.SUSPENDED, listener.getState());
+        // Fix here
+        listener.onStart(Behavior.FAIL_COMPILE);
+        tempFailSubscriber =
+                new WaitForStateSubscriber(listener, Collections.singleton(ActivityState.TEMPORARILY_FAILED));
+        recoveringSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        Action resume = users[1].resumeActivity(listener);
+        resume.sync();
+        assertSuccess(resume);
+        tempFailSubscriber.sync();
+        recoveringSubscriber.sync();
+        ActivityState state = listener.getState();
+        Assert.assertTrue(state == ActivityState.RECOVERING || state == ActivityState.TEMPORARILY_FAILED);
+        Assert.assertNotNull(listener.getRecoveryTask());
+        runningSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        listener.onStart(Behavior.SUCCEED);
+        runningSubscriber.sync();
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testSuspendDuringRecoveryAttemptThatSucceedsThenResumeFailsRuntime() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_SUCCEED);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        listener.onStop(Behavior.SUCCEED);
+        Action suspend = users[1].suspendActivity(listener);
+        listener.allowStep();
+        runningSubscriber.sync();
+        suspend.sync();
+        assertSuccess(suspend);
+        Assert.assertEquals(ActivityState.SUSPENDED, listener.getState());
+        // Fix here
+        listener.onStart(Behavior.FAIL_RUNTIME);
+        tempFailSubscriber =
+                new WaitForStateSubscriber(listener, Collections.singleton(ActivityState.TEMPORARILY_FAILED));
+        recoveringSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        Action resume = users[1].resumeActivity(listener);
+        resume.sync();
+        assertSuccess(resume);
+        tempFailSubscriber.sync();
+        recoveringSubscriber.sync();
+        ActivityState state = listener.getState();
+        Assert.assertTrue(state == ActivityState.RECOVERING || state == ActivityState.TEMPORARILY_FAILED);
+        Assert.assertNotNull(listener.getRecoveryTask());
+        runningSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        listener.onStart(Behavior.SUCCEED);
+        runningSubscriber.sync();
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testSuspendDuringRecoveryAttemptThatFailsCompile() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_COMPILE);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        Assert.assertEquals(ActivityState.RECOVERING, listener.getState());
+        tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        Action suspend = users[1].suspendActivity(listener);
+        listener.onStart(Behavior.FAIL_COMPILE);
+        listener.allowStep();
+        tempFailSubscriber.sync();
+        suspend.sync();
+        assertSuccess(suspend);
+        Assert.assertEquals(ActivityState.SUSPENDED, listener.getState());
+        tempFailSubscriber =
+                new WaitForStateSubscriber(listener, Collections.singleton(ActivityState.TEMPORARILY_FAILED));
+        recoveringSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        Action resume = users[1].resumeActivity(listener);
+        resume.sync();
+        assertSuccess(resume);
+        tempFailSubscriber.sync();
+        recoveringSubscriber.sync();
+        ActivityState state = listener.getState();
+        Assert.assertTrue(state == ActivityState.RECOVERING || state == ActivityState.TEMPORARILY_FAILED);
+        Assert.assertNotNull(listener.getRecoveryTask());
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        listener.onStart(Behavior.SUCCEED);
+        runningSubscriber.sync();
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testSuspendDuringRecoveryAttemptThatFailsRuntime() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_RUNTIME);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_RUNTIME);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        Assert.assertEquals(ActivityState.RECOVERING, listener.getState());
+        tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        Action suspend = users[1].suspendActivity(listener);
+        listener.onStart(Behavior.FAIL_RUNTIME);
+        listener.allowStep();
+        tempFailSubscriber.sync();
+        suspend.sync();
+        assertSuccess(suspend);
+        Assert.assertEquals(ActivityState.SUSPENDED, listener.getState());
+        tempFailSubscriber =
+                new WaitForStateSubscriber(listener, Collections.singleton(ActivityState.TEMPORARILY_FAILED));
+        recoveringSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        Action resume = users[1].resumeActivity(listener);
+        resume.sync();
+        assertSuccess(resume);
+        tempFailSubscriber.sync();
+        recoveringSubscriber.sync();
+        ActivityState state = listener.getState();
+        Assert.assertTrue(state == ActivityState.RECOVERING || state == ActivityState.TEMPORARILY_FAILED);
+        Assert.assertNotNull(listener.getRecoveryTask());
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        listener.onStart(Behavior.SUCCEED);
+        runningSubscriber.sync();
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testCreateNewShadowDuringRecoveryAttemptThatSucceeds() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_SUCCEED);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        Dataset newDataset =
+                new Dataset(dataverseName, "newDataset", null, null, null, null, null, null, null, null, 0, 0);
+        Action add = users[1].addDataset(newDataset, listener);
+        listener.allowStep();
+        runningSubscriber.sync();
+        add.sync();
+        assertFailure(add, ErrorCode.CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY);
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testCreateNewShadowDuringRecoveryAttemptThatFailsCompile() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_COMPILE);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        Dataset newDataset =
+                new Dataset(dataverseName, "newDataset", null, null, null, null, null, null, null, null, 0, 0);
+        Action add = users[1].addDataset(newDataset, listener);
+        listener.allowStep();
+        tempFailSubscriber.sync();
+        add.sync();
+        assertFailure(add, ErrorCode.CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY);
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testCreateNewShadowDuringRecoveryAttemptThatFailsRuntime() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_RUNTIME);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        Dataset newDataset =
+                new Dataset(dataverseName, "newDataset", null, null, null, null, null, null, null, null, 0, 0);
+        Action add = users[1].addDataset(newDataset, listener);
+        listener.allowStep();
+        tempFailSubscriber.sync();
+        add.sync();
+        assertFailure(add, ErrorCode.CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY);
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @Test
+    public void testCreateNewShadowWhileStarting() throws Exception {
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStart(Behavior.STEP_SUCCEED);
+        Action startAction = users[0].startActivity(listener);
+        WaitForStateSubscriber subscriber =
+                new WaitForStateSubscriber(listener, Collections.singleton(ActivityState.STARTING));
+        subscriber.sync();
+        Dataset newDataset =
+                new Dataset(dataverseName, "newDataset", null, null, null, null, null, null, null, null, 0, 0);
+        Action createDatasetAction = users[1].addDataset(newDataset, listener);
+        listener.allowStep();
+        startAction.sync();
+        assertSuccess(startAction);
+        createDatasetAction.sync();
+        assertFailure(createDatasetAction, ErrorCode.CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @Test
+    public void testCreateNewShadowWhileRunning() throws Exception {
+        testStartWhenStartSucceed();
+        Dataset newDataset =
+                new Dataset(dataverseName, "newDataset", null, null, null, null, null, null, null, null, 0, 0);
+        Action createDatasetAction = users[1].addDataset(newDataset, listener);
+        createDatasetAction.sync();
+        assertFailure(createDatasetAction, ErrorCode.CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @Test
+    public void testCreateNewShadowWhileSuspended() throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.STEP_SUCCEED);
+        Action suspension = users[1].suspendActivity(listener);
+        WaitForStateSubscriber subscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.SUSPENDING, ActivityState.SUSPENDED));
+        subscriber.sync();
+        Dataset newDataset =
+                new Dataset(dataverseName, "newDataset", null, null, null, null, null, null, null, null, 0, 0);
+        Action createDatasetAction = users[0].addDataset(newDataset, listener);
+        listener.allowStep();
+        listener.allowStep();
+        suspension.sync();
+        assertSuccess(suspension);
+        users[1].resumeActivity(listener);
+        createDatasetAction.sync();
+        assertFailure(createDatasetAction, ErrorCode.CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @Test
+    public void testCreateNewShadowWhilePermanentFailure() throws Exception {
+        testRecoveryFailureAfterOneAttemptCompilationFailure();
+        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Dataset newDataset =
+                new Dataset(dataverseName, "newDataset", null, null, null, null, null, null, null, null, 0, 0);
+        Action createDatasetAction = users[0].addDataset(newDataset, listener);
+        createDatasetAction.sync();
+        assertSuccess(createDatasetAction);
+        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(3, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testDeleteShadowDuringRecoveryAttemptThatSucceeds() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_SUCCEED);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        Action drop = users[1].dropDataset(firstDataset, listener);
+        listener.allowStep();
+        runningSubscriber.sync();
+        drop.sync();
+        assertFailure(drop, ErrorCode.CANNOT_REMOVE_DATASET_FROM_ACTIVE_ENTITY);
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testDeleteShadowDuringRecoveryAttemptThatFailsCompile() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_COMPILE);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        Action drop = users[1].dropDataset(firstDataset, listener);
+        listener.allowStep();
+        tempFailSubscriber.sync();
+        drop.sync();
+        assertFailure(drop, ErrorCode.CANNOT_REMOVE_DATASET_FROM_ACTIVE_ENTITY);
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testDeleteShadowDuringRecoveryAttemptThatFailsRuntime() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_RUNTIME);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        Action drop = users[1].dropDataset(firstDataset, listener);
+        listener.allowStep();
+        tempFailSubscriber.sync();
+        drop.sync();
+        assertFailure(drop, ErrorCode.CANNOT_REMOVE_DATASET_FROM_ACTIVE_ENTITY);
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @Test
+    public void testDeleteShadowWhileStarting() throws Exception {
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStart(Behavior.STEP_SUCCEED);
+        Action startAction = users[0].startActivity(listener);
+        WaitForStateSubscriber subscriber =
+                new WaitForStateSubscriber(listener, Collections.singleton(ActivityState.STARTING));
+        subscriber.sync();
+        Action dropDatasetAction = users[1].dropDataset(firstDataset, listener);
+        listener.allowStep();
+        startAction.sync();
+        assertSuccess(startAction);
+        dropDatasetAction.sync();
+        assertFailure(dropDatasetAction, ErrorCode.CANNOT_REMOVE_DATASET_FROM_ACTIVE_ENTITY);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @Test
+    public void testDeleteShadowWhileRunning() throws Exception {
+        testStartWhenStartSucceed();
+        Action dropDatasetAction = users[1].dropDataset(firstDataset, listener);
+        dropDatasetAction.sync();
+        assertFailure(dropDatasetAction, ErrorCode.CANNOT_REMOVE_DATASET_FROM_ACTIVE_ENTITY);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @Test
+    public void testDeleteShadowWhilePermanentFailure() throws Exception {
+        testRecoveryFailureAfterOneAttemptCompilationFailure();
+        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Action dropDatasetAction = users[0].dropDataset(secondDataset, listener);
+        dropDatasetAction.sync();
+        assertSuccess(dropDatasetAction);
+        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(1, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @Test
+    public void testDeleteShadowWhileSuspended() throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.STEP_SUCCEED);
+        Action suspension = users[1].suspendActivity(listener);
+        WaitForStateSubscriber subscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.SUSPENDING, ActivityState.SUSPENDED));
+        subscriber.sync();
+        Action dropDatasetAction = users[0].dropDataset(secondDataset, listener);
+        listener.allowStep();
+        listener.allowStep();
+        suspension.sync();
+        assertSuccess(suspension);
+        users[1].resumeActivity(listener);
+        dropDatasetAction.sync();
+        assertFailure(dropDatasetAction, ErrorCode.CANNOT_REMOVE_DATASET_FROM_ACTIVE_ENTITY);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        Assert.assertEquals(2, listener.getDatasets().size());
+        Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testCreateNewIndexDuringRecoveryAttemptThatSucceeds() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_SUCCEED);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        Action add = users[1].addIndex(firstDataset, listener);
+        listener.allowStep();
+        runningSubscriber.sync();
+        add.sync();
+        assertFailure(add, ErrorCode.CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testCreateNewIndexDuringRecoveryAttemptThatFailsCompile() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_COMPILE);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        Action add = users[1].addIndex(firstDataset, listener);
+        listener.allowStep();
+        tempFailSubscriber.sync();
+        add.sync();
+        assertFailure(add, ErrorCode.CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testCreateNewIndexDuringRecoveryAttemptThatFailsRuntime() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_RUNTIME);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_COMPILE);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        Action add = users[1].addIndex(firstDataset, listener);
+        listener.allowStep();
+        tempFailSubscriber.sync();
+        add.sync();
+        assertFailure(add, ErrorCode.CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+    }
+
+    @Test
+    public void testCreateNewIndexWhileStarting() throws Exception {
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStart(Behavior.STEP_SUCCEED);
+        Action startAction = users[0].startActivity(listener);
+        WaitForStateSubscriber subscriber =
+                new WaitForStateSubscriber(listener, Collections.singleton(ActivityState.STARTING));
+        subscriber.sync();
+        Action add = users[1].addIndex(firstDataset, listener);
+        listener.allowStep();
+        startAction.sync();
+        assertSuccess(startAction);
+        add.sync();
+        assertFailure(add, ErrorCode.CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @Test
+    public void testCreateNewIndexWhileRunning() throws Exception {
+        testStartWhenStartSucceed();
+        Action add = users[1].addIndex(firstDataset, listener);
+        add.sync();
+        assertFailure(add, ErrorCode.CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+    }
+
+    @Test
+    public void testCreateNewIndexWhilePermanentFailure() throws Exception {
+        testRecoveryFailureAfterOneAttemptCompilationFailure();
+        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Action add = users[1].addIndex(firstDataset, listener);
+        add.sync();
+        assertSuccess(add);
+    }
+
+    @Test
+    public void testCreateNewIndexWhileSuspended() throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.STEP_SUCCEED);
+        Action suspension = users[1].suspendActivity(listener);
+        WaitForStateSubscriber subscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.SUSPENDING, ActivityState.SUSPENDED));
+        subscriber.sync();
+        Action add = users[0].addIndex(firstDataset, listener);
+        listener.allowStep();
+        listener.allowStep();
+        suspension.sync();
+        assertSuccess(suspension);
+        users[1].resumeActivity(listener);
+        add.sync();
+        assertFailure(add, ErrorCode.CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testDeleteIndexDuringRecoveryAttemptThatSucceeds() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_SUCCEED);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber runningSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        Action drop = users[1].dropIndex(firstDataset, listener);
+        listener.allowStep();
+        runningSubscriber.sync();
+        drop.sync();
+        assertFailure(drop, ErrorCode.CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testDeleteIndexDuringRecoveryAttemptThatFailsCompile() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_COMPILE);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        Action drop = users[1].dropIndex(firstDataset, listener);
+        listener.allowStep();
+        tempFailSubscriber.sync();
+        drop.sync();
+        assertFailure(drop, ErrorCode.CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    public void testDeleteIndexDuringRecoveryAttemptThatFailsRuntime() throws Exception {
+        testStartWhenStartSucceed();
+        listener.onStart(Behavior.FAIL_COMPILE);
+        WaitForStateSubscriber tempFailSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
+                Collections.singletonList(new HyracksDataException("Runtime Failure")));
+        // recovery is ongoing
+        listener.onStart(Behavior.STEP_FAIL_RUNTIME);
+        tempFailSubscriber.sync();
+        WaitForStateSubscriber recoveringSubscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
+        recoveringSubscriber.sync();
+        tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
+        Action drop = users[1].dropIndex(firstDataset, listener);
+        listener.allowStep();
+        tempFailSubscriber.sync();
+        drop.sync();
+        assertFailure(drop, ErrorCode.CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+    }
+
+    @Test
+    public void testDeleteIndexwWhileStarting() throws Exception {
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStart(Behavior.STEP_SUCCEED);
+        Action startAction = users[0].startActivity(listener);
+        WaitForStateSubscriber subscriber =
+                new WaitForStateSubscriber(listener, Collections.singleton(ActivityState.STARTING));
+        subscriber.sync();
+        Action drop = users[1].dropIndex(firstDataset, listener);
+        listener.allowStep();
+        startAction.sync();
+        assertSuccess(startAction);
+        drop.sync();
+        assertFailure(drop, ErrorCode.CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @Test
+    public void testDeleteIndexWhileRunning() throws Exception {
+        testStartWhenStartSucceed();
+        Action drop = users[1].dropIndex(firstDataset, listener);
+        drop.sync();
+        assertFailure(drop, ErrorCode.CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @Test
+    public void testDeleteIndexWhilePermanentFailure() throws Exception {
+        testRecoveryFailureAfterOneAttemptCompilationFailure();
+        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Action drop = users[1].dropIndex(firstDataset, listener);
+        drop.sync();
+        assertSuccess(drop);
+    }
+
+    @Test
+    public void testDeleteIndexWhileSuspended() throws Exception {
+        testStartWhenStartSucceed();
+        // suspend
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.STEP_SUCCEED);
+        Action suspension = users[1].suspendActivity(listener);
+        WaitForStateSubscriber subscriber =
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.SUSPENDING, ActivityState.SUSPENDED));
+        subscriber.sync();
+        Action drop = users[0].dropIndex(firstDataset, listener);
+        listener.allowStep();
+        listener.allowStep();
+        suspension.sync();
+        assertSuccess(suspension);
+        users[1].resumeActivity(listener);
+        drop.sync();
+        assertFailure(drop, ErrorCode.CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY);
+    }
+
+    @Test
+    public void testSuspendedAllActivities() throws Exception {
+        TestEventsListener[] additionalListeners = new TestEventsListener[3];
+        for (int i = 0; i < additionalListeners.length; i++) {
+            String entityName = "entityName" + i;
+            EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, entityName);
+            ClusterControllerService ccService = Mockito.mock(ClusterControllerService.class);
+            CCServiceContext ccServiceCtx = Mockito.mock(CCServiceContext.class);
+            CcApplicationContext ccAppCtx = Mockito.mock(CcApplicationContext.class);
+            IStatementExecutor statementExecutor = Mockito.mock(IStatementExecutor.class);
+            IHyracksClientConnection hcc = Mockito.mock(IHyracksClientConnection.class);
+            Mockito.when(ccAppCtx.getActiveNotificationHandler()).thenReturn(handler);
+            Mockito.when(ccAppCtx.getMetadataLockManager()).thenReturn(lockManager);
+            Mockito.when(ccAppCtx.getServiceContext()).thenReturn(ccServiceCtx);
+            Mockito.when(ccAppCtx.getClusterStateManager()).thenReturn(clusterStateManager);
+            Mockito.when(ccServiceCtx.getControllerService()).thenReturn(ccService);
+            Mockito.when(ccService.getExecutor()).thenReturn(executor);
+            Mockito.when(ccAppCtx.getStorageComponentProvider()).thenReturn(componentProvider);
+            AlgebricksAbsolutePartitionConstraint locations = new AlgebricksAbsolutePartitionConstraint(nodes);
+            MetadataProvider metadataProvider = new MetadataProvider(ccAppCtx, null);
+            metadataProvider.setConfig(new HashMap<>());
+            additionalListeners[i] = listener = new TestEventsListener(clusterController, nodeControllers, jobIdFactory,
+                    entityId, new ArrayList<>(allDatasets), statementExecutor, ccAppCtx, hcc, locations,
+                    new InfiniteRetryPolicyFactory());
+        }
+        Action suspension = users[0].suspendAllActivities(handler);
+        suspension.sync();
+        assertSuccess(suspension);
+        Action query = users[1].query(firstDataset, new Semaphore(1));
+        query.sync();
+        assertSuccess(query);
+        Dataset newDataset =
+                new Dataset(dataverseName, "newDataset", null, null, null, null, null, null, null, null, 0, 0);
+        Action addDataset = users[1].addDataset(newDataset, listener);
+        // blocked by suspension
+        Assert.assertFalse(addDataset.isDone());
+        Action resumption = users[0].resumeAllActivities(handler);
+        resumption.sync();
+        assertSuccess(resumption);
+        addDataset.sync();
+        assertSuccess(addDataset);
+    }
+
+    private void assertFailure(Action action, int errorCode) throws Exception {
+        HyracksDataException exception = action.getFailure();
+        try {
+            Assert.assertTrue(action.hasFailed());
+            Assert.assertNotNull(exception);
+            Assert.assertEquals(errorCode, exception.getErrorCode());
+        } catch (Exception e) {
+            throw new Exception("Expected failure: " + errorCode + ". Found failure: " + exception);
+        }
+    }
+
+    private void assertSuccess(Action action) throws Exception {
+        if (action.hasFailed()) {
+            System.err.println("Action failed while it was expected to succeed");
+            action.getFailure().printStackTrace();
+            throw action.getFailure();
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
index 956d111..f8baa0e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
@@ -21,23 +21,29 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
-import org.apache.asterix.active.ActiveJobNotificationHandler;
-import org.apache.asterix.active.ActiveLifecycleListener;
 import org.apache.asterix.active.ActiveRuntimeId;
 import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.IActiveRuntime;
+import org.apache.asterix.active.NoRetryPolicyFactory;
 import org.apache.asterix.active.message.ActivePartitionMessage;
+import org.apache.asterix.algebra.base.ILangExtension.Language;
+import org.apache.asterix.app.active.ActiveEntityEventsListener;
+import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.app.cc.CCExtensionManager;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.metadata.IDataset;
-import org.apache.asterix.external.feed.management.ActiveEntityEventsListener;
 import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.test.runtime.ExecutionTestUtil;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.SessionOutput;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobId;
@@ -65,14 +71,14 @@
         EntityId entityId = new EntityId("MockExtension", "MockDataverse", "MockEntity");
         ActiveRuntimeId activeRuntimeId =
                 new ActiveRuntimeId(entityId, FeedIntakeOperatorNodePushable.class.getSimpleName(), 0);
-        List<IDataset> datasetList = new ArrayList<>();
+        List<Dataset> datasetList = new ArrayList<>();
         AlgebricksAbsolutePartitionConstraint partitionConstraint =
                 new AlgebricksAbsolutePartitionConstraint(new String[] { "asterix_nc1" });
         String requestedStats;
         CcApplicationContext appCtx =
                 (CcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext();
-        ActiveLifecycleListener activeLifecycleListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
-        ActiveJobNotificationHandler activeJobNotificationHandler = activeLifecycleListener.getNotificationHandler();
+        ActiveNotificationHandler activeJobNotificationHandler =
+                (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
         JobId jobId = new JobId(1);
 
         // Mock ActiveRuntime
@@ -82,14 +88,19 @@
 
         // Mock JobSpecification
         JobSpecification jobSpec = Mockito.mock(JobSpecification.class);
-        Mockito.when(jobSpec.getProperty(ActiveJobNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME))
-                .thenReturn(entityId);
+        Mockito.when(jobSpec.getProperty(ActiveNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME)).thenReturn(entityId);
 
+        // Mock MetadataProvider
+        CCExtensionManager extensionManager = (CCExtensionManager) appCtx.getExtensionManager();
+        IStatementExecutor statementExecutor = extensionManager
+                .getStatementExecutorFactory(appCtx.getServiceContext().getControllerService().getExecutor())
+                .create(appCtx, Collections.emptyList(), Mockito.mock(SessionOutput.class),
+                        extensionManager.getCompilationProvider(Language.SQLPP), appCtx.getStorageComponentProvider());
+        MetadataProvider mdProvider = new MetadataProvider(appCtx, null);
         // Add event listener
-        ActiveEntityEventsListener eventsListener = new ActiveEntityEventsListener(appCtx, entityId, datasetList,
-                partitionConstraint, FeedIntakeOperatorNodePushable.class.getSimpleName());
-        activeJobNotificationHandler.registerListener(eventsListener);
-
+        ActiveEntityEventsListener eventsListener = new DummyFeedEventsListener(statementExecutor, appCtx, null,
+                entityId, datasetList, partitionConstraint, FeedIntakeOperatorNodePushable.class.getSimpleName(),
+                NoRetryPolicyFactory.INSTANCE, null, Collections.emptyList());
         // Register mock runtime
         NCAppRuntimeContext nc1AppCtx =
                 (NCAppRuntimeContext) ExecutionTestUtil.integrationUtil.ncs[0].getApplicationContext();
@@ -103,26 +114,30 @@
         eventsListener.refreshStats(1000);
         requestedStats = eventsListener.getStats();
         Assert.assertTrue(requestedStats.contains("N/A"));
-        WaitForStateSubscriber startingSubscriber = new WaitForStateSubscriber(eventsListener, ActivityState.STARTING);
-        eventsListener.subscribe(startingSubscriber);
+        WaitForStateSubscriber startingSubscriber =
+                new WaitForStateSubscriber(eventsListener, Collections.singleton(ActivityState.STARTING));
         // Update stats of created/started job without joined partition
-        activeJobNotificationHandler.notifyJobCreation(jobId, jobSpec);
-        activeLifecycleListener.notifyJobStart(jobId);
+        TestUserActor user = new TestUserActor("Xikui", mdProvider, null);
+        Action start = user.startActivity(eventsListener);
         startingSubscriber.sync();
+        activeJobNotificationHandler.notifyJobCreation(jobId, jobSpec);
+        activeJobNotificationHandler.notifyJobStart(jobId);
         eventsListener.refreshStats(1000);
         requestedStats = eventsListener.getStats();
         Assert.assertTrue(requestedStats.contains("N/A"));
-
         // Fake partition message and notify eventListener
-        WaitForStateSubscriber startedSubscriber = new WaitForStateSubscriber(eventsListener, ActivityState.STARTED);
-        eventsListener.subscribe(startedSubscriber);
         ActivePartitionMessage partitionMessage = new ActivePartitionMessage(activeRuntimeId, jobId,
                 ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED, null);
         partitionMessage.handle(appCtx);
-        startedSubscriber.sync();
+        start.sync();
+        if (start.hasFailed()) {
+            throw start.getFailure();
+        }
         eventsListener.refreshStats(100000);
         requestedStats = eventsListener.getStats();
-        Assert.assertTrue(requestedStats.contains(EXPECTED_STATS));
+        if (!requestedStats.contains(EXPECTED_STATS)) {
+            throw new Exception("Expected stats to contain " + EXPECTED_STATS + " but found stats = " + requestedStats);
+        }
         ObjectMapper objectMapper = new ObjectMapper();
         try {
             objectMapper.readTree(requestedStats);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/Actor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/Actor.java
new file mode 100644
index 0000000..3f68651
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/Actor.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.test.active;
+
+import org.apache.asterix.active.SingleThreadEventProcessor;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+
+class Actor extends SingleThreadEventProcessor<Action> {
+
+    private final MetadataProvider actorMdProvider;
+
+    public Actor(String name, MetadataProvider metadataProvider) {
+        super(Actor.class.getSimpleName() + ":" + name);
+        this.actorMdProvider = metadataProvider;
+    }
+
+    @Override
+    protected void handle(Action action) throws Exception {
+        action.execute(actorMdProvider);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/DummyFeedEventsListener.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/DummyFeedEventsListener.java
new file mode 100644
index 0000000..961b731
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/DummyFeedEventsListener.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.test.active;
+
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.IActiveEntityEventSubscriber;
+import org.apache.asterix.active.IRetryPolicyFactory;
+import org.apache.asterix.app.active.FeedEventsListener;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class DummyFeedEventsListener extends FeedEventsListener {
+
+    public DummyFeedEventsListener(IStatementExecutor statementExecutor, ICcApplicationContext appCtx,
+            IHyracksClientConnection hcc, EntityId entityId, List<Dataset> datasets,
+            AlgebricksAbsolutePartitionConstraint locations, String runtimeName, IRetryPolicyFactory retryPolicyFactory,
+            Feed feed, List<FeedConnection> feedConnections) throws HyracksDataException {
+        super(statementExecutor, appCtx, hcc, entityId, datasets, locations, runtimeName, retryPolicyFactory, feed,
+                feedConnections);
+    }
+
+    @Override
+    protected void doStart(MetadataProvider metadataProvider) throws HyracksDataException, AlgebricksException {
+        IActiveEntityEventSubscriber eventSubscriber =
+                new WaitForStateSubscriber(this, Collections.singleton(ActivityState.RUNNING));
+        try {
+            eventSubscriber.sync();
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    protected Void doStop(MetadataProvider metadataProvider) throws HyracksDataException, AlgebricksException {
+        IActiveEntityEventSubscriber eventSubscriber =
+                new WaitForStateSubscriber(this, EnumSet.of(ActivityState.RUNNING, ActivityState.PERMANENTLY_FAILED));
+        try {
+            eventSubscriber.sync();
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+        return null;
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestClusterControllerActor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestClusterControllerActor.java
new file mode 100644
index 0000000..52d4225
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestClusterControllerActor.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.test.active;
+
+import java.util.List;
+
+import org.apache.asterix.active.ActiveEvent;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.JobStatus;
+import org.mockito.Mockito;
+
+public class TestClusterControllerActor extends Actor {
+
+    private final ActiveNotificationHandler handler;
+    private final List<Dataset> allDatasets;
+
+    public TestClusterControllerActor(String name, ActiveNotificationHandler handler, List<Dataset> allDatasets) {
+        super(name, null);
+        this.handler = handler;
+        this.allDatasets = allDatasets;
+    }
+
+    public Action startActiveJob(JobId jobId, EntityId entityId) {
+        Action startJob = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider actorMdProvider) throws Exception {
+                // succeed
+                JobSpecification jobSpecification = Mockito.mock(JobSpecification.class);
+                Mockito.when(jobSpecification.getProperty(ActiveNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME))
+                        .thenReturn(entityId);
+                handler.notifyJobCreation(jobId, jobSpecification);
+                handler.notifyJobStart(jobId);
+            }
+        };
+        add(startJob);
+        return startJob;
+    }
+
+    public Action activeEvent(ActiveEvent event) {
+        Action delivery = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider actorMdProvider) throws Exception {
+                handler.add(event);
+            }
+        };
+        add(delivery);
+        return delivery;
+    }
+
+    public Action jobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions) {
+        Action delivery = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider actorMdProvider) throws Exception {
+                handler.notifyJobFinish(jobId, jobStatus, exceptions);
+            }
+        };
+        add(delivery);
+        return delivery;
+    }
+
+    public List<Dataset> getAllDatasets() {
+        return allDatasets;
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java
new file mode 100644
index 0000000..d896995
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.test.active;
+
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Semaphore;
+
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.IRetryPolicyFactory;
+import org.apache.asterix.app.active.ActiveEntityEventsListener;
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.metadata.LockList;
+import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobIdFactory;
+import org.apache.hyracks.api.job.JobStatus;
+
+public class TestEventsListener extends ActiveEntityEventsListener {
+
+    public static enum Behavior {
+        SUCCEED,
+        RUNNING_JOB_FAIL,
+        FAIL_COMPILE,
+        FAIL_RUNTIME,
+        STEP_SUCCEED,
+        STEP_FAIL_COMPILE,
+        STEP_FAIL_RUNTIME
+    }
+
+    private final Semaphore step = new Semaphore(0);
+    private final TestClusterControllerActor clusterController;
+    private final TestNodeControllerActor[] nodeControllers;
+    private final JobIdFactory jobIdFactory;
+    private Behavior onStart = Behavior.FAIL_COMPILE;
+    private Behavior onStop = Behavior.FAIL_COMPILE;
+
+    public TestEventsListener(TestClusterControllerActor clusterController, TestNodeControllerActor[] nodeControllers,
+            JobIdFactory jobIdFactory, EntityId entityId, List<Dataset> datasets, IStatementExecutor statementExecutor,
+            ICcApplicationContext appCtx, IHyracksClientConnection hcc, AlgebricksAbsolutePartitionConstraint locations,
+            IRetryPolicyFactory retryPolicyFactory) throws HyracksDataException {
+        super(statementExecutor, appCtx, hcc, entityId, datasets, locations, TestEventsListener.class.getSimpleName(),
+                retryPolicyFactory);
+        this.clusterController = clusterController;
+        this.nodeControllers = nodeControllers;
+        this.jobIdFactory = jobIdFactory;
+    }
+
+    public void allowStep() {
+        step.release();
+    }
+
+    private void step(Behavior behavior) throws HyracksDataException {
+        if (behavior == Behavior.STEP_FAIL_COMPILE || behavior == Behavior.STEP_FAIL_RUNTIME
+                || behavior == Behavior.STEP_SUCCEED) {
+            takeStep();
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    private void failCompile(Behavior behavior) throws HyracksDataException {
+        if (behavior == Behavior.FAIL_COMPILE || behavior == Behavior.STEP_FAIL_COMPILE) {
+            throw new HyracksDataException("Compilation Failure");
+        }
+    }
+
+    private synchronized void takeStep() throws HyracksDataException {
+        try {
+            while (!step.tryAcquire()) {
+                notifyAll();
+                wait(10);
+            }
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    protected void doStart(MetadataProvider metadataProvider) throws HyracksDataException, AlgebricksException {
+        step(onStart);
+        failCompile(onStart);
+        JobId jobId = jobIdFactory.create();
+        Action startJob = clusterController.startActiveJob(jobId, entityId);
+        try {
+            startJob.sync();
+        } catch (InterruptedException e) {
+            throw HyracksDataException.create(e);
+        }
+        WaitForStateSubscriber subscriber = new WaitForStateSubscriber(this,
+                EnumSet.of(ActivityState.RUNNING, ActivityState.TEMPORARILY_FAILED, ActivityState.PERMANENTLY_FAILED));
+        if (onStart == Behavior.FAIL_RUNTIME || onStart == Behavior.STEP_FAIL_RUNTIME) {
+            clusterController.jobFinish(jobId, JobStatus.FAILURE,
+                    Collections.singletonList(new HyracksDataException("RuntimeFailure")));
+        } else {
+            for (int i = 0; i < nodeControllers.length; i++) {
+                TestNodeControllerActor nodeController = nodeControllers[0];
+                nodeController.registerRuntime(jobId, entityId, i);
+            }
+        }
+        try {
+            subscriber.sync();
+            if (subscriber.getFailure() != null) {
+                throw HyracksDataException.create(subscriber.getFailure());
+            }
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    protected Void doStop(MetadataProvider metadataProvider) throws HyracksDataException {
+        step(onStop);
+        failCompile(onStop);
+        try {
+            Set<ActivityState> waitFor;
+            if (state == ActivityState.STOPPING) {
+                waitFor = EnumSet.of(ActivityState.STOPPED, ActivityState.PERMANENTLY_FAILED);
+            } else if (state == ActivityState.SUSPENDING) {
+                waitFor = EnumSet.of(ActivityState.SUSPENDED, ActivityState.TEMPORARILY_FAILED);
+            } else {
+                throw new IllegalStateException("stop with what intention??");
+            }
+            WaitForStateSubscriber subscriber = new WaitForStateSubscriber(this, waitFor);
+            if (onStop == Behavior.RUNNING_JOB_FAIL) {
+                clusterController.jobFinish(jobId, JobStatus.FAILURE,
+                        Collections.singletonList(new HyracksDataException("RuntimeFailure")));
+            } else {
+                for (int i = 0; i < nodeControllers.length; i++) {
+                    TestNodeControllerActor nodeController = nodeControllers[0];
+                    nodeController.deRegisterRuntime(jobId, entityId, i).sync();
+                }
+                clusterController.jobFinish(jobId, JobStatus.TERMINATED, Collections.emptyList());
+            }
+            subscriber.sync();
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+        return null;
+    }
+
+    public void onStart(Behavior behavior) {
+        this.onStart = behavior;
+    }
+
+    public void onStop(Behavior behavior) {
+        this.onStop = behavior;
+    }
+
+    @Override
+    protected void setRunning(MetadataProvider metadataProvider, boolean running)
+            throws HyracksDataException, AlgebricksException {
+        try {
+            IMetadataLockManager lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
+            LockList locks = metadataProvider.getLocks();
+            lockManager.acquireDataverseReadLock(locks, entityId.getDataverse());
+            lockManager.acquireActiveEntityWriteLock(locks, entityId.getDataverse() + '.' + entityId.getEntityName());
+            // persist entity
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    protected Void doSuspend(MetadataProvider metadataProvider) throws HyracksDataException, AlgebricksException {
+        return doStop(metadataProvider);
+    }
+
+    @Override
+    protected void doResume(MetadataProvider metadataProvider) throws HyracksDataException, AlgebricksException {
+        doStart(metadataProvider);
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestNodeControllerActor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestNodeControllerActor.java
new file mode 100644
index 0000000..e7e21b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestNodeControllerActor.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.test.active;
+
+import org.apache.asterix.active.ActiveEvent;
+import org.apache.asterix.active.ActiveEvent.Kind;
+import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.message.ActivePartitionMessage;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.hyracks.api.job.JobId;
+
+public class TestNodeControllerActor extends Actor {
+
+    private final String id;
+    private final TestClusterControllerActor clusterController;
+
+    public TestNodeControllerActor(String name, TestClusterControllerActor clusterController) {
+        super("NC: " + name, null);
+        this.id = name;
+        this.clusterController = clusterController;
+    }
+
+    public Action registerRuntime(JobId jobId, EntityId entityId, int partition) {
+        Action registration = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider actorMdProvider) throws Exception {
+                ActiveEvent event = new ActiveEvent(jobId, Kind.PARTITION_EVENT, entityId,
+                        new ActivePartitionMessage(new ActiveRuntimeId(entityId, id, partition), jobId,
+                                ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED, null));
+                clusterController.activeEvent(event);
+            }
+        };
+        add(registration);
+        return registration;
+    }
+
+    public Action deRegisterRuntime(JobId jobId, EntityId entityId, int partition) {
+        Action registration = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider actorMdProvider) throws Exception {
+                ActiveEvent event = new ActiveEvent(jobId, Kind.PARTITION_EVENT, entityId,
+                        new ActivePartitionMessage(new ActiveRuntimeId(entityId, id, partition), jobId,
+                                ActivePartitionMessage.ACTIVE_RUNTIME_DEREGISTERED, null));
+                clusterController.activeEvent(event);
+            }
+        };
+        add(registration);
+        return registration;
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
new file mode 100644
index 0000000..5f715af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.test.active;
+
+import java.util.List;
+import java.util.concurrent.Semaphore;
+
+import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.metadata.api.IActiveEntityController;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.MetadataLockUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class TestUserActor extends Actor {
+
+    private TestClusterControllerActor clusterController;
+    private IMetadataLockManager lockManager;
+
+    public TestUserActor(String name, MetadataProvider metadataProvider, TestClusterControllerActor clusterController) {
+        super(name, metadataProvider);
+        this.clusterController = clusterController;
+        this.lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
+    }
+
+    public Action startActivity(IActiveEntityController actionListener) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                String dataverseName = actionListener.getEntityId().getDataverse();
+                String entityName = actionListener.getEntityId().getEntityName();
+                try {
+                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
+                    List<Dataset> datasets = actionListener.getDatasets();
+                    for (Dataset dataset : datasets) {
+                        MetadataLockUtil.modifyDatasetBegin(lockManager, mdProvider.getLocks(), dataverseName,
+                                DatasetUtil.getFullyQualifiedName(dataset));
+                    }
+                    actionListener.start(mdProvider);
+                } finally {
+                    mdProvider.getLocks().reset();
+                }
+            }
+        };
+        add(action);
+        return action;
+    }
+
+    public Action stopActivity(IActiveEntityController actionListener) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                String dataverseName = actionListener.getEntityId().getDataverse();
+                String entityName = actionListener.getEntityId().getEntityName();
+                try {
+                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
+                    List<Dataset> datasets = actionListener.getDatasets();
+                    for (Dataset dataset : datasets) {
+                        MetadataLockUtil.modifyDatasetBegin(lockManager, mdProvider.getLocks(), dataverseName,
+                                DatasetUtil.getFullyQualifiedName(dataset));
+                    }
+                    actionListener.stop(mdProvider);
+                } finally {
+                    mdProvider.getLocks().reset();
+                }
+            }
+        };
+        add(action);
+        return action;
+    }
+
+    public Action suspendActivity(IActiveEntityController actionListener) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                String dataverseName = actionListener.getEntityId().getDataverse();
+                String entityName = actionListener.getEntityId().getEntityName();
+                List<Dataset> datasets = actionListener.getDatasets();
+                try {
+                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
+                    for (Dataset dataset : datasets) {
+                        lockManager.acquireDatasetExclusiveModificationLock(mdProvider.getLocks(),
+                                DatasetUtil.getFullyQualifiedName(dataset));
+                    }
+                    actionListener.suspend(mdProvider);
+                } catch (Exception e) {
+                    // only release in case of failure
+                    mdProvider.getLocks().reset();
+                    throw e;
+                }
+            }
+        };
+        add(action);
+        return action;
+    }
+
+    public Action resumeActivity(IActiveEntityController actionListener) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                String dataverseName = actionListener.getEntityId().getDataverse();
+                String entityName = actionListener.getEntityId().getEntityName();
+                try {
+                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
+                    List<Dataset> datasets = actionListener.getDatasets();
+                    for (Dataset dataset : datasets) {
+                        lockManager.upgradeDatasetLockToWrite(mdProvider.getLocks(),
+                                DatasetUtil.getFullyQualifiedName(dataset));
+                        lockManager.downgradeDatasetLockToExclusiveModify(mdProvider.getLocks(),
+                                DatasetUtil.getFullyQualifiedName(dataset));
+                    }
+                    actionListener.resume(mdProvider);
+                } finally {
+                    mdProvider.getLocks().reset();
+                }
+            }
+        };
+        add(action);
+        return action;
+    }
+
+    public Action addDataset(Dataset dataset, IActiveEntityController actionListener) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                String entityDataverseName = actionListener.getEntityId().getDataverse();
+                String entityName = actionListener.getEntityId().getEntityName();
+                try {
+                    lockManager.acquireActiveEntityReadLock(mdProvider.getLocks(),
+                            entityDataverseName + '.' + entityName);
+                    lockManager.acquireDatasetWriteLock(mdProvider.getLocks(),
+                            DatasetUtil.getFullyQualifiedName(dataset));
+                    List<Dataset> datasets = clusterController.getAllDatasets();
+                    if (datasets.contains(dataset)) {
+                        throw new HyracksDataException("Dataset " + dataset + " already exists");
+                    }
+                    actionListener.add(dataset);
+                    datasets.add(dataset);
+                } finally {
+                    mdProvider.getLocks().reset();
+                }
+            }
+        };
+        add(action);
+        return action;
+    }
+
+    public Action dropDataset(Dataset dataset, IActiveEntityController actionListener) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                String entityDataverseName = actionListener.getEntityId().getDataverse();
+                String entityName = actionListener.getEntityId().getEntityName();
+                try {
+                    lockManager.acquireActiveEntityReadLock(mdProvider.getLocks(),
+                            entityDataverseName + '.' + entityName); // we have to first read lock all active entities before deleting a dataset
+                    lockManager.acquireDatasetWriteLock(mdProvider.getLocks(),
+                            DatasetUtil.getFullyQualifiedName(dataset));
+                    List<Dataset> datasets = clusterController.getAllDatasets();
+                    if (!datasets.contains(dataset)) {
+                        throw new HyracksDataException("Dataset " + dataset + " does not exist");
+                    }
+                    actionListener.remove(dataset);
+                    datasets.remove(dataset);
+                } finally {
+                    mdProvider.getLocks().reset();
+                }
+            }
+        };
+        add(action);
+        return action;
+    }
+
+    public Action addIndex(Dataset dataset, IActiveEntityController actionListener) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                String dataverseName = dataset.getDataverseName();
+                String datasetFullyQualifiedName = dataverseName + '.' + dataset.getDatasetName();
+                String indexFullyQualifiedName = datasetFullyQualifiedName + ".index";
+                try {
+                    MetadataLockUtil.createIndexBegin(lockManager, mdProvider.getLocks(), dataverseName,
+                            datasetFullyQualifiedName);
+                    if (actionListener.isActive()) {
+                        throw new RuntimeDataException(ErrorCode.CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY,
+                                indexFullyQualifiedName, actionListener.getEntityId(), actionListener.getState());
+                    }
+                } finally {
+                    mdProvider.getLocks().reset();
+                }
+            }
+        };
+        add(action);
+        return action;
+    }
+
+    public Action dropIndex(Dataset dataset, IActiveEntityController actionListener) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                String dataverseName = dataset.getDataverseName();
+                String datasetFullyQualifiedName = dataverseName + '.' + dataset.getDatasetName();
+                try {
+                    MetadataLockUtil.dropIndexBegin(lockManager, mdProvider.getLocks(), dataverseName,
+                            datasetFullyQualifiedName);
+                    if (actionListener.isActive()) {
+                        throw new RuntimeDataException(
+                                ErrorCode.CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY,
+                                datasetFullyQualifiedName + ".index", actionListener.getEntityId(),
+                                actionListener.getState());
+                    }
+                } finally {
+                    mdProvider.getLocks().reset();
+                }
+            }
+        };
+        add(action);
+        return action;
+    }
+
+    public Action query(Dataset dataset, Semaphore semaphore) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                String dataverseName = dataset.getDataverseName();
+                String datasetFullyQualifiedName = dataverseName + '.' + dataset.getDatasetName();
+                try {
+                    lockManager.acquireDataverseReadLock(mdProvider.getLocks(), dataverseName);
+                    lockManager.acquireDatasetReadLock(mdProvider.getLocks(), datasetFullyQualifiedName);
+                    if (!semaphore.tryAcquire()) {
+                        semaphore.acquire();
+                    }
+                } finally {
+                    mdProvider.getLocks().reset();
+                }
+            }
+        };
+        add(action);
+        return action;
+    }
+
+    public Action suspendAllActivities(ActiveNotificationHandler handler) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                handler.suspend(mdProvider);
+            }
+        };
+        add(action);
+        return action;
+    }
+
+    public Action resumeAllActivities(ActiveNotificationHandler handler) {
+        Action action = new Action() {
+            @Override
+            protected void doExecute(MetadataProvider mdProvider) throws Exception {
+                try {
+                    handler.resume(mdProvider);
+                } finally {
+                    mdProvider.getLocks().reset();
+                }
+            }
+        };
+        add(action);
+        return action;
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 8442162..10b528f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -472,8 +472,10 @@
             } catch (Exception e) {
                 // whoops, not JSON (e.g. 404) - just include the body
                 GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, errorBody);
-                throw new Exception("HTTP operation failed:" + "\nSTATUS LINE: " + httpResponse.getStatusLine()
-                        + "\nERROR_BODY: " + errorBody, e);
+                Exception failure = new Exception("HTTP operation failed:" + "\nSTATUS LINE: "
+                        + httpResponse.getStatusLine() + "\nERROR_BODY: " + errorBody);
+                failure.addSuppressed(e);
+                throw failure;
             }
             throw new ParsedException("HTTP operation failed: " + errors[0] + "\nSTATUS LINE: "
                     + httpResponse.getStatusLine() + "\nSUMMARY: " + errors[2].split("\n")[0], errors[2]);
@@ -1173,7 +1175,8 @@
                 break;
             } catch (TimeoutException e) {
                 if (responsesReceived == 0) {
-                    throw new Exception("Poll limit (" + timeoutSecs + "s) exceeded without obtaining *any* result from server");
+                    throw new Exception(
+                            "Poll limit (" + timeoutSecs + "s) exceeded without obtaining *any* result from server");
                 } else {
                     throw new Exception("Poll limit (" + timeoutSecs + "s) exceeded without obtaining expected result");
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index a5e6fbb..e99686d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -237,13 +237,13 @@
     <test-case FilePath="feeds">
       <compilation-unit name="start-started-feed">
         <output-dir compare="Text">start-started-feed</output-dir>
-        <expected-error>Feed TweetFeed is started already.</expected-error>
+        <expected-error>experiments.TweetFeed(Feed) is already started</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
       <compilation-unit name="stop-stopped-feed">
         <output-dir compare="Text">stop-stopped-feed</output-dir>
-        <expected-error>Feed TweetFeed is not started.</expected-error>
+        <expected-error>new_experiments.TweetFeed(Feed) cannot be stopped because its state is STOPPED</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -6421,7 +6421,7 @@
     <test-case FilePath="cross-dataverse">
       <compilation-unit name="drop-dataverse">
         <output-dir compare="Text">drop-dataverse</output-dir>
-        <expected-error>org.apache.asterix.metadata.MetadataException: Cannot drop dataverse. Type a.a used by dataset b.b1</expected-error>
+        <expected-error>Cannot drop dataverse. Type a.a used by dataset b.b1</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="cross-dataverse">
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterEventsSubscriber.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterEventsSubscriber.java
index c3cf86b..61ded27 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterEventsSubscriber.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterEventsSubscriber.java
@@ -55,7 +55,7 @@
      * @param previousState
      * @param newState
      */
-    default void notifyStateChange(ClusterState previousState, ClusterState newState) {
+    default void notifyStateChange(ClusterState newState) {
         // default is no-op
     }
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
new file mode 100644
index 0000000..4408d84
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.common.api;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.metadata.LockList;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+public interface IMetadataLockManager {
+
+    /**
+     * Acquire read lock on the dataverse
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param dataverseName
+     *            the dataverse name
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireDataverseReadLock(LockList locks, String dataverseName) throws AsterixException;
+
+    /**
+     * Acquire write lock on the dataverse
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param dataverseName
+     *            the dataverse name
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireDataverseWriteLock(LockList locks, String dataverseName) throws AsterixException;
+
+    /**
+     * Acquire read lock on the dataset (for queries)
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param datasetFullyQualifiedName
+     *            the fully qualified name of the dataset
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireDatasetReadLock(LockList locks, String datasetFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire write lock on the dataset (for dataset create, dataset drop, and index drop)
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param datasetFullyQualifiedName
+     *            the fully qualified name of the dataset
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireDatasetWriteLock(LockList locks, String datasetFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire modify lock on the dataset (for inserts, upserts, deletes) Mutually exclusive with create index lock
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param datasetFullyQualifiedName
+     *            the fully qualified name of the dataset
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireDatasetModifyLock(LockList locks, String datasetFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire create index lock on the dataset (for index creation) Mutually exclusive with modify lock
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param datasetFullyQualifiedName
+     *            the fully qualified name of the dataset
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireDatasetCreateIndexLock(LockList locks, String datasetFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire exclusive modify lock on the dataset. only a single thread can acquire this lock and it is mutually
+     * exclusive with modify locks and index build lock
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param datasetFullyQualifiedName
+     *            the fully qualified name of the dataset
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireDatasetExclusiveModificationLock(LockList locks, String datasetFullyQualifiedName)
+            throws AsterixException;
+
+    /**
+     * Acquire read lock on the function
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param functionFullyQualifiedName
+     *            the fully qualified name of the function
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireFunctionReadLock(LockList locks, String functionFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire write lock on the function
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param functionFullyQualifiedName
+     *            the fully qualified name of the function
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireFunctionWriteLock(LockList locks, String functionFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire read lock on the node group
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param nodeGroupName
+     *            the name of the node group
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireNodeGroupReadLock(LockList locks, String nodeGroupName) throws AsterixException;
+
+    /**
+     * Acquire write lock on the node group
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param nodeGroupName
+     *            the name of the node group
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireNodeGroupWriteLock(LockList locks, String nodeGroupName) throws AsterixException;
+
+    /**
+     * Acquire read lock on the active entity
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param entityFullyQualifiedName
+     *            the fully qualified name of the active entity
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireActiveEntityReadLock(LockList locks, String entityFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire write lock on the active entity
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param entityFullyQualifiedName
+     *            the fully qualified name of the active entity
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireActiveEntityWriteLock(LockList locks, String entityFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire read lock on the feed policy
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param feedPolicyFullyQualifiedName
+     *            the fully qualified name of the feed policy
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireFeedPolicyWriteLock(LockList locks, String feedPolicyFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire write lock on the feed policy
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param feedPolicyFullyQualifiedName
+     *            the fully qualified name of the feed policy
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireFeedPolicyReadLock(LockList locks, String feedPolicyFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire read lock on the merge policy
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param mergePolicyFullyQualifiedName
+     *            the fully qualified name of the merge policy
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireMergePolicyReadLock(LockList locks, String mergePolicyFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire write lock on the merge policy
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param mergePolicyFullyQualifiedName
+     *            the fully qualified name of the merge policy
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireMergePolicyWriteLock(LockList locks, String mergePolicyFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire read lock on the data type
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param datatypeFullyQualifiedName
+     *            the fully qualified name of the data type
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireDataTypeReadLock(LockList locks, String datatypeFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire write lock on the data type
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param datatypeFullyQualifiedName
+     *            the fully qualified name of the data type
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireDataTypeWriteLock(LockList locks, String datatypeFullyQualifiedName) throws AsterixException;
+
+    /**
+     * Acquire read lock on the extension entity
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param extension
+     *            the extension key
+     * @param extensionEntityFullyQualifiedName
+     *            the fully qualified name of the extension entity
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireExtensionReadLock(LockList locks, String extension, String extensionEntityFullyQualifiedName)
+            throws AsterixException;
+
+    /**
+     * Acquire write lock on the extension entity
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param extension
+     *            the extension key
+     * @param extensionEntityFullyQualifiedName
+     *            the fully qualified name of the extension entity
+     * @throws AsterixException
+     *             if lock couldn't be acquired
+     */
+    void acquireExtensionWriteLock(LockList locks, String extension, String extensionEntityFullyQualifiedName)
+            throws AsterixException;
+
+    /**
+     * Upgrade a previously acquired exclusive modification lock on the dataset to a write lock
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param datasetFullyQualifiedName
+     *            the fully qualified name of the dataset
+     * @throws AlgebricksException
+     *             if lock couldn't be upgraded
+     */
+    void upgradeDatasetLockToWrite(LockList locks, String datasetFullyQualifiedName) throws AlgebricksException;
+
+    /**
+     * Downgrade an upgraded dataset write lock to an exclusive modification lock
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param datasetFullyQualifiedName
+     *            the fully qualified name of the dataset
+     * @throws AlgebricksException
+     *             if lock couldn't be downgraded
+     */
+    void downgradeDatasetLockToExclusiveModify(LockList locks, String datasetFullyQualifiedName)
+            throws AlgebricksException;
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java
index 6211af4..b4559c8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java
@@ -20,11 +20,22 @@
 
 import org.apache.asterix.common.api.IClusterEventsSubscriber;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IGlobalRecoveryManager extends IClusterEventsSubscriber {
 
     /**
-     * Starts the global recovery process if the cluster state changed to ACTIVE.
+     * Starts the global recovery process after the cluster state has changed to ACTIVE.
+     *
+     * @param appCtx
+     *            the application context
+     * @throws HyracksDataException
+     *             if the global recovery fails
      */
-    public void startGlobalRecovery(ICcApplicationContext appCtx);
+    void startGlobalRecovery(ICcApplicationContext appCtx) throws HyracksDataException;
+
+    /**
+     * @return true, if global recovery has been completed successfully
+     */
+    boolean isRecoveryCompleted();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
index 249bd56..3eff214 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
@@ -19,6 +19,8 @@
 package org.apache.asterix.common.dataflow;
 
 import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.replication.IFaultToleranceStrategy;
@@ -63,9 +65,9 @@
     IFaultToleranceStrategy getFaultToleranceStrategy();
 
     /**
-     * @return the active lifecycle listener at Cluster controller
+     * @return the active notification handler at Cluster controller
      */
-    IJobLifecycleListener getActiveLifecycleListener();
+    IJobLifecycleListener getActiveNotificationHandler();
 
     /**
      * @return a new instance of {@link IHyracksClientConnection}
@@ -90,4 +92,14 @@
      * @return the extension manager instance
      */
     Object getExtensionManager();
+
+    /**
+     * @return the metadata lock manager
+     */
+    IMetadataLockManager getMetadataLockManager();
+
+    /**
+     * @return the cluster state manager
+     */
+    IClusterStateManager getClusterStateManager();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
index 7cb7d8a..dc884f1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
@@ -25,6 +25,10 @@
 public class AsterixException extends AlgebricksException {
     private static final long serialVersionUID = 1L;
 
+    /**
+     * @deprecated Instead, use a constructor with error code
+     */
+    @Deprecated
     public AsterixException(String message) {
         super(message);
     }
@@ -34,10 +38,11 @@
 
     }
 
-    public static AsterixException create(int errorCode, Serializable... params) {
-        return new AsterixException(errorCode, params);
-    }
-
+    /**
+     * @deprecated When creating a constructor with cause,
+     *             create AlgebricksException using AlgebricksException.create(Throwable th);
+     */
+    @Deprecated
     public AsterixException(Throwable cause) {
         super(cause);
     }
@@ -47,7 +52,15 @@
         addSuppressed(cause);
     }
 
+    /**
+     * @deprecated Instead, use a constructor with error code
+     */
+    @Deprecated
     public AsterixException(String message, Throwable cause) {
         super(message, cause);
     }
+
+    public static AsterixException create(int errorCode, Serializable... params) {
+        return new AsterixException(errorCode, params);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
index b53eec4..693b1c0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
@@ -31,8 +31,7 @@
     }
 
     public CompilationException(int errorCode, Throwable cause, Serializable... params) {
-        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
-        addSuppressed(cause);
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, params);
     }
 
     /**
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 9d889ea..a00ed99 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -67,6 +67,7 @@
     public static final int POLYGON_INVALID_COORDINATE = 24;
     public static final int POLYGON_3_POINTS = 25;
     public static final int POLYGON_INVALID = 26;
+    public static final int OPERATION_NOT_SUPPORTED = 27;
 
     public static final int INSTANTIATION_ERROR = 100;
 
@@ -114,6 +115,12 @@
     public static final int INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL = 1041;
     public static final int INDEX_ILLEGAL_NON_ENFORCED_TYPED = 1042;
     public static final int INDEX_RTREE_MULTIPLE_FIELDS_NOT_ALLOWED = 1043;
+    public static final int REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE = 1044;
+    public static final int ILLEGAL_LOCK_UPGRADE_OPERATION = 1045;
+    public static final int ILLEGAL_LOCK_DOWNGRADE_OPERATION = 1046;
+    public static final int UPGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED = 1047;
+    public static final int DOWNGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED = 1048;
+    public static final int LOCK_WAS_ACQUIRED_DIFFERENT_OPERATION = 1049;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
@@ -200,6 +207,26 @@
     public static final int PROVIDER_STREAM_RECORD_READER_WRONG_CONFIGURATION = 3086;
     public static final int FEED_CONNECT_FEED_APPLIED_INVALID_FUNCTION = 3087;
     public static final int ACTIVE_MANAGER_INVALID_RUNTIME = 3088;
+    public static final int ACTIVE_ENTITY_ALREADY_STARTED = 3089;
+    public static final int ACTIVE_ENTITY_CANNOT_BE_STOPPED = 3090;
+    public static final int CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY = 3091;
+    public static final int CANNOT_REMOVE_DATASET_FROM_ACTIVE_ENTITY = 3092;
+    public static final int ACTIVE_ENTITY_IS_ALREADY_REGISTERED = 3093;
+    public static final int CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY = 3094;
+    public static final int CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY = 3095;
+    public static final int ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED = 3096;
+    public static final int ACTIVE_ENTITY_LISTENER_IS_NOT_REGISTERED = 3097;
+    public static final int CANNOT_DERIGESTER_ACTIVE_ENTITY_LISTENER = 3098;
+    public static final int DOUBLE_INITIALIZATION_OF_ACTIVE_NOTIFICATION_HANDLER = 3099;
+    public static final int FAILED_TO_SHUTDOWN_EVENT_PROCESSOR = 3100;
+    public static final int DOUBLE_RECOVERY_ATTEMPTS = 3101;
+    public static final int UNREPORTED_TASK_FAILURE_EXCEPTION = 3102;
+    public static final int ACTIVE_ENTITY_ALREADY_SUSPENDED = 3103;
+    public static final int ACTIVE_ENTITY_CANNOT_RESUME_FROM_STATE = 3104;
+    public static final int ACTIVE_RUNTIME_IS_ALREADY_REGISTERED = 3105;
+    public static final int ACTIVE_RUNTIME_IS_NOT_REGISTERED = 3106;
+    public static final int ACTIVE_EVENT_HANDLER_ALREADY_SUSPENDED = 3107;
+    public static final int FEED_STOPPED_WHILE_WAITING_FOR_A_NEW_RECORD = 3108;
 
     // Lifecycle management errors
     public static final int DUPLICATE_PARTITION_ID = 4000;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
new file mode 100644
index 0000000..f04d19d
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.common.exceptions;
+
+import java.io.Serializable;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class MetadataException extends CompilationException {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER = Logger.getLogger(MetadataException.class.getName());
+
+    @Deprecated
+    /**
+     * @Deprecated Instead, use a constructor with error code
+     * @param message
+     */
+    public MetadataException(String message) {
+        super(message);
+    }
+
+    @Deprecated
+    /**
+     * @Deprecated When creating a constructor with cause,
+     *             create AlgebricksException using AlgebricksException.create(Throwable th);
+     * @param cause
+     */
+    public MetadataException(Throwable cause) {
+        super(cause);
+    }
+
+    @Deprecated
+    /**
+     * @Deprecated When creating a constructor with cause,
+     *             create AlgebricksException using AlgebricksException.create(Throwable th);
+     * @param cause
+     */
+    public MetadataException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public MetadataException(int errorCode, Serializable... params) {
+        super(errorCode, params);
+    }
+
+    public MetadataException(int errorCode, Throwable cause, Serializable... params) {
+        super(errorCode, cause, params);
+    }
+
+    public static MetadataException create(Throwable cause) {
+        if (cause instanceof MetadataException || cause == null) {
+            return (MetadataException) cause;
+        }
+        if (cause instanceof InterruptedException && !Thread.currentThread().isInterrupted()) {
+            LOGGER.log(Level.WARNING, "Wrapping an InterruptedException in " + MetadataException.class.getSimpleName()
+                    + " and current thread is not interrupted", cause);
+        }
+        return new MetadataException(cause);
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
index 0c099fb..85bfaa5 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
@@ -31,7 +31,6 @@
     }
 
     public RuntimeDataException(int errorCode, Throwable cause, Serializable... params) {
-        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
-        addSuppressed(cause);
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, params);
     }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
index a0e6e71..276e294 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
@@ -23,6 +23,16 @@
 public interface IDataset {
 
     /**
+     * @return the dataverse name
+     */
+    String getDataverseName();
+
+    /**
+     * @return the dataset name
+     */
+    String getDatasetName();
+
+    /**
      * @return the list of primary keys for the dataset
      */
     List<List<String>> getPrimaryKeys();
@@ -31,5 +41,4 @@
      * @return the bloom filter fields indexes for the primary index of the dataset
      */
     int[] getPrimaryBloomFilterFields();
-
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLock.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLock.java
new file mode 100644
index 0000000..ba17b0c
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLock.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.common.metadata;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.MetadataException;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+/**
+ * A Metadata lock local to compilation node
+ */
+public interface IMetadataLock {
+
+    enum Mode {
+        READ,
+        MODIFY,
+        INDEX_BUILD,
+        EXCLUSIVE_MODIFY,
+        UPGRADED_WRITE,
+        WRITE;
+
+        public boolean contains(Mode mode) {
+            if (mode == this) {
+                return true;
+            }
+            if (this == Mode.WRITE) {
+                return true;
+            }
+            return mode == Mode.READ;
+        }
+    }
+
+    /**
+     * Acquire a lock
+     *
+     * @param mode
+     *            lock mode
+     */
+    void lock(IMetadataLock.Mode mode);
+
+    /**
+     * Release a lock
+     *
+     * @param mode
+     *            lock mode
+     */
+    void unlock(IMetadataLock.Mode mode);
+
+    /**
+     * Get the lock's key
+     *
+     * @return the key identiying the lock
+     */
+    String getKey();
+
+    /**
+     * upgrade the lock
+     *
+     * @param from
+     * @param to
+     * @throws AlgebricksException
+     */
+    default void upgrade(Mode from, Mode to) throws AlgebricksException {
+        throw new MetadataException(ErrorCode.ILLEGAL_LOCK_UPGRADE_OPERATION, from, to);
+    }
+
+    /**
+     * downgrade the lock
+     *
+     * @param from
+     * @param to
+     * @throws AlgebricksException
+     */
+    default void downgrade(Mode from, Mode to) throws AlgebricksException {
+        throw new MetadataException(ErrorCode.ILLEGAL_LOCK_DOWNGRADE_OPERATION, from, to);
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/LockList.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/LockList.java
new file mode 100644
index 0000000..6f2bc39
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/LockList.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.common.metadata;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.IMetadataLock.Mode;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+/**
+ * The LockList is used for two phase locking.
+ */
+public class LockList {
+    private final List<MutablePair<IMetadataLock, IMetadataLock.Mode>> locks = new ArrayList<>();
+    private final HashMap<String, Integer> indexes = new HashMap<>();
+    private boolean lockPhase = true;
+
+    /**
+     * Acquires a lock.
+     *
+     * @param mode
+     *            the lock mode.
+     * @param lock
+     *            the lock object.
+     */
+    public void add(IMetadataLock.Mode mode, IMetadataLock lock) throws AsterixException {
+        if (isContained(mode, lock)) {
+            return;
+        }
+        lock.lock(mode);
+        indexes.put(lock.getKey(), locks.size());
+        locks.add(MutablePair.of(lock, mode));
+    }
+
+    private boolean isContained(Mode mode, IMetadataLock lock) throws AsterixException {
+        if (!lockPhase) {
+            throw new AsterixException(ErrorCode.COMPILATION_TWO_PHASE_LOCKING_VIOLATION);
+        }
+        Integer index = indexes.get(lock.getKey());
+        if (index != null) {
+            Mode acquired = locks.get(index).right;
+            if (!acquired.contains(mode)) {
+                throw new AsterixException(ErrorCode.LOCK_WAS_ACQUIRED_DIFFERENT_OPERATION, mode, acquired);
+            }
+            return true;
+        }
+        return false;
+    }
+
+    public void upgrade(Mode to, IMetadataLock lock) throws AlgebricksException {
+        if (!lockPhase) {
+            throw new AsterixException(ErrorCode.COMPILATION_TWO_PHASE_LOCKING_VIOLATION);
+        }
+        Integer index = indexes.get(lock.getKey());
+        if (index == null) {
+            throw new AsterixException(ErrorCode.UPGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED);
+        }
+        MutablePair<IMetadataLock, Mode> pair = locks.get(index);
+        Mode from = pair.getRight();
+        if (from == to) {
+            return;
+        }
+        lock.upgrade(from, to);
+        pair.setRight(to);
+    }
+
+    public void downgrade(Mode mode, IMetadataLock lock) throws AlgebricksException {
+        if (!lockPhase) {
+            throw new AsterixException(ErrorCode.COMPILATION_TWO_PHASE_LOCKING_VIOLATION);
+        }
+        Integer index = indexes.get(lock.getKey());
+        if (index == null) {
+            throw new AsterixException(ErrorCode.DOWNGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED);
+        }
+        MutablePair<IMetadataLock, Mode> pair = locks.get(index);
+        Mode acquired = pair.getRight();
+        lock.downgrade(acquired, mode);
+        pair.setRight(mode);
+    }
+
+    /**
+     * Once unlock() is called, no caller can call add(IMetadataLock.Mode mode, IMetadataLock lock),
+     * except that reset() is called.
+     */
+    public void unlock() {
+        for (int i = locks.size() - 1; i >= 0; i--) {
+            MutablePair<IMetadataLock, Mode> pair = locks.get(i);
+            pair.getLeft().unlock(pair.getRight());
+        }
+        locks.clear();
+        indexes.clear();
+        lockPhase = false;
+    }
+
+    /**
+     * Clears the state and starts another pass of two phase locking again.
+     */
+    public void reset() {
+        unlock();
+        lockPhase = true;
+    }
+
+    @Override
+    public String toString() {
+        return "{\"phase\" : \"" + (lockPhase ? "lock" : "unlock") + "\", \"locks\" : " + locks + "}";
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 9d4db56..9ed6aa5 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -49,7 +49,7 @@
 12 = Invalid implicit scalar to collection coercion in %1$s
 14 = Property %1$s not set
 15 = Storage metadata directory of %1$s in %2$s already exists
-16 = Storage metadata directory of %1$s in %2$s couldn't be created
+16 = Storage metadata directory of %1$s in %2$s could not be created
 17 = Unknown external file pending operation %1$s
 18 = Cannot convert the %1$s type to the %2$s type.
 19 = Cannot convert integer types. The source type should be one of %1$s.
@@ -60,6 +60,7 @@
 24 = Invalid coordinate
 25 = Polygon must have at least 3 points
 26 = %1$s can not be an instance of polygon
+27 = Operation not supported
 
 100 = Unable to instantiate class %1$s
 
@@ -83,7 +84,7 @@
 1024 = Identifier %1$s is not found in AQL+ meta-scope
 1025 = There is no such join type in AQL+
 1026 = The given function expression %1$s cannot utilize index
-1027 = Dataset of type %1$s doesn't have a primary index
+1027 = Dataset of type %1$s does not have a primary index
 1028 = Query parameter %1$s is not supported
 1029 = No metadata exists for dataset %1$s
 1030 = The subtree does not have any data source
@@ -100,6 +101,12 @@
 1041 = Cannot create enforced index on \"%1$s\" field with non-optional type
 1042 = Cannot create non-enforced typed index of this kind: %1$s
 1043 = Cannot use %1$s fields as key for the R-tree index. There can be only one field as a key for the R-tree index.
+1044 = Communication-related exception occurred during the execution of a remote method call
+1045 = Illegal attempt to upgrade metadata lock from %1$s to %2$s
+1046 = Illegal attempt to downgrade metadata lock from %1$s to %2$s
+1047 = Metadata lock cannot be upgraded! because it was not acquired before
+1048 = Metadata lock cannot be downgraded! because it was not acquired before
+1049 = Metadata lock cannot be acquired for %1$s since it is already acquired for %2$s
 
 # Feed Errors
 3001 = Illegal state.
@@ -111,7 +118,7 @@
 3007 = Twitter4J library not found!
 3008 = Unable to ingest data
 3009 = Exception in get record type %1$s for feed
-3010 = Doesn't support Hive data with list of non-primitive types
+3010 = Does not support Hive data with list of non-primitive types
 3011 = Cannot get hive type for field of type %1$s
 3012 = Failed to get columns of record
 3013 = Cannot deserialize Hive records with no closed columns
@@ -189,6 +196,26 @@
 3086 = Cannot find record reader %1$s with specified configuration
 3087 = Cannot find function %1$s
 3088 = %1$s is not a valid runtime Id
+3089 = %1$s is already started and has state %2$s
+3090 = %1$s cannot be stopped because its state is %2$s
+3091 = Cannot add dataset to %1$s because its state is %2$s
+3092 = Cannot remove dataset from %1$s because its state is %2$s
+3093 = %1$s is already registered
+3094 = Cannot create index on dataset %1$s because it is connected to %2$s with state %3$s
+3095 = Cannot drop index of dataset %1$s because it is connected to %2$s with state %3$s
+3096 = Active Notification Handler is suspended
+3097 = Active Entity %1$s has not been registered
+3098 = Cannot deregister %1$s because it is active
+3099 = Attempt to initialize an initialized Active Notification Handler
+3100 = Failed to shutdown event processor for %1$s
+3101 = Recovery request while recovery is currently ongoing
+3102 = Unreported exception causing task failure
+3103 = %1$s is already suspended and has state %2$s
+3104 = %1$s cannot be resumed from state %2$s
+3105 = %1$s is already registered
+3106 = %1$s is not registered
+3107 = Active Notification Handler is already suspended
+3108 = Feed stopped while waiting for a new record
 
 # Lifecycle management errors
 4000 = Partition id %1$d for node %2$s already in use by node %3$s
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataSourceAdapter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataSourceAdapter.java
index 48df79b..e62672d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataSourceAdapter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataSourceAdapter.java
@@ -29,6 +29,7 @@
  * to be implemented by each adapter irrespective of the the kind of
  * adapter(pull or push).
  */
+@FunctionalInterface
 public interface IDataSourceAdapter extends Serializable {
 
     public enum AdapterType {
@@ -38,6 +39,7 @@
 
     /**
      * Triggers the adapter to begin ingesting data from the external source.
+     *
      * @param partition
      *            The adapter could be running with a degree of parallelism.
      *            partition corresponds to the i'th parallel instance.
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractFeedDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractFeedDataFlowController.java
index bcf5e25..c87fe2d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractFeedDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractFeedDataFlowController.java
@@ -63,4 +63,8 @@
     public abstract boolean handleException(Throwable th) throws HyracksDataException;
 
     public abstract String getStats();
+
+    public void fail() throws HyracksDataException {
+        tupleForwarder.fail();
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
index 1e62159..5b9b96f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
@@ -21,6 +21,8 @@
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.api.IRawRecord;
 import org.apache.asterix.external.api.IRecordDataParser;
 import org.apache.asterix.external.api.IRecordReader;
@@ -30,6 +32,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
 public class FeedRecordDataFlowController<T> extends AbstractFeedDataFlowController {
@@ -57,8 +60,8 @@
         try {
             failed = false;
             tupleForwarder.initialize(ctx, writer);
-            while (recordReader.hasNext()) {
-                IRawRecord<? extends T> record = recordReader.next();
+            while (hasNext()) {
+                IRawRecord<? extends T> record = next();
                 if (record == null) {
                     flush();
                     Thread.sleep(INTERVAL); // NOSONAR: No one notifies the sleeping thread
@@ -70,25 +73,46 @@
                     failedRecordsCount++;
                 }
             }
-        } catch (InterruptedException e) {
-            //TODO: Find out what could cause an interrupted exception beside termination of a job/feed
-            LOGGER.warn("Feed has been interrupted. Closing the feed", e);
-            failed = true;
-            try {
-                finish();
-            } catch (HyracksDataException hde) {
-                e.addSuppressed(hde);
+        } catch (HyracksDataException e) {
+            LOGGER.log(Level.WARN, e);
+            //if interrupted while waiting for a new record, then it is safe to not fail forward
+            if (e.getComponent() == ErrorCode.ASTERIX
+                    && e.getErrorCode() == ErrorCode.FEED_STOPPED_WHILE_WAITING_FOR_A_NEW_RECORD) {
+                // Do nothing
+            } else {
+                failed = true;
+                throw e;
             }
-            throw e;
         } catch (Exception e) {
             failed = true;
-            tupleForwarder.flush();
             LOGGER.warn("Failure while operating a feed source", e);
             throw HyracksDataException.create(e);
         }
         finish();
     }
 
+    private IRawRecord<? extends T> next() throws HyracksDataException {
+        try {
+            return recordReader.next();
+        } catch (InterruptedException e) { // NOSONAR Gracefully handling interrupt to push records in the pipeline
+            throw new RuntimeDataException(ErrorCode.FEED_STOPPED_WHILE_WAITING_FOR_A_NEW_RECORD, e);
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private boolean hasNext() throws HyracksDataException {
+        boolean hasNext;
+        try {
+            hasNext = recordReader.hasNext();
+        } catch (InterruptedException e) { // NOSONAR Gracefully handling interrupt to push records in the pipeline
+            throw new RuntimeDataException(ErrorCode.FEED_STOPPED_WHILE_WAITING_FOR_A_NEW_RECORD, e);
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+        return hasNext;
+    }
+
     private void finish() throws HyracksDataException {
         HyracksDataException hde = null;
         try {
@@ -194,6 +218,7 @@
         return dataParser;
     }
 
+    @Override
     public String getStats() {
         return "{\"incoming-records-count\": " + incomingRecordsCount + ", \"failed-at-parser-records-count\": "
                 + failedRecordsCount + "}";
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
index 4177ea6..3a8130b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
@@ -40,6 +40,7 @@
     private IFrameWriter writer;
     private boolean paused = false;
     private boolean initialized;
+    private boolean failed;
 
     public FeedTupleForwarder(FeedLogManager feedLogManager) {
         this.feedLogManager = feedLogManager;
@@ -67,7 +68,8 @@
                     try {
                         wait();
                     } catch (InterruptedException e) {
-                        throw new HyracksDataException(e);
+                        Thread.currentThread().interrupt();
+                        throw HyracksDataException.create(e);
                     }
                 }
             }
@@ -88,7 +90,7 @@
     public void close() throws HyracksDataException {
         Throwable throwable = null;
         try {
-            if (appender.getTupleCount() > 0) {
+            if (!failed && appender.getTupleCount() > 0) {
                 FrameUtils.flushFrame(frame.getBuffer(), writer);
             }
         } catch (Throwable th) {
@@ -116,4 +118,9 @@
     public void flush() throws HyracksDataException {
         appender.flush(writer);
     }
+
+    public void fail() throws HyracksDataException {
+        failed = true;
+        writer.fail();
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedAdapter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedAdapter.java
index e6d81d3..9f32a25 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedAdapter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedAdapter.java
@@ -55,4 +55,8 @@
     public String getStats() {
         return controller.getStats();
     }
+
+    public void fail() throws HyracksDataException {
+        controller.fail();
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ActiveEntityEventsListener.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ActiveEntityEventsListener.java
deleted file mode 100644
index 3216bfe..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ActiveEntityEventsListener.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.asterix.external.feed.management;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.active.ActiveEvent;
-import org.apache.asterix.active.ActiveEvent.Kind;
-import org.apache.asterix.active.ActiveLifecycleListener;
-import org.apache.asterix.active.ActiveRuntimeId;
-import org.apache.asterix.active.ActivityState;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.active.IActiveEntityEventsListener;
-import org.apache.asterix.active.IActiveEventSubscriber;
-import org.apache.asterix.active.message.ActiveManagerMessage;
-import org.apache.asterix.active.message.ActivePartitionMessage;
-import org.apache.asterix.active.message.StatsRequestMessage;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.common.messaging.api.ICCMessageBroker;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.common.metadata.IDataset;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobStatus;
-
-public class ActiveEntityEventsListener implements IActiveEntityEventsListener {
-
-    private static final Logger LOGGER = Logger.getLogger(ActiveEntityEventsListener.class.getName());
-
-    enum RequestState {
-        INIT,
-        STARTED,
-        FINISHED
-    }
-
-    // members
-    protected volatile ActivityState state;
-    protected JobId jobId;
-    protected final List<IActiveEventSubscriber> subscribers = new ArrayList<>();
-    protected final ICcApplicationContext appCtx;
-    protected final EntityId entityId;
-    protected final List<IDataset> datasets;
-    protected final ActiveEvent statsUpdatedEvent;
-    protected long statsTimestamp;
-    protected String stats;
-    protected RequestState statsRequestState;
-    protected final String runtimeName;
-    protected final AlgebricksAbsolutePartitionConstraint locations;
-    protected int numRegistered;
-
-    public ActiveEntityEventsListener(ICcApplicationContext appCtx, EntityId entityId, List<IDataset> datasets,
-            AlgebricksAbsolutePartitionConstraint locations, String runtimeName) {
-        this.appCtx = appCtx;
-        this.entityId = entityId;
-        this.datasets = datasets;
-        this.state = ActivityState.STOPPED;
-        this.statsTimestamp = -1;
-        this.statsRequestState = RequestState.INIT;
-        this.statsUpdatedEvent = new ActiveEvent(null, Kind.STATS_UPDATED, entityId);
-        this.stats = "{\"Stats\":\"N/A\"}";
-        this.runtimeName = runtimeName;
-        this.locations = locations;
-        this.numRegistered = 0;
-    }
-
-    @Override
-    public synchronized void notify(ActiveEvent event) {
-        try {
-            LOGGER.finer("EventListener is notified.");
-            ActiveEvent.Kind eventKind = event.getEventKind();
-            switch (eventKind) {
-                case JOB_CREATED:
-                    state = ActivityState.CREATED;
-                    break;
-                case JOB_STARTED:
-                    start(event);
-                    break;
-                case JOB_FINISHED:
-                    finish();
-                    break;
-                case PARTITION_EVENT:
-                    handle((ActivePartitionMessage) event.getEventObject());
-                    break;
-                default:
-                    LOGGER.log(Level.WARNING, "Unhandled feed event notification: " + event);
-                    break;
-            }
-            notifySubscribers(event);
-        } catch (Exception e) {
-            LOGGER.log(Level.SEVERE, "Unhandled Exception", e);
-        }
-    }
-
-    protected synchronized void handle(ActivePartitionMessage message) {
-        if (message.getEvent() == ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED) {
-            numRegistered++;
-            if (numRegistered == locations.getLocations().length) {
-                state = ActivityState.STARTED;
-            }
-        }
-    }
-
-    private void finish() throws Exception {
-        IHyracksClientConnection hcc = appCtx.getHcc();
-        JobStatus status = hcc.getJobStatus(jobId);
-        state = status.equals(JobStatus.FAILURE) ? ActivityState.FAILED : ActivityState.STOPPED;
-        ActiveLifecycleListener activeLcListener = (ActiveLifecycleListener) appCtx.getActiveLifecycleListener();
-        activeLcListener.getNotificationHandler().removeListener(this);
-    }
-
-    private void start(ActiveEvent event) {
-        this.jobId = event.getJobId();
-        state = ActivityState.STARTING;
-    }
-
-    @Override
-    public synchronized void subscribe(IActiveEventSubscriber subscriber) throws HyracksDataException {
-        if (this.state == ActivityState.FAILED) {
-            throw new RuntimeDataException(ErrorCode.CANNOT_SUBSCRIBE_TO_FAILED_ACTIVE_ENTITY);
-        }
-        subscriber.subscribed(this);
-        if (!subscriber.isDone()) {
-            subscribers.add(subscriber);
-        }
-    }
-
-    @Override
-    public EntityId getEntityId() {
-        return entityId;
-    }
-
-    @Override
-    public ActivityState getState() {
-        return state;
-    }
-
-    @Override
-    public boolean isEntityUsingDataset(IDataset dataset) {
-        return datasets.contains(dataset);
-    }
-
-    public JobId getJobId() {
-        return jobId;
-    }
-
-    @Override
-    public String getStats() {
-        return stats;
-    }
-
-    @Override
-    public long getStatsTimeStamp() {
-        return statsTimestamp;
-    }
-
-    public String formatStats(List<String> responses) {
-        StringBuilder strBuilder = new StringBuilder();
-        strBuilder.append("{\"Stats\": [").append(responses.get(0));
-        for (int i = 1; i < responses.size(); i++) {
-            strBuilder.append(", ").append(responses.get(i));
-        }
-        strBuilder.append("]}");
-        return strBuilder.toString();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public void refreshStats(long timeout) throws HyracksDataException {
-        LOGGER.log(Level.INFO, "refreshStats called");
-        synchronized (this) {
-            if (state != ActivityState.STARTED || statsRequestState == RequestState.STARTED) {
-                LOGGER.log(Level.INFO, "returning immediately since state = " + state + " and statsRequestState = "
-                        + statsRequestState);
-                return;
-            } else {
-                statsRequestState = RequestState.STARTED;
-            }
-        }
-        ICCMessageBroker messageBroker = (ICCMessageBroker) appCtx.getServiceContext().getMessageBroker();
-        long reqId = messageBroker.newRequestId();
-        List<INcAddressedMessage> requests = new ArrayList<>();
-        List<String> ncs = Arrays.asList(locations.getLocations());
-        for (int i = 0; i < ncs.size(); i++) {
-            requests.add(new StatsRequestMessage(ActiveManagerMessage.REQUEST_STATS,
-                    new ActiveRuntimeId(entityId, runtimeName, i), reqId));
-        }
-        try {
-            List<String> responses = (List<String>) messageBroker.sendSyncRequestToNCs(reqId, ncs, requests, timeout);
-            stats = formatStats(responses);
-            statsTimestamp = System.currentTimeMillis();
-            notifySubscribers(statsUpdatedEvent);
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
-        }
-        // Same as above
-        statsRequestState = RequestState.FINISHED;
-    }
-
-    protected synchronized void notifySubscribers(ActiveEvent event) {
-        notifyAll();
-        Iterator<IActiveEventSubscriber> it = subscribers.iterator();
-        while (it.hasNext()) {
-            IActiveEventSubscriber subscriber = it.next();
-            if (subscriber.isDone()) {
-                it.remove();
-            } else {
-                try {
-                    subscriber.notify(event);
-                } catch (HyracksDataException e) {
-                    LOGGER.log(Level.WARNING, "Failed to notify subscriber", e);
-                }
-                if (subscriber.isDone()) {
-                    it.remove();
-                }
-            }
-        }
-    }
-
-    public AlgebricksAbsolutePartitionConstraint getLocations() {
-        return locations;
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
index e6ac265..d102d0c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
@@ -77,10 +77,14 @@
                 // Adapter has completed execution
                 continueIngestion = false;
             } catch (InterruptedException e) {
+                adapter.fail();
                 throw e;
             } catch (Exception e) {
                 LOGGER.error("Exception during feed ingestion ", e);
                 continueIngestion = adapter.handleException(e);
+                if (!continueIngestion) {
+                    adapter.fail();
+                }
                 failedIngestion = !continueIngestion;
                 restartCount++;
             }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
index 822d725..e21f9eb 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
@@ -18,39 +18,48 @@
  */
 package org.apache.asterix.external.feed.watch;
 
+import org.apache.asterix.active.IActiveEntityEventSubscriber;
 import org.apache.asterix.active.IActiveEntityEventsListener;
-import org.apache.asterix.active.IActiveEventSubscriber;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public abstract class AbstractSubscriber implements IActiveEventSubscriber {
+public abstract class AbstractSubscriber implements IActiveEntityEventSubscriber {
 
     protected final IActiveEntityEventsListener listener;
-    private boolean done = false;
+    private volatile boolean done = false;
+    private volatile Exception failure = null;
 
     public AbstractSubscriber(IActiveEntityEventsListener listener) {
         this.listener = listener;
     }
 
     @Override
-    public synchronized boolean isDone() {
+    public boolean isDone() {
         return done;
     }
 
-    public synchronized void complete() throws HyracksDataException {
-        done = true;
-        notifyAll();
-    }
-
-    @Override
-    public synchronized void sync() throws InterruptedException {
-        while (!done) {
-            wait();
+    public void complete(Exception failure) {
+        synchronized (listener) {
+            if (failure != null) {
+                this.failure = failure;
+            }
+            done = true;
+            listener.notifyAll();
         }
     }
 
     @Override
-    public synchronized void unsubscribe() {
-        done = true;
-        notifyAll();
+    public void sync() throws HyracksDataException, InterruptedException {
+        synchronized (listener) {
+            while (!done) {
+                if (failure != null) {
+                    throw HyracksDataException.create(failure);
+                }
+                listener.wait();
+            }
+        }
+    }
+
+    public Exception getFailure() {
+        return failure;
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
index 42f7a74..8230b48 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
@@ -19,14 +19,14 @@
 package org.apache.asterix.external.feed.watch;
 
 import org.apache.asterix.active.ActiveEvent;
+import org.apache.asterix.active.IActiveEntityEventSubscriber;
 import org.apache.asterix.active.IActiveEntityEventsListener;
-import org.apache.asterix.active.IActiveEventSubscriber;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * An event subscriber that does not listen to any events
  */
-public class NoOpSubscriber implements IActiveEventSubscriber {
+public class NoOpSubscriber implements IActiveEntityEventSubscriber {
 
     public static final NoOpSubscriber INSTANCE = new NoOpSubscriber();
 
@@ -49,11 +49,6 @@
     }
 
     @Override
-    public void unsubscribe() {
-        // no op
-    }
-
-    @Override
     public void subscribed(IActiveEntityEventsListener eventsListener) throws HyracksDataException {
         // no op
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java
index fa2fa7f..a571904 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java
@@ -32,7 +32,17 @@
     @Override
     public void notify(ActiveEvent event) throws HyracksDataException {
         if (event.getEventKind() == ActiveEvent.Kind.STATS_UPDATED) {
-            complete();
+            try {
+                complete(null);
+            } catch (Exception e) {
+                throw HyracksDataException.create(e);
+            }
+        } else if (event.getEventKind() == ActiveEvent.Kind.FAILURE) {
+            try {
+                complete((Exception) event.getEventObject());
+            } catch (Exception e) {
+                throw HyracksDataException.create(e);
+            }
         }
     }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
index 7bab421..a1cdfb0 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
@@ -18,38 +18,46 @@
  */
 package org.apache.asterix.external.feed.watch;
 
+import java.util.Set;
+
 import org.apache.asterix.active.ActiveEvent;
 import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.IActiveEntityEventsListener;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class WaitForStateSubscriber extends AbstractSubscriber {
 
-    private final ActivityState targetState;
+    private final Set<ActivityState> targetStates;
 
-    public WaitForStateSubscriber(IActiveEntityEventsListener listener, ActivityState targetState)
+    public WaitForStateSubscriber(IActiveEntityEventsListener listener, Set<ActivityState> targetStates)
             throws HyracksDataException {
         super(listener);
-        this.targetState = targetState;
+        this.targetStates = targetStates;
         listener.subscribe(this);
     }
 
     @Override
     public void notify(ActiveEvent event) throws HyracksDataException {
-        if (listener.getState() == targetState) {
-            complete();
+        if (targetStates.contains(listener.getState())) {
+            if (listener.getState() == ActivityState.PERMANENTLY_FAILED
+                    || listener.getState() == ActivityState.TEMPORARILY_FAILED) {
+                complete(listener.getJobFailure());
+            } else {
+                complete(null);
+            }
+        } else if (event != null && event.getEventKind() == ActiveEvent.Kind.FAILURE) {
+            try {
+                complete((Exception) event.getEventObject());
+            } catch (Exception e) {
+                throw HyracksDataException.create(e);
+            }
         }
     }
 
     @Override
     public void subscribed(IActiveEntityEventsListener eventsListener) throws HyracksDataException {
-        if (eventsListener.getState() == ActivityState.FAILED) {
-            throw new RuntimeDataException(ErrorCode.CANNOT_SUBSCRIBE_TO_FAILED_ACTIVE_ENTITY);
-        }
-        if (listener.getState() == targetState) {
-            complete();
+        if (targetStates.contains(listener.getState())) {
+            complete(null);
         }
     }
 }
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
index 1d8ae5e..f0539c6 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
@@ -25,6 +25,7 @@
 
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.external.feed.management.FeedConnectionRequest;
 import org.apache.asterix.external.feed.watch.FeedActivityDetails;
@@ -37,7 +38,6 @@
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.entities.Feed;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
index 3289d68..3d8fc68 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.lang.common.statement;
 
+import java.util.List;
+
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Statement;
@@ -26,9 +28,6 @@
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-import java.util.ArrayList;
-import java.util.List;
-
 public class ConnectFeedStatement implements Statement {
 
     private final Identifier dataverseName;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
index c84a5bd..08ca03b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
@@ -57,8 +57,7 @@
     // Key is dataverse name. Key of value map is dataset name.
     protected final Map<String, Map<String, Dataset>> datasets = new HashMap<>();
     // Key is dataverse name. Key of value map is dataset name. Key of value map of value map is index name.
-    protected final Map<String, Map<String, Map<String, Index>>> indexes =
-            new HashMap<>();
+    protected final Map<String, Map<String, Map<String, Index>>> indexes = new HashMap<>();
     // Key is dataverse name. Key of value map is datatype name.
     protected final Map<String, Map<String, Datatype>> datatypes = new HashMap<>();
     // Key is dataverse name.
@@ -66,19 +65,16 @@
     // Key is function Identifier . Key of value map is function name.
     protected final Map<FunctionSignature, Function> functions = new HashMap<>();
     // Key is adapter dataverse. Key of value map is the adapter name
-    protected final Map<String, Map<String, DatasourceAdapter>> adapters =
-            new HashMap<>();
+    protected final Map<String, Map<String, DatasourceAdapter>> adapters = new HashMap<>();
 
     // Key is DataverseName, Key of the value map is the Policy name
-    protected final Map<String, Map<String, FeedPolicyEntity>> feedPolicies =
-            new HashMap<>();
+    protected final Map<String, Map<String, FeedPolicyEntity>> feedPolicies = new HashMap<>();
     // Key is library dataverse. Key of value map is the library name
     protected final Map<String, Map<String, Library>> libraries = new HashMap<>();
     // Key is library dataverse. Key of value map is the feed name
     protected final Map<String, Map<String, Feed>> feeds = new HashMap<>();
     // Key is DataverseName, Key of the value map is the Policy name
-    protected final Map<String, Map<String, CompactionPolicy>> compactionPolicies =
-            new HashMap<>();
+    protected final Map<String, Map<String, CompactionPolicy>> compactionPolicies = new HashMap<>();
     // Key is DataverseName, Key of value map is feedConnectionId
     protected final Map<String, Map<String, FeedConnection>> feedConnections = new HashMap<>();
 
@@ -247,8 +243,7 @@
                                             datatypes.remove(dataverse.getDataverseName());
                                             adapters.remove(dataverse.getDataverseName());
                                             compactionPolicies.remove(dataverse.getDataverseName());
-                                            List<FunctionSignature> markedFunctionsForRemoval =
-                                                    new ArrayList<>();
+                                            List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<>();
                                             for (FunctionSignature signature : functions.keySet()) {
                                                 if (signature.getNamespace().equals(dataverse.getDataverseName())) {
                                                     markedFunctionsForRemoval.add(signature);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataException.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataException.java
deleted file mode 100644
index 61c21f4..0000000
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataException.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.asterix.metadata;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-
-import java.io.Serializable;
-
-public class MetadataException extends CompilationException {
-    private static final long serialVersionUID = 1L;
-
-    public MetadataException(String message) {
-        super(message);
-    }
-
-    public MetadataException(Throwable cause) {
-        super(cause);
-    }
-
-    public MetadataException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public MetadataException(int errorCode, Serializable... params) {
-        super(errorCode, params);
-    }
-}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 11645e8..5e6d11a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -28,6 +28,9 @@
 
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.MetadataException;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.external.indexing.ExternalFile;
@@ -149,7 +152,7 @@
         try {
             metadataNode.addDataverse(ctx.getJobId(), dataverse);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.addDataverse(dataverse);
     }
@@ -159,7 +162,7 @@
         try {
             metadataNode.dropDataverse(ctx.getJobId(), dataverseName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.dropDataverse(dataverseName);
     }
@@ -169,7 +172,7 @@
         try {
             return metadataNode.getDataverses(ctx.getJobId());
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -196,7 +199,7 @@
         try {
             dataverse = metadataNode.getDataverse(ctx.getJobId(), dataverseName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         // We fetched the dataverse from the MetadataNode. Add it to the cache
         // when this transaction commits.
@@ -227,7 +230,7 @@
             // metadata node.
             dataverseDatasets.addAll(metadataNode.getDataverseDatasets(ctx.getJobId(), dataverseName));
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         // Don't update the cache to avoid checking against the transaction's
         // uncommitted datasets.
@@ -241,7 +244,7 @@
             try {
                 metadataNode.addDataset(ctx.getJobId(), dataset);
             } catch (RemoteException e) {
-                throw new MetadataException(e);
+                throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
         }
 
@@ -258,7 +261,7 @@
             try {
                 metadataNode.dropDataset(ctx.getJobId(), dataverseName, datasetName);
             } catch (RemoteException e) {
-                throw new MetadataException(e);
+                throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
         }
 
@@ -292,7 +295,7 @@
         try {
             dataset = metadataNode.getDataset(ctx.getJobId(), dataverseName, datasetName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         // We fetched the dataset from the MetadataNode. Add it to the cache
         // when this transaction commits.
@@ -318,7 +321,7 @@
                 // for persistent datasets
                 datasetIndexes = metadataNode.getDatasetIndexes(ctx.getJobId(), dataverseName, datasetName);
             } catch (RemoteException e) {
-                throw new MetadataException(e);
+                throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
         }
         return datasetIndexes;
@@ -330,7 +333,7 @@
         try {
             metadataNode.addCompactionPolicy(mdTxnCtx.getJobId(), compactionPolicy);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         mdTxnCtx.addCompactionPolicy(compactionPolicy);
     }
@@ -343,7 +346,7 @@
         try {
             compactionPolicy = metadataNode.getCompactionPolicy(ctx.getJobId(), dataverse, policyName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         return compactionPolicy;
     }
@@ -353,13 +356,13 @@
         try {
             metadataNode.addDatatype(ctx.getJobId(), datatype);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         try {
             ctx.addDatatype(
                     metadataNode.getDatatype(ctx.getJobId(), datatype.getDataverseName(), datatype.getDatatypeName()));
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -369,7 +372,7 @@
         try {
             metadataNode.dropDatatype(ctx.getJobId(), dataverseName, datatypeName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.dropDataDatatype(dataverseName, datatypeName);
     }
@@ -406,7 +409,7 @@
         try {
             datatype = metadataNode.getDatatype(ctx.getJobId(), dataverseName, datatypeName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         // We fetched the datatype from the MetadataNode. Add it to the cache
         // when this transaction commits.
@@ -425,7 +428,7 @@
             try {
                 metadataNode.addIndex(ctx.getJobId(), index);
             } catch (RemoteException e) {
-                throw new MetadataException(e);
+                throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
         }
         ctx.addIndex(index);
@@ -436,7 +439,7 @@
         try {
             metadataNode.addAdapter(mdTxnCtx.getJobId(), adapter);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         mdTxnCtx.addAdapter(adapter);
 
@@ -452,7 +455,7 @@
             try {
                 metadataNode.dropIndex(ctx.getJobId(), dataverseName, datasetName, indexName);
             } catch (RemoteException e) {
-                throw new MetadataException(e);
+                throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
         }
         ctx.dropIndex(dataverseName, datasetName, indexName);
@@ -485,7 +488,7 @@
         try {
             index = metadataNode.getIndex(ctx.getJobId(), dataverseName, datasetName, indexName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         // We fetched the index from the MetadataNode. Add it to the cache
         // when this transaction commits.
@@ -500,7 +503,7 @@
         try {
             metadataNode.addNode(ctx.getJobId(), node);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -509,7 +512,7 @@
         try {
             metadataNode.addNodeGroup(ctx.getJobId(), nodeGroup);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.addNodeGroup(nodeGroup);
     }
@@ -521,7 +524,7 @@
         try {
             dropped = metadataNode.dropNodegroup(ctx.getJobId(), nodeGroupName, failSilently);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         if (dropped) {
             ctx.dropNodeGroup(nodeGroupName);
@@ -551,7 +554,7 @@
         try {
             nodeGroup = metadataNode.getNodeGroup(ctx.getJobId(), nodeGroupName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         // We fetched the nodeGroup from the MetadataNode. Add it to the cache
         // when this transaction commits.
@@ -566,7 +569,7 @@
         try {
             metadataNode.updateFunction(mdTxnCtx.getJobId(), function);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         mdTxnCtx.dropFunction(function);
         mdTxnCtx.addFunction(function);
@@ -577,7 +580,7 @@
         try {
             metadataNode.addFunction(mdTxnCtx.getJobId(), function);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         mdTxnCtx.addFunction(function);
     }
@@ -588,7 +591,7 @@
         try {
             metadataNode.dropFunction(ctx.getJobId(), functionSignature);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.dropFunction(functionSignature);
     }
@@ -622,7 +625,7 @@
         try {
             function = metadataNode.getFunction(ctx.getJobId(), functionSignature);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         // We fetched the function from the MetadataNode. Add it to the cache
         // when this transaction commits.
@@ -639,7 +642,7 @@
         try {
             metadataNode.addFeedPolicy(mdTxnCtx.getJobId(), feedPolicy);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         mdTxnCtx.addFeedPolicy(feedPolicy);
     }
@@ -649,7 +652,7 @@
         try {
             metadataNode.initializeDatasetIdFactory(ctx.getJobId());
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -658,7 +661,7 @@
         try {
             return metadataNode.getMostRecentDatasetId();
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -671,7 +674,7 @@
             // metadata node.
             dataverseFunctions = metadataNode.getDataverseFunctions(ctx.getJobId(), dataverseName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         // Don't update the cache to avoid checking against the transaction's
         // uncommitted functions.
@@ -684,7 +687,7 @@
         try {
             metadataNode.dropAdapter(ctx.getJobId(), dataverseName, name);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -695,7 +698,7 @@
         try {
             adapter = metadataNode.getAdapter(ctx.getJobId(), dataverseName, name);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         return adapter;
     }
@@ -706,7 +709,7 @@
         try {
             metadataNode.dropLibrary(ctx.getJobId(), dataverseName, libraryName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.dropLibrary(dataverseName, libraryName);
     }
@@ -720,7 +723,7 @@
             // metadata node.
             dataverseLibaries = metadataNode.getDataverseLibraries(ctx.getJobId(), dataverseName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         // Don't update the cache to avoid checking against the transaction's
         // uncommitted functions.
@@ -732,7 +735,7 @@
         try {
             metadataNode.addLibrary(ctx.getJobId(), library);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.addLibrary(library);
     }
@@ -744,7 +747,7 @@
         try {
             library = metadataNode.getLibrary(ctx.getJobId(), dataverseName, libraryName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         return library;
     }
@@ -777,7 +780,7 @@
         try {
             feedPolicy = metadataNode.getFeedPolicy(ctx.getJobId(), dataverse, policyName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         return feedPolicy;
     }
@@ -788,7 +791,7 @@
         try {
             feed = metadataNode.getFeed(ctx.getJobId(), dataverse, feedName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         return feed;
     }
@@ -806,7 +809,7 @@
                 ctx.dropFeedConnection(dataverse, feedName, feedConnection.getDatasetName());
             }
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.dropFeed(feed);
     }
@@ -816,7 +819,7 @@
         try {
             metadataNode.addFeed(ctx.getJobId(), feed);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.addFeed(feed);
     }
@@ -827,7 +830,7 @@
         try {
             metadataNode.addFeedConnection(ctx.getJobId(), feedConnection);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.addFeedConnection(feedConnection);
     }
@@ -838,7 +841,7 @@
         try {
             metadataNode.dropFeedConnection(ctx.getJobId(), dataverseName, feedName, datasetName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.dropFeedConnection(dataverseName, feedName, datasetName);
     }
@@ -849,7 +852,7 @@
         try {
             return metadataNode.getFeedConnection(ctx.getJobId(), dataverseName, feedName, datasetName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -859,7 +862,7 @@
         try {
             return metadataNode.getFeedConnections(ctx.getJobId(), dataverseName, feedName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -870,7 +873,7 @@
         try {
             dataverseAdapters = metadataNode.getDataverseAdapters(mdTxnCtx.getJobId(), dataverse);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         return dataverseAdapters;
     }
@@ -883,7 +886,7 @@
             feedPolicy = metadataNode.getFeedPolicy(mdTxnCtx.getJobId(), dataverseName, policyName);
             metadataNode.dropFeedPolicy(mdTxnCtx.getJobId(), dataverseName, policyName);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         mdTxnCtx.dropFeedPolicy(feedPolicy);
     }
@@ -894,7 +897,7 @@
         try {
             dataverseFeedPolicies = metadataNode.getDataversePolicies(mdTxnCtx.getJobId(), dataverse);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         return dataverseFeedPolicies;
     }
@@ -906,7 +909,7 @@
         try {
             externalFiles = metadataNode.getExternalFiles(mdTxnCtx.getJobId(), dataset);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         return externalFiles;
     }
@@ -916,7 +919,7 @@
         try {
             metadataNode.addExternalFile(ctx.getJobId(), externalFile);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -926,7 +929,7 @@
             metadataNode.dropExternalFile(ctx.getJobId(), externalFile.getDataverseName(),
                     externalFile.getDatasetName(), externalFile.getFileNumber());
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -937,7 +940,7 @@
         try {
             file = metadataNode.getExternalFile(ctx.getJobId(), dataverseName, datasetName, fileNumber);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         return file;
     }
@@ -949,7 +952,7 @@
         try {
             metadataNode.dropExternalFiles(mdTxnCtx.getJobId(), dataset);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -958,7 +961,7 @@
         try {
             metadataNode.updateDataset(ctx.getJobId(), dataset);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         // reflect the dataset into the cache
         ctx.dropDataset(dataset.getDataverseName(), dataset.getDatasetName());
@@ -984,7 +987,17 @@
         try {
             metadataNode.addEntity(mdTxnCtx.getJobId(), entity);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+    }
+
+    @Override
+    public <T extends IExtensionMetadataEntity> void upsertEntity(MetadataTransactionContext mdTxnCtx, T entity)
+            throws MetadataException {
+        try {
+            metadataNode.upsertEntity(mdTxnCtx.getJobId(), entity);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -994,7 +1007,7 @@
         try {
             metadataNode.deleteEntity(mdTxnCtx.getJobId(), entity);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -1004,7 +1017,7 @@
         try {
             return metadataNode.getEntities(mdTxnCtx.getJobId(), searchKey);
         } catch (RemoteException e) {
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
     }
 
@@ -1046,9 +1059,9 @@
                 }
             } catch (InterruptedException e) {
                 Thread.currentThread().interrupt();
-                throw new HyracksDataException(e);
+                throw HyracksDataException.create(e);
             } catch (RemoteException e) {
-                throw new HyracksDataException(e);
+                throw new RuntimeDataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
             super.init();
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 84cb671..265e533 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -34,6 +34,7 @@
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.dataflow.LSMIndexUtil;
 import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
 import org.apache.asterix.common.transactions.AbstractOperationCallback;
@@ -97,6 +98,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallback;
+import org.apache.asterix.transaction.management.opcallbacks.UpsertOperationCallback;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.asterix.transaction.management.service.transaction.TransactionContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
@@ -124,8 +126,8 @@
 public class MetadataNode implements IMetadataNode {
     private static final long serialVersionUID = 1L;
     private static final Logger LOGGER = Logger.getLogger(MetadataNode.class.getName());
-    private static final DatasetId METADATA_DATASET_ID = new ImmutableDatasetId(
-            MetadataPrimaryIndexes.PROPERTIES_METADATA.getDatasetId());
+    private static final DatasetId METADATA_DATASET_ID =
+            new ImmutableDatasetId(MetadataPrimaryIndexes.PROPERTIES_METADATA.getDatasetId());
 
     // shared between core and extension
     private IDatasetLifecycleManager datasetLifecycleManager;
@@ -173,8 +175,8 @@
     @Override
     public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
         try {
-            ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId,
-                    false);
+            ITransactionContext txnCtx =
+                    transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, DatasetId.NULL, -1);
         } catch (ACIDException e) {
             LOGGER.log(Level.WARNING, "Exception aborting transaction", e);
@@ -215,6 +217,25 @@
     }
 
     /**
+     * Upsert entity to index
+     *
+     * @param jobId
+     * @param entity
+     * @param tupleTranslator
+     * @param index
+     * @throws MetadataException
+     */
+    private <T> void upsertEntity(JobId jobId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
+            IMetadataIndex index) throws MetadataException {
+        try {
+            ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
+            upsertTupleIntoIndex(jobId, index, tuple);
+        } catch (HyracksDataException | ACIDException e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    /**
      * Delete entity from index
      *
      * @param jobId
@@ -271,6 +292,18 @@
 
     @SuppressWarnings("unchecked")
     @Override
+    public <T extends IExtensionMetadataEntity> void upsertEntity(JobId jobId, T entity)
+            throws MetadataException, RemoteException {
+        ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(entity.getDatasetId());
+        if (index == null) {
+            throw new MetadataException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
+        }
+        IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
+        upsertEntity(jobId, entity, tupleTranslator, index);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
     public <T extends IExtensionMetadataEntity> void deleteEntity(JobId jobId, T entity)
             throws MetadataException, RemoteException {
         ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(entity.getDatasetId());
@@ -285,8 +318,8 @@
     @Override
     public <T extends IExtensionMetadataEntity> List<T> getEntities(JobId jobId, IExtensionMetadataSearchKey searchKey)
             throws MetadataException, RemoteException {
-        ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets
-                .get(searchKey.getDatasetId());
+        ExtensionMetadataDataset<T> index =
+                (ExtensionMetadataDataset<T>) extensionDatasets.get(searchKey.getDatasetId());
         if (index == null) {
             throw new MetadataException("Metadata Extension Index: " + searchKey.getDatasetId() + " was not found");
         }
@@ -396,8 +429,8 @@
     @Override
     public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException {
         try {
-            DatatypeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this,
-                    true);
+            DatatypeTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this, true);
             ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
             insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
         } catch (HyracksDataException e) {
@@ -443,13 +476,13 @@
             datasetLifecycleManager.open(resourceName);
 
             // prepare a Callback for logging
-            IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID,
-                    metadataIndex, lsmIndex, Operation.INSERT);
+            IModificationOperationCallback modCallback =
+                    createIndexModificationCallback(jobId, resourceID, metadataIndex, lsmIndex, Operation.INSERT);
 
             ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
-            ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId,
-                    false);
+            ITransactionContext txnCtx =
+                    transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             txnCtx.setWriteTxn(true);
             txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
                     metadataIndex.isPrimaryIndex());
@@ -471,6 +504,39 @@
         }
     }
 
+    private void upsertTupleIntoIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
+            throws ACIDException, HyracksDataException {
+        long resourceId = metadataIndex.getResourceId();
+        String resourceName = metadataIndex.getFile().getRelativePath();
+        ILSMIndex lsmIndex = (ILSMIndex) datasetLifecycleManager.get(resourceName);
+        datasetLifecycleManager.open(resourceName);
+        try {
+            // prepare a Callback for logging
+            ITransactionContext txnCtx =
+                    transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+            IModificationOperationCallback modCallback =
+                    new UpsertOperationCallback(metadataIndex.getDatasetId(), metadataIndex.getPrimaryKeyIndexes(),
+                            txnCtx, transactionSubsystem.getLockManager(), transactionSubsystem, resourceId,
+                            metadataStoragePartition, ResourceType.LSM_BTREE, Operation.UPSERT);
+            ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+            txnCtx.setWriteTxn(true);
+            txnCtx.registerIndexAndCallback(resourceId, lsmIndex, (AbstractOperationCallback) modCallback,
+                    metadataIndex.isPrimaryIndex());
+            LSMIndexUtil.checkAndSetFirstLSN((AbstractLSMIndex) lsmIndex, transactionSubsystem.getLogManager());
+            indexAccessor.forceUpsert(tuple);
+            //Manually complete the operation after the insert. This is to decrement the resource counters within the
+            //index that determine how many tuples are still 'in-flight' within the index. Normally the log flusher
+            //does this. The only exception is the index registered as the "primary" which we will let be decremented
+            //by the job commit log event
+            if (!((TransactionContext) txnCtx).getPrimaryIndexOpTracker().equals(lsmIndex.getOperationTracker())) {
+                lsmIndex.getOperationTracker().completeOperation(lsmIndex, LSMOperationType.FORCE_MODIFICATION, null,
+                        modCallback);
+            }
+        } finally {
+            datasetLifecycleManager.close(resourceName);
+        }
+    }
+
     private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
             IMetadataIndex metadataIndex, ILSMIndex lsmIndex, Operation indexOp) throws ACIDException {
         ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
@@ -748,12 +814,12 @@
         try {
             datasetLifecycleManager.open(resourceName);
             // prepare a Callback for logging
-            IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID,
-                    metadataIndex, lsmIndex, Operation.DELETE);
+            IModificationOperationCallback modCallback =
+                    createIndexModificationCallback(jobId, resourceID, metadataIndex, lsmIndex, Operation.DELETE);
             ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
-            ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId,
-                    false);
+            ITransactionContext txnCtx =
+                    transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             txnCtx.setWriteTxn(true);
             txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
                     metadataIndex.isPrimaryIndex());
@@ -852,8 +918,8 @@
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
-            DatatypeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this,
-                    false);
+            DatatypeTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this, false);
             IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Datatype> results = new ArrayList<>();
             searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
@@ -897,8 +963,8 @@
     public List<Datatype> getAllDatatypes(JobId jobId) throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = null;
-            DatatypeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this,
-                    false);
+            DatatypeTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this, false);
             IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Datatype> results = new ArrayList<>();
             searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
@@ -1010,8 +1076,8 @@
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
-            IndexTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getIndexTupleTranslator(jobId, this,
-                    false);
+            IndexTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getIndexTupleTranslator(jobId, this, false);
             IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Index> results = new ArrayList<>();
             searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
@@ -1029,8 +1095,8 @@
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datasetName);
-            IndexTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getIndexTupleTranslator(jobId, this,
-                    false);
+            IndexTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getIndexTupleTranslator(jobId, this, false);
             IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Index> results = new ArrayList<>();
             searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
@@ -1045,8 +1111,8 @@
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datatypeName);
-            DatatypeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this,
-                    false);
+            DatatypeTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this, false);
             IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Datatype> results = new ArrayList<>();
             searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
@@ -1111,8 +1177,8 @@
                     "" + functionSignature.getArity());
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'function' dataset.
-            ITupleReference functionTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
-                    searchKey);
+            ITupleReference functionTuple =
+                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey);
             deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
 
             // TODO: Change this to be a BTree specific exception, e.g.,
@@ -1153,8 +1219,8 @@
             String resourceName = index.getFile().toString();
             IIndex indexInstance = datasetLifecycleManager.get(resourceName);
             datasetLifecycleManager.open(resourceName);
-            IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
-                    NoOpOperationCallback.INSTANCE);
+            IIndexAccessor indexAccessor =
+                    indexInstance.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
 
             RangePredicate rangePred = null;
@@ -1174,8 +1240,8 @@
             index = MetadataPrimaryIndexes.DATASET_DATASET;
             indexInstance = datasetLifecycleManager.get(resourceName);
             datasetLifecycleManager.open(resourceName);
-            indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
-                    NoOpOperationCallback.INSTANCE);
+            indexAccessor =
+                    indexInstance.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
 
             rangePred = null;
@@ -1196,8 +1262,8 @@
             index = MetadataPrimaryIndexes.INDEX_DATASET;
             indexInstance = datasetLifecycleManager.get(resourceName);
             datasetLifecycleManager.open(resourceName);
-            indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
-                    NoOpOperationCallback.INSTANCE);
+            indexAccessor =
+                    indexInstance.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
 
             rangePred = null;
@@ -1232,8 +1298,8 @@
         String resourceName = index.getFile().getRelativePath();
         IIndex indexInstance = datasetLifecycleManager.get(resourceName);
         datasetLifecycleManager.open(resourceName);
-        IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
-                NoOpOperationCallback.INSTANCE);
+        IIndexAccessor indexAccessor =
+                indexInstance.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
         ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
 
         IBinaryComparator[] searchCmps = null;
@@ -1271,8 +1337,8 @@
             IIndex indexInstance = datasetLifecycleManager.get(resourceName);
             datasetLifecycleManager.open(resourceName);
             try {
-                IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
-                        NoOpOperationCallback.INSTANCE);
+                IIndexAccessor indexAccessor =
+                        indexInstance.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
                 IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
 
                 DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
@@ -1310,8 +1376,8 @@
     // Hyracks version.
     public static ITupleReference createTuple(String... fields) {
         @SuppressWarnings("unchecked")
-        ISerializerDeserializer<AString> stringSerde = SerializerDeserializerProvider.INSTANCE
-                .getSerializerDeserializer(BuiltinType.ASTRING);
+        ISerializerDeserializer<AString> stringSerde =
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
         AMutableString aString = new AMutableString("");
         ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fields.length);
         for (String s : fields) {
@@ -1348,8 +1414,8 @@
     public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
         try {
             // Insert into the 'Adapter' dataset.
-            DatasourceAdapterTupleTranslator tupleReaderWriter = tupleTranslatorProvider
-                    .getAdapterTupleTranslator(true);
+            DatasourceAdapterTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getAdapterTupleTranslator(true);
             ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
             insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
         } catch (HyracksDataException e) {
@@ -1376,8 +1442,8 @@
             ITupleReference searchKey = createTuple(dataverseName, adapterName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'Adapter' dataset.
-            ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET,
-                    searchKey);
+            ITupleReference datasetTuple =
+                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
             deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
 
             // TODO: Change this to be a BTree specific exception, e.g.,
@@ -1400,8 +1466,8 @@
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, adapterName);
-            DatasourceAdapterTupleTranslator tupleReaderWriter = tupleTranslatorProvider
-                    .getAdapterTupleTranslator(false);
+            DatasourceAdapterTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getAdapterTupleTranslator(false);
             List<DatasourceAdapter> results = new ArrayList<>();
             IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
@@ -1419,8 +1485,8 @@
             throws MetadataException, RemoteException {
         try {
             // Insert into the 'CompactionPolicy' dataset.
-            CompactionPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider
-                    .getCompactionPolicyTupleTranslator(true);
+            CompactionPolicyTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getCompactionPolicyTupleTranslator(true);
             ITupleReference compactionPolicyTuple = tupleReaderWriter.getTupleFromMetadataEntity(compactionPolicy);
             insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, compactionPolicyTuple);
         } catch (HyracksDataException e) {
@@ -1440,8 +1506,8 @@
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverse, policyName);
-            CompactionPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider
-                    .getCompactionPolicyTupleTranslator(false);
+            CompactionPolicyTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getCompactionPolicyTupleTranslator(false);
             List<CompactionPolicy> results = new ArrayList<>();
             IValueExtractor<CompactionPolicy> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             searchIndex(jobId, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, searchKey, valueExtractor, results);
@@ -1459,8 +1525,8 @@
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
-            DatasourceAdapterTupleTranslator tupleReaderWriter = tupleTranslatorProvider
-                    .getAdapterTupleTranslator(false);
+            DatasourceAdapterTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getAdapterTupleTranslator(false);
             IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<DatasourceAdapter> results = new ArrayList<>();
             searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
@@ -1502,8 +1568,8 @@
             ITupleReference searchKey = createTuple(dataverseName, libraryName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'Adapter' dataset.
-            ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET,
-                    searchKey);
+            ITupleReference datasetTuple =
+                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey);
             deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, datasetTuple);
 
             // TODO: Change this to be a BTree specific exception, e.g.,
@@ -1630,8 +1696,8 @@
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, feedName, datasetName);
-            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET,
-                    searchKey);
+            ITupleReference tuple =
+                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey);
             deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, tuple);
         } catch (HyracksDataException | ACIDException e) {
             throw new MetadataException(e);
@@ -1734,8 +1800,8 @@
     public void addExternalFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException {
         try {
             // Insert into the 'externalFiles' dataset.
-            ExternalFileTupleTranslator tupleReaderWriter = tupleTranslatorProvider
-                    .getExternalFileTupleTranslator(true);
+            ExternalFileTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getExternalFileTupleTranslator(true);
             ITupleReference externalFileTuple = tupleReaderWriter.getTupleFromMetadataEntity(externalFile);
             insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, externalFileTuple);
         } catch (HyracksDataException e) {
@@ -1755,8 +1821,8 @@
     public List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataset.getDataverseName(), dataset.getDatasetName());
-            ExternalFileTupleTranslator tupleReaderWriter = tupleTranslatorProvider
-                    .getExternalFileTupleTranslator(false);
+            ExternalFileTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getExternalFileTupleTranslator(false);
             IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<ExternalFile> results = new ArrayList<>();
             searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
@@ -1774,8 +1840,8 @@
             ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'ExternalFile' dataset.
-            ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET,
-                    searchKey);
+            ITupleReference datasetTuple =
+                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey);
             deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, datasetTuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
@@ -1803,10 +1869,10 @@
     @SuppressWarnings("unchecked")
     public ITupleReference createExternalFileSearchTuple(String dataverseName, String datasetName, int fileNumber)
             throws HyracksDataException {
-        ISerializerDeserializer<AString> stringSerde = SerializerDeserializerProvider.INSTANCE
-                .getSerializerDeserializer(BuiltinType.ASTRING);
-        ISerializerDeserializer<AInt32> intSerde = SerializerDeserializerProvider.INSTANCE
-                .getSerializerDeserializer(BuiltinType.AINT32);
+        ISerializerDeserializer<AString> stringSerde =
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
+        ISerializerDeserializer<AInt32> intSerde =
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
 
         AMutableString aString = new AMutableString("");
         ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(3);
@@ -1835,8 +1901,8 @@
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
-            ExternalFileTupleTranslator tupleReaderWriter = tupleTranslatorProvider
-                    .getExternalFileTupleTranslator(false);
+            ExternalFileTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getExternalFileTupleTranslator(false);
             IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<ExternalFile> results = new ArrayList<>();
             searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
@@ -1858,8 +1924,8 @@
             searchKey = createTuple(dataset.getDataverseName(), dataset.getDatasetName());
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'dataset' dataset.
-            ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET,
-                    searchKey);
+            ITupleReference datasetTuple =
+                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
             deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
             // Previous tuple was deleted
             // Insert into the 'dataset' dataset.
@@ -1876,10 +1942,10 @@
         try {
             // remove old function
             ITupleReference searchKey;
-            searchKey = createTuple(function.getDataverseName(), function.getName(),
-                    Integer.toString(function.getArity()));
-            ITupleReference functionTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
-                    searchKey);
+            searchKey =
+                    createTuple(function.getDataverseName(), function.getName(), Integer.toString(function.getArity()));
+            ITupleReference functionTuple =
+                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey);
             deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
             // add new function
             FunctionTupleTranslator functionTupleTranslator = tupleTranslatorProvider.getFunctionTupleTranslator(true);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
new file mode 100644
index 0000000..c73a433
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.metadata.api;
+
+import java.util.List;
+
+import org.apache.asterix.active.IActiveEntityEventsListener;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IActiveEntityController extends IActiveEntityEventsListener {
+
+    /**
+     * Start the active entity
+     *
+     * @param metadataProvider
+     * @throws HyracksDataException
+     * @throws InterruptedException
+     */
+    void start(MetadataProvider metadataProvider) throws HyracksDataException, InterruptedException;
+
+    /**
+     * Stop the active entity
+     *
+     * @param metadataProvider
+     * @throws HyracksDataException
+     * @throws InterruptedException
+     */
+    void stop(MetadataProvider metadataProvider) throws HyracksDataException, InterruptedException;
+
+    /**
+     * Suspend the active entity
+     * This call stops and freezes the active entity. The calling thread must call resume to release
+     * locks on the entity
+     *
+     * @param metadataProvider
+     * @throws HyracksDataException
+     * @throws InterruptedException
+     */
+    void suspend(MetadataProvider metadataProvider) throws HyracksDataException, InterruptedException;
+
+    /**
+     * Resumes the active entity activity prior to the suspend call
+     *
+     * @param metadataProvider
+     * @throws HyracksDataException
+     * @throws InterruptedException
+     */
+    void resume(MetadataProvider metadataProvider) throws HyracksDataException, InterruptedException;
+
+    /**
+     * Start failure recovery
+     *
+     * @throws HyracksDataException
+     */
+    void recover() throws HyracksDataException;
+
+    /**
+     * Add dataset to the list of associated datasets
+     *
+     * @param dataset
+     *            the dataset to add
+     * @throws HyracksDataException
+     *             if the entity is active
+     */
+    void add(Dataset dataset) throws HyracksDataException;
+
+    /**
+     * Remove dataset to the list of associated datasets
+     *
+     * @param dataset
+     *            the dataset to add
+     * @throws HyracksDataException
+     *             if the entity is active
+     */
+    void remove(Dataset dataset) throws HyracksDataException;
+
+    /**
+     * @return the list of associated datasets
+     */
+    List<Dataset> getDatasets();
+
+    /**
+     * replace the dataset object with the passed updated object
+     *
+     * @param target
+     */
+    void replace(Dataset dataset);
+
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java
index 9a9f18d..fbeb353 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java
@@ -23,7 +23,7 @@
 import java.io.Serializable;
 import java.rmi.RemoteException;
 
-import org.apache.asterix.metadata.MetadataException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 57fcc5e..6ad1a20 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -23,10 +23,10 @@
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.IMetadataBootstrap;
 import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.entities.CompactionPolicy;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -671,6 +671,16 @@
             throws MetadataException;
 
     /**
+     * Upsert an extension entity to its extension dataset under the ongoing metadata transaction
+     *
+     * @param mdTxnCtx
+     * @param entity
+     * @throws MetadataException
+     */
+    <T extends IExtensionMetadataEntity> void upsertEntity(MetadataTransactionContext mdTxnCtx, T entity)
+            throws MetadataException;
+
+    /**
      * Deletes an extension entity from its extension dataset under the ongoing metadata transaction
      *
      * @param mdTxnCtx
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index cfbbda3..78360fd 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -25,10 +25,10 @@
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.entities.CompactionPolicy;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
@@ -749,6 +749,17 @@
             throws MetadataException, RemoteException;
 
     /**
+     * Upserts an extension entity under the ongoing transaction job id
+     *
+     * @param jobId
+     * @param entity
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+    <T extends IExtensionMetadataEntity> void upsertEntity(JobId jobId, T entity)
+            throws MetadataException, RemoteException;
+
+    /**
      * Deletes an extension entity under the ongoing transaction job id
      *
      * @param jobId
@@ -781,4 +792,5 @@
 
     List<FeedConnection> getFeedConnections(JobId jobId, String dataverseName, String feedName)
             throws MetadataException, RemoteException;
+
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java
index c8db613..c2b61ee 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java
@@ -22,8 +22,8 @@
 import java.io.IOException;
 import java.rmi.RemoteException;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 93b19f1..dc38749 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -36,6 +36,7 @@
 import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
@@ -44,7 +45,6 @@
 import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.metadata.IDatasetDetails;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.api.IMetadataIndex;
@@ -54,7 +54,6 @@
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Dataverse;
-import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
@@ -160,7 +159,7 @@
                 insertNodes(mdTxnCtx);
                 insertInitialGroups(mdTxnCtx);
                 insertInitialAdapters(mdTxnCtx);
-                insertInitialFeedPolicies(mdTxnCtx);
+                BuiltinFeedPolicies.insertInitialFeedPolicies(mdTxnCtx);
                 insertInitialCompactionPolicies(mdTxnCtx);
                 if (LOGGER.isLoggable(Level.INFO)) {
                     LOGGER.info("Finished creating metadata B-trees.");
@@ -262,15 +261,6 @@
         }
     }
 
-    private static void insertInitialFeedPolicies(MetadataTransactionContext mdTxnCtx) throws MetadataException {
-        for (FeedPolicyEntity feedPolicy : BuiltinFeedPolicies.policies) {
-            MetadataManager.INSTANCE.addFeedPolicy(mdTxnCtx, feedPolicy);
-        }
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Finished adding built-in feed policies.");
-        }
-    }
-
     private static void insertInitialCompactionPolicies(MetadataTransactionContext mdTxnCtx) throws MetadataException {
         String[] builtInCompactionPolicyClassNames =
                 new String[] { ConstantMergePolicyFactory.class.getName(), PrefixMergePolicyFactory.class.getName(),
@@ -358,7 +348,8 @@
             if (index.getResourceId() != resource.getId()) {
                 throw new HyracksDataException("Resource Id doesn't match expected metadata index resource id");
             }
-            IndexDataflowHelper indexHelper = new IndexDataflowHelper(ncServiceCtx, storageComponentProvider.getStorageManager(), file);
+            IndexDataflowHelper indexHelper =
+                    new IndexDataflowHelper(ncServiceCtx, storageComponentProvider.getStorageManager(), file);
             indexHelper.open(); // Opening the index through the helper will ensure it gets instantiated
             indexHelper.close();
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
index 49b32c0..c23755d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
@@ -22,7 +22,7 @@
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.metadata.MetadataException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.entities.Dataset;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 68a2448..7d28a06 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -35,6 +35,8 @@
 import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.MetadataException;
+import org.apache.asterix.common.metadata.LockList;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -56,7 +58,6 @@
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.LinearizeComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.dataset.hints.DatasetHints.DatasetCardinalityHint;
@@ -70,8 +71,6 @@
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
 import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
-import org.apache.asterix.metadata.lock.LockList;
-import org.apache.asterix.metadata.lock.MetadataLockManager;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
@@ -140,7 +139,7 @@
 public class MetadataProvider implements IMetadataProvider<DataSourceId, String> {
 
     private final ICcApplicationContext appCtx;
-    private final IStorageComponentProvider storaegComponentProvider;
+    private final IStorageComponentProvider storageComponentProvider;
     private final StorageProperties storageProperties;
     private final Dataverse defaultDataverse;
     private final LockList locks;
@@ -158,11 +157,10 @@
     private boolean isTemporaryDatasetWriteJob = true;
     private boolean blockingOperatorDisabled = false;
 
-    public MetadataProvider(ICcApplicationContext appCtx, Dataverse defaultDataverse,
-            IStorageComponentProvider componentProvider) {
+    public MetadataProvider(ICcApplicationContext appCtx, Dataverse defaultDataverse) {
         this.appCtx = appCtx;
         this.defaultDataverse = defaultDataverse;
-        this.storaegComponentProvider = componentProvider;
+        this.storageComponentProvider = appCtx.getStorageComponentProvider();
         storageProperties = appCtx.getStorageProperties();
         locks = new LockList();
     }
@@ -293,8 +291,8 @@
             return null;
         }
         String fqName = dv + '.' + dataset;
-        MetadataLockManager.INSTANCE.acquireDataverseReadLock(locks, dv);
-        MetadataLockManager.INSTANCE.acquireDatasetReadLock(locks, fqName);
+        appCtx.getMetadataLockManager().acquireDataverseReadLock(locks, dv);
+        appCtx.getMetadataLockManager().acquireDatasetReadLock(locks, fqName);
         return MetadataManagerUtil.findDataset(mdTxnCtx, dv, dataset);
     }
 
@@ -398,23 +396,23 @@
     }
 
     public Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> buildFeedIntakeRuntime(
-            JobSpecification jobSpec, Feed primaryFeed, FeedPolicyAccessor policyAccessor) throws Exception {
+            JobSpecification jobSpec, Feed feed, FeedPolicyAccessor policyAccessor) throws Exception {
         Triple<IAdapterFactory, RecordDescriptor, IDataSourceAdapter.AdapterType> factoryOutput;
-        factoryOutput = FeedMetadataUtil.getPrimaryFeedFactoryAndOutput(primaryFeed, policyAccessor, mdTxnCtx,
+        factoryOutput = FeedMetadataUtil.getPrimaryFeedFactoryAndOutput(feed, policyAccessor, mdTxnCtx,
                 getApplicationContext());
-        ARecordType recordType = FeedMetadataUtil.getOutputType(primaryFeed, primaryFeed.getAdapterConfiguration(),
+        ARecordType recordType = FeedMetadataUtil.getOutputType(feed, feed.getAdapterConfiguration(),
                 ExternalDataConstants.KEY_TYPE_NAME);
         IAdapterFactory adapterFactory = factoryOutput.first;
         FeedIntakeOperatorDescriptor feedIngestor = null;
         switch (factoryOutput.third) {
             case INTERNAL:
-                feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, primaryFeed, adapterFactory, recordType,
+                feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, feed, adapterFactory, recordType,
                         policyAccessor, factoryOutput.second);
                 break;
             case EXTERNAL:
-                String libraryName = primaryFeed.getAdapterName().trim()
-                        .split(FeedConstants.NamingConstants.LIBRARY_NAME_SEPARATOR)[0];
-                feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, primaryFeed, libraryName,
+                String libraryName =
+                        feed.getAdapterName().trim().split(FeedConstants.NamingConstants.LIBRARY_NAME_SEPARATOR)[0];
+                feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, feed, libraryName,
                         adapterFactory.getClass().getName(), recordType, policyAccessor, factoryOutput.second);
                 break;
             default:
@@ -449,7 +447,7 @@
             }
 
             ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
-                    storaegComponentProvider, theIndex, jobId, IndexOperation.SEARCH, primaryKeyFields);
+                    storageComponentProvider, theIndex, jobId, IndexOperation.SEARCH, primaryKeyFields);
             IStorageManager storageManager = getStorageComponentProvider().getStorageManager();
             IIndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(storageManager, spPc.first);
             BTreeSearchOperatorDescriptor btreeSearchOp;
@@ -493,10 +491,10 @@
             }
 
             ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
-                    storaegComponentProvider, secondaryIndex, jobId, IndexOperation.SEARCH, primaryKeyFields);
+                    storageComponentProvider, secondaryIndex, jobId, IndexOperation.SEARCH, primaryKeyFields);
             RTreeSearchOperatorDescriptor rtreeSearchOp;
             IIndexDataflowHelperFactory indexDataflowHelperFactory =
-                    new IndexDataflowHelperFactory(storaegComponentProvider.getStorageManager(), spPc.first);
+                    new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
             if (dataset.getDatasetType() == DatasetType.INTERNAL) {
                 rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
                         indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
@@ -585,8 +583,8 @@
             // right callback
             // (ex. what's the expected behavior when there is an error during
             // bulkload?)
-            IIndexDataflowHelperFactory indexHelperFactory =
-                    new IndexDataflowHelperFactory(storaegComponentProvider.getStorageManager(), splitsAndConstraint.first);
+            IIndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(
+                    storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
             TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad =
                     new TreeIndexBulkLoadOperatorDescriptor(spec, null, fieldPermutation,
                             GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, true, indexHelperFactory);
@@ -804,11 +802,11 @@
                     dataset.getDatasetName(), fileIndexName);
             // Create the file index data flow helper
             IIndexDataflowHelperFactory indexDataflowHelperFactory =
-                    new IndexDataflowHelperFactory(storaegComponentProvider.getStorageManager(), spPc.first);
+                    new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
             // Create the out record descriptor, appContext and fileSplitProvider for the files index
             RecordDescriptor outRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
             ISearchOperationCallbackFactory searchOpCallbackFactory = dataset
-                    .getSearchCallbackFactory(storaegComponentProvider, fileIndex, jobId, IndexOperation.SEARCH, null);
+                    .getSearchCallbackFactory(storageComponentProvider, fileIndex, jobId, IndexOperation.SEARCH, null);
             // Create the operator
             ExternalLookupOperatorDescriptor op = new ExternalLookupOperatorDescriptor(jobSpec, adapterFactory,
                     outRecDesc, indexDataflowHelperFactory, searchOpCallbackFactory,
@@ -865,7 +863,6 @@
                 context.getMissingWriterFactory());
     }
 
-
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetDataScannerRuntime(
             JobSpecification jobSpec, IAType itemType, IAdapterFactory adapterFactory, IDataFormat format)
             throws AlgebricksException {
@@ -985,9 +982,9 @@
                 primaryKeyFields[i] = i;
             }
             IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
-                    storaegComponentProvider, primaryIndex, jobId, indexOp, primaryKeyFields);
-            IIndexDataflowHelperFactory idfh =
-                    new IndexDataflowHelperFactory(storaegComponentProvider.getStorageManager(), splitsAndConstraint.first);
+                    storageComponentProvider, primaryIndex, jobId, indexOp, primaryKeyFields);
+            IIndexDataflowHelperFactory idfh = new IndexDataflowHelperFactory(
+                    storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
             IOperatorDescriptor op;
             if (bulkload) {
                 long numElementsHint = getCardinalityPerPartitionHint(dataset);
@@ -1119,9 +1116,9 @@
             // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
-                    storaegComponentProvider, secondaryIndex, jobId, indexOp, modificationCallbackPrimaryKeyFields);
-            IIndexDataflowHelperFactory idfh =
-                    new IndexDataflowHelperFactory(storaegComponentProvider.getStorageManager(), splitsAndConstraint.first);
+                    storageComponentProvider, secondaryIndex, jobId, indexOp, modificationCallbackPrimaryKeyFields);
+            IIndexDataflowHelperFactory idfh = new IndexDataflowHelperFactory(
+                    storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
             IOperatorDescriptor op;
             if (bulkload) {
                 long numElementsHint = getCardinalityPerPartitionHint(dataset);
@@ -1221,9 +1218,9 @@
             // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
-                    storaegComponentProvider, secondaryIndex, jobId, indexOp, modificationCallbackPrimaryKeyFields);
-            IIndexDataflowHelperFactory indexDataflowHelperFactory =
-                    new IndexDataflowHelperFactory(storaegComponentProvider.getStorageManager(), splitsAndConstraint.first);
+                    storageComponentProvider, secondaryIndex, jobId, indexOp, modificationCallbackPrimaryKeyFields);
+            IIndexDataflowHelperFactory indexDataflowHelperFactory = new IndexDataflowHelperFactory(
+                    storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
             IOperatorDescriptor op;
             if (bulkload) {
                 long numElementsHint = getCardinalityPerPartitionHint(dataset);
@@ -1337,9 +1334,9 @@
             // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
-                    storaegComponentProvider, secondaryIndex, jobId, indexOp, modificationCallbackPrimaryKeyFields);
-            IIndexDataflowHelperFactory indexDataFlowFactory =
-                    new IndexDataflowHelperFactory(storaegComponentProvider.getStorageManager(), splitsAndConstraint.first);
+                    storageComponentProvider, secondaryIndex, jobId, indexOp, modificationCallbackPrimaryKeyFields);
+            IIndexDataflowHelperFactory indexDataFlowFactory = new IndexDataflowHelperFactory(
+                    storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
             IOperatorDescriptor op;
             if (bulkload) {
                 long numElementsHint = getCardinalityPerPartitionHint(dataset);
@@ -1562,7 +1559,7 @@
     }
 
     public IStorageComponentProvider getStorageComponentProvider() {
-        return storaegComponentProvider;
+        return storageComponentProvider;
     }
 
     public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> getSplitProviderAndConstraints(Dataset ds)
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
index fd55815..1f0a08e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
@@ -25,8 +25,8 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 020ff6c..fa60bba 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -27,8 +27,8 @@
 import java.util.logging.Logger;
 import java.util.stream.IntStream;
 
-import org.apache.asterix.active.ActiveLifecycleListener;
 import org.apache.asterix.active.IActiveEntityEventsListener;
+import org.apache.asterix.active.IActiveNotificationHandler;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
 import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -40,8 +40,8 @@
 import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
 import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 import org.apache.asterix.common.metadata.IDataset;
-import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
+import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.common.utils.JobUtils.ProgressState;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
@@ -57,7 +57,6 @@
 import org.apache.asterix.metadata.declared.BTreeResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
-import org.apache.asterix.metadata.lock.MetadataLockManager;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.ExternalIndexingOperations;
 import org.apache.asterix.metadata.utils.IndexUtil;
@@ -67,6 +66,7 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.asterix.transaction.management.opcallbacks.LockThenSearchOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
@@ -77,7 +77,6 @@
 import org.apache.asterix.transaction.management.opcallbacks.TempDatasetPrimaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.TempDatasetSecondaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.UpsertOperationCallbackFactory;
-import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.asterix.transaction.management.resource.DatasetLocalResourceFactory;
 import org.apache.asterix.transaction.management.runtime.CommitRuntimeFactory;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
@@ -197,10 +196,12 @@
         this.rebalanceCount = rebalanceCount;
     }
 
+    @Override
     public String getDataverseName() {
         return dataverseName;
     }
 
+    @Override
     public String getDatasetName() {
         return datasetName;
     }
@@ -316,9 +317,9 @@
         Map<FeedConnectionId, Pair<JobSpecification, Boolean>> disconnectJobList = new HashMap<>();
         if (getDatasetType() == DatasetType.INTERNAL) {
             // prepare job spec(s) that would disconnect any active feeds involving the dataset.
-            ActiveLifecycleListener activeListener =
-                    (ActiveLifecycleListener) metadataProvider.getApplicationContext().getActiveLifecycleListener();
-            IActiveEntityEventsListener[] activeListeners = activeListener.getNotificationHandler().getEventListeners();
+            IActiveNotificationHandler activeListener = (IActiveNotificationHandler) metadataProvider
+                    .getApplicationContext().getActiveNotificationHandler();
+            IActiveEntityEventsListener[] activeListeners = activeListener.getEventListeners();
             for (IActiveEntityEventsListener listener : activeListeners) {
                 if (listener.isEntityUsingDataset(this)) {
                     throw new CompilationException(ErrorCode.COMPILATION_CANT_DROP_ACTIVE_DATASET,
@@ -403,7 +404,8 @@
 
         // Drops the associated nodegroup if it is no longer used by any other dataset.
         if (dropCorrespondingNodeGroup) {
-            MetadataLockManager.INSTANCE.acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodeGroupName);
+            metadataProvider.getApplicationContext().getMetadataLockManager()
+                    .acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodeGroupName);
             MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx.getValue(), nodeGroupName, true);
         }
     }
@@ -697,8 +699,8 @@
     public RecordDescriptor getPrimaryRecordDescriptor(MetadataProvider metadataProvider) throws AlgebricksException {
         List<List<String>> partitioningKeys = getPrimaryKeys();
         int numPrimaryKeys = partitioningKeys.size();
-        ISerializerDeserializer[] primaryRecFields = new ISerializerDeserializer[numPrimaryKeys + 1
-                + (hasMetaPart() ? 1 : 0)];
+        ISerializerDeserializer[] primaryRecFields =
+                new ISerializerDeserializer[numPrimaryKeys + 1 + (hasMetaPart() ? 1 : 0)];
         ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1 + (hasMetaPart() ? 1 : 0)];
         ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
         List<Integer> indicators = null;
@@ -710,9 +712,9 @@
 
         // Set the serde/traits for primary keys
         for (int i = 0; i < numPrimaryKeys; i++) {
-            IAType keyType = (indicators == null || indicators.get(i) == 0)
-                    ? itemType.getSubFieldType(partitioningKeys.get(i))
-                    : metaType.getSubFieldType(partitioningKeys.get(i));
+            IAType keyType =
+                    (indicators == null || indicators.get(i) == 0) ? itemType.getSubFieldType(partitioningKeys.get(i))
+                            : metaType.getSubFieldType(partitioningKeys.get(i));
             primaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
             primaryTypeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
         }
@@ -722,8 +724,8 @@
         primaryTypeTraits[numPrimaryKeys] = TypeTraitProvider.INSTANCE.getTypeTrait(itemType);
         if (hasMetaPart()) {
             // Set the serde and traits for the meta record field
-            primaryRecFields[numPrimaryKeys + 1] = SerializerDeserializerProvider.INSTANCE
-                    .getSerializerDeserializer(metaType);
+            primaryRecFields[numPrimaryKeys + 1] =
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(metaType);
             primaryTypeTraits[numPrimaryKeys + 1] = TypeTraitProvider.INSTANCE.getTypeTrait(itemType);
         }
         return new RecordDescriptor(primaryRecFields, primaryTypeTraits);
@@ -749,9 +751,9 @@
             indicators = ((InternalDatasetDetails) getDatasetDetails()).getKeySourceIndicator();
         }
         for (int i = 0; i < numPrimaryKeys; i++) {
-            IAType keyType = (indicators == null || indicators.get(i) == 0)
-                    ? recordType.getSubFieldType(partitioningKeys.get(i))
-                    : metaType.getSubFieldType(partitioningKeys.get(i));
+            IAType keyType =
+                    (indicators == null || indicators.get(i) == 0) ? recordType.getSubFieldType(partitioningKeys.get(i))
+                            : metaType.getSubFieldType(partitioningKeys.get(i));
             cmpFactories[i] = cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
         }
         return cmpFactories;
@@ -802,8 +804,8 @@
 
     // Gets an array of partition numbers for this dataset.
     protected int[] getDatasetPartitions(MetadataProvider metadataProvider) throws AlgebricksException {
-        FileSplit[] splitsForDataset = metadataProvider.splitsForIndex(metadataProvider.getMetadataTxnContext(), this,
-                getDatasetName());
+        FileSplit[] splitsForDataset =
+                metadataProvider.splitsForIndex(metadataProvider.getMetadataTxnContext(), this, getDatasetName());
         return IntStream.range(0, splitsForDataset.length).toArray();
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FeedConnection.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FeedConnection.java
index b05e61e..151c9ca 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FeedConnection.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FeedConnection.java
@@ -24,9 +24,7 @@
 import org.apache.asterix.external.util.FeedUtils;
 import org.apache.asterix.metadata.MetadataCache;
 import org.apache.asterix.metadata.api.IMetadataEntity;
-import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
 
-import java.util.ArrayList;
 import java.util.List;
 
 /**
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index 57f0a07..bf19b03 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -275,4 +275,9 @@
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, indexType);
         }
     }
+
+    @Override
+    public String toString() {
+        return dataverseName + '.' + datasetName + '.' + indexName;
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
index 2ca7215..3c409da 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
@@ -23,8 +23,8 @@
 import java.io.DataInput;
 import java.io.DataInputStream;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.CompactionPolicy;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index b9464ed..8da34c0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -37,9 +37,9 @@
 import org.apache.asterix.builders.UnorderedListBuilder;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.TransactionState;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.IDatasetDetails;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.Dataset;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
index 7f4e28d..9cd73d7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
@@ -24,10 +24,10 @@
 import java.io.DataInputStream;
 import java.util.Calendar;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.external.api.IDataSourceAdapter;
 import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index 44b5548..fe0b40d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -29,9 +29,9 @@
 import org.apache.asterix.builders.IARecordBuilder;
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
index 89aaad3..4b20a5b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
@@ -24,8 +24,8 @@
 import java.io.DataInputStream;
 import java.util.Calendar;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.Dataverse;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
index ea04f1d..a29048e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
@@ -24,9 +24,9 @@
 import java.util.Date;
 
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.om.base.ADateTime;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java
index 420e4fc..2a2387d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java
@@ -27,9 +27,9 @@
 
 import org.apache.asterix.builders.IARecordBuilder;
 import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.FeedConnection;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
index 7b631b6..8ddac7a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
@@ -29,8 +29,8 @@
 import org.apache.asterix.builders.IARecordBuilder;
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
index 4737d79..2541e97 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
@@ -30,8 +30,8 @@
 import org.apache.asterix.builders.IARecordBuilder;
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.Feed;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
index 7cbc6ac..eed082c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
@@ -26,8 +26,8 @@
 import java.util.List;
 
 import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.Function;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index 7b1d9a6..fec4ddc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -29,9 +29,9 @@
 
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
index 1d8cff8..9c2b824 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
@@ -24,8 +24,8 @@
 import java.io.DataInputStream;
 import java.util.Calendar;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.Library;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java
index c2cdf0d..5329df4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java
@@ -27,8 +27,8 @@
 import java.util.List;
 
 import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.NodeGroup;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
index dae11bc..b1847fd 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
@@ -19,8 +19,8 @@
 
 package org.apache.asterix.metadata.entitytupletranslators;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.Node;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java
index dfa00ab..41b356a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java
@@ -20,13 +20,20 @@
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 
 public class BuiltinFeedPolicies {
 
+    private static final Logger LOGGER = Logger.getLogger(BuiltinFeedPolicies.class.getName());
+
     public static final FeedPolicyEntity BASIC = initializeBasicPolicy();
 
     public static final FeedPolicyEntity ADVANCED_FT_DISCARD = initializeAdvancedFTDiscardPolicy();
@@ -35,15 +42,18 @@
 
     public static final FeedPolicyEntity ELASTIC = initializeAdvancedFTElasticPolicy();
 
-    public static final FeedPolicyEntity[] policies = new FeedPolicyEntity[] { BASIC, ADVANCED_FT_DISCARD,
-            ADVANCED_FT_SPILL, ELASTIC };
+    private static final FeedPolicyEntity[] POLICIES =
+            new FeedPolicyEntity[] { BASIC, ADVANCED_FT_DISCARD, ADVANCED_FT_SPILL, ELASTIC };
 
     public static final FeedPolicyEntity DEFAULT_POLICY = BASIC;
 
     public static final String CONFIG_FEED_POLICY_KEY = "policy";
 
+    private BuiltinFeedPolicies() {
+    }
+
     public static FeedPolicyEntity getFeedPolicy(String policyName) {
-        for (FeedPolicyEntity policy : policies) {
+        for (FeedPolicyEntity policy : POLICIES) {
             if (policy.getPolicyName().equalsIgnoreCase(policyName)) {
                 return policy;
             }
@@ -53,8 +63,8 @@
 
     //Basic
     private static FeedPolicyEntity initializeBasicPolicy() {
-        Map<String, String> policyParams = new HashMap<String, String>();
-        policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+        Map<String, String> policyParams = new HashMap<>();
+        policyParams.put(FeedPolicyAccessor.ELASTIC, Boolean.toString(false));
 
         String description = "Basic";
         return new FeedPolicyEntity(MetadataConstants.METADATA_DATAVERSE_NAME, "Basic", description, policyParams);
@@ -62,40 +72,47 @@
 
     // Discard
     private static FeedPolicyEntity initializeAdvancedFTDiscardPolicy() {
-        Map<String, String> policyParams = new HashMap<String, String>();
-        policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
-        policyParams.put(FeedPolicyAccessor.FLOWCONTROL_ENABLED, "true");
-        policyParams.put(FeedPolicyAccessor.MAX_SPILL_SIZE_ON_DISK, "false");
+        Map<String, String> policyParams = new HashMap<>();
+        policyParams.put(FeedPolicyAccessor.ELASTIC, Boolean.toString(false));
+        policyParams.put(FeedPolicyAccessor.FLOWCONTROL_ENABLED, Boolean.toString(true));
+        policyParams.put(FeedPolicyAccessor.MAX_SPILL_SIZE_ON_DISK, Boolean.toString(false));
         policyParams.put(FeedPolicyAccessor.MAX_FRACTION_DISCARD, "100");
-        policyParams.put(FeedPolicyAccessor.LOGGING_STATISTICS, "true");
+        policyParams.put(FeedPolicyAccessor.LOGGING_STATISTICS, Boolean.toString(true));
 
         String description = "FlowControl 100% Discard during congestion";
-        return new FeedPolicyEntity(MetadataConstants.METADATA_DATAVERSE_NAME, "Discard", description,
-                policyParams);
+        return new FeedPolicyEntity(MetadataConstants.METADATA_DATAVERSE_NAME, "Discard", description, policyParams);
     }
 
     // Spill
     private static FeedPolicyEntity initializeAdvancedFTSpillPolicy() {
-        Map<String, String> policyParams = new HashMap<String, String>();
-        policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
-        policyParams.put(FeedPolicyAccessor.FLOWCONTROL_ENABLED, "true");
+        Map<String, String> policyParams = new HashMap<>();
+        policyParams.put(FeedPolicyAccessor.ELASTIC, Boolean.toString(false));
+        policyParams.put(FeedPolicyAccessor.FLOWCONTROL_ENABLED, Boolean.toString(true));
         policyParams.put(FeedPolicyAccessor.SPILL_TO_DISK_ON_CONGESTION, "" + Boolean.TRUE);
-        policyParams.put(FeedPolicyAccessor.MAX_SPILL_SIZE_ON_DISK, "" + FeedPolicyAccessor.NO_LIMIT);
+        policyParams.put(FeedPolicyAccessor.MAX_SPILL_SIZE_ON_DISK, "" + Long.toString(FeedPolicyAccessor.NO_LIMIT));
 
         String description = "FlowControl 100% Spill during congestion";
-        return new FeedPolicyEntity(MetadataConstants.METADATA_DATAVERSE_NAME, "Spill", description,
-                policyParams);
+        return new FeedPolicyEntity(MetadataConstants.METADATA_DATAVERSE_NAME, "Spill", description, policyParams);
     }
 
     // AdvancedFT_Elastic
     private static FeedPolicyEntity initializeAdvancedFTElasticPolicy() {
-        Map<String, String> policyParams = new HashMap<String, String>();
-        policyParams.put(FeedPolicyAccessor.ELASTIC, "true");
-        policyParams.put(FeedPolicyAccessor.FLOWCONTROL_ENABLED, "true");
-        policyParams.put(FeedPolicyAccessor.LOGGING_STATISTICS, "true");
+        Map<String, String> policyParams = new HashMap<>();
+        policyParams.put(FeedPolicyAccessor.ELASTIC, Boolean.toString(true));
+        policyParams.put(FeedPolicyAccessor.FLOWCONTROL_ENABLED, Boolean.toString(true));
+        policyParams.put(FeedPolicyAccessor.LOGGING_STATISTICS, Boolean.toString(true));
         String description = "Basic Monitored Fault-Tolerant Elastic";
         return new FeedPolicyEntity(MetadataConstants.METADATA_DATAVERSE_NAME, "AdvancedFT_Elastic", description,
                 policyParams);
     }
 
+    public static void insertInitialFeedPolicies(MetadataTransactionContext mdTxnCtx) throws MetadataException {
+        for (FeedPolicyEntity feedPolicy : BuiltinFeedPolicies.POLICIES) {
+            MetadataManager.INSTANCE.addFeedPolicy(mdTxnCtx, feedPolicy);
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Finished adding built-in feed policies.");
+        }
+    }
+
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 94b36ed..3f9c002 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.api.IDataSourceAdapter;
 import org.apache.asterix.external.api.IDataSourceAdapter.AdapterType;
@@ -36,7 +37,6 @@
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -59,8 +59,11 @@
  */
 public class FeedMetadataUtil {
 
+    private FeedMetadataUtil() {
+    }
+
     public static Dataset validateIfDatasetExists(MetadataProvider metadataProvider, String dataverse,
-            String datasetName, MetadataTransactionContext ctx) throws AlgebricksException {
+            String datasetName) throws AlgebricksException {
         Dataset dataset = metadataProvider.findDataset(dataverse, datasetName);
         if (dataset == null) {
             throw new CompilationException("Unknown target dataset :" + datasetName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
index 9cc9d4a..c0d9bda 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -23,7 +23,7 @@
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.asterix.metadata.MetadataException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.entities.Datatype;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/DatasetLock.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/DatasetLock.java
index 542dd72..31f2089 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/DatasetLock.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/DatasetLock.java
@@ -21,65 +21,83 @@
 import java.util.Objects;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.MetadataException;
+import org.apache.asterix.common.metadata.IMetadataLock;
+import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 public class DatasetLock implements IMetadataLock {
 
     private final String key;
-    private final ReentrantReadWriteLock dsLock;
+    private final ReentrantReadWriteLock lock;
+    private final ReentrantReadWriteLock dsReadLock;
     private final ReentrantReadWriteLock dsModifyLock;
-    private final AMutableInt32 indexBuildCounter;
+    private final MutableInt indexBuildCounter;
 
     public DatasetLock(String key) {
         this.key = key;
-        dsLock = new ReentrantReadWriteLock(true);
+        lock = new ReentrantReadWriteLock(true);
+        dsReadLock = new ReentrantReadWriteLock(true);
         dsModifyLock = new ReentrantReadWriteLock(true);
-        indexBuildCounter = new AMutableInt32(0);
+        indexBuildCounter = new MutableInt(0);
     }
 
-    private void acquireReadLock() {
+    private void readLock() {
         // query
         // build index
         // insert
-        dsLock.readLock().lock();
+        lock.readLock().lock();
     }
 
-    private void releaseReadLock() {
+    private void readUnlock() {
         // query
         // build index
         // insert
-        dsLock.readLock().unlock();
+        lock.readLock().unlock();
     }
 
-    private void acquireWriteLock() {
+    private void writeLock() {
         // create ds
         // delete ds
         // drop index
-        dsLock.writeLock().lock();
+        lock.writeLock().lock();
     }
 
-    private void releaseWriteLock() {
+    private void writeUnlock() {
         // create ds
         // delete ds
         // drop index
-        dsLock.writeLock().unlock();
+        lock.writeLock().unlock();
     }
 
-    private void acquireReadModifyLock() {
+    private void readReadLock() {
+        dsReadLock.readLock().lock();
+    }
+
+    private void modifyReadLock() {
         // insert
         dsModifyLock.readLock().lock();
     }
 
-    private void releaseReadModifyLock() {
+    private void modifyReadUnlock() {
         // insert
         dsModifyLock.readLock().unlock();
     }
 
-    private void acquireWriteModifyLock() {
+    private void readReadUnlock() {
+        dsReadLock.readLock().unlock();
+    }
+
+    private void readWriteUnlock() {
+        dsReadLock.writeLock().unlock();
+    }
+
+    private void modifySharedWriteLock() {
         // Build index statement
         synchronized (indexBuildCounter) {
-            if (indexBuildCounter.getIntegerValue() > 0) {
-                indexBuildCounter.setValue(indexBuildCounter.getIntegerValue() + 1);
+            if (indexBuildCounter.getValue() > 0) {
+                indexBuildCounter.setValue(indexBuildCounter.getValue() + 1);
             } else {
                 dsModifyLock.writeLock().lock();
                 indexBuildCounter.setValue(1);
@@ -87,73 +105,104 @@
         }
     }
 
-    private void releaseWriteModifyLock() {
+    private void modifySharedWriteUnlock() {
         // Build index statement
         synchronized (indexBuildCounter) {
-            if (indexBuildCounter.getIntegerValue() == 1) {
+            if (indexBuildCounter.getValue() == 1) {
                 dsModifyLock.writeLock().unlock();
             }
-            indexBuildCounter.setValue(indexBuildCounter.getIntegerValue() - 1);
+            indexBuildCounter.setValue(indexBuildCounter.getValue() - 1);
         }
     }
 
-    private void acquireRefreshLock() {
-        // Refresh External Dataset statement
+    private void modifyExclusiveWriteLock() {
         dsModifyLock.writeLock().lock();
     }
 
-    private void releaseRefreshLock() {
-        // Refresh External Dataset statement
+    private void modifyExclusiveWriteUnlock() {
         dsModifyLock.writeLock().unlock();
     }
 
     @Override
-    public void acquire(IMetadataLock.Mode mode) {
-        switch (mode) {
-            case INDEX_BUILD:
-                acquireReadLock();
-                acquireWriteModifyLock();
-                break;
-            case MODIFY:
-                acquireReadLock();
-                acquireReadModifyLock();
-                break;
-            case REFRESH:
-                acquireReadLock();
-                acquireRefreshLock();
-                break;
-            case INDEX_DROP:
-            case WRITE:
-                acquireWriteLock();
-                break;
-            default:
-                acquireReadLock();
-                break;
+    public void upgrade(IMetadataLock.Mode from, IMetadataLock.Mode to) throws AlgebricksException {
+        if (from == IMetadataLock.Mode.EXCLUSIVE_MODIFY && to == IMetadataLock.Mode.UPGRADED_WRITE) {
+            dsReadLock.readLock().unlock();
+            dsReadLock.writeLock().lock();
+        } else {
+            throw new MetadataException(ErrorCode.ILLEGAL_LOCK_UPGRADE_OPERATION, from, to);
         }
     }
 
     @Override
-    public void release(IMetadataLock.Mode mode) {
+    public void downgrade(IMetadataLock.Mode from, IMetadataLock.Mode to) throws AlgebricksException {
+        if (from == IMetadataLock.Mode.UPGRADED_WRITE && to == IMetadataLock.Mode.EXCLUSIVE_MODIFY) {
+            dsReadLock.writeLock().unlock();
+            dsReadLock.readLock().lock();
+        } else {
+            throw new MetadataException(ErrorCode.ILLEGAL_LOCK_DOWNGRADE_OPERATION, from, to);
+        }
+    }
+
+    @Override
+    public void lock(IMetadataLock.Mode mode) {
         switch (mode) {
             case INDEX_BUILD:
-                releaseWriteModifyLock();
-                releaseReadLock();
+                readLock();
+                modifySharedWriteLock();
                 break;
             case MODIFY:
-                releaseReadModifyLock();
-                releaseReadLock();
+                readLock();
+                readReadLock();
+                modifyReadLock();
                 break;
-            case REFRESH:
-                releaseRefreshLock();
-                releaseReadLock();
+            case EXCLUSIVE_MODIFY:
+                readLock();
+                readReadLock();
+                modifyExclusiveWriteLock();
                 break;
-            case INDEX_DROP:
             case WRITE:
-                releaseWriteLock();
+                writeLock();
+                break;
+            case READ:
+                readLock();
+                readReadLock();
                 break;
             default:
-                releaseReadLock();
+                throw new IllegalStateException("locking mode " + mode + " is not supported");
+        }
+    }
+
+    @Override
+    public void unlock(IMetadataLock.Mode mode) {
+        switch (mode) {
+            case INDEX_BUILD:
+                modifySharedWriteUnlock();
+                readUnlock();
                 break;
+            case MODIFY:
+                modifyReadUnlock();
+                readReadUnlock();
+                readUnlock();
+                break;
+            case EXCLUSIVE_MODIFY:
+                modifyExclusiveWriteUnlock();
+                readReadUnlock();
+                readUnlock();
+                break;
+            case WRITE:
+                writeUnlock();
+                break;
+            case READ:
+                readReadUnlock();
+                readUnlock();
+                break;
+            case UPGRADED_WRITE:
+                readWriteUnlock();
+                modifyExclusiveWriteUnlock();
+                readUnlock();
+                break;
+            default:
+                throw new IllegalStateException("unlocking mode " + mode + " is not supported");
         }
     }
 
@@ -177,4 +226,9 @@
         }
         return Objects.equals(key, ((DatasetLock) o).key);
     }
+
+    @Override
+    public String toString() {
+        return key;
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/IMetadataLock.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/IMetadataLock.java
deleted file mode 100644
index 4d7fa93..0000000
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/IMetadataLock.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.asterix.metadata.lock;
-
-/**
- * A Metadata lock local to compilation node
- */
-public interface IMetadataLock {
-
-    enum Mode {
-        READ,
-        MODIFY,
-        REFRESH,
-        INDEX_BUILD,
-        INDEX_DROP,
-        WRITE
-    }
-
-    /**
-     * Acquire a lock
-     *
-     * @param mode
-     *            lock mode
-     */
-    void acquire(IMetadataLock.Mode mode);
-
-    /**
-     * Release a lock
-     *
-     * @param mode
-     *            lock mode
-     */
-    void release(IMetadataLock.Mode mode);
-
-    /**
-     * Get the lock's key
-     *
-     * @return the key identiying the lock
-     */
-    String getKey();
-}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/LockList.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/LockList.java
deleted file mode 100644
index c5c9fe9..0000000
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/LockList.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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.asterix.metadata.lock;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * The LockList is used for two phase locking.
- */
-public class LockList {
-    private final List<Pair<IMetadataLock.Mode, IMetadataLock>> locks = new ArrayList<>();
-    private final Set<String> lockSet = new HashSet<>();
-    private boolean lockPhase = true;
-
-    /**
-     * Acquires a lock.
-     *
-     * @param mode
-     *            the lock mode.
-     * @param lock
-     *            the lock object.
-     */
-    public void add(IMetadataLock.Mode mode, IMetadataLock lock) throws AsterixException {
-        if (!lockPhase) {
-            throw new AsterixException(ErrorCode.COMPILATION_TWO_PHASE_LOCKING_VIOLATION);
-        }
-        if (lockSet.contains(lock.getKey())) {
-            return;
-        }
-        lock.acquire(mode);
-        locks.add(Pair.of(mode, lock));
-        lockSet.add(lock.getKey());
-    }
-
-    /**
-     * Once unlock() is called, no caller can call add(IMetadataLock.Mode mode, IMetadataLock lock),
-     * except that reset() is called.
-     */
-    public void unlock() {
-        for (int i = locks.size() - 1; i >= 0; i--) {
-            Pair<IMetadataLock.Mode, IMetadataLock> pair = locks.get(i);
-            pair.getRight().release(pair.getLeft());
-        }
-        locks.clear();
-        lockSet.clear();
-        lockPhase = false;
-    }
-
-    /**
-     * Clears the state and starts another pass of two phase locking again.
-     */
-    public void reset() {
-        unlock();
-        lockPhase = true;
-    }
-}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLock.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLock.java
index 6c20e9f..48f315b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLock.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLock.java
@@ -21,6 +21,8 @@
 import java.util.Objects;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import org.apache.asterix.common.metadata.IMetadataLock;
+
 public class MetadataLock implements IMetadataLock {
     private final String key;
     private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
@@ -30,7 +32,7 @@
     }
 
     @Override
-    public void acquire(IMetadataLock.Mode mode) {
+    public void lock(IMetadataLock.Mode mode) {
         switch (mode) {
             case WRITE:
                 lock.writeLock().lock();
@@ -42,7 +44,7 @@
     }
 
     @Override
-    public void release(IMetadataLock.Mode mode) {
+    public void unlock(IMetadataLock.Mode mode) {
         switch (mode) {
             case WRITE:
                 lock.writeLock().unlock();
@@ -73,4 +75,9 @@
         }
         return Objects.equals(key, ((MetadataLock) o).key);
     }
+
+    @Override
+    public String toString() {
+        return key;
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
index 43d72e3..779fe2a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
@@ -18,17 +18,17 @@
  */
 package org.apache.asterix.metadata.lock;
 
-import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.Function;
 
+import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.metadata.entities.FeedConnection;
-import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.common.metadata.IMetadataLock;
+import org.apache.asterix.common.metadata.LockList;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
-public class MetadataLockManager {
+public class MetadataLockManager implements IMetadataLockManager {
 
-    public static final MetadataLockManager INSTANCE = new MetadataLockManager();
     private static final Function<String, MetadataLock> LOCK_FUNCTION = MetadataLock::new;
     private static final Function<String, DatasetLock> DATASET_LOCK_FUNCTION = DatasetLock::new;
 
@@ -38,286 +38,175 @@
     private static final String DATASET_PREFIX = "Dataset:";
     private static final String FUNCTION_PREFIX = "Function:";
     private static final String NODE_GROUP_PREFIX = "NodeGroup:";
-    private static final String FEED_PREFIX = "Feed:";
+    private static final String ACTIVE_PREFIX = "Active:";
     private static final String FEED_POLICY_PREFIX = "FeedPolicy:";
     private static final String MERGE_POLICY_PREFIX = "MergePolicy:";
     private static final String DATATYPE_PREFIX = "DataType:";
     private static final String EXTENSION_PREFIX = "Extension:";
 
-    private MetadataLockManager() {
+    public MetadataLockManager() {
         mdlocks = new ConcurrentHashMap<>();
     }
 
-    // Dataverse
+    @Override
     public void acquireDataverseReadLock(LockList locks, String dataverseName) throws AsterixException {
         String key = DATAVERSE_PREFIX + dataverseName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.READ, lock);
     }
 
+    @Override
     public void acquireDataverseWriteLock(LockList locks, String dataverseName) throws AsterixException {
         String key = DATAVERSE_PREFIX + dataverseName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.WRITE, lock);
     }
 
-    // Dataset
+    @Override
     public void acquireDatasetReadLock(LockList locks, String datasetName) throws AsterixException {
         String key = DATASET_PREFIX + datasetName;
         DatasetLock lock = (DatasetLock) mdlocks.computeIfAbsent(key, DATASET_LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.READ, lock);
     }
 
+    @Override
     public void acquireDatasetWriteLock(LockList locks, String datasetName) throws AsterixException {
         String key = DATASET_PREFIX + datasetName;
         DatasetLock lock = (DatasetLock) mdlocks.computeIfAbsent(key, DATASET_LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.WRITE, lock);
     }
 
+    @Override
     public void acquireDatasetModifyLock(LockList locks, String datasetName) throws AsterixException {
         String key = DATASET_PREFIX + datasetName;
         DatasetLock lock = (DatasetLock) mdlocks.computeIfAbsent(key, DATASET_LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.MODIFY, lock);
     }
 
+    @Override
     public void acquireDatasetCreateIndexLock(LockList locks, String datasetName) throws AsterixException {
-        String key = DATASET_PREFIX + datasetName;
-        DatasetLock lock = (DatasetLock) mdlocks.computeIfAbsent(key, DATASET_LOCK_FUNCTION);
+        String dsKey = DATASET_PREFIX + datasetName;
+        DatasetLock lock = (DatasetLock) mdlocks.computeIfAbsent(dsKey, DATASET_LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.INDEX_BUILD, lock);
     }
 
-    public void acquireExternalDatasetRefreshLock(LockList locks, String datasetName) throws AsterixException {
+    @Override
+    public void acquireDatasetExclusiveModificationLock(LockList locks, String datasetName) throws AsterixException {
         String key = DATASET_PREFIX + datasetName;
         DatasetLock lock = (DatasetLock) mdlocks.computeIfAbsent(key, DATASET_LOCK_FUNCTION);
-        locks.add(IMetadataLock.Mode.INDEX_BUILD, lock);
+        locks.add(IMetadataLock.Mode.EXCLUSIVE_MODIFY, lock);
     }
 
-    // Function
+    @Override
     public void acquireFunctionReadLock(LockList locks, String functionName) throws AsterixException {
         String key = FUNCTION_PREFIX + functionName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.READ, lock);
     }
 
+    @Override
     public void acquireFunctionWriteLock(LockList locks, String functionName) throws AsterixException {
         String key = FUNCTION_PREFIX + functionName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.WRITE, lock);
     }
 
-    // Node Group
+    @Override
     public void acquireNodeGroupReadLock(LockList locks, String nodeGroupName) throws AsterixException {
         String key = NODE_GROUP_PREFIX + nodeGroupName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.READ, lock);
     }
 
+    @Override
     public void acquireNodeGroupWriteLock(LockList locks, String nodeGroupName) throws AsterixException {
         String key = NODE_GROUP_PREFIX + nodeGroupName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.WRITE, lock);
     }
 
-    // Feeds
-    public void acquireFeedReadLock(LockList locks, String feedName) throws AsterixException {
-        String key = FEED_PREFIX + feedName;
+    @Override
+    public void acquireActiveEntityReadLock(LockList locks, String entityName) throws AsterixException {
+        String key = ACTIVE_PREFIX + entityName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.READ, lock);
     }
 
-    public void acquireFeedWriteLock(LockList locks, String feedName) throws AsterixException {
-        String key = FEED_PREFIX + feedName;
+    @Override
+    public void acquireActiveEntityWriteLock(LockList locks, String entityName) throws AsterixException {
+        String key = ACTIVE_PREFIX + entityName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.WRITE, lock);
     }
 
+    @Override
     public void acquireFeedPolicyWriteLock(LockList locks, String feedPolicyName) throws AsterixException {
         String key = FEED_POLICY_PREFIX + feedPolicyName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.WRITE, lock);
     }
 
+    @Override
     public void acquireFeedPolicyReadLock(LockList locks, String feedPolicyName) throws AsterixException {
         String key = FEED_POLICY_PREFIX + feedPolicyName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.READ, lock);
     }
 
-    // CompactionPolicy
+    @Override
     public void acquireMergePolicyReadLock(LockList locks, String mergePolicyName) throws AsterixException {
         String key = MERGE_POLICY_PREFIX + mergePolicyName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.READ, lock);
     }
 
+    @Override
     public void acquireMergePolicyWriteLock(LockList locks, String mergePolicyName) throws AsterixException {
         String key = MERGE_POLICY_PREFIX + mergePolicyName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.WRITE, lock);
     }
 
-    // DataType
+    @Override
     public void acquireDataTypeReadLock(LockList locks, String datatypeName) throws AsterixException {
         String key = DATATYPE_PREFIX + datatypeName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.READ, lock);
     }
 
+    @Override
     public void acquireDataTypeWriteLock(LockList locks, String datatypeName) throws AsterixException {
         String key = DATATYPE_PREFIX + datatypeName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.WRITE, lock);
     }
 
-    // Extensions
-    public void acquireExtensionReadLock(LockList locks, String extensionName) throws AsterixException {
-        String key = EXTENSION_PREFIX + extensionName;
+    @Override
+    public void acquireExtensionReadLock(LockList locks, String extension, String entityName) throws AsterixException {
+        String key = EXTENSION_PREFIX + extension + entityName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.READ, lock);
     }
 
-    public void acquireExtensionWriteLock(LockList locks, String extensionName) throws AsterixException {
-        String key = EXTENSION_PREFIX + extensionName;
+    @Override
+    public void acquireExtensionWriteLock(LockList locks, String extension, String entityName) throws AsterixException {
+        String key = EXTENSION_PREFIX + extension + entityName;
         IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
         locks.add(IMetadataLock.Mode.WRITE, lock);
     }
 
-    public void createDatasetBegin(LockList locks, String dataverseName, String itemTypeDataverseName,
-            String itemTypeFullyQualifiedName, String metaItemTypeDataverseName, String metaItemTypeFullyQualifiedName,
-            String nodeGroupName, String compactionPolicyName, String datasetFullyQualifiedName,
-            boolean isDefaultCompactionPolicy) throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        if (!dataverseName.equals(itemTypeDataverseName)) {
-            acquireDataverseReadLock(locks, itemTypeDataverseName);
-        }
-        if (metaItemTypeDataverseName != null && !metaItemTypeDataverseName.equals(dataverseName)
-                && !metaItemTypeDataverseName.equals(itemTypeDataverseName)) {
-            acquireDataverseReadLock(locks, metaItemTypeDataverseName);
-        }
-        acquireDataTypeReadLock(locks, itemTypeFullyQualifiedName);
-        if (metaItemTypeFullyQualifiedName != null
-                && !metaItemTypeFullyQualifiedName.equals(itemTypeFullyQualifiedName)) {
-            acquireDataTypeReadLock(locks, metaItemTypeFullyQualifiedName);
-        }
-        if (nodeGroupName != null) {
-            acquireNodeGroupReadLock(locks, nodeGroupName);
-        }
-        if (!isDefaultCompactionPolicy) {
-            acquireMergePolicyReadLock(locks, compactionPolicyName);
-        }
-        acquireDatasetWriteLock(locks, datasetFullyQualifiedName);
+    @Override
+    public void upgradeDatasetLockToWrite(LockList locks, String fullyQualifiedName) throws AlgebricksException {
+        String key = DATASET_PREFIX + fullyQualifiedName;
+        DatasetLock lock = (DatasetLock) mdlocks.computeIfAbsent(key, DATASET_LOCK_FUNCTION);
+        locks.upgrade(IMetadataLock.Mode.UPGRADED_WRITE, lock);
     }
 
-    public void createIndexBegin(LockList locks, String dataverseName, String datasetFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireDatasetCreateIndexLock(locks, datasetFullyQualifiedName);
-    }
-
-    public void createTypeBegin(LockList locks, String dataverseName, String itemTypeFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireDataTypeWriteLock(locks, itemTypeFullyQualifiedName);
-    }
-
-    public void dropDatasetBegin(LockList locks, String dataverseName, String datasetFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireDatasetWriteLock(locks, datasetFullyQualifiedName);
-    }
-
-    public void dropIndexBegin(LockList locks, String dataverseName, String datasetFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireDatasetWriteLock(locks, datasetFullyQualifiedName);
-    }
-
-    public void dropTypeBegin(LockList locks, String dataverseName, String dataTypeFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireDataTypeWriteLock(locks, dataTypeFullyQualifiedName);
-    }
-
-    public void functionStatementBegin(LockList locks, String dataverseName, String functionFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireFunctionWriteLock(locks, functionFullyQualifiedName);
-    }
-
-    public void modifyDatasetBegin(LockList locks, String dataverseName, String datasetFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireDatasetModifyLock(locks, datasetFullyQualifiedName);
-    }
-
-    public void insertDeleteUpsertBegin(LockList locks, String datasetFullyQualifiedName) throws AsterixException {
-        acquireDataverseReadLock(locks, DatasetUtil.getDataverseFromFullyQualifiedName(datasetFullyQualifiedName));
-        acquireDatasetModifyLock(locks, datasetFullyQualifiedName);
-    }
-
-    public void dropFeedBegin(LockList locks, String dataverseName, String feedFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireFeedWriteLock(locks, feedFullyQualifiedName);
-    }
-
-    public void dropFeedPolicyBegin(LockList locks, String dataverseName, String policyName) throws AsterixException {
-        acquireFeedWriteLock(locks, policyName);
-        acquireDataverseReadLock(locks, dataverseName);
-    }
-
-    public void startFeedBegin(LockList locks, String dataverseName, String feedName,
-            List<FeedConnection> feedConnections) throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireFeedReadLock(locks, feedName);
-        for (FeedConnection feedConnection : feedConnections) {
-            // what if the dataset is in a different dataverse
-            String fqName = dataverseName + "." + feedConnection.getDatasetName();
-            acquireDatasetReadLock(locks, fqName);
-        }
-    }
-
-    public void stopFeedBegin(LockList locks, String dataverseName, String feedName) throws AsterixException {
-        // TODO: dataset lock?
-        // Dataset locks are not required here since datasets are protected by the active event listener
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireFeedReadLock(locks, feedName);
-    }
-
-    public void createFeedBegin(LockList locks, String dataverseName, String feedFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireFeedWriteLock(locks, feedFullyQualifiedName);
-    }
-
-    public void connectFeedBegin(LockList locks, String dataverseName, String datasetFullyQualifiedName,
-            String feedFullyQualifiedName) throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireDatasetReadLock(locks, datasetFullyQualifiedName);
-        acquireFeedReadLock(locks, feedFullyQualifiedName);
-    }
-
-    public void createFeedPolicyBegin(LockList locks, String dataverseName, String policyName) throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireFeedPolicyWriteLock(locks, policyName);
-    }
-
-    public void disconnectFeedBegin(LockList locks, String dataverseName, String datasetFullyQualifiedName,
-            String feedFullyQualifiedName) throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireDatasetReadLock(locks, datasetFullyQualifiedName);
-        acquireFeedReadLock(locks, feedFullyQualifiedName);
-    }
-
-    public void compactBegin(LockList locks, String dataverseName, String datasetFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireDatasetReadLock(locks, datasetFullyQualifiedName);
-    }
-
-    public void refreshDatasetBegin(LockList locks, String dataverseName, String datasetFullyQualifiedName)
-            throws AsterixException {
-        acquireDataverseReadLock(locks, dataverseName);
-        acquireExternalDatasetRefreshLock(locks, datasetFullyQualifiedName);
+    @Override
+    public void downgradeDatasetLockToExclusiveModify(LockList locks, String fullyQualifiedName)
+            throws AlgebricksException {
+        String key = DATASET_PREFIX + fullyQualifiedName;
+        DatasetLock lock = (DatasetLock) mdlocks.computeIfAbsent(key, DATASET_LOCK_FUNCTION);
+        locks.downgrade(IMetadataLock.Mode.EXCLUSIVE_MODIFY, lock);
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index 154e1b5..e4a6ca8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -34,14 +34,15 @@
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.context.ITransactionSubsystemProvider;
 import org.apache.asterix.common.context.TransactionSubsystemProvider;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -51,7 +52,6 @@
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.NodeGroup;
-import org.apache.asterix.metadata.lock.MetadataLockManager;
 import org.apache.asterix.om.base.AMutableString;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.types.ARecordType;
@@ -283,22 +283,22 @@
             metaItemType = (ARecordType) metadataProvider.findMetaType(dataset);
         }
         JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
-                .getSplitProviderAndConstraints(dataset);
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(dataset);
         FileSplit[] fs = splitsAndConstraint.first.getFileSplits();
         StringBuilder sb = new StringBuilder();
         for (int i = 0; i < fs.length; i++) {
             sb.append(fs[i] + " ");
         }
         LOGGER.info("CREATING File Splits: " + sb.toString());
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtil.getMergePolicyFactory(dataset,
-                metadataProvider.getMetadataTxnContext());
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         IResourceFactory resourceFactory = dataset.getResourceFactory(metadataProvider, index, itemType, metaItemType,
                 compactionInfo.first, compactionInfo.second);
-        IndexBuilderFactory indexBuilderFactory = new IndexBuilderFactory(
-                metadataProvider.getStorageComponentProvider().getStorageManager(), splitsAndConstraint.first,
-                resourceFactory, !dataset.isTemp());
+        IndexBuilderFactory indexBuilderFactory =
+                new IndexBuilderFactory(metadataProvider.getStorageComponentProvider().getStorageManager(),
+                        splitsAndConstraint.first, resourceFactory, !dataset.isTemp());
         IndexCreateOperatorDescriptor indexCreateOp = new IndexCreateOperatorDescriptor(spec, indexBuilderFactory);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
                 splitsAndConstraint.second);
@@ -344,8 +344,8 @@
      */
     public static IOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec, MetadataProvider metadataProvider,
             Dataset dataset, JobId jobId) throws AlgebricksException {
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
-                .getSplitProviderAndConstraints(dataset);
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(dataset);
         IFileSplitProvider primaryFileSplitProvider = primarySplitsAndConstraint.first;
         AlgebricksPartitionConstraint primaryPartitionConstraint = primarySplitsAndConstraint.second;
         // -Infinity
@@ -396,8 +396,8 @@
         try {
             Index primaryIndex = metadataProvider.getIndex(dataset.getDataverseName(), dataset.getDatasetName(),
                     dataset.getDatasetName());
-            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
-                    .getSplitProviderAndConstraints(dataset);
+            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                    metadataProvider.getSplitProviderAndConstraints(dataset);
 
             // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -405,8 +405,8 @@
             for (int i = 0; i < numKeys; i++) {
                 primaryKeyFields[i] = i;
             }
-            boolean hasSecondaries = metadataProvider
-                    .getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName()).size() > 1;
+            boolean hasSecondaries =
+                    metadataProvider.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName()).size() > 1;
             IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
             IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
                     storageComponentProvider, primaryIndex, jobId, IndexOperation.UPSERT, primaryKeyFields);
@@ -426,8 +426,8 @@
             f++;
             // add the previous meta second
             if (dataset.hasMetaPart()) {
-                outputSerDes[f] = FormatUtils.getDefaultFormat().getSerdeProvider()
-                        .getSerializerDeserializer(metaItemType);
+                outputSerDes[f] =
+                        FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(metaItemType);
                 outputTypeTraits[f] = FormatUtils.getDefaultFormat().getTypeTraitProvider().getTypeTrait(metaItemType);
                 f++;
             }
@@ -477,8 +477,8 @@
      */
     public static IOperatorDescriptor createDummyKeyProviderOp(JobSpecification spec, Dataset dataset,
             MetadataProvider metadataProvider) throws AlgebricksException {
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
-                .getSplitProviderAndConstraints(dataset);
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(dataset);
         AlgebricksPartitionConstraint primaryPartitionConstraint = primarySplitsAndConstraint.second;
 
         // Build dummy tuple containing one field with a dummy value inside.
@@ -506,9 +506,8 @@
         return datasetName.indexOf('.') > 0; //NOSONAR a fully qualified name can't start with a .
     }
 
-    public static String getDataverseFromFullyQualifiedName(String datasetName) {
-        int idx = datasetName.indexOf('.');
-        return datasetName.substring(0, idx);
+    public static String getFullyQualifiedName(Dataset dataset) {
+        return dataset.getDataverseName() + '.' + dataset.getDatasetName();
     }
 
     /***
@@ -548,13 +547,14 @@
      */
     public static String createNodeGroupForNewDataset(String dataverseName, String datasetName, long rebalanceCount,
             Set<String> ncNames, MetadataProvider metadataProvider) throws Exception {
+        ICcApplicationContext appCtx = metadataProvider.getApplicationContext();
         String nodeGroup = dataverseName + "." + datasetName + (rebalanceCount == 0L ? "" : "_" + rebalanceCount);
         MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        MetadataLockManager.INSTANCE.acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodeGroup);
+        appCtx.getMetadataLockManager().acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodeGroup);
         NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, nodeGroup);
         if (ng != null) {
             nodeGroup = nodeGroup + "_" + UUID.randomUUID().toString();
-            MetadataLockManager.INSTANCE.acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodeGroup);
+            appCtx.getMetadataLockManager().acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodeGroup);
         }
         MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, new NodeGroup(nodeGroup, new ArrayList<>(ncNames)));
         return nodeGroup;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
new file mode 100644
index 0000000..4cf25f7
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.asterix.metadata.utils;
+
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.metadata.LockList;
+
+public class MetadataLockUtil {
+
+    private MetadataLockUtil() {
+    }
+
+    public static void createDatasetBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String itemTypeDataverseName, String itemTypeFullyQualifiedName, String metaItemTypeDataverseName,
+            String metaItemTypeFullyQualifiedName, String nodeGroupName, String compactionPolicyName,
+            String datasetFullyQualifiedName, boolean isDefaultCompactionPolicy) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        if (!dataverseName.equals(itemTypeDataverseName)) {
+            lockMgr.acquireDataverseReadLock(locks, itemTypeDataverseName);
+        }
+        if (metaItemTypeDataverseName != null && !metaItemTypeDataverseName.equals(dataverseName)
+                && !metaItemTypeDataverseName.equals(itemTypeDataverseName)) {
+            lockMgr.acquireDataverseReadLock(locks, metaItemTypeDataverseName);
+        }
+        lockMgr.acquireDataTypeReadLock(locks, itemTypeFullyQualifiedName);
+        if (metaItemTypeFullyQualifiedName != null
+                && !metaItemTypeFullyQualifiedName.equals(itemTypeFullyQualifiedName)) {
+            lockMgr.acquireDataTypeReadLock(locks, metaItemTypeFullyQualifiedName);
+        }
+        if (nodeGroupName != null) {
+            lockMgr.acquireNodeGroupReadLock(locks, nodeGroupName);
+        }
+        if (!isDefaultCompactionPolicy) {
+            lockMgr.acquireMergePolicyReadLock(locks, compactionPolicyName);
+        }
+        lockMgr.acquireDatasetWriteLock(locks, datasetFullyQualifiedName);
+    }
+
+    public static void createIndexBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String datasetFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDatasetCreateIndexLock(locks, datasetFullyQualifiedName);
+    }
+
+    public static void dropIndexBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String datasetFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDatasetWriteLock(locks, datasetFullyQualifiedName);
+    }
+
+    public static void createTypeBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String itemTypeFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDataTypeWriteLock(locks, itemTypeFullyQualifiedName);
+    }
+
+    public static void dropDatasetBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String datasetFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDatasetWriteLock(locks, datasetFullyQualifiedName);
+    }
+
+    public static void dropTypeBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String dataTypeFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDataTypeWriteLock(locks, dataTypeFullyQualifiedName);
+    }
+
+    public static void functionStatementBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String functionFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireFunctionWriteLock(locks, functionFullyQualifiedName);
+    }
+
+    public static void modifyDatasetBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String datasetFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDatasetModifyLock(locks, datasetFullyQualifiedName);
+    }
+
+    public static void insertDeleteUpsertBegin(IMetadataLockManager lockMgr, LockList locks,
+            String datasetFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks,
+                MetadataUtil.getDataverseFromFullyQualifiedName(datasetFullyQualifiedName));
+        lockMgr.acquireDatasetModifyLock(locks, datasetFullyQualifiedName);
+    }
+
+    public static void dropFeedBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String feedFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireActiveEntityWriteLock(locks, feedFullyQualifiedName);
+    }
+
+    public static void dropFeedPolicyBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String policyName) throws AsterixException {
+        lockMgr.acquireActiveEntityWriteLock(locks, policyName);
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+    }
+
+    public static void startFeedBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String feedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireActiveEntityReadLock(locks, feedName);
+    }
+
+    public static void stopFeedBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String feedName) throws AsterixException {
+        // TODO: dataset lock?
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireActiveEntityReadLock(locks, feedName);
+    }
+
+    public static void createFeedBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String feedFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireActiveEntityWriteLock(locks, feedFullyQualifiedName);
+    }
+
+    public static void connectFeedBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String datasetFullyQualifiedName, String feedFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireActiveEntityReadLock(locks, feedFullyQualifiedName);
+        lockMgr.acquireDatasetReadLock(locks, datasetFullyQualifiedName);
+    }
+
+    public static void createFeedPolicyBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String policyName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireFeedPolicyWriteLock(locks, policyName);
+    }
+
+    public static void disconnectFeedBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String datasetFullyQualifiedName, String feedFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireActiveEntityReadLock(locks, feedFullyQualifiedName);
+        lockMgr.acquireDatasetReadLock(locks, datasetFullyQualifiedName);
+    }
+
+    public static void compactBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String datasetFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDatasetReadLock(locks, datasetFullyQualifiedName);
+    }
+
+    public static void refreshDatasetBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+            String datasetFullyQualifiedName) throws AsterixException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDatasetExclusiveModificationLock(locks, datasetFullyQualifiedName);
+    }
+
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java
index 3133aba..e5d4721 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java
@@ -38,4 +38,9 @@
                 return "Unknown Pending Operation";
         }
     }
+
+    public static String getDataverseFromFullyQualifiedName(String datasetName) {
+        int idx = datasetName.indexOf('.');
+        return datasetName.substring(0, idx);
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
index e634d4e..6825f10 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
@@ -24,11 +24,12 @@
 
 import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.config.ClusterProperties;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -59,12 +60,12 @@
     public static FileSplit[] getIndexSplits(Dataset dataset, String indexName, MetadataTransactionContext mdTxnCtx)
             throws AlgebricksException {
         try {
-            List<String> nodeGroup =
-                    MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName()).getNodeNames();
+            NodeGroup nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName());
             if (nodeGroup == null) {
                 throw new AlgebricksException("Couldn't find node group " + dataset.getNodeGroupName());
             }
-            return getIndexSplits(dataset, indexName, nodeGroup);
+            List<String> nodeList = nodeGroup.getNodeNames();
+            return getIndexSplits(dataset, indexName, nodeList);
         } catch (MetadataException me) {
             throw new AlgebricksException(me);
         }
@@ -85,11 +86,10 @@
 
             for (int k = 0; k < numPartitions; k++) {
                 // format: 'storage dir name'/partition_#/dataverse/dataset_idx_index
-                File f = new File(
-                        StoragePathUtil.prepareStoragePartitionPath(storageDirName, nodePartitions[k].getPartitionId())
-                                + (dataset.isTemp() ? (File.separator + StoragePathUtil.TEMP_DATASETS_STORAGE_FOLDER)
-                                        : "")
-                                + File.separator + relPathFile);
+                File f = new File(StoragePathUtil.prepareStoragePartitionPath(storageDirName,
+                        nodePartitions[k].getPartitionId())
+                        + (dataset.isTemp() ? (File.separator + StoragePathUtil.TEMP_DATASETS_STORAGE_FOLDER) : "")
+                        + File.separator + relPathFile);
                 splits.add(StoragePathUtil.getFileSplitForClusterPartition(nodePartitions[k], f.getPath()));
             }
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
index 5bedffa..ff65994 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
@@ -21,8 +21,8 @@
 
 import java.rmi.RemoteException;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.api.IMetadataEntityTupleTranslator;
 import org.apache.asterix.metadata.api.IValueExtractor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
index 32bed0d..5f0525b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
@@ -24,8 +24,8 @@
 import java.io.DataInputStream;
 import java.io.IOException;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.api.IValueExtractor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
index fcf69d5..1928d7e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
@@ -21,8 +21,8 @@
 
 import java.nio.ByteBuffer;
 
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.api.IValueExtractor;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
index 6f3bd30..e5d0d7d 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
@@ -24,7 +24,7 @@
 import java.util.Map;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.metadata.MetadataException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index fe76c6b..cdbaad3 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -30,7 +30,7 @@
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.metadata.MetadataException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Datatype;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
index 4cd243c..28c480f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
@@ -21,6 +21,8 @@
 import java.io.IOException;
 import java.util.function.Supplier;
 
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.ActiveProperties;
 import org.apache.asterix.common.config.BuildProperties;
@@ -74,12 +76,13 @@
     private Object extensionManager;
     private IFaultToleranceStrategy ftStrategy;
     private IJobLifecycleListener activeLifeCycleListener;
+    private IMetadataLockManager mdLockManager;
 
     public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
             ILibraryManager libraryManager, IResourceIdManager resourceIdManager,
             Supplier<IMetadataBootstrap> metadataBootstrapSupplier, IGlobalRecoveryManager globalRecoveryManager,
             IFaultToleranceStrategy ftStrategy, IJobLifecycleListener activeLifeCycleListener,
-            IStorageComponentProvider storageComponentProvider)
+            IStorageComponentProvider storageComponentProvider, IMetadataLockManager mdLockManager)
             throws AsterixException, IOException {
         this.ccServiceCtx = ccServiceCtx;
         this.hcc = hcc;
@@ -105,6 +108,7 @@
         this.metadataBootstrapSupplier = metadataBootstrapSupplier;
         this.globalRecoveryManager = globalRecoveryManager;
         this.storageComponentProvider = storageComponentProvider;
+        this.mdLockManager = mdLockManager;
     }
 
     @Override
@@ -210,7 +214,7 @@
     }
 
     @Override
-    public IJobLifecycleListener getActiveLifecycleListener() {
+    public IJobLifecycleListener getActiveNotificationHandler() {
         return activeLifeCycleListener;
     }
 
@@ -218,4 +222,14 @@
     public IStorageComponentProvider getStorageComponentProvider() {
         return storageComponentProvider;
     }
+
+    @Override
+    public IMetadataLockManager getMetadataLockManager() {
+        return mdLockManager;
+    }
+
+    @Override
+    public IClusterStateManager getClusterStateManager() {
+        return ClusterStateManager.INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index da341af..4717a7b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -66,17 +66,11 @@
     public static final ClusterStateManager INSTANCE = new ClusterStateManager();
     private final Map<String, Map<IOption, Object>> activeNcConfiguration = new HashMap<>();
     private Set<String> pendingRemoval = new HashSet<>();
-
     private final Cluster cluster;
     private ClusterState state = ClusterState.UNUSABLE;
-
     private AlgebricksAbsolutePartitionConstraint clusterPartitionConstraint;
-
-    private boolean globalRecoveryCompleted = false;
-
     private Map<String, ClusterPartition[]> node2PartitionsMap;
     private SortedMap<Integer, ClusterPartition> clusterPartitions;
-
     private String currentMetadataNode = null;
     private boolean metadataNodeActive = false;
     private Set<String> failedNodes = new HashSet<>();
@@ -117,7 +111,9 @@
 
     @Override
     public synchronized void setState(ClusterState state) {
+        LOGGER.info("updating cluster state from " + this.state + " to " + state.name());
         this.state = state;
+        appCtx.getGlobalRecoveryManager().notifyStateChange(state);
         LOGGER.info("Cluster State is now " + state.name());
         // Notify any waiting threads for the cluster state to change.
         notifyAll();
@@ -262,16 +258,8 @@
                 clusterActiveLocations.add(p.getActiveNodeId());
             }
         }
-        clusterPartitionConstraint = new AlgebricksAbsolutePartitionConstraint(
-                clusterActiveLocations.toArray(new String[] {}));
-    }
-
-    public boolean isGlobalRecoveryCompleted() {
-        return globalRecoveryCompleted;
-    }
-
-    public void setGlobalRecoveryCompleted(boolean globalRecoveryCompleted) {
-        this.globalRecoveryCompleted = globalRecoveryCompleted;
+        clusterPartitionConstraint =
+                new AlgebricksAbsolutePartitionConstraint(clusterActiveLocations.toArray(new String[] {}));
     }
 
     public boolean isClusterActive() {
@@ -384,13 +372,12 @@
 
     @Override
     public synchronized void deregisterNodePartitions(String nodeId) {
-        ClusterPartition [] nodePartitions = node2PartitionsMap.remove(nodeId);
+        ClusterPartition[] nodePartitions = node2PartitionsMap.remove(nodeId);
         if (nodePartitions == null) {
             LOGGER.info("deregisterNodePartitions unknown node " + nodeId + " (already removed?)");
         } else {
             if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("deregisterNodePartitions for node " + nodeId + ": " +
-                        Arrays.toString(nodePartitions));
+                LOGGER.info("deregisterNodePartitions for node " + nodeId + ": " + Arrays.toString(nodePartitions));
             }
             for (ClusterPartition nodePartition : nodePartitions) {
                 clusterPartitions.remove(nodePartition.getPartitionId());
@@ -413,7 +400,7 @@
             LOGGER.info("Deregistering intention to remove node id " + nodeId);
         }
         if (!pendingRemoval.remove(nodeId)) {
-            LOGGER.warning("Cannot deregister intention to remove node id " + nodeId  + " that was not registered");
+            LOGGER.warning("Cannot deregister intention to remove node id " + nodeId + " that was not registered");
             return false;
         } else {
             return true;
diff --git a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
index e4a94a9..3b2d0ce 100644
--- a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
+++ b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
@@ -47,13 +47,13 @@
 import org.apache.asterix.common.annotations.UndeclaredFieldsDataGen;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.lang.aql.parser.AQLParserFactory;
 import org.apache.asterix.lang.aql.parser.ParseException;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
diff --git a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java
index a525b1b..94389b2 100644
--- a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java
+++ b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java
@@ -24,10 +24,10 @@
 
 import org.apache.asterix.common.annotations.TypeDataGen;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.statement.DataverseDecl;
 import org.apache.asterix.lang.common.statement.TypeDecl;
-import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeSignature;
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java
index 443edf1..f5695a5 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java
@@ -46,7 +46,7 @@
             int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
             log(pkHash, after, before);
         } catch (ACIDException e) {
-            throw new HyracksDataException(e);
+            throw HyracksDataException.create(e);
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetPartitionManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetPartitionManager.java
index 3119ddd..008f0be 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetPartitionManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetPartitionManager.java
@@ -24,21 +24,23 @@
 import org.apache.hyracks.api.job.JobId;
 
 public interface IDatasetPartitionManager extends IDatasetManager {
-    public IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
+    IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
             boolean asyncMode, int partition, int nPartitions) throws HyracksException;
 
-    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, int partition, int nPartitions,
+    void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, int partition, int nPartitions,
             boolean orderedResult, boolean emptyResult) throws HyracksException;
 
-    public void reportPartitionWriteCompletion(JobId jobId, ResultSetId resultSetId, int partition)
+    void reportPartitionWriteCompletion(JobId jobId, ResultSetId resultSetId, int partition) throws HyracksException;
+
+    void initializeDatasetPartitionReader(JobId jobId, ResultSetId resultSetId, int partition, IFrameWriter noc)
             throws HyracksException;
 
-    public void initializeDatasetPartitionReader(JobId jobId, ResultSetId resultSetId, int partition, IFrameWriter noc)
-            throws HyracksException;
+    void removePartition(JobId jobId, ResultSetId resultSetId, int partition);
 
-    public void removePartition(JobId jobId, ResultSetId resultSetId, int partition);
+    void abortReader(JobId jobId);
 
-    public void abortReader(JobId jobId);
+    void abortAllReaders();
 
-    public void close();
+    void close();
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
index c9cdb2d..0a99ea6 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
@@ -53,11 +53,6 @@
         return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, params);
     }
 
-    public static HyracksDataException create(HyracksDataException e, String nodeId) {
-        return new HyracksDataException(e.getComponent(), e.getErrorCode(), e.getMessage(), e.getCause(), nodeId,
-                e.getParams());
-    }
-
     public static HyracksDataException suppress(HyracksDataException root, Throwable th) {
         if (root == null) {
             return HyracksDataException.create(th);
@@ -124,7 +119,7 @@
     }
 
     public HyracksDataException(String component, int errorCode, Throwable cause, Serializable... params) {
-        super(component, errorCode, cause.getMessage(), cause, null, params);
+        super(component, errorCode, cause, params);
     }
 
     public HyracksDataException(String component, int errorCode, String message, Throwable cause,
@@ -132,4 +127,8 @@
         super(component, errorCode, message, cause, null, params);
     }
 
+    public static HyracksDataException create(HyracksDataException e, String nodeId) {
+        return new HyracksDataException(e.getComponent(), e.getErrorCode(), e.getMessage(), e.getCause(), nodeId,
+                e.getParams());
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobLifecycleListener.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobLifecycleListener.java
index 30ffebe..338c331 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobLifecycleListener.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobLifecycleListener.java
@@ -18,12 +18,38 @@
  */
 package org.apache.hyracks.api.job;
 
+import java.util.List;
+
 import org.apache.hyracks.api.exceptions.HyracksException;
 
+/**
+ * A listener for job related events
+ */
 public interface IJobLifecycleListener {
-    public void notifyJobCreation(JobId jobId, JobSpecification spec) throws HyracksException;
+    /**
+     * Notify the listener that a job has been created
+     *
+     * @param jobId
+     * @param spec
+     * @throws HyracksException
+     */
+    void notifyJobCreation(JobId jobId, JobSpecification spec) throws HyracksException;
 
-    public void notifyJobStart(JobId jobId) throws HyracksException;
+    /**
+     * Notify the listener that the job has started on the cluster controller
+     *
+     * @param jobId
+     * @throws HyracksException
+     */
+    void notifyJobStart(JobId jobId) throws HyracksException;
 
-    public void notifyJobFinish(JobId jobId) throws HyracksException;
+    /**
+     * Notify the listener that the job has been terminated, passing exceptions in case of failure
+     *
+     * @param jobId
+     * @param jobStatus
+     * @param exceptions
+     * @throws HyracksException
+     */
+    void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions) throws HyracksException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
index 627f406..e1c218f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
@@ -167,7 +167,7 @@
         threadDumpRunMap = Collections.synchronizedMap(new HashMap<>());
 
         // Node manager is in charge of cluster membership management.
-        nodeManager = new NodeManager(ccConfig, resourceManager);
+        nodeManager = new NodeManager(this, ccConfig, resourceManager);
 
         jobIdFactory = new JobIdFactory();
     }
@@ -193,9 +193,9 @@
         clusterIPC = new IPCSystem(new InetSocketAddress(ccConfig.getClusterListenPort()), ccIPCI,
                 new CCNCFunctions.SerializerDeserializer());
         IIPCI ciIPCI = new ClientInterfaceIPCI(this, jobIdFactory);
-        clientIPC = new IPCSystem(
-                new InetSocketAddress(ccConfig.getClientListenAddress(), ccConfig.getClientListenPort()),
-                ciIPCI, new JavaSerializationBasedPayloadSerializerDeserializer());
+        clientIPC =
+                new IPCSystem(new InetSocketAddress(ccConfig.getClientListenAddress(), ccConfig.getClientListenPort()),
+                        ciIPCI, new JavaSerializationBasedPayloadSerializerDeserializer());
         webServer = new WebServer(this, ccConfig.getConsoleListenPort());
         clusterIPC.start();
         clientIPC.start();
@@ -222,9 +222,9 @@
 
         // Job manager is in charge of job lifecycle management.
         try {
-            Constructor<?> jobManagerConstructor = this.getClass().getClassLoader()
-                    .loadClass(ccConfig.getJobManagerClass())
-                    .getConstructor(CCConfig.class, ClusterControllerService.class, IJobCapacityController.class);
+            Constructor<?> jobManagerConstructor =
+                    this.getClass().getClassLoader().loadClass(ccConfig.getJobManagerClass()).getConstructor(
+                            CCConfig.class, ClusterControllerService.class, IJobCapacityController.class);
             jobManager = (IJobManager) jobManagerConstructor.newInstance(ccConfig, this, jobCapacityController);
         } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | NoSuchMethodException
                 | InvocationTargetException e) {
@@ -263,13 +263,14 @@
             @Override
             public void notifyNodeJoin(String nodeId, Map<IOption, Object> ncConfiguration) throws HyracksException {
                 // no-op, we don't care
+                LOGGER.log(Level.WARNING, "Getting notified that node: " + nodeId + " has joined. and we don't care");
             }
 
             @Override
             public void notifyNodeFailure(Collection<String> deadNodeIds) throws HyracksException {
+                LOGGER.log(Level.WARNING, "Getting notified that nodes: " + deadNodeIds + " has failed");
                 for (String nodeId : deadNodeIds) {
                     Pair<String, Integer> ncService = getNCService(nodeId);
-
                     final TriggerNCWork triggerWork = new TriggerNCWork(ClusterControllerService.this,
                             ncService.getLeft(), ncService.getRight(), nodeId);
                     workQueue.schedule(triggerWork);
@@ -396,8 +397,8 @@
 
         @Override
         public void getIPAddressNodeMap(Map<InetAddress, Set<String>> map) throws HyracksDataException {
-            GetIpAddressNodeNameMapWork ginmw = new GetIpAddressNodeNameMapWork(
-                    ClusterControllerService.this.getNodeManager(), map);
+            GetIpAddressNodeNameMapWork ginmw =
+                    new GetIpAddressNodeNameMapWork(ClusterControllerService.this.getNodeManager(), map);
             try {
                 workQueue.scheduleAndSync(ginmw);
             } catch (Exception e) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCServiceContext.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCServiceContext.java
index 5075081..26245e1 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCServiceContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCServiceContext.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.api.job.IJobLifecycleListener;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.service.IControllerService;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.common.application.ServiceContext;
@@ -88,14 +89,14 @@
         }
     }
 
-    public synchronized void notifyJobFinish(JobId jobId) throws HyracksException {
+    public synchronized void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions)
+            throws HyracksException {
         for (IJobLifecycleListener l : jobLifecycleListeners) {
-            l.notifyJobFinish(jobId);
+            l.notifyJobFinish(jobId, jobStatus, exceptions);
         }
     }
 
-    public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec)
-            throws HyracksException {
+    public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec) throws HyracksException {
         for (IJobLifecycleListener l : jobLifecycleListeners) {
             l.notifyJobCreation(jobId, spec);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
index 47e78a3..8cca1e0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
@@ -38,19 +38,24 @@
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.resource.NodeCapacity;
+import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.job.IJobManager;
+import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.cc.scheduler.IResourceManager;
 import org.apache.hyracks.control.common.controllers.CCConfig;
 
 public class NodeManager implements INodeManager {
     private static final Logger LOGGER = Logger.getLogger(NodeManager.class.getName());
 
+    private final ClusterControllerService ccs;
     private final CCConfig ccConfig;
     private final IResourceManager resourceManager;
     private final Map<String, NodeControllerState> nodeRegistry;
     private final Map<InetAddress, Set<String>> ipAddressNodeNameMap;
 
-    public NodeManager(CCConfig ccConfig, IResourceManager resourceManager) {
+    public NodeManager(ClusterControllerService ccs, CCConfig ccConfig, IResourceManager resourceManager) {
+        this.ccs = ccs;
         this.ccConfig = ccConfig;
         this.resourceManager = resourceManager;
         this.nodeRegistry = new LinkedHashMap<>();
@@ -79,15 +84,18 @@
 
     @Override
     public void addNode(String nodeId, NodeControllerState ncState) throws HyracksException {
+        LOGGER.warning("addNode(" + nodeId + ") called");
         if (nodeId == null || ncState == null) {
             throw HyracksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
         }
         // Updates the node registry.
         if (nodeRegistry.containsKey(nodeId)) {
-            LOGGER.warning("Node with name " + nodeId + " has already registered; re-registering");
+            LOGGER.warning(
+                    "Node with name " + nodeId + " has already registered; failing the node then re-registering.");
+            removeDeadNode(nodeId);
         }
+        LOGGER.warning("adding node to registry");
         nodeRegistry.put(nodeId, ncState);
-
         // Updates the IP address to node names map.
         try {
             InetAddress ipAddress = getIpAddress(ncState);
@@ -98,8 +106,8 @@
             nodeRegistry.remove(nodeId);
             throw e;
         }
-
         // Updates the cluster capacity.
+        LOGGER.warning("updating cluster capacity");
         resourceManager.update(nodeId, ncState.getCapacity());
     }
 
@@ -147,6 +155,27 @@
         return Pair.of(deadNodes, affectedJobIds);
     }
 
+    public void removeDeadNode(String nodeId) throws HyracksException {
+        NodeControllerState state = nodeRegistry.get(nodeId);
+        Set<JobId> affectedJobIds = state.getActiveJobIds();
+        // Removes the node from node map.
+        nodeRegistry.remove(nodeId);
+        // Removes the node from IP map.
+        removeNodeFromIpAddressMap(nodeId, state);
+        // Updates the cluster capacity.
+        resourceManager.update(nodeId, new NodeCapacity(0L, 0));
+        LOGGER.info(nodeId + " considered dead");
+        IJobManager jobManager = ccs.getJobManager();
+        Set<String> collection = Collections.singleton(nodeId);
+        for (JobId jobId : affectedJobIds) {
+            JobRun run = jobManager.get(jobId);
+            if (run != null) {
+                run.getExecutor().notifyNodeFailures(collection);
+            }
+        }
+        ccs.getContext().notifyNodeFailure(collection);
+    }
+
     @Override
     public void apply(NodeFunction nodeFunction) {
         nodeRegistry.forEach(nodeFunction::apply);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/dataset/DatasetDirectoryService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/dataset/DatasetDirectoryService.java
index 8401fcf..2685f60 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/dataset/DatasetDirectoryService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/dataset/DatasetDirectoryService.java
@@ -41,6 +41,7 @@
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.control.cc.PreDistributedJobStore;
 import org.apache.hyracks.control.common.dataset.ResultStateSweeper;
 import org.apache.hyracks.control.common.work.IResultCallback;
@@ -69,7 +70,7 @@
         this.resultTTL = resultTTL;
         this.resultSweepThreshold = resultSweepThreshold;
         this.preDistributedJobStore = preDistributedJobStore;
-        jobResultLocations = new LinkedHashMap<JobId, JobResultInfo>();
+        jobResultLocations = new LinkedHashMap<>();
     }
 
     @Override
@@ -94,7 +95,7 @@
     }
 
     @Override
-    public void notifyJobFinish(JobId jobId) throws HyracksException {
+    public void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions) throws HyracksException {
         // Auto-generated method stub
     }
 
@@ -179,7 +180,7 @@
 
     @Override
     public synchronized long getResultTimestamp(JobId jobId) {
-        if (preDistributedJobStore.jobIsPredistributed(jobId)){
+        if (preDistributedJobStore.jobIsPredistributed(jobId)) {
             return -1;
         }
         return getState(jobId).getTimestamp();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
index 45c7711..c1a7899 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
@@ -132,8 +132,8 @@
         // Removes a pending job.
         JobRun jobRun = jobQueue.remove(jobId);
         if (jobRun != null) {
-            List<Exception> exceptions = Collections
-                    .singletonList(HyracksException.create(ErrorCode.JOB_CANCELED, jobId));
+            List<Exception> exceptions =
+                    Collections.singletonList(HyracksException.create(ErrorCode.JOB_CANCELED, jobId));
             // Since the job has not been executed, we only need to update its status and lifecyle here.
             jobRun.setStatus(JobStatus.FAILURE, exceptions);
             runMapArchive.put(jobId, jobRun);
@@ -179,7 +179,7 @@
                 } catch (Exception e) {
                     LOGGER.log(Level.SEVERE, e.getMessage(), e);
                     if (caughtException == null) {
-                        caughtException = new HyracksException(e);
+                        caughtException = HyracksException.create(e);
                     } else {
                         caughtException.addSuppressed(e);
                     }
@@ -208,7 +208,7 @@
         CCServiceContext serviceCtx = ccs.getContext();
         if (serviceCtx != null) {
             try {
-                serviceCtx.notifyJobFinish(jobId);
+                serviceCtx.notifyJobFinish(jobId, run.getPendingStatus(), run.getPendingExceptions());
             } catch (HyracksException e) {
                 LOGGER.log(Level.SEVERE, e.getMessage(), e);
                 caughtException = e;
@@ -249,8 +249,6 @@
         }
     }
 
-
-
     @Override
     public Collection<JobRun> getRunningJobs() {
         return activeRunMap.values();
@@ -320,9 +318,8 @@
         try {
             run.getExecutor().startJob();
         } catch (Exception e) {
-            ccs.getWorkQueue()
-                    .schedule(new JobCleanupWork(ccs.getJobManager(), run.getJobId(), JobStatus.FAILURE,
-                            Collections.singletonList(e)));
+            ccs.getWorkQueue().schedule(new JobCleanupWork(ccs.getJobManager(), run.getJobId(), JobStatus.FAILURE,
+                    Collections.singletonList(e)));
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
index f44fca0..bf0846f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
@@ -54,6 +54,7 @@
         CCNCFunctions.NodeRegistrationResult result;
         Map<IOption, Object> ncConfiguration = new HashMap<>();
         try {
+            LOGGER.log(Level.WARNING, "Registering INodeController: id = " + id);
             INodeController nc = new NodeControllerRemoteProxy(ccs.getClusterIPC(), reg.getNodeControllerAddress());
             NodeControllerState state = new NodeControllerState(nc, reg);
             INodeManager nodeManager = ccs.getNodeManager();
@@ -71,9 +72,12 @@
             result = new CCNCFunctions.NodeRegistrationResult(params, null);
             ccs.getJobIdFactory().ensureMinimumId(reg.getMaxJobId() + 1);
         } catch (Exception e) {
+            LOGGER.log(Level.WARNING, "Node registration failed", e);
             result = new CCNCFunctions.NodeRegistrationResult(null, e);
         }
+        LOGGER.warning("sending registration response to node");
         ncIPCHandle.send(-1, result, null);
+        LOGGER.warning("notifying node join");
         ccs.getContext().notifyNodeJoin(id, ncConfiguration);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
index dde3bad..fa6580e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
@@ -45,7 +45,7 @@
     @Test
     public void testNormal() throws HyracksException {
         IResourceManager resourceManager = new ResourceManager();
-        INodeManager nodeManager = new NodeManager(makeCCConfig(), resourceManager);
+        INodeManager nodeManager = new NodeManager(null, makeCCConfig(), resourceManager);
         NodeControllerState ncState1 = mockNodeControllerState(NODE1, false);
         NodeControllerState ncState2 = mockNodeControllerState(NODE2, false);
 
@@ -70,7 +70,7 @@
     @Test
     public void testException() throws HyracksException {
         IResourceManager resourceManager = new ResourceManager();
-        INodeManager nodeManager = new NodeManager(makeCCConfig(), resourceManager);
+        INodeManager nodeManager = new NodeManager(null, makeCCConfig(), resourceManager);
         NodeControllerState ncState1 = mockNodeControllerState(NODE1, true);
 
         boolean invalidNetworkAddress = false;
@@ -89,7 +89,7 @@
     @Test
     public void testNullNode() throws HyracksException {
         IResourceManager resourceManager = new ResourceManager();
-        INodeManager nodeManager = new NodeManager(makeCCConfig(), resourceManager);
+        INodeManager nodeManager = new NodeManager(null, makeCCConfig(), resourceManager);
 
         boolean invalidParameter = false;
         // Verifies states after a failure during adding nodes.
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ExceptionUtils.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ExceptionUtils.java
index 3bc549e..1d506ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ExceptionUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ExceptionUtils.java
@@ -59,10 +59,13 @@
         List<Exception> newExceptions = new ArrayList<>();
         for (Exception e : exceptions) {
             if (e instanceof HyracksDataException) {
-                newExceptions.add(HyracksDataException.create((HyracksDataException) e, nodeId));
+                if (((HyracksDataException) e).getNodeId() == null) {
+                    newExceptions.add(HyracksDataException.create((HyracksDataException) e, nodeId));
+                } else {
+                    newExceptions.add(e);
+                }
             } else {
-                newExceptions.add(new HyracksDataException(ErrorCode.HYRACKS, ErrorCode.FAILURE_ON_NODE, e.getMessage(),
-                        e, nodeId));
+                newExceptions.add(new HyracksDataException(ErrorCode.HYRACKS, ErrorCode.FAILURE_ON_NODE, e, nodeId));
             }
         }
         exceptions.clear();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index fc911c0..b1f39f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -179,8 +179,8 @@
         this.application = application;
         id = ncConfig.getNodeId();
 
-        ioManager = new IOManager(IODeviceHandle.getDevices(ncConfig.getIODevices()),
-                application.getFileDeviceResolver());
+        ioManager =
+                new IOManager(IODeviceHandle.getDevices(ncConfig.getIODevices()), application.getFileDeviceResolver());
         if (id == null) {
             throw new HyracksException("id not set");
         }
@@ -274,8 +274,7 @@
         partitionManager = new PartitionManager(this);
         netManager = new NetworkManager(ncConfig.getDataListenAddress(), ncConfig.getDataListenPort(), partitionManager,
                 ncConfig.getNetThreadCount(), ncConfig.getNetBufferCount(), ncConfig.getDataPublicAddress(),
-                ncConfig.getDataPublicPort(),
-                FullFrameChannelInterfaceFactory.INSTANCE);
+                ncConfig.getDataPublicPort(), FullFrameChannelInterfaceFactory.INSTANCE);
         netManager.start();
 
         startApplication();
@@ -288,16 +287,17 @@
         this.ccs = new ClusterControllerRemoteProxy(ipc,
                 new InetSocketAddress(ncConfig.getClusterAddress(), ncConfig.getClusterPort()),
                 ncConfig.getClusterConnectRetries(), new IControllerRemoteProxyIPCEventListener() {
-            @Override
-            public void ipcHandleRestored(IIPCHandle handle) throws IPCException {
-                // we need to re-register in case the NC -> CC connection reset was due to CC shutdown
-                try {
-                    registerNode();
-                } catch (Exception e) {
-                    throw new IPCException(e);
-                }
-            }
-        });
+                    @Override
+                    public void ipcHandleRestored(IIPCHandle handle) throws IPCException {
+                        // we need to re-register in case of NC -> CC connection reset
+                        try {
+                            registerNode();
+                        } catch (Exception e) {
+                            LOGGER.log(Level.WARNING, "Failed Registering with cc", e);
+                            throw new IPCException(e);
+                        }
+                    }
+                });
         registerNode();
 
         workQueue.start();
@@ -332,15 +332,15 @@
         // Use "public" versions of network addresses and ports
         NetworkAddress datasetAddress = datasetNetworkManager.getPublicNetworkAddress();
         NetworkAddress netAddress = netManager.getPublicNetworkAddress();
-        NetworkAddress meesagingPort = messagingNetManager != null ? messagingNetManager.getPublicNetworkAddress()
-                : null;
+        NetworkAddress meesagingPort =
+                messagingNetManager != null ? messagingNetManager.getPublicNetworkAddress() : null;
         int allCores = osMXBean.getAvailableProcessors();
         nodeRegistration = new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netAddress, datasetAddress,
-                osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), allCores,
-                runtimeMXBean.getVmName(), runtimeMXBean.getVmVersion(), runtimeMXBean.getVmVendor(),
-                runtimeMXBean.getClassPath(), runtimeMXBean.getLibraryPath(), runtimeMXBean.getBootClassPath(),
-                runtimeMXBean.getInputArguments(), runtimeMXBean.getSystemProperties(), hbSchema, meesagingPort,
-                application.getCapacity(), PidHelper.getPid(), maxJobId.get());
+                osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), allCores, runtimeMXBean.getVmName(),
+                runtimeMXBean.getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean.getClassPath(),
+                runtimeMXBean.getLibraryPath(), runtimeMXBean.getBootClassPath(), runtimeMXBean.getInputArguments(),
+                runtimeMXBean.getSystemProperties(), hbSchema, meesagingPort, application.getCapacity(),
+                PidHelper.getPid(), maxJobId.get());
 
         ccs.registerNode(nodeRegistration);
 
@@ -572,12 +572,12 @@
 
     private static INCApplication getApplication(NCConfig config)
             throws ClassNotFoundException, IllegalAccessException, InstantiationException {
-            if (config.getAppClass() != null) {
-                Class<?> c = Class.forName(config.getAppClass());
-                return (INCApplication) c.newInstance();
-            } else {
-                return BaseNCApplication.INSTANCE;
-            }
+        if (config.getAppClass() != null) {
+            Class<?> c = Class.forName(config.getAppClass());
+            return (INCApplication) c.newInstance();
+        } else {
+            return BaseNCApplication.INSTANCE;
+        }
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionManager.java
index 361ee37..962d541 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionManager.java
@@ -44,7 +44,7 @@
 
     private final Executor executor;
 
-    private final Map<JobId, IDatasetStateRecord> partitionResultStateMap;
+    private final Map<JobId, ResultSetMap> partitionResultStateMap;
 
     private final DefaultDeallocatableRegistry deallocatableRegistry;
 
@@ -76,8 +76,7 @@
             dpw = new DatasetPartitionWriter(ctx, this, jobId, rsId, asyncMode, orderedResult, partition, nPartitions,
                     datasetMemoryManager, fileFactory);
 
-            ResultSetMap rsIdMap = (ResultSetMap) partitionResultStateMap.computeIfAbsent(jobId,
-                    k -> new ResultSetMap());
+            ResultSetMap rsIdMap = partitionResultStateMap.computeIfAbsent(jobId, k -> new ResultSetMap());
 
             ResultState[] resultStates = rsIdMap.createOrGetResultStates(rsId, nPartitions);
             resultStates[partition] = dpw.getResultState();
@@ -122,7 +121,7 @@
 
     protected synchronized ResultState getResultState(JobId jobId, ResultSetId resultSetId, int partition)
             throws HyracksException {
-        ResultSetMap rsIdMap = (ResultSetMap) partitionResultStateMap.get(jobId);
+        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
         if (rsIdMap == null) {
             throw new HyracksException("Unknown JobId " + jobId);
         }
@@ -139,7 +138,7 @@
 
     @Override
     public synchronized void removePartition(JobId jobId, ResultSetId resultSetId, int partition) {
-        ResultSetMap rsIdMap = (ResultSetMap) partitionResultStateMap.get(jobId);
+        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
         if (rsIdMap != null && rsIdMap.removePartition(jobId, resultSetId, partition)) {
             partitionResultStateMap.remove(jobId);
         }
@@ -147,13 +146,20 @@
 
     @Override
     public synchronized void abortReader(JobId jobId) {
-        ResultSetMap rsIdMap = (ResultSetMap) partitionResultStateMap.get(jobId);
+        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
         if (rsIdMap != null) {
             rsIdMap.abortAll();
         }
     }
 
     @Override
+    public synchronized void abortAllReaders() {
+        for (ResultSetMap rsIdMap : partitionResultStateMap.values()) {
+            rsIdMap.abortAll();
+        }
+    }
+
+    @Override
     public synchronized void close() {
         for (JobId jobId : getJobIds()) {
             deinit(jobId);
@@ -167,7 +173,7 @@
     }
 
     @Override
-    public IDatasetStateRecord getState(JobId jobId) {
+    public ResultSetMap getState(JobId jobId) {
         return partitionResultStateMap.get(jobId);
     }
 
@@ -187,7 +193,7 @@
     }
 
     private synchronized void deinit(JobId jobId) {
-        ResultSetMap rsIdMap = (ResultSetMap) partitionResultStateMap.get(jobId);
+        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
         if (rsIdMap != null) {
             rsIdMap.closeAndDeleteAll();
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PipelinedPartition.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PipelinedPartition.java
index fd434d7..16e5027 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PipelinedPartition.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PipelinedPartition.java
@@ -39,9 +39,7 @@
 
     private IFrameWriter delegate;
 
-    private boolean pendingConnection;
-
-    private boolean failed;
+    private volatile boolean pendingConnection = true;
 
     public PipelinedPartition(IHyracksTaskContext ctx, PartitionManager manager, PartitionId pid, TaskAttemptId taId) {
         this.ctx = ctx;
@@ -74,16 +72,13 @@
     @Override
     public void open() throws HyracksDataException {
         manager.registerPartition(pid, taId, this, PartitionState.STARTED, false);
-        failed = false;
         pendingConnection = true;
         ensureConnected();
     }
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        if (!failed) {
-            delegate.nextFrame(buffer);
-        }
+        delegate.nextFrame(buffer);
     }
 
     private void ensureConnected() throws HyracksDataException {
@@ -93,7 +88,8 @@
                     try {
                         wait();
                     } catch (InterruptedException e) {
-                        throw new HyracksDataException(e);
+                        Thread.currentThread().interrupt();
+                        throw HyracksDataException.create(e);
                     }
                 }
             }
@@ -104,22 +100,21 @@
 
     @Override
     public void fail() throws HyracksDataException {
-        failed = true;
-        if (delegate != null) {
+        if (!pendingConnection) {
             delegate.fail();
         }
     }
 
     @Override
     public void close() throws HyracksDataException {
-        if (!failed) {
+        if (!pendingConnection) {
             delegate.close();
         }
     }
 
     @Override
     public void flush() throws HyracksDataException {
-        if (!failed) {
+        if (!pendingConnection) {
             delegate.flush();
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortAllTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortAllTasksWork.java
new file mode 100644
index 0000000..4fb4bf6
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortAllTasksWork.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.hyracks.control.nc.work;
+
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
+import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.hyracks.control.nc.Joblet;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.hyracks.control.nc.Task;
+
+public class AbortAllTasksWork extends SynchronizableWork {
+
+    private static final Logger LOGGER = Logger.getLogger(AbortAllTasksWork.class.getName());
+    private final NodeControllerService ncs;
+
+    public AbortAllTasksWork(NodeControllerService ncs) {
+        this.ncs = ncs;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Aborting all tasks");
+        }
+        IDatasetPartitionManager dpm = ncs.getDatasetPartitionManager();
+        if (dpm != null) {
+            ncs.getDatasetPartitionManager().abortAllReaders();
+        }
+        for (Joblet ji : ncs.getJobletMap().values()) {
+            Map<TaskAttemptId, Task> taskMap = ji.getTaskMap();
+            for (Task task : taskMap.values()) {
+                if (task != null) {
+                    task.abort();
+                }
+            }
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortTasksWork.java
index 07e1ad2..5870e76 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortTasksWork.java
@@ -65,8 +65,8 @@
                 }
             }
         } else {
-            LOGGER.log(Level.WARNING, "Joblet couldn't be found. Tasks of job " + jobId
-                    + " have all either completed or failed");
+            LOGGER.log(Level.WARNING,
+                    "Joblet couldn't be found. Tasks of job " + jobId + " have all either completed or failed");
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/FrameDebugUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/FrameDebugUtils.java
index 66e7ae0..d09b890 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/FrameDebugUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/FrameDebugUtils.java
@@ -39,11 +39,12 @@
 
     /**
      * Debugging method
+     *
      * @param fta
      * @param recordDescriptor
      * @param prefix
      */
-    public void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, String prefix) {
+    public static void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, String prefix) {
         try (ByteBufferInputStream bbis = new ByteBufferInputStream();
                 DataInputStream dis = new DataInputStream(bbis)) {
             int tc = fta.getTupleCount();
@@ -60,19 +61,21 @@
 
     /**
      * Debugging method
+     *
      * @param fta
      * @param recordDescriptor
      */
-    public void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor) {
+    public static void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor) {
         prettyPrint(fta, recordDescriptor, "");
     }
 
     /**
      * Debugging method
+     *
      * @param fta
      * @param operator
      */
-    public void prettyPrintTags(IFrameTupleAccessor fta, String operator) {
+    public static void prettyPrintTags(IFrameTupleAccessor fta, String operator) {
         try (ByteBufferInputStream bbis = new ByteBufferInputStream();
                 DataInputStream dis = new DataInputStream(bbis)) {
             int tc = fta.getTupleCount();
@@ -90,14 +93,15 @@
 
     /**
      * Debugging method
+     *
      * @param fta
      * @param tid
      * @param bbis
      * @param dis
      * @param sb
      */
-    protected void prettyPrintTag(IFrameTupleAccessor fta, int tid, ByteBufferInputStream bbis, DataInputStream dis,
-            StringBuilder sb) {
+    protected static void prettyPrintTag(IFrameTupleAccessor fta, int tid, ByteBufferInputStream bbis,
+            DataInputStream dis, StringBuilder sb) {
         sb.append(" tid" + tid + ":(" + fta.getTupleStartOffset(tid) + ", " + fta.getTupleEndOffset(tid) + ")[");
         for (int j = 0; j < fta.getFieldCount(); ++j) {
             sb.append(" ");
@@ -115,6 +119,7 @@
 
     /**
      * Debugging method
+     *
      * @param fta
      * @param recordDescriptor
      * @param tid
@@ -122,9 +127,8 @@
      * @param dis
      * @param sb
      */
-    protected void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, int tid,
-            ByteBufferInputStream bbis, DataInputStream dis,
-            StringBuilder sb) {
+    protected static void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, int tid,
+            ByteBufferInputStream bbis, DataInputStream dis, StringBuilder sb) {
         sb.append(" tid" + tid + ":(" + fta.getTupleStartOffset(tid) + ", " + fta.getTupleEndOffset(tid) + ")[");
         for (int j = 0; j < fta.getFieldCount(); ++j) {
             sb.append(" ");
@@ -133,8 +137,8 @@
             }
             sb.append("f" + j + ":(" + fta.getFieldStartOffset(tid, j) + ", " + fta.getFieldEndOffset(tid, j) + ") ");
             sb.append("{");
-            bbis.setByteBuffer(fta.getBuffer(), fta.getTupleStartOffset(tid) + fta.getFieldSlotsLength() + fta
-                    .getFieldStartOffset(tid, j));
+            bbis.setByteBuffer(fta.getBuffer(),
+                    fta.getTupleStartOffset(tid) + fta.getFieldSlotsLength() + fta.getFieldStartOffset(tid, j));
             try {
                 sb.append(recordDescriptor.getFields()[j].deserialize(dis));
             } catch (Exception e) {
@@ -146,14 +150,14 @@
         sb.append("\n");
     }
 
-
     /**
      * Debugging method
+     *
      * @param fta
      * @param recordDescriptor
      * @param tid
      */
-    public void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, int tid) {
+    public static void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, int tid) {
         try (ByteBufferInputStream bbis = new ByteBufferInputStream();
                 DataInputStream dis = new DataInputStream(bbis)) {
             StringBuilder sb = new StringBuilder();
@@ -169,13 +173,14 @@
      * They are safe as they don't print records. Printing records
      * using IserializerDeserializer can print incorrect results or throw exceptions.
      * A better way yet would be to use record pointable.
+     *
      * @param fta
      * @param recordDescriptor
      * @param prefix
      * @param recordFields
      * @throws IOException
      */
-    public void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, String prefix,
+    public static void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, String prefix,
             int[] recordFields) throws IOException {
         try (ByteBufferInputStream bbis = new ByteBufferInputStream();
                 DataInputStream dis = new DataInputStream(bbis)) {
@@ -191,14 +196,15 @@
 
     /**
      * Debugging method
+     *
      * @param fta
      * @param recordDescriptor
      * @param tIdx
      * @param recordFields
      * @throws IOException
      */
-    public void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, int tIdx, int[] recordFields)
-            throws IOException {
+    public static void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, int tIdx,
+            int[] recordFields) throws IOException {
         try (ByteBufferInputStream bbis = new ByteBufferInputStream();
                 DataInputStream dis = new DataInputStream(bbis)) {
             StringBuilder sb = new StringBuilder();
@@ -209,14 +215,14 @@
 
     /**
      * Debugging method
+     *
      * @param tuple
      * @param fieldsIdx
      * @param descIdx
      * @throws HyracksDataException
      */
-    public void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, ITupleReference tuple,
-            int fieldsIdx, int descIdx)
-            throws HyracksDataException {
+    public static void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, ITupleReference tuple,
+            int fieldsIdx, int descIdx) throws HyracksDataException {
         try (ByteBufferInputStream bbis = new ByteBufferInputStream();
                 DataInputStream dis = new DataInputStream(bbis)) {
             StringBuilder sb = new StringBuilder();
@@ -237,11 +243,12 @@
 
     /**
      * Debugging method
+     *
      * @param tuple
      * @param descF
      * @throws HyracksDataException
      */
-    public void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, ITupleReference tuple,
+    public static void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, ITupleReference tuple,
             int[] descF) throws HyracksDataException {
         try (ByteBufferInputStream bbis = new ByteBufferInputStream();
                 DataInputStream dis = new DataInputStream(bbis)) {
@@ -265,6 +272,7 @@
 
     /**
      * Debugging method
+     *
      * @param fta
      * @param recordDescriptor
      * @param tid
@@ -274,17 +282,15 @@
      * @param recordFields
      * @throws IOException
      */
-    protected void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, int tid,
-            ByteBufferInputStream bbis, DataInputStream dis,
-            StringBuilder sb,
-            int[] recordFields) throws IOException {
+    protected static void prettyPrint(IFrameTupleAccessor fta, RecordDescriptor recordDescriptor, int tid,
+            ByteBufferInputStream bbis, DataInputStream dis, StringBuilder sb, int[] recordFields) throws IOException {
         Arrays.sort(recordFields);
         sb.append(" tid" + tid + ":(" + fta.getTupleStartOffset(tid) + ", " + fta.getTupleEndOffset(tid) + ")[");
         for (int j = 0; j < fta.getFieldCount(); ++j) {
             sb.append("f" + j + ":(" + fta.getFieldStartOffset(tid, j) + ", " + fta.getFieldEndOffset(tid, j) + ") ");
             sb.append("{");
-            bbis.setByteBuffer(fta.getBuffer(), fta.getTupleStartOffset(tid) + fta
-                    .getFieldSlotsLength() + fta.getFieldStartOffset(tid, j));
+            bbis.setByteBuffer(fta.getBuffer(),
+                    fta.getTupleStartOffset(tid) + fta.getFieldSlotsLength() + fta.getFieldStartOffset(tid, j));
             if (Arrays.binarySearch(recordFields, j) >= 0) {
                 sb.append("{a record field: only print using pointable:");
                 sb.append("tag->" + dis.readByte() + "}");
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
index 0cc0170..d7d5c27 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
@@ -123,14 +123,15 @@
             try {
                 wait();
             } catch (InterruptedException e) {
-                throw new HyracksDataException(e);
+                Thread.currentThread().interrupt();
+                throw HyracksDataException.create(e);
             }
         }
     }
 
     public synchronized void close() throws HyracksDataException {
-        for (int i = closedSenders.nextClearBit(0); i >= 0
-                && i < nSenderPartitions; i = closedSenders.nextClearBit(i + 1)) {
+        for (int i = closedSenders.nextClearBit(0); i >= 0 && i < nSenderPartitions; i =
+                closedSenders.nextClearBit(i + 1)) {
             if (channels[i] != null) {
                 channels[i].close();
                 channels[i] = null;