Feed Connection Refactoring

1. The feed subscription network using FeedJoint is removed.
2. FeedConnection metadata dataset is added (pkeys: dataverseName,
   feedName, datasetName).
3. Replaced the old intake job + collect job combination with one single
   job using SplitOperator.
4. Now one feed can connect to multiple datasets.
5. The disconnect feed job is replaced by ActiveManagerMessage.
6. The new feed life cycle is:
   - Create feed
   - Connect feed to dataset0, dataset1, dataset2, etc.
   - Start feed
   - Stop feed
   - Disconnect feed
 7. New feedEventListner framework by Abdullah

Change-Id: Ic36267eb9a10df21734ce1cc1f38583e23c9e8f0
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1259
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <tillw@apache.org>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <bamousaa@gmail.com>
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 c907a36..2669990 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
@@ -22,25 +22,27 @@
 
 public class ActiveEvent {
 
-    private final JobId jobId;
-    private final EntityId entityId;
-    private final EventKind eventKind;
-    private final Object eventObject;
-
-    public enum EventKind {
-        JOB_START,
-        JOB_FINISH,
-        PARTITION_EVENT
+    public enum Kind {
+        JOB_CREATED,
+        JOB_STARTED,
+        JOB_FINISHED,
+        PARTITION_EVENT,
+        EXTENSION_EVENT
     }
 
-    public ActiveEvent(JobId jobId, ActiveEvent.EventKind eventKind, EntityId entityId, Object eventObject) {
+    private final JobId jobId;
+    private final EntityId entityId;
+    private final Kind eventKind;
+    private final Object eventObject;
+
+    public ActiveEvent(JobId jobId, Kind eventKind, EntityId entityId, Object eventObject) {
         this.jobId = jobId;
         this.entityId = entityId;
         this.eventKind = eventKind;
         this.eventObject = eventObject;
     }
 
-    public ActiveEvent(JobId jobId, ActiveEvent.EventKind eventKind, EntityId entityId) {
+    public ActiveEvent(JobId jobId, Kind eventKind, EntityId entityId) {
         this(jobId, eventKind, entityId, null);
     }
 
@@ -52,11 +54,16 @@
         return entityId;
     }
 
-    public EventKind getEventKind() {
+    public Kind getEventKind() {
         return eventKind;
     }
 
     public Object getEventObject() {
         return eventObject;
     }
+
+    @Override
+    public String toString() {
+        return "JobId:" + jobId + ", " + "EntityId:" + entityId + ", " + "Kind" + eventKind;
+    }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJob.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJob.java
deleted file mode 100644
index 1e3eca1..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveJob.java
+++ /dev/null
@@ -1,92 +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.io.Serializable;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class ActiveJob implements Serializable {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(ActiveJob.class.getName());
-    protected final EntityId entityId;
-    protected JobId jobId;
-    protected final Serializable jobObject;
-    protected ActivityState state;
-    protected JobSpecification spec;
-
-    public ActiveJob(EntityId entityId, JobId jobId, ActivityState state, Serializable jobInfo, JobSpecification spec) {
-        this.entityId = entityId;
-        this.state = state;
-        this.jobId = jobId;
-        this.jobObject = jobInfo;
-        this.spec = spec;
-    }
-
-    public ActiveJob(EntityId entityId, ActivityState state, Serializable jobInfo, JobSpecification spec) {
-        this.entityId = entityId;
-        this.state = state;
-        this.jobObject = jobInfo;
-        this.spec = spec;
-    }
-
-    public JobId getJobId() {
-        return jobId;
-    }
-
-    public void setJobId(JobId jobId) {
-        this.jobId = jobId;
-    }
-
-    public ActivityState getState() {
-        return state;
-    }
-
-    public void setState(ActivityState state) {
-        this.state = state;
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(this + " is in " + state + " state.");
-        }
-    }
-
-    public Object getJobObject() {
-        return jobObject;
-    }
-
-    public JobSpecification getSpec() {
-        return spec;
-    }
-
-    public void setSpec(JobSpecification spec) {
-        this.spec = spec;
-    }
-
-    @Override
-    public String toString() {
-        return jobId + " [" + jobObject + "]";
-    }
-
-    public EntityId getEntityId() {
-        return entityId;
-    }
-
-}
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
index 0d1d8ab..e4491bd 100644
--- 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
@@ -24,7 +24,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.active.ActiveEvent.EventKind;
+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;
@@ -36,7 +36,7 @@
     private static final boolean DEBUG = false;
     private final LinkedBlockingQueue<ActiveEvent> eventInbox;
     private final Map<EntityId, IActiveEntityEventsListener> entityEventListeners;
-    private final Map<JobId, ActiveJob> jobId2ActiveJobInfos;
+    private final Map<JobId, EntityId> jobId2ActiveJobInfos;
 
     private ActiveJobNotificationHandler() {
         this.eventInbox = new LinkedBlockingQueue<>();
@@ -51,16 +51,20 @@
         while (!Thread.interrupted()) {
             try {
                 ActiveEvent event = getEventInbox().take();
-                ActiveJob jobInfo = jobId2ActiveJobInfos.get(event.getJobId());
-                EntityId entityId = jobInfo.getEntityId();
-                IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-                if (DEBUG) {
-                    LOGGER.log(Level.WARNING, "Next event is of type " + event.getEventKind());
-                    LOGGER.log(Level.WARNING, "Notifying the listener");
-                }
-                listener.notify(event);
-                if (event.getEventKind() == EventKind.JOB_FINISH) {
-                    removeJob(event.getJobId(), listener);
+                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());
+                    LOGGER.log(Level.FINER, "Notifying the listener");
+                    listener.notify(event);
+                    if (event.getEventKind() == Kind.JOB_FINISHED) {
+                        LOGGER.log(Level.FINER, "Removing the job");
+                        jobId2ActiveJobInfos.remove(event.getJobId());
+                        LOGGER.log(Level.FINER, "Removing the listener since it is not active anymore");
+                        entityEventListeners.remove(listener.getEntityId());
+                    }
+                } else {
+                    LOGGER.log(Level.SEVERE, "Entity not found for received message for job " + event.getJobId());
                 }
             } catch (InterruptedException e) {
                 Thread.currentThread().interrupt();
@@ -71,29 +75,6 @@
         LOGGER.log(Level.INFO, "Stopped " + ActiveJobNotificationHandler.class.getSimpleName());
     }
 
-    public void removeJob(JobId jobId, IActiveEntityEventsListener listener) {
-        removeFinishedJob(jobId, listener);
-        removeInactiveListener(listener);
-    }
-
-    private void removeFinishedJob(JobId jobId, IActiveEntityEventsListener listener) {
-        if (!listener.isEntityActive()) {
-            if (DEBUG) {
-                LOGGER.log(Level.WARNING, "Remove job" + jobId);
-            }
-            jobId2ActiveJobInfos.remove(jobId);
-        }
-    }
-
-    private void removeInactiveListener(IActiveEntityEventsListener listener) {
-        if (!listener.isEntityActive()) {
-            if (DEBUG) {
-                LOGGER.log(Level.WARNING, "Removing the listener since it is not active anymore");
-            }
-            entityEventListeners.remove(listener.getEntityId());
-        }
-    }
-
     public IActiveEntityEventsListener getActiveEntityListener(EntityId entityId) {
         if (DEBUG) {
             LOGGER.log(Level.WARNING, "getActiveEntityListener(EntityId entityId) was called with entity " + entityId);
@@ -103,61 +84,27 @@
         return entityEventListeners.get(entityId);
     }
 
-    public synchronized ActiveJob[] getActiveJobs() {
-        if (DEBUG) {
-            LOGGER.log(Level.WARNING, "getActiveJobs()  was called");
-            LOGGER.log(Level.WARNING, "Number of jobs found: " + jobId2ActiveJobInfos.size());
-        }
-        return jobId2ActiveJobInfos.values().toArray(new ActiveJob[jobId2ActiveJobInfos.size()]);
-    }
-
-    public boolean isActiveJob(JobId jobId) {
-        if (DEBUG) {
-            LOGGER.log(Level.WARNING, "isActiveJob(JobId jobId) called with jobId: " + jobId);
-            boolean found = jobId2ActiveJobInfos.get(jobId) != null;
-            LOGGER.log(Level.WARNING, "Job was found to be: " + (found ? "Active" : "Inactive"));
-        }
-        return jobId2ActiveJobInfos.get(jobId) != null;
-    }
-
     public EntityId getEntity(JobId jobId) {
-        ActiveJob jobInfo = jobId2ActiveJobInfos.get(jobId);
-        return jobInfo == null ? null : jobInfo.getEntityId();
+        return jobId2ActiveJobInfos.get(jobId);
     }
 
     public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification) {
-        if (DEBUG) {
-            LOGGER.log(Level.WARNING,
-                    "notifyJobCreation(JobId jobId, JobSpecification jobSpecification) was called with jobId = "
-                            + jobId);
-        }
+        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 ActiveJob)) {
-            if (DEBUG) {
-                LOGGER.log(Level.WARNING, "Job was is not active. property found to be: " + property);
-            }
+        if (property == null || !(property instanceof EntityId)) {
+            LOGGER.log(Level.FINER, "Job was is not active. property found to be: " + property);
             return;
-        } else {
-            monitorJob(jobId, (ActiveJob) property);
         }
-        if (DEBUG) {
-            boolean found = jobId2ActiveJobInfos.get(jobId) != null;
-            LOGGER.log(Level.WARNING, "Job was found to be: " + (found ? "Active" : "Inactive"));
+        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) {
+            listener.notify(new ActiveEvent(jobId, Kind.JOB_CREATED, entityId, jobSpecification));
         }
-        ActiveJob jobInfo = jobId2ActiveJobInfos.get(jobId);
-        if (jobInfo != null) {
-            EntityId entityId = jobInfo.getEntityId();
-            IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-            listener.notifyJobCreation(jobId, jobSpecification);
-            if (DEBUG) {
-                LOGGER.log(Level.WARNING, "Listener was notified" + jobId);
-            }
-        } else {
-            if (DEBUG) {
-                LOGGER.log(Level.WARNING,
-                        "Listener was not notified since it was not registered for the job " + jobId);
-            }
-        }
+        LOGGER.log(Level.FINER, "Listener was notified" + jobId);
     }
 
     public LinkedBlockingQueue<ActiveEvent> getEventInbox() {
@@ -174,9 +121,8 @@
 
     public synchronized void registerListener(IActiveEntityEventsListener listener) throws HyracksDataException {
         if (DEBUG) {
-            LOGGER.log(Level.WARNING,
-                    "registerListener(IActiveEntityEventsListener listener) was called for the entity "
-                            + listener.getEntityId());
+            LOGGER.log(Level.FINER, "registerListener(IActiveEntityEventsListener listener) was called for the entity "
+                    + listener.getEntityId());
         }
         if (entityEventListeners.containsKey(listener.getEntityId())) {
             throw new HyracksDataException(
@@ -185,13 +131,23 @@
         entityEventListeners.put(listener.getEntityId(), listener);
     }
 
-    public synchronized void monitorJob(JobId jobId, ActiveJob activeJob) {
+    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.getEntityId())) {
+        if (entityEventListeners.containsKey(activeJob)) {
             if (jobId2ActiveJobInfos.containsKey(jobId)) {
                 LOGGER.severe("Job is already being monitored for job: " + jobId);
                 return;
@@ -199,22 +155,9 @@
             if (DEBUG) {
                 LOGGER.log(Level.WARNING, "monitoring started for job id: " + jobId);
             }
-            jobId2ActiveJobInfos.put(jobId, activeJob);
         } else {
-            LOGGER.severe("No listener was found for the entity: " + activeJob.getEntityId());
+            LOGGER.severe("No listener was found for the entity: " + activeJob);
         }
-    }
-
-    public synchronized void unregisterListener(IActiveEntityEventsListener listener) throws HyracksDataException {
-        if (DEBUG) {
-            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 HyracksDataException(
-                    "Active Entity Listener " + listener.getEntityId() + " hasn't been registered");
-        }
+        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
index fad30fa..6a10b0c 100644
--- 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
@@ -24,6 +24,7 @@
 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;
@@ -48,7 +49,7 @@
     public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
         EntityId entityId = ActiveJobNotificationHandler.INSTANCE.getEntity(jobId);
         if (entityId != null) {
-            jobEventInbox.add(new ActiveEvent(jobId, ActiveEvent.EventKind.JOB_START, entityId));
+            jobEventInbox.add(new ActiveEvent(jobId, Kind.JOB_STARTED, entityId));
         }
     }
 
@@ -56,7 +57,7 @@
     public synchronized void notifyJobFinish(JobId jobId) throws HyracksException {
         EntityId entityId = ActiveJobNotificationHandler.INSTANCE.getEntity(jobId);
         if (entityId != null) {
-            jobEventInbox.add(new ActiveEvent(jobId, ActiveEvent.EventKind.JOB_FINISH, entityId));
+            jobEventInbox.add(new ActiveEvent(jobId, Kind.JOB_FINISHED, entityId));
         } else {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("NO NEED TO NOTIFY JOB FINISH!");
@@ -70,10 +71,8 @@
     }
 
     public void receive(ActivePartitionMessage message) {
-        if (ActiveJobNotificationHandler.INSTANCE.isActiveJob(message.getJobId())) {
-            jobEventInbox.add(new ActiveEvent(message.getJobId(), ActiveEvent.EventKind.PARTITION_EVENT,
-                    message.getActiveRuntimeId().getEntityId(), message));
-        }
+        jobEventInbox.add(new ActiveEvent(message.getJobId(), Kind.PARTITION_EVENT,
+                message.getActiveRuntimeId().getEntityId(), message));
     }
 
     public void stop() {
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 1301535..c8abb84 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
@@ -18,11 +18,25 @@
  */
 package org.apache.asterix.active;
 
-// TODO: Document the state machine and its transition.
 public enum ActivityState {
-    CREATED,
-    INACTIVE,
-    ACTIVE,
-    UNDER_RECOVERY,
-    ENDED
+    /**
+     * The starting state and a possible terminal state. Next state can only be {@code ActivityState.STARTING}
+     */
+    STOPPED,
+    /**
+     * A terminal state
+     */
+    FAILED,
+    /**
+     * An intermediate state. Next state can only be {@code ActivityState.STARTED} or {@code ActivityState.FAILED}
+     */
+    STARTING,
+    /**
+     * An intermediate state. Next state can only be {@code ActivityState.STOPPING} or {@code ActivityState.FAILED}
+     */
+    STARTED,
+    /**
+     * An intermediate state. Next state can only be {@code ActivityState.STOPPED} or {@code ActivityState.FAILED}
+     */
+    STOPPING
 }
\ No newline at end of file
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 d0fb5e8..ee8e776 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
@@ -18,19 +18,48 @@
  */
 package org.apache.asterix.active;
 
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.asterix.common.metadata.IDataset;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IActiveEntityEventsListener {
 
-    public void notify(ActiveEvent message);
+    /**
+     * Notify the listener that an event related to the entity has taken place
+     * Examples of such events include
+     * 1. Job created
+     * 2. Job completed
+     * 3. Partition event
+     *
+     * @param event
+     *            the event that took place
+     */
+    void notify(ActiveEvent event);
 
-    public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification);
+    /**
+     * @return the state of the entity
+     */
+    ActivityState getState();
 
-    public boolean isEntityActive();
+    /**
+     * get a subscriber that waits till state has been reached.
+     *
+     * @param state
+     *            the desired state
+     * @throws HyracksDataException
+     *             a failure happened while waiting for the state
+     */
+    IActiveEventSubscriber subscribe(ActivityState state) throws HyracksDataException;
 
-    public EntityId getEntityId();
+    /**
+     * @return the active entity id
+     */
+    EntityId getEntityId();
 
-    public boolean isEntityUsingDataset(String dataverseName, String datasetName);
+    /**
+     * dataset
+     *
+     * @return
+     */
+    boolean isEntityUsingDataset(IDataset dataset);
 
 }
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/IActiveEventSubscriber.java
new file mode 100644
index 0000000..7be5737
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEventSubscriber.java
@@ -0,0 +1,48 @@
+/*
+ * 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;
+
+/**
+ * An active event subscriber that subscribe to events related to active entity
+ */
+public interface IActiveEventSubscriber {
+
+    /**
+     * Notify the subscriber of a new event
+     * @param event
+     */
+    void notify(ActiveEvent event);
+
+    /**
+     * Checkcs whether the subscriber is done receiving events
+     * @return
+     */
+    boolean done();
+
+    /**
+     * Wait until the terminal event has been received
+     * @throws InterruptedException
+     */
+    void sync() throws InterruptedException;
+
+    /**
+     * Stop watching events
+     */
+    void unsubscribe();
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
index 6ff8a56..1a0ecd9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
@@ -21,6 +21,7 @@
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.FeedDataSource;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -59,8 +60,8 @@
         }
 
         final FeedDataSource feedDataSource = (FeedDataSource) dataSource;
-        Feed feed = feedDataSource.getFeed();
-        if (feed.getAppliedFunction() == null) {
+        FeedConnection feedConnection = feedDataSource.getFeedConnection();
+        if (feedConnection.getAppliedFunctions() == null) {
             return false;
         }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
index 74e6ec5..46b421b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
@@ -33,6 +33,7 @@
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
@@ -214,6 +215,8 @@
         Dataset dataset = metadataProvider.findDataset(aqlId.getDataverseName(), targetDataset);
         ARecordType feedOutputType = (ARecordType) metadataProvider.findType(aqlId.getDataverseName(), outputType);
         Feed sourceFeed = metadataProvider.findFeed(aqlId.getDataverseName(), sourceFeedName);
+        FeedConnection feedConnection = metadataProvider.findFeedConnection(aqlId.getDataverseName(), sourceFeedName,
+                targetDataset);
         ARecordType metaType = null;
         // Does dataset have meta?
         if (dataset.hasMetaPart()) {
@@ -260,8 +263,8 @@
         }
         FeedDataSource feedDataSource = new FeedDataSource(sourceFeed, aqlId, targetDataset, feedOutputType, metaType,
                 pkTypes, partitioningKeys, keyAccessScalarFunctionCallExpression, sourceFeed.getFeedId(),
-                sourceFeed.getFeedType(), FeedRuntimeType.valueOf(subscriptionLocation), locations.split(","),
-                context.getComputationNodeDomain());
+                 FeedRuntimeType.valueOf(subscriptionLocation), locations.split(","),
+                context.getComputationNodeDomain(), feedConnection);
         feedDataSource.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy);
         return feedDataSource;
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJoint.java
deleted file mode 100644
index 0614d14..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJoint.java
+++ /dev/null
@@ -1,212 +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.app.external;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.active.ActiveJobNotificationHandler;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.app.external.FeedWorkCollection.SubscribeFeedWork;
-import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedConnectionRequest;
-import org.apache.asterix.external.feed.management.FeedEventsListener;
-import org.apache.asterix.external.feed.management.FeedJointKey;
-import org.apache.asterix.external.feed.management.FeedWorkManager;
-import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
-
-public class FeedJoint implements IFeedJoint {
-
-    private static final long serialVersionUID = 1L;
-
-    private static final Logger LOGGER = Logger.getLogger(FeedJoint.class.getName());
-
-    /** A unique key associated with the feed point **/
-    private final FeedJointKey key;
-
-    /** The state associated with the FeedJoint **/
-    private State state;
-
-    /** A list of subscribers that receive data from this FeedJoint **/
-    private final List<FeedConnectionId> receivers;
-
-    /** The feedId on which the feedPoint resides **/
-    private final EntityId ownerFeedId;
-
-    /** A list of feed subscription requests submitted for subscribing to the FeedPoint's data **/
-    private final List<FeedConnectionRequest> connectionRequests;
-
-    private final FeedRuntimeType connectionLocation;
-
-    private final FeedJointType type;
-
-    private FeedConnectionId provider;
-
-    public FeedJoint(FeedJointKey key, EntityId ownerFeedId, FeedRuntimeType subscriptionLocation, FeedJointType type,
-            FeedConnectionId provider) {
-        this.key = key;
-        this.ownerFeedId = ownerFeedId;
-        this.type = type;
-        this.receivers = new ArrayList<FeedConnectionId>();
-        this.state = State.CREATED;
-        this.connectionLocation = subscriptionLocation;
-        this.connectionRequests = new ArrayList<FeedConnectionRequest>();
-        this.provider = provider;
-    }
-
-    @Override
-    public int hashCode() {
-        return key.hashCode();
-    }
-
-    @Override
-    public void addReceiver(FeedConnectionId connectionId) {
-        receivers.add(connectionId);
-    }
-
-    @Override
-    public void removeReceiver(FeedConnectionId connectionId) {
-        receivers.remove(connectionId);
-    }
-
-    @Override
-    public synchronized void addConnectionRequest(FeedConnectionRequest request) {
-        connectionRequests.add(request);
-        if (state.equals(State.ACTIVE)) {
-            handlePendingConnectionRequest();
-        }
-    }
-
-    @Override
-    public synchronized void setState(State state) {
-        if (this.state.equals(state)) {
-            return;
-        }
-        this.state = state;
-        if (this.state.equals(State.ACTIVE)) {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Feed joint " + this + " is now " + State.ACTIVE);
-            }
-            handlePendingConnectionRequest();
-        }
-    }
-
-    private void handlePendingConnectionRequest() {
-        for (FeedConnectionRequest connectionRequest : connectionRequests) {
-            FeedConnectionId connectionId =
-                    new FeedConnectionId(connectionRequest.getReceivingFeedId(), connectionRequest.getTargetDataset());
-            try {
-                FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
-                        .getActiveEntityListener(connectionId.getFeedId());
-                SubscribeFeedWork work = new SubscribeFeedWork(
-                        listener.getConnectionLocations(this, connectionRequest).toArray(new String[] {}),
-                        connectionRequest);
-                FeedWorkManager.INSTANCE.submitWork(work, new SubscribeFeedWork.FeedSubscribeWorkEventListener());
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Submitted feed connection request " + connectionRequest + " at feed joint " + this);
-                }
-                addReceiver(connectionId);
-            } catch (Exception e) {
-                if (LOGGER.isLoggable(Level.WARNING)) {
-                    LOGGER.warning("Unsuccessful attempt at submitting connection request " + connectionRequest
-                            + " at feed joint " + this + ". Message " + e.getMessage());
-                }
-                e.printStackTrace();
-            }
-        }
-        connectionRequests.clear();
-    }
-
-    @Override
-    public FeedConnectionId getReceiver(FeedConnectionId connectionId) {
-        for (FeedConnectionId cid : receivers) {
-            if (cid.equals(connectionId)) {
-                return cid;
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public String toString() {
-        return key.toString() + " [" + connectionLocation + "]" + "[" + state + "]";
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (o == null) {
-            return false;
-        }
-        if (o == this) {
-            return true;
-        }
-        if (!(o instanceof FeedJoint)) {
-            return false;
-        }
-        return ((FeedJoint) o).getFeedJointKey().equals(this.key);
-    }
-
-    @Override
-    public EntityId getOwnerFeedId() {
-        return ownerFeedId;
-    }
-
-    @Override
-    public List<FeedConnectionRequest> getConnectionRequests() {
-        return connectionRequests;
-    }
-
-    @Override
-    public FeedRuntimeType getConnectionLocation() {
-        return connectionLocation;
-    }
-
-    @Override
-    public FeedJointType getType() {
-        return type;
-    }
-
-    @Override
-    public FeedConnectionId getProvider() {
-        return provider;
-    }
-
-    @Override
-    public List<FeedConnectionId> getReceivers() {
-        return receivers;
-    }
-
-    public FeedJointKey getKey() {
-        return key;
-    }
-
-    @Override
-    public synchronized State getState() {
-        return state;
-    }
-
-    @Override
-    public FeedJointKey getFeedJointKey() {
-        return key;
-    }
-
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
deleted file mode 100644
index ec7c239..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
+++ /dev/null
@@ -1,140 +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.app.external;
-
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.List;
-
-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.compiler.provider.AqlCompilationProvider;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.external.feed.api.IFeedWork;
-import org.apache.asterix.external.feed.api.IFeedWorkEventListener;
-import org.apache.asterix.external.feed.management.FeedConnectionRequest;
-import org.apache.asterix.external.feed.management.FeedConnectionRequest.ConnectionStatus;
-import org.apache.asterix.file.StorageComponentProvider;
-import org.apache.asterix.lang.aql.statement.SubscribeFeedStatement;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.statement.DataverseDecl;
-import org.apache.asterix.lang.common.struct.Identifier;
-import org.apache.asterix.runtime.utils.AppContextInfo;
-import org.apache.asterix.translator.IStatementExecutor;
-import org.apache.asterix.translator.SessionConfig;
-import org.apache.asterix.translator.SessionConfig.OutputFormat;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * A collection of feed management related task, each represented as an implementation of {@code IFeedWork}.
- */
-public class FeedWorkCollection {
-
-    private static Logger LOGGER = Logger.getLogger(FeedWorkCollection.class.getName());
-    private static final ILangCompilationProvider compilationProvider = new AqlCompilationProvider();
-    private static final IStorageComponentProvider storageComponentProvider = new StorageComponentProvider();
-
-    /**
-     * The task of subscribing to a feed to obtain data.
-     */
-    public static class SubscribeFeedWork implements IFeedWork {
-
-        private final Runnable runnable;
-
-        private final FeedConnectionRequest request;
-
-        @Override
-        public Runnable getRunnable() {
-            return runnable;
-        }
-
-        public SubscribeFeedWork(String[] locations, FeedConnectionRequest request) {
-            this.runnable = new SubscribeFeedWorkRunnable(locations, request);
-            this.request = request;
-        }
-
-        private static class SubscribeFeedWorkRunnable implements Runnable {
-
-            private static final DefaultStatementExecutorFactory qtFactory = new DefaultStatementExecutorFactory();
-            private final FeedConnectionRequest request;
-            private final String[] locations;
-
-            public SubscribeFeedWorkRunnable(String[] locations, FeedConnectionRequest request) {
-                this.request = request;
-                this.locations = locations;
-            }
-
-            @Override
-            public void run() {
-                try {
-                    //TODO(amoudi): route PrintWriter to log file
-                    PrintWriter writer = new PrintWriter(System.err, true);
-                    SessionConfig pc = new SessionConfig(writer, OutputFormat.ADM);
-                    DataverseDecl dataverseDecl = new DataverseDecl(
-                            new Identifier(request.getReceivingFeedId().getDataverse()));
-                    SubscribeFeedStatement subscribeStmt = new SubscribeFeedStatement(locations, request);
-                    List<Statement> statements = new ArrayList<>();
-                    statements.add(dataverseDecl);
-                    statements.add(subscribeStmt);
-                    IStatementExecutor translator = qtFactory.create(statements, pc, compilationProvider,
-                            storageComponentProvider);
-                    translator.compileAndExecute(AppContextInfo.INSTANCE.getHcc(), null,
-                            QueryTranslator.ResultDelivery.IMMEDIATE);
-                    if (LOGGER.isEnabledFor(Level.INFO)) {
-                        LOGGER.info("Submitted connection requests for execution: " + request);
-                    }
-                } catch (Exception e) {
-                    if (LOGGER.isEnabledFor(Level.FATAL)) {
-                        LOGGER.fatal("Exception in executing " + request, e);
-                    }
-                }
-            }
-        }
-
-        public static class FeedSubscribeWorkEventListener implements IFeedWorkEventListener {
-
-            @Override
-            public void workFailed(IFeedWork work, Exception e) {
-                if (LOGGER.isEnabledFor(Level.WARN)) {
-                    LOGGER.warn(" Feed subscription request " + ((SubscribeFeedWork) work).request
-                            + " failed with exception " + e);
-                }
-            }
-
-            @Override
-            public void workCompleted(IFeedWork work) {
-                ((SubscribeFeedWork) work).request.setSubscriptionStatus(ConnectionStatus.ACTIVE);
-                if (LOGGER.isEnabledFor(Level.INFO)) {
-                    LOGGER.info(" Feed subscription request " + ((SubscribeFeedWork) work).request + " completed ");
-                }
-            }
-        }
-
-        public FeedConnectionRequest getRequest() {
-            return request;
-        }
-
-        @Override
-        public String toString() {
-            return "SubscribeFeedWork for [" + request + "]";
-        }
-    }
-}
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 978c2eb..f111c54 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
@@ -42,10 +42,10 @@
 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.algebra.extension.IExtensionStatement;
 import org.apache.asterix.api.common.APIFramework;
 import org.apache.asterix.api.http.server.ApiServlet;
-import org.apache.asterix.app.external.FeedJoint;
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.app.result.ResultUtil;
@@ -60,31 +60,18 @@
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.CompilationException;
 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.api.IAdapterFactory;
-import org.apache.asterix.external.feed.api.IActiveLifecycleEventSubscriber;
-import org.apache.asterix.external.feed.api.IActiveLifecycleEventSubscriber.ActiveLifecycleEvent;
-import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.api.IFeed.FeedType;
-import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.feed.api.IFeedJoint.FeedJointType;
-import org.apache.asterix.external.feed.management.ActiveLifecycleEventSubscriber;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedConnectionRequest;
 import org.apache.asterix.external.feed.management.FeedEventsListener;
-import org.apache.asterix.external.feed.management.FeedJointKey;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.watch.FeedActivityDetails;
-import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
-import org.apache.asterix.external.feed.watch.FeedIntakeInfo;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
+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.aql.statement.SubscribeFeedStatement;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.base.IStatementRewriter;
@@ -97,8 +84,6 @@
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
-import org.apache.asterix.lang.common.statement.CreatePrimaryFeedStatement;
-import org.apache.asterix.lang.common.statement.CreateSecondaryFeedStatement;
 import org.apache.asterix.lang.common.statement.DatasetDecl;
 import org.apache.asterix.lang.common.statement.DataverseDecl;
 import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -121,11 +106,12 @@
 import org.apache.asterix.lang.common.statement.RefreshExternalDatasetStatement;
 import org.apache.asterix.lang.common.statement.RunStatement;
 import org.apache.asterix.lang.common.statement.SetStatement;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.statement.StopFeedStatement;
 import org.apache.asterix.lang.common.statement.TypeDecl;
 import org.apache.asterix.lang.common.statement.TypeDropStatement;
 import org.apache.asterix.lang.common.statement.WriteStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
-import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
@@ -141,14 +127,13 @@
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.Function;
 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.feeds.BuiltinFeedPolicies;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
-import org.apache.asterix.metadata.feeds.FeedOperations;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.ExternalDatasetsRegistry;
 import org.apache.asterix.metadata.utils.ExternalIndexingOperations;
@@ -165,11 +150,9 @@
 import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.asterix.translator.AbstractLangTranslator;
-import org.apache.asterix.translator.CompiledStatements.CompiledConnectFeedStatement;
 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.CompiledSubscribeFeedStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledUpsertStatement;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.IStatementExecutor;
@@ -177,13 +160,14 @@
 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;
 import org.apache.asterix.utils.FlushDatasetUtil;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.mutable.MutableBoolean;
 import org.apache.commons.lang3.mutable.MutableObject;
+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.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
 import org.apache.hyracks.algebricks.data.IAWriterFactory;
 import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
@@ -221,8 +205,9 @@
     protected final IStorageComponentProvider componentProvider;
     protected final ExecutorService executorService;
 
-    public QueryTranslator(List<Statement> statements, SessionConfig conf, ILangCompilationProvider compliationProvider,
-            IStorageComponentProvider componentProvider, ExecutorService executorService) {
+    public QueryTranslator(List<Statement> statements, SessionConfig conf,
+            ILangCompilationProvider compliationProvider, IStorageComponentProvider componentProvider,
+            ExecutorService executorService) {
         this.statements = statements;
         this.sessionConfig = conf;
         this.componentProvider = componentProvider;
@@ -349,8 +334,7 @@
                     case Statement.Kind.DELETE:
                         handleDeleteStatement(metadataProvider, stmt, hcc, false);
                         break;
-                    case Statement.Kind.CREATE_PRIMARY_FEED:
-                    case Statement.Kind.CREATE_SECONDARY_FEED:
+                    case Statement.Kind.CREATE_FEED:
                         handleCreateFeedStatement(metadataProvider, stmt);
                         break;
                     case Statement.Kind.DROP_FEED:
@@ -360,13 +344,16 @@
                         handleDropFeedPolicyStatement(metadataProvider, stmt);
                         break;
                     case Statement.Kind.CONNECT_FEED:
-                        handleConnectFeedStatement(metadataProvider, stmt, hcc);
+                        handleConnectFeedStatement(metadataProvider, stmt);
                         break;
                     case Statement.Kind.DISCONNECT_FEED:
-                        handleDisconnectFeedStatement(metadataProvider, stmt, hcc);
+                        handleDisconnectFeedStatement(metadataProvider, stmt);
                         break;
-                    case Statement.Kind.SUBSCRIBE_FEED:
-                        handleSubscribeFeedStatement(metadataProvider, stmt, hcc);
+                    case Statement.Kind.START_FEED:
+                        handleStartFeedStatement(metadataProvider, stmt, hcc);
+                        break;
+                    case Statement.Kind.STOP_FEED:
+                        handleStopFeedStatement(metadataProvider, stmt);
                         break;
                     case Statement.Kind.CREATE_FEED_POLICY:
                         handleCreateFeedPolicyStatement(metadataProvider, stmt);
@@ -702,12 +689,11 @@
         }
     }
 
-    protected void validateIfResourceIsActiveInFeed(String dataverseName, String datasetName)
-            throws CompilationException {
+    protected void validateIfResourceIsActiveInFeed(Dataset dataset) throws CompilationException {
         StringBuilder builder = null;
         IActiveEntityEventsListener[] listeners = ActiveJobNotificationHandler.INSTANCE.getEventListeners();
         for (IActiveEntityEventsListener listener : listeners) {
-            if (listener.isEntityUsingDataset(dataverseName, datasetName)) {
+            if (listener.isEntityUsingDataset(dataset)) {
                 if (builder == null) {
                     builder = new StringBuilder();
                 }
@@ -715,8 +701,8 @@
             }
         }
         if (builder != null) {
-            throw new CompilationException("Dataset " + dataverseName + "." + datasetName + " is currently being "
-                    + "fed into by the following active entities.\n" + builder.toString());
+            throw new CompilationException("Dataset " + dataset.getDataverseName() + "." + dataset.getDatasetName()
+                    + " is currently being " + "fed into by the following active entities.\n" + builder.toString());
         }
     }
 
@@ -907,7 +893,7 @@
             }
 
             if (ds.getDatasetType() == DatasetType.INTERNAL) {
-                validateIfResourceIsActiveInFeed(dataverseName, datasetName);
+                validateIfResourceIsActiveInFeed(ds);
             } else {
                 // External dataset
                 // Check if the dataset is indexible
@@ -1204,11 +1190,8 @@
                 EntityId activeEntityId = listener.getEntityId();
                 if (activeEntityId.getExtensionName().equals(Feed.EXTENSION_NAME)
                         && activeEntityId.getDataverse().equals(dataverseName)) {
-                    FeedEventsListener feedEventListener = (FeedEventsListener) listener;
-                    FeedConnectionId[] connections = feedEventListener.getConnections();
-                    for (FeedConnectionId conn : connections) {
-                        disconnectFeedBeforeDelete(dvId, activeEntityId, conn, metadataProvider, hcc);
-                    }
+                    stopFeedBeforeDelete(new Pair<>(dvId, new Identifier(activeEntityId.getEntityName())),
+                            metadataProvider);
                     // prepare job to remove feed log storage
                     jobsToExecute.add(FeedOperations.buildRemoveFeedStorageJob(MetadataManager.INSTANCE
                             .getFeed(mdTxnCtx, dataverseName, activeEntityId.getEntityName())));
@@ -1316,20 +1299,16 @@
         }
     }
 
-    protected void disconnectFeedBeforeDelete(Identifier dvId, EntityId activeEntityId, FeedConnectionId conn,
-            MetadataProvider metadataProvider, IHyracksClientConnection hcc) {
-        DisconnectFeedStatement disStmt = new DisconnectFeedStatement(dvId,
-                new Identifier(activeEntityId.getEntityName()), new Identifier(conn.getDatasetName()));
+    protected void stopFeedBeforeDelete(Pair<Identifier, Identifier> feedNameComp, MetadataProvider metadataProvider) {
+        StopFeedStatement disStmt = new StopFeedStatement(feedNameComp);
         try {
-            handleDisconnectFeedStatement(metadataProvider, disStmt, hcc);
+            handleStopFeedStatement(metadataProvider, disStmt);
             if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Disconnected feed " + activeEntityId.getEntityName() + " from dataset "
-                        + conn.getDatasetName());
+                LOGGER.info("Stopped feed " + feedNameComp.second.getValue());
             }
         } catch (Exception exception) {
             if (LOGGER.isLoggable(Level.WARNING)) {
-                LOGGER.warning("Unable to disconnect feed " + activeEntityId.getEntityName() + " from dataset "
-                        + conn.getDatasetName() + ". Encountered exception " + exception);
+                LOGGER.warning("Unable to stop feed " + feedNameComp.second.getValue() + exception);
             }
         }
     }
@@ -1407,7 +1386,7 @@
             // prepare job spec(s) that would disconnect any active feeds involving the dataset.
             IActiveEntityEventsListener[] activeListeners = ActiveJobNotificationHandler.INSTANCE.getEventListeners();
             for (IActiveEntityEventsListener listener : activeListeners) {
-                if (listener.isEntityUsingDataset(dataverseName, datasetName) && listener.isEntityActive()) {
+                if (listener.isEntityUsingDataset(ds)) {
                     throw new CompilationException(
                             "Can't drop dataset since it is connected to active entity: " + listener.getEntityId());
                 }
@@ -1523,7 +1502,7 @@
             IActiveEntityEventsListener[] listeners = ActiveJobNotificationHandler.INSTANCE.getEventListeners();
             StringBuilder builder = null;
             for (IActiveEntityEventsListener listener : listeners) {
-                if (listener.isEntityUsingDataset(dataverseName, datasetName)) {
+                if (listener.isEntityUsingDataset(ds)) {
                     if (builder == null) {
                         builder = new StringBuilder();
                     }
@@ -1991,22 +1970,8 @@
                     throw new AlgebricksException("A feed with this name " + feedName + " already exists.");
                 }
             }
-
-            switch (stmt.getKind()) {
-                case Statement.Kind.CREATE_PRIMARY_FEED:
-                    CreatePrimaryFeedStatement cpfs = (CreatePrimaryFeedStatement) stmt;
-                    String adaptorName = cpfs.getAdaptorName();
-                    feed = new Feed(dataverseName, feedName, cfs.getAppliedFunction(), FeedType.PRIMARY, feedName,
-                            adaptorName, cpfs.getAdaptorConfiguration());
-                    break;
-                case Statement.Kind.CREATE_SECONDARY_FEED:
-                    CreateSecondaryFeedStatement csfs = (CreateSecondaryFeedStatement) stmt;
-                    feed = new Feed(dataverseName, feedName, csfs.getAppliedFunction(), FeedType.SECONDARY,
-                            csfs.getSourceFeedName(), null, null);
-                    break;
-                default:
-                    throw new IllegalStateException();
-            }
+            String adaptorName = cfs.getAdaptorName();
+            feed = new Feed(dataverseName, feedName, adaptorName, cfs.getAdaptorConfiguration());
             FeedMetadataUtil.validateFeed(feed, mdTxnCtx, metadataProvider.getLibraryManager());
             MetadataManager.INSTANCE.addFeed(metadataProvider.getMetadataTxnContext(), feed);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2103,12 +2068,8 @@
             FeedEventsListener listener =
                     (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(feedId);
             if (listener != null) {
-                StringBuilder builder = new StringBuilder();
-                for (FeedConnectionId connectionId : listener.getConnections()) {
-                    builder.append(connectionId.getDatasetName() + "\n");
-                }
                 throw new AlgebricksException("Feed " + feedId
-                        + " is currently active and connected to the following dataset(s) \n" + builder.toString());
+                        + " is currently active and connected to the following dataset(s) \n" + listener.toString());
             } else {
                 JobSpecification spec = FeedOperations.buildRemoveFeedStorageJob(
                         MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName()));
@@ -2156,236 +2117,155 @@
         }
     }
 
-    protected void handleConnectFeedStatement(MetadataProvider metadataProvider, Statement stmt,
+    private void handleStartFeedStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
+        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();
+        FeedEventsListener listener =
+                (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
+        if (listener != null) {
+            throw new AlgebricksException("Feed " + feedName + " is started already.");
+        }
+        // Start
+        try {
+            MetadataLockManager.INSTANCE.startFeedBegin(dataverseName, dataverseName + "." + feedName,
+                    feedConnections);
+            // Prepare policy
+            List<IDataset> datasets = new ArrayList<>();
+            for (FeedConnection connection : feedConnections) {
+                datasets.add(MetadataManager.INSTANCE.getDataset(mdTxnCtx, connection.getDataverseName(),
+                        connection.getDatasetName()));
+            }
+
+            org.apache.commons.lang3.tuple.Pair<JobSpecification, AlgebricksAbsolutePartitionConstraint> jobInfo =
+                    FeedOperations.buildStartFeedJob(sessionConfig, metadataProvider, feed, feedConnections,
+                            compilationProvider, storageComponentProvider, qtFactory, hcc);
+
+            JobSpecification feedJob = jobInfo.getLeft();
+            listener = new FeedEventsListener(entityId, datasets, jobInfo.getRight().getLocations());
+            ActiveJobNotificationHandler.INSTANCE.registerListener(listener);
+            IActiveEventSubscriber eventSubscriber = listener.subscribe(ActivityState.STARTED);
+            feedJob.setProperty(ActiveJobNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME, entityId);
+            JobUtils.runJob(hcc, feedJob,
+                    Boolean.valueOf(metadataProvider.getConfig().get(StartFeedStatement.WAIT_FOR_COMPLETION)));
+            eventSubscriber.sync();
+            LOGGER.log(Level.INFO, "Submitted");
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            if (listener != null) {
+                ActiveJobNotificationHandler.INSTANCE.unregisterListener(listener);
+            }
+            throw e;
+        } finally {
+            MetadataLockManager.INSTANCE.startFeedEnd(dataverseName, dataverseName + "." + feedName, feedConnections);
+        }
+    }
+
+    private void handleStopFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
+        StopFeedStatement sfst = (StopFeedStatement) stmt;
+        String dataverseName = getActiveDataverse(sfst.getDataverseName());
+        String feedName = sfst.getFeedName().getValue();
+        EntityId feedId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
+        // Obtain runtime info from ActiveListener
+        FeedEventsListener listener =
+                (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(feedId);
+        if (listener == null) {
+            throw new AlgebricksException("Feed " + feedName + " is not started.");
+        }
+        IActiveEventSubscriber eventSubscriber = listener.subscribe(ActivityState.STOPPED);
+        // Transaction
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        MetadataLockManager.INSTANCE.StopFeedBegin(dataverseName, feedName);
+        try {
+            // validate
+            FeedMetadataUtil.validateIfFeedExists(dataverseName, feedName, mdTxnCtx);
+            // Construct ActiveMessage
+            for (int i = 0; i < listener.getSources().length; i++) {
+                String intakeLocation = listener.getSources()[i];
+                FeedOperations.SendStopMessageToNode(feedId, intakeLocation, i);
+            }
+            eventSubscriber.sync();
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        } finally {
+            MetadataLockManager.INSTANCE.StopFeedEnd(dataverseName, feedName);
+        }
+    }
+
+    private void handleConnectFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
+        FeedConnection fc;
         ConnectFeedStatement cfs = (ConnectFeedStatement) stmt;
         String dataverseName = getActiveDataverse(cfs.getDataverseName());
         String feedName = cfs.getFeedName();
         String datasetName = cfs.getDatasetName().getValue();
-        boolean bActiveTxn = true;
+        String policyName = cfs.getPolicy();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        metadataProvider.disableBlockingOperator();
-        boolean subscriberRegistered = false;
-        IActiveLifecycleEventSubscriber eventSubscriber = new ActiveLifecycleEventSubscriber();
-        FeedConnectionId feedConnId = null;
-        EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, cfs.getFeedName());
-        FeedEventsListener listener =
-                (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
+        // validation
+        Feed feed = FeedMetadataUtil.validateIfFeedExists(dataverseName, feedName,
+                metadataProvider.getMetadataTxnContext());
+        ARecordType outputType = FeedMetadataUtil.getOutputType(feed, feed.getAdapterConfiguration(),
+                ExternalDataConstants.KEY_TYPE_NAME);
+        List<FunctionSignature> appliedFunctions = cfs.getAppliedFunctions();
+        // Transaction handling
         MetadataLockManager.INSTANCE.connectFeedBegin(dataverseName, dataverseName + "." + datasetName,
                 dataverseName + "." + feedName);
         try {
-            metadataProvider.setWriteTransaction(true);
-            CompiledConnectFeedStatement cbfs = new CompiledConnectFeedStatement(dataverseName, cfs.getFeedName(),
-                    cfs.getDatasetName().getValue(), cfs.getPolicy(), cfs.getQuery(), cfs.getVarCounter());
-            FeedMetadataUtil.validateIfDatasetExists(dataverseName, cfs.getDatasetName().getValue(),
-                    metadataProvider.getMetadataTxnContext());
-            Feed feed = FeedMetadataUtil.validateIfFeedExists(dataverseName, cfs.getFeedName(),
-                    metadataProvider.getMetadataTxnContext());
-            feedConnId = new FeedConnectionId(dataverseName, cfs.getFeedName(), cfs.getDatasetName().getValue());
-            if (listener != null) {
-                subscriberRegistered = listener.isFeedConnectionActive(feedConnId, eventSubscriber);
+            fc = MetadataManager.INSTANCE.getFeedConnection(metadataProvider.getMetadataTxnContext(), dataverseName,
+                    feedName, datasetName);
+            if (fc != null) {
+                throw new AlgebricksException("Feed" + feedName + " is already connected dataset " + datasetName);
             }
-            if (subscriberRegistered) {
-                throw new CompilationException("Feed " + cfs.getFeedName() + " is already connected to dataset "
-                        + cfs.getDatasetName().getValue());
-            }
-            FeedPolicyEntity feedPolicy =
-                    FeedMetadataUtil.validateIfPolicyExists(dataverseName, cbfs.getPolicyName(), mdTxnCtx);
-            // All Metadata checks have passed. Feed connect request is valid. //
-            if (listener == null) {
-                listener = new FeedEventsListener(entityId);
-                ActiveJobNotificationHandler.INSTANCE.registerListener(listener);
-            }
-            FeedPolicyAccessor policyAccessor = new FeedPolicyAccessor(feedPolicy.getProperties());
-            Triple<FeedConnectionRequest, Boolean, List<IFeedJoint>> triple =
-                    getFeedConnectionRequest(dataverseName, feed, cbfs.getDatasetName(), feedPolicy, mdTxnCtx);
-            FeedConnectionRequest connectionRequest = triple.first;
-            boolean createFeedIntakeJob = triple.second;
-            listener.registerFeedEventSubscriber(eventSubscriber);
-            subscriberRegistered = true;
-            if (createFeedIntakeJob) {
-                EntityId feedId = connectionRequest.getFeedJointKey().getFeedId();
-                Feed primaryFeed =
-                        MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName());
-                Pair<JobSpecification, IAdapterFactory> pair =
-                        FeedOperations.buildFeedIntakeJobSpec(primaryFeed, metadataProvider, policyAccessor);
-                // adapter configuration are valid at this stage
-                // register the feed joints (these are auto-de-registered)
-                int numOfPrividers = pair.second.getPartitionConstraint().getLocations().length;
-                for (IFeedJoint fj : triple.third) {
-                    listener.registerFeedJoint(fj, numOfPrividers);
-                }
-                FeedIntakeInfo activeJob = new FeedIntakeInfo(null, ActivityState.ACTIVE, feed.getFeedId(),
-                        triple.third.get(0), pair.first);
-                pair.first.setProperty(ActiveJobNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME, activeJob);
-                JobUtils.runJob(hcc, pair.first, false);
-                eventSubscriber.assertEvent(ActiveLifecycleEvent.FEED_INTAKE_STARTED);
-            } else {
-                for (IFeedJoint fj : triple.third) {
-                    listener.registerFeedJoint(fj, 0);
-                }
-            }
+            fc = new FeedConnection(dataverseName, feedName, datasetName, appliedFunctions, policyName,
+                    outputType.toString());
+            MetadataManager.INSTANCE.addFeedConnection(metadataProvider.getMetadataTxnContext(), fc);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-            bActiveTxn = false;
-            eventSubscriber.assertEvent(ActiveLifecycleEvent.FEED_COLLECT_STARTED);
-            if (Boolean.valueOf(metadataProvider.getConfig().get(ConnectFeedStatement.WAIT_FOR_COMPLETION))) {
-                eventSubscriber.assertEvent(ActiveLifecycleEvent.FEED_COLLECT_ENDED); // blocking call
-            }
         } catch (Exception e) {
-            if (bActiveTxn) {
-                abort(e, e, mdTxnCtx);
-            }
+            abort(e, e, mdTxnCtx);
             throw e;
         } finally {
             MetadataLockManager.INSTANCE.connectFeedEnd(dataverseName, dataverseName + "." + datasetName,
                     dataverseName + "." + feedName);
-            if (subscriberRegistered) {
-                listener.deregisterFeedEventSubscriber(eventSubscriber);
-            }
         }
     }
 
-    /**
-     * Generates a subscription request corresponding to a connect feed request. In addition, provides a boolean
-     * flag indicating if feed intake job needs to be started (source primary feed not found to be active).
-     *
-     * @param dataverse
-     * @param feed
-     * @param dataset
-     * @param feedPolicy
-     * @param mdTxnCtx
-     * @return
-     * @throws CompilationException
-     */
-    protected Triple<FeedConnectionRequest, Boolean, List<IFeedJoint>> getFeedConnectionRequest(String dataverse,
-            Feed feed, String dataset, FeedPolicyEntity feedPolicy, MetadataTransactionContext mdTxnCtx)
-            throws CompilationException {
-        IFeedJoint sourceFeedJoint;
-        FeedConnectionRequest request;
-        List<String> functionsToApply = new ArrayList<>();
-        boolean needIntakeJob = false;
-        List<IFeedJoint> jointsToRegister = new ArrayList<>();
-        FeedConnectionId connectionId = new FeedConnectionId(feed.getFeedId(), dataset);
-        FeedRuntimeType connectionLocation;
-        FeedJointKey feedJointKey = getFeedJointKey(feed, mdTxnCtx);
-        EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverse, feed.getFeedName());
-        FeedEventsListener listener =
-                (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
-        if (listener == null) {
-            throw new CompilationException("Feed Listener is not registered");
-        }
-
-        boolean isFeedJointAvailable = listener.isFeedJointAvailable(feedJointKey);
-        if (!isFeedJointAvailable) {
-            sourceFeedJoint = listener.getAvailableFeedJoint(feedJointKey);
-            if (sourceFeedJoint == null) { // the feed is currently not being ingested, i.e., it is unavailable.
-                connectionLocation = FeedRuntimeType.INTAKE;
-                EntityId sourceFeedId = feedJointKey.getFeedId(); // the root/primary feedId
-                Feed primaryFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverse, sourceFeedId.getEntityName());
-                FeedJointKey intakeFeedJointKey = new FeedJointKey(sourceFeedId, new ArrayList<>());
-                sourceFeedJoint = new FeedJoint(intakeFeedJointKey, primaryFeed.getFeedId(), connectionLocation,
-                        FeedJointType.INTAKE, connectionId);
-                jointsToRegister.add(sourceFeedJoint);
-                needIntakeJob = true;
-            } else {
-                connectionLocation = sourceFeedJoint.getConnectionLocation();
-            }
-
-            String[] functions = feedJointKey.getStringRep()
-                    .substring(sourceFeedJoint.getFeedJointKey().getStringRep().length()).trim().split(":");
-            for (String f : functions) {
-                if (f.trim().length() > 0) {
-                    functionsToApply.add(f);
-                }
-            }
-            // register the compute feed point that represents the final output from the collection of
-            // functions that will be applied.
-            if (!functionsToApply.isEmpty()) {
-                FeedJointKey computeFeedJointKey = new FeedJointKey(feed.getFeedId(), functionsToApply);
-                IFeedJoint computeFeedJoint = new FeedJoint(computeFeedJointKey, feed.getFeedId(),
-                        FeedRuntimeType.COMPUTE, FeedJointType.COMPUTE, connectionId);
-                jointsToRegister.add(computeFeedJoint);
-            }
-        } else {
-            sourceFeedJoint = listener.getFeedJoint(feedJointKey);
-            connectionLocation = sourceFeedJoint.getConnectionLocation();
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Feed joint " + sourceFeedJoint + " is available! need not apply any further computation");
-            }
-        }
-
-        request = new FeedConnectionRequest(sourceFeedJoint.getFeedJointKey(), connectionLocation, functionsToApply,
-                dataset, feedPolicy.getPolicyName(), feedPolicy.getProperties(), feed.getFeedId());
-
-        sourceFeedJoint.addConnectionRequest(request);
-        return new Triple<>(request, needIntakeJob, jointsToRegister);
-    }
-
-    /*
-     * Gets the feed joint corresponding to the feed definition. Tuples constituting the feed are
-     * available at this feed joint.
-     */
-    protected FeedJointKey getFeedJointKey(Feed feed, MetadataTransactionContext ctx) throws MetadataException {
-        Feed sourceFeed = feed;
-        List<String> appliedFunctions = new ArrayList<>();
-        while (sourceFeed.getFeedType().equals(IFeed.FeedType.SECONDARY)) {
-            if (sourceFeed.getAppliedFunction() != null) {
-                appliedFunctions.add(0, sourceFeed.getAppliedFunction().getName());
-            }
-            Feed parentFeed =
-                    MetadataManager.INSTANCE.getFeed(ctx, feed.getDataverseName(), sourceFeed.getSourceFeedName());
-            sourceFeed = parentFeed;
-        }
-
-        if (sourceFeed.getAppliedFunction() != null) {
-            appliedFunctions.add(0, sourceFeed.getAppliedFunction().getName());
-        }
-
-        return new FeedJointKey(sourceFeed.getFeedId(), appliedFunctions);
-    }
-
-    protected void handleDisconnectFeedStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc) throws Exception {
+    protected void handleDisconnectFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         DisconnectFeedStatement cfs = (DisconnectFeedStatement) stmt;
         String dataverseName = getActiveDataverse(cfs.getDataverseName());
         String datasetName = cfs.getDatasetName().getValue();
+        String feedName = cfs.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-
-        FeedMetadataUtil.validateIfDatasetExists(dataverseName, cfs.getDatasetName().getValue(), mdTxnCtx);
-        Feed feed = FeedMetadataUtil.validateIfFeedExists(dataverseName, cfs.getFeedName().getValue(), mdTxnCtx);
-        EntityId entityId = new EntityId(Feed.EXTENSION_NAME, feed.getDataverseName(), feed.getFeedName());
-        FeedConnectionId connectionId = new FeedConnectionId(feed.getFeedId(), cfs.getDatasetName().getValue());
-        IActiveLifecycleEventSubscriber eventSubscriber = new ActiveLifecycleEventSubscriber();
-        FeedEventsListener listener =
-                (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
-        if (listener == null || !listener.isEntityUsingDataset(dataverseName, datasetName)) {
-            throw new CompilationException("Feed " + feed.getFeedId().getEntityName()
-                    + " is currently not connected to " + cfs.getDatasetName().getValue() + ". Invalid operation!");
-        }
-        listener.registerFeedEventSubscriber(eventSubscriber);
         MetadataLockManager.INSTANCE.disconnectFeedBegin(dataverseName, dataverseName + "." + datasetName,
                 dataverseName + "." + cfs.getFeedName());
         try {
-            Dataset dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(),
-                    dataverseName, cfs.getDatasetName().getValue());
-            if (dataset == null) {
-                throw new CompilationException(
-                        "Unknown dataset :" + cfs.getDatasetName().getValue() + " in dataverse " + dataverseName);
+            FeedMetadataUtil.validateIfDatasetExists(dataverseName, cfs.getDatasetName().getValue(), mdTxnCtx);
+            FeedMetadataUtil.validateIfFeedExists(dataverseName, cfs.getFeedName().getValue(), mdTxnCtx);
+            FeedConnection fc = MetadataManager.INSTANCE.getFeedConnection(metadataProvider.getMetadataTxnContext(),
+                    dataverseName, feedName, datasetName);
+            if (fc == null) {
+                throw new CompilationException("Feed " + feedName + " is currently not connected to "
+                        + cfs.getDatasetName().getValue() + ". Invalid operation!");
             }
-            Pair<JobSpecification, Boolean> specDisconnectType =
-                    FeedOperations.buildDisconnectFeedJobSpec(connectionId);
-            JobSpecification jobSpec = specDisconnectType.first;
-            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-            bActiveTxn = false;
-            JobUtils.runJob(hcc, jobSpec, true);
-            eventSubscriber.assertEvent(ActiveLifecycleEvent.FEED_COLLECT_ENDED);
+            MetadataManager.INSTANCE.dropFeedConnection(mdTxnCtx, dataverseName, feedName, datasetName);
         } catch (Exception e) {
-            if (bActiveTxn) {
-                abort(e, e, mdTxnCtx);
-            }
+            abort(e, e, mdTxnCtx);
             throw e;
         } finally {
             MetadataLockManager.INSTANCE.disconnectFeedEnd(dataverseName, dataverseName + "." + datasetName,
@@ -2393,71 +2273,6 @@
         }
     }
 
-    protected void handleSubscribeFeedStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc) throws Exception {
-
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Subscriber Feed Statement :" + stmt);
-        }
-
-        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        boolean bActiveTxn = true;
-        metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        metadataProvider.setWriteTransaction(true);
-        metadataProvider.disableBlockingOperator();
-        SubscribeFeedStatement bfs = (SubscribeFeedStatement) stmt;
-        bfs.initialize(metadataProvider.getMetadataTxnContext());
-
-        CompiledSubscribeFeedStatement csfs =
-                new CompiledSubscribeFeedStatement(bfs.getSubscriptionRequest(), bfs.getVarCounter());
-        metadataProvider.getConfig().put(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
-        metadataProvider.getConfig().put(FeedActivityDetails.FEED_POLICY_NAME, "" + bfs.getPolicy());
-        metadataProvider.getConfig().put(FeedActivityDetails.COLLECT_LOCATIONS,
-                StringUtils.join(bfs.getLocations(), ','));
-
-        JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, bfs.getQuery(), csfs);
-        FeedConnectionId feedConnectionId = new FeedConnectionId(bfs.getSubscriptionRequest().getReceivingFeedId(),
-                bfs.getSubscriptionRequest().getTargetDataset());
-        String dataverse = feedConnectionId.getFeedId().getDataverse();
-        String dataset = feedConnectionId.getDatasetName();
-        MetadataLockManager.INSTANCE.subscribeFeedBegin(dataverse, dataverse + "." + dataset,
-                dataverse + "." + feedConnectionId.getFeedId().getEntityName());
-        try {
-            JobSpecification alteredJobSpec = FeedMetadataUtil.alterJobSpecificationForFeed(jobSpec, feedConnectionId,
-                    bfs.getSubscriptionRequest().getPolicyParameters());
-            FeedPolicyEntity policy = metadataProvider.findFeedPolicy(dataverse, bfs.getPolicy());
-            if (policy == null) {
-                policy = BuiltinFeedPolicies.getFeedPolicy(bfs.getPolicy());
-                if (policy == null) {
-                    throw new AlgebricksException("Unknown feed policy:" + bfs.getPolicy());
-                }
-            }
-            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-            bActiveTxn = false;
-
-            if (jobSpec != null) {
-                FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
-                        .getActiveEntityListener(bfs.getSubscriptionRequest().getReceivingFeedId());
-                FeedConnectJobInfo activeJob = new FeedConnectJobInfo(
-                        bfs.getSubscriptionRequest().getReceivingFeedId(), null, ActivityState.ACTIVE,
-                        new FeedConnectionId(bfs.getSubscriptionRequest().getReceivingFeedId(), dataset),
-                        listener.getSourceFeedJoint(), null, alteredJobSpec, policy.getProperties());
-                alteredJobSpec.setProperty(ActiveJobNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME, activeJob);
-                JobUtils.runJob(hcc, alteredJobSpec, false);
-            }
-
-        } catch (Exception e) {
-            LOGGER.log(Level.WARNING, e.getMessage(), e);
-            if (bActiveTxn) {
-                abort(e, e, mdTxnCtx);
-            }
-            throw e;
-        } finally {
-            MetadataLockManager.INSTANCE.subscribeFeedEnd(dataverse, dataverse + "." + dataset,
-                    dataverse + "." + feedConnectionId.getFeedId().getEntityName());
-        }
-    }
-
     protected void handleCompactStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         CompactStatement compactStatement = (CompactStatement) stmt;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
new file mode 100644
index 0000000..f8b5496
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
@@ -0,0 +1,397 @@
+/*
+ * 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.utils;
+
+import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.active.message.ActiveManagerMessage;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.messaging.api.ICCMessageBroker;
+import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.utils.StoragePathUtil;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.feed.management.FeedConnectionId;
+import org.apache.asterix.external.feed.management.FeedConnectionRequest;
+import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
+import org.apache.asterix.external.feed.watch.FeedActivityDetails;
+import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
+import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
+import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
+import org.apache.asterix.external.operators.FeedMetaOperatorDescriptor;
+import org.apache.asterix.external.util.FeedConstants;
+import org.apache.asterix.external.util.FeedUtils;
+import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
+import org.apache.asterix.lang.aql.statement.SubscribeFeedStatement;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.statement.DataverseDecl;
+import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
+import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
+import org.apache.asterix.metadata.feeds.LocationConstraint;
+import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
+import org.apache.asterix.runtime.job.listener.MultiTransactionJobletEventListenerFactory;
+import org.apache.asterix.runtime.utils.AppContextInfo;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
+import org.apache.asterix.translator.CompiledStatements;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.SessionConfig;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import org.apache.hyracks.algebricks.runtime.operators.std.StreamSelectRuntimeFactory;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.constraints.Constraint;
+import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
+import org.apache.hyracks.api.constraints.expressions.ConstantExpression;
+import org.apache.hyracks.api.constraints.expressions.ConstraintExpression;
+import org.apache.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import org.apache.hyracks.api.constraints.expressions.PartitionCountExpression;
+import org.apache.hyracks.api.constraints.expressions.PartitionLocationExpression;
+import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
+import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileSplit;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningWithMessageConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.file.FileRemoveOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.misc.ReplicateOperatorDescriptor;
+
+/**
+ * Provides helper method(s) for creating JobSpec for operations on a feed.
+ */
+public class FeedOperations {
+
+    private FeedOperations() {
+    }
+
+    private static Pair<JobSpecification, IAdapterFactory> buildFeedIntakeJobSpec(Feed feed,
+            MetadataProvider metadataProvider, FeedPolicyAccessor policyAccessor) throws Exception {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        spec.setFrameSize(FeedConstants.JobConstants.DEFAULT_FRAME_SIZE);
+        IAdapterFactory adapterFactory;
+        IOperatorDescriptor feedIngestor;
+        AlgebricksPartitionConstraint ingesterPc;
+        Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> t =
+                metadataProvider.buildFeedIntakeRuntime(spec, feed, policyAccessor);
+        feedIngestor = t.first;
+        ingesterPc = t.second;
+        adapterFactory = t.third;
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, feedIngestor, ingesterPc);
+        NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, nullSink, ingesterPc);
+        spec.connect(new OneToOneConnectorDescriptor(spec), feedIngestor, 0, nullSink, 0);
+        spec.addRoot(nullSink);
+        return Pair.of(spec, adapterFactory);
+    }
+
+    public static JobSpecification buildRemoveFeedStorageJob(Feed feed) throws AsterixException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        AlgebricksAbsolutePartitionConstraint allCluster = ClusterStateManager.INSTANCE.getClusterLocations();
+        Set<String> nodes = new TreeSet<>();
+        for (String node : allCluster.getLocations()) {
+            nodes.add(node);
+        }
+        AlgebricksAbsolutePartitionConstraint locations =
+                new AlgebricksAbsolutePartitionConstraint(nodes.toArray(new String[nodes.size()]));
+        FileSplit[] feedLogFileSplits =
+                FeedUtils.splitsForAdapter(feed.getDataverseName(), feed.getFeedName(), locations);
+        org.apache.hyracks.algebricks.common.utils.Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spC =
+                StoragePathUtil.splitProviderAndPartitionConstraints(feedLogFileSplits);
+        FileRemoveOperatorDescriptor frod = new FileRemoveOperatorDescriptor(spec, spC.first, true);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, frod, spC.second);
+        spec.addRoot(frod);
+        return spec;
+    }
+
+    private static JobSpecification getConnectionJob(SessionConfig sessionConfig, MetadataProvider metadataProvider,
+            FeedConnection feedConnection, String[] locations, ILangCompilationProvider compilationProvider,
+            IStorageComponentProvider storageComponentProvider, DefaultStatementExecutorFactory qtFactory,
+            IHyracksClientConnection hcc) throws AlgebricksException, RemoteException, ACIDException {
+        DataverseDecl dataverseDecl = new DataverseDecl(new Identifier(feedConnection.getDataverseName()));
+        FeedConnectionRequest fcr =
+                new FeedConnectionRequest(FeedRuntimeType.INTAKE, feedConnection.getAppliedFunctions(),
+                        feedConnection.getDatasetName(), feedConnection.getPolicyName(), feedConnection.getFeedId());
+        SubscribeFeedStatement subscribeStmt = new SubscribeFeedStatement(locations, fcr);
+        subscribeStmt.initialize(metadataProvider.getMetadataTxnContext());
+        List<Statement> statements = new ArrayList<>();
+        statements.add(dataverseDecl);
+        statements.add(subscribeStmt);
+        IStatementExecutor translator =
+                qtFactory.create(statements, sessionConfig, compilationProvider, storageComponentProvider);
+        // configure the metadata provider
+        metadataProvider.getConfig().put(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
+        metadataProvider.getConfig().put(FeedActivityDetails.FEED_POLICY_NAME, "" + subscribeStmt.getPolicy());
+        metadataProvider.getConfig().put(FeedActivityDetails.COLLECT_LOCATIONS,
+                StringUtils.join(subscribeStmt.getLocations(), ','));
+
+        CompiledStatements.CompiledSubscribeFeedStatement csfs = new CompiledStatements.CompiledSubscribeFeedStatement(
+                subscribeStmt.getSubscriptionRequest(), subscribeStmt.getVarCounter());
+        return translator.rewriteCompileQuery(hcc, metadataProvider, subscribeStmt.getQuery(), csfs);
+    }
+
+    private static JobSpecification combineIntakeCollectJobs(MetadataProvider metadataProvider, Feed feed,
+            JobSpecification intakeJob, List<JobSpecification> jobsList, List<FeedConnection> feedConnections,
+            String[] intakeLocations) throws AlgebricksException, HyracksDataException {
+        JobSpecification jobSpec = new JobSpecification(intakeJob.getFrameSize());
+
+        // copy ingestor
+        FeedIntakeOperatorDescriptor firstOp =
+                (FeedIntakeOperatorDescriptor) intakeJob.getOperatorMap().get(new OperatorDescriptorId(0));
+        FeedIntakeOperatorDescriptor ingestionOp;
+        if (firstOp.getAdaptorFactory() == null) {
+            ingestionOp = new FeedIntakeOperatorDescriptor(jobSpec, feed, firstOp.getAdaptorLibraryName(),
+                    firstOp.getAdaptorFactoryClassName(), firstOp.getAdapterOutputType(), firstOp.getPolicyAccessor(),
+                    firstOp.getOutputRecordDescriptors()[0]);
+        } else {
+            ingestionOp = new FeedIntakeOperatorDescriptor(jobSpec, feed, firstOp.getAdaptorFactory(),
+                    firstOp.getAdapterOutputType(), firstOp.getPolicyAccessor(),
+                    firstOp.getOutputRecordDescriptors()[0]);
+        }
+        // create replicator
+        ReplicateOperatorDescriptor replicateOp =
+                new ReplicateOperatorDescriptor(jobSpec, ingestionOp.getOutputRecordDescriptors()[0], jobsList.size());
+        jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), ingestionOp, 0, replicateOp, 0);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, ingestionOp, intakeLocations);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, replicateOp, intakeLocations);
+        // Loop over the jobs to copy operators and connections
+        Map<OperatorDescriptorId, OperatorDescriptorId> operatorIdMapping = new HashMap<>();
+        Map<ConnectorDescriptorId, ConnectorDescriptorId> connectorIdMapping = new HashMap<>();
+        Map<OperatorDescriptorId, List<LocationConstraint>> operatorLocations = new HashMap<>();
+        Map<OperatorDescriptorId, Integer> operatorCounts = new HashMap<>();
+        List<JobId> jobIds = new ArrayList<>();
+
+        for (int iter1 = 0; iter1 < jobsList.size(); iter1++) {
+            FeedConnection curFeedConnection = feedConnections.get(iter1);
+            JobSpecification subJob = jobsList.get(iter1);
+            operatorIdMapping.clear();
+            Map<OperatorDescriptorId, IOperatorDescriptor> operatorsMap = subJob.getOperatorMap();
+
+            FeedPolicyEntity feedPolicyEntity =
+                    FeedMetadataUtil.validateIfPolicyExists(curFeedConnection.getDataverseName(),
+                            curFeedConnection.getPolicyName(), metadataProvider.getMetadataTxnContext());
+
+            for (Map.Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operatorsMap.entrySet()) {
+                IOperatorDescriptor opDesc = entry.getValue();
+                OperatorDescriptorId oldId = opDesc.getOperatorId();
+                OperatorDescriptorId opId;
+                if (opDesc instanceof LSMTreeInsertDeleteOperatorDescriptor
+                        && ((LSMTreeInsertDeleteOperatorDescriptor) opDesc).isPrimary()) {
+                    String operandId = ((LSMTreeInsertDeleteOperatorDescriptor) opDesc).getIndexName();
+                    FeedMetaOperatorDescriptor metaOp = new FeedMetaOperatorDescriptor(jobSpec,
+                            new FeedConnectionId(ingestionOp.getEntityId(),
+                                    feedConnections.get(iter1).getDatasetName()),
+                            opDesc, feedPolicyEntity.getProperties(), FeedRuntimeType.STORE, false, operandId);
+                    opId = metaOp.getOperatorId();
+                    opDesc.setOperatorId(opId);
+                } else {
+                    if (opDesc instanceof AlgebricksMetaOperatorDescriptor) {
+                        AlgebricksMetaOperatorDescriptor algOp = (AlgebricksMetaOperatorDescriptor) opDesc;
+                        for (IPushRuntimeFactory runtimeFactory : algOp.getPipeline().getRuntimeFactories()) {
+                            if (runtimeFactory instanceof StreamSelectRuntimeFactory) {
+                                ((StreamSelectRuntimeFactory) runtimeFactory).retainMissing(true, 0);
+                            }
+                        }
+                    }
+                    opId = jobSpec.createOperatorDescriptorId(opDesc);
+                }
+                operatorIdMapping.put(oldId, opId);
+            }
+
+            // copy connectors
+            connectorIdMapping.clear();
+            for (Entry<ConnectorDescriptorId, IConnectorDescriptor> entry : subJob.getConnectorMap().entrySet()) {
+                IConnectorDescriptor connDesc = entry.getValue();
+                ConnectorDescriptorId newConnId;
+                if (entry.getKey().getId() == 0) {
+                    continue;
+                }
+                if (connDesc instanceof MToNPartitioningConnectorDescriptor) {
+                    MToNPartitioningConnectorDescriptor m2nConn = (MToNPartitioningConnectorDescriptor) connDesc;
+                    connDesc = new MToNPartitioningWithMessageConnectorDescriptor(jobSpec,
+                            m2nConn.getTuplePartitionComputerFactory());
+                    newConnId = connDesc.getConnectorId();
+                } else {
+                    newConnId = jobSpec.createConnectorDescriptor(connDesc);
+                }
+                connectorIdMapping.put(entry.getKey(), newConnId);
+            }
+
+            // make connections between operators
+            for (Entry<ConnectorDescriptorId,
+                    Pair<Pair<IOperatorDescriptor, Integer>,Pair<IOperatorDescriptor, Integer>>> entry :
+                          subJob.getConnectorOperatorMap().entrySet()) {
+                ConnectorDescriptorId newId = connectorIdMapping.get(entry.getKey());
+                IConnectorDescriptor connDesc = jobSpec.getConnectorMap().get(newId);
+                Pair<IOperatorDescriptor, Integer> leftOp = entry.getValue().getLeft();
+                Pair<IOperatorDescriptor, Integer> rightOp = entry.getValue().getRight();
+                IOperatorDescriptor leftOpDesc = jobSpec.getOperatorMap().get(leftOp.getLeft().getOperatorId());
+                IOperatorDescriptor rightOpDesc = jobSpec.getOperatorMap().get(rightOp.getLeft().getOperatorId());
+                if (leftOp.getLeft() instanceof FeedCollectOperatorDescriptor) {
+                    jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), replicateOp, iter1, leftOpDesc,
+                            leftOp.getRight());
+                    jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), leftOpDesc, leftOp.getRight(),
+                            rightOpDesc, rightOp.getRight());
+                } else {
+                    jobSpec.connect(connDesc, leftOpDesc, leftOp.getRight(), rightOpDesc, rightOp.getRight());
+                }
+            }
+
+            // prepare for setting partition constraints
+            operatorLocations.clear();
+            operatorCounts.clear();
+
+            for (Constraint constraint : subJob.getUserConstraints()) {
+                LValueConstraintExpression lexpr = constraint.getLValue();
+                ConstraintExpression cexpr = constraint.getRValue();
+                OperatorDescriptorId opId;
+                switch (lexpr.getTag()) {
+                    case PARTITION_COUNT:
+                        opId = ((PartitionCountExpression) lexpr).getOperatorDescriptorId();
+                        if (opId.getId() == 0) {
+                            continue;
+                        }
+                        operatorCounts.put(operatorIdMapping.get(opId), (int) ((ConstantExpression) cexpr).getValue());
+                        break;
+                    case PARTITION_LOCATION:
+                        opId = ((PartitionLocationExpression) lexpr).getOperatorDescriptorId();
+                        if (opId.getId() == 0) {
+                            continue;
+                        }
+                        IOperatorDescriptor opDesc = jobSpec.getOperatorMap().get(operatorIdMapping.get(opId));
+                        List<LocationConstraint> locations = operatorLocations.get(opDesc.getOperatorId());
+                        if (locations == null) {
+                            locations = new ArrayList<>();
+                            operatorLocations.put(opDesc.getOperatorId(), locations);
+                        }
+                        String location = (String) ((ConstantExpression) cexpr).getValue();
+                        LocationConstraint lc =
+                                new LocationConstraint(location, ((PartitionLocationExpression) lexpr).getPartition());
+                        locations.add(lc);
+                        break;
+                    default:
+                        break;
+                }
+            }
+
+            // set absolute location constraints
+            for (Entry<OperatorDescriptorId, List<LocationConstraint>> entry : operatorLocations.entrySet()) {
+                IOperatorDescriptor opDesc = jobSpec.getOperatorMap().get(entry.getKey());
+                // why do we need to sort?
+                Collections.sort(entry.getValue(), (LocationConstraint o1, LocationConstraint o2) -> {
+                    return o1.partition - o2.partition;
+                });
+                String[] locations = new String[entry.getValue().size()];
+                for (int j = 0; j < locations.length; ++j) {
+                    locations[j] = entry.getValue().get(j).location;
+                }
+                PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, opDesc, locations);
+            }
+
+            // set count constraints
+            for (Entry<OperatorDescriptorId, Integer> entry : operatorCounts.entrySet()) {
+                IOperatorDescriptor opDesc = jobSpec.getOperatorMap().get(entry.getKey());
+                if (!operatorLocations.keySet().contains(entry.getKey())) {
+                    PartitionConstraintHelper.addPartitionCountConstraint(jobSpec, opDesc, entry.getValue());
+                }
+            }
+            // roots
+            for (OperatorDescriptorId root : subJob.getRoots()) {
+                jobSpec.addRoot(jobSpec.getOperatorMap().get(operatorIdMapping.get(root)));
+            }
+            jobIds.add(((JobEventListenerFactory) subJob.getJobletEventListenerFactory()).getJobId());
+        }
+
+        // jobEventListenerFactory
+        jobSpec.setJobletEventListenerFactory(new MultiTransactionJobletEventListenerFactory(jobIds, true));
+        // useConnectorSchedulingPolicy
+        jobSpec.setUseConnectorPolicyForScheduling(jobsList.get(0).isUseConnectorPolicyForScheduling());
+        // connectorAssignmentPolicy
+        jobSpec.setConnectorPolicyAssignmentPolicy(jobsList.get(0).getConnectorPolicyAssignmentPolicy());
+        return jobSpec;
+    }
+
+    public static Pair<JobSpecification, AlgebricksAbsolutePartitionConstraint> buildStartFeedJob(
+            SessionConfig sessionConfig, MetadataProvider metadataProvider, Feed feed,
+            List<FeedConnection> feedConnections, ILangCompilationProvider compilationProvider,
+            IStorageComponentProvider storageComponentProvider, DefaultStatementExecutorFactory qtFactory,
+            IHyracksClientConnection hcc) throws Exception {
+        FeedPolicyAccessor fpa = new FeedPolicyAccessor(new HashMap<>());
+        // TODO: Change the default Datasource to use all possible partitions
+        Pair<JobSpecification, IAdapterFactory> intakeInfo = buildFeedIntakeJobSpec(feed, metadataProvider, fpa);
+        //TODO: Add feed policy accessor
+        List<JobSpecification> jobsList = new ArrayList<>();
+        // Construct the ingestion Job
+        JobSpecification intakeJob = intakeInfo.getLeft();
+        IAdapterFactory ingestionAdaptorFactory = intakeInfo.getRight();
+        String[] ingestionLocations = ingestionAdaptorFactory.getPartitionConstraint().getLocations();
+        // Add connection job
+        for (FeedConnection feedConnection : feedConnections) {
+            JobSpecification connectionJob = getConnectionJob(sessionConfig, metadataProvider, feedConnection,
+                    ingestionLocations, compilationProvider, storageComponentProvider, qtFactory, hcc);
+            jobsList.add(connectionJob);
+        }
+        return Pair.of(combineIntakeCollectJobs(metadataProvider, feed, intakeJob, jobsList, feedConnections,
+                ingestionLocations), intakeInfo.getRight().getPartitionConstraint());
+    }
+
+    public static void SendStopMessageToNode(EntityId feedId, String intakeNodeLocation, Integer partition)
+            throws Exception {
+        ActiveManagerMessage stopFeedMessage = new ActiveManagerMessage(ActiveManagerMessage.STOP_ACTIVITY, "SRC",
+                new ActiveRuntimeId(feedId, FeedIntakeOperatorNodePushable.class.getSimpleName(), partition));
+        SendActiveMessage(stopFeedMessage, intakeNodeLocation);
+    }
+
+    private static void SendActiveMessage(ActiveManagerMessage activeManagerMessage, String nodeId) throws Exception {
+        ICCMessageBroker messageBroker =
+                (ICCMessageBroker) AppContextInfo.INSTANCE.getCCApplicationContext().getMessageBroker();
+        messageBroker.sendApplicationMessageToNC(activeManagerMessage, nodeId);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
index 390a955..cdaaf1b 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
@@ -1,13 +1,14 @@
-{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DatatypeDataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 13, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DatatypeDataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DatatypeDataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DatatypeDataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DatatypeDataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ] ], "PrimaryKey": [ [ "DataverseName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DatatypeDataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 14, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Feed", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FeedName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FeedName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 10, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 12, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DatatypeDataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DatatypeDataverseName": "Metadata", "DatatypeName": "IndexRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "DatatypeDataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DatatypeDataverseName": "Metadata", "DatatypeName": "NodeRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "NodeName" ] ], "PrimaryKey": [ [ "NodeName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DatatypeDataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "GroupName" ] ], "PrimaryKey": [ [ "GroupName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "DatasetId": 6, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DatatypeDataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 13, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DatatypeDataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DatatypeDataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DatatypeDataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DatatypeDataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ] ], "PrimaryKey": [ [ "DataverseName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DatatypeDataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 14, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FeedName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FeedName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedConnection", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FeedName" ], [ "DatasetName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FeedName" ], [ "DatasetName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DatatypeDataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DatatypeDataverseName": "Metadata", "DatatypeName": "IndexRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DatatypeDataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DatatypeDataverseName": "Metadata", "DatatypeName": "NodeRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "NodeName" ] ], "PrimaryKey": [ [ "NodeName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DatatypeDataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "GroupName" ] ], "PrimaryKey": [ [ "GroupName" ] ], "Autogenerated": false }, "Hints": {{  }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 6, "PendingOp": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index 91639a8..7229aa8 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -1,65 +1,66 @@
-{ "DataverseName": "Metadata", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [  ] } }, "Timestamp": "Wed Sep 14 23:26:12 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeDataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetType", "FieldType": "string", "IsNullable": false }, { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicyProperties", "FieldType": "DatasetRecordType_CompactionPolicyProperties", "IsNullable": false }, { "FieldName": "InternalDetails", "FieldType": "DatasetRecordType_InternalDetails", "IsNullable": true }, { "FieldName": "ExternalDetails", "FieldType": "DatasetRecordType_ExternalDetails", "IsNullable": true }, { "FieldName": "Hints", "FieldType": "DatasetRecordType_Hints", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetId", "FieldType": "int32", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_CompactionPolicyProperties_Item" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "DatasetRecordType_ExternalDetails_Properties", "IsNullable": false }, { "FieldName": "LastRefreshTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "TransactionState", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_ExternalDetails_Properties_Item" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "DatasetRecordType_Hints_Item" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningStrategy", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningKey", "FieldType": "DatasetRecordType_InternalDetails_PartitioningKey", "IsNullable": false }, { "FieldName": "PrimaryKey", "FieldType": "DatasetRecordType_InternalDetails_PrimaryKey", "IsNullable": false }, { "FieldName": "Autogenerated", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PartitioningKey_Item" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PrimaryKey_Item" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false }, { "FieldName": "Type", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Derived", "FieldType": "DatatypeRecordType_Derived", "IsNullable": true }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsAnonymous", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Record", "FieldType": "DatatypeRecordType_Derived_Record", "IsNullable": true }, { "FieldName": "UnorderedList", "FieldType": "string", "IsNullable": true }, { "FieldName": "OrderedList", "FieldType": "string", "IsNullable": true } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Fields", "FieldType": "DatatypeRecordType_Derived_Record_Fields", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatatypeRecordType_Derived_Record_Fields_Item" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FieldType", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsNullable", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DataFormat", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileNumber", "FieldType": "int32", "IsNullable": false }, { "FieldName": "FileName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileSize", "FieldType": "int64", "IsNullable": false }, { "FieldName": "FileModTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Description", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "FeedPolicyRecordType_Properties", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedPolicyRecordType_Properties_Item" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Function", "FieldType": "string", "IsNullable": true }, { "FieldName": "FeedType", "FieldType": "string", "IsNullable": false }, { "FieldName": "PrimaryTypeDetails", "FieldType": "FeedRecordType_PrimaryTypeDetails", "IsNullable": true }, { "FieldName": "SecondaryTypeDetails", "FieldType": "FeedRecordType_SecondaryTypeDetails", "IsNullable": true }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_PrimaryTypeDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "AdapterName", "FieldType": "string", "IsNullable": false }, { "FieldName": "AdapterConfiguration", "FieldType": "FeedRecordType_PrimaryTypeDetails_AdapterConfiguration", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_PrimaryTypeDetails_AdapterConfiguration", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "DatasetRecordType_ExternalDetails_Properties_Item" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_SecondaryTypeDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "SourceFeedName", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Arity", "FieldType": "string", "IsNullable": false }, { "FieldName": "Params", "FieldType": "FunctionRecordType_Params", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "Definition", "FieldType": "string", "IsNullable": false }, { "FieldName": "Language", "FieldType": "string", "IsNullable": false }, { "FieldName": "Kind", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Params", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "SearchKey", "FieldType": "IndexRecordType_SearchKey", "IsNullable": false }, { "FieldName": "IsPrimary", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "IndexRecordType_SearchKey_Item" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NodeNames", "FieldType": "NodeGroupRecordType_NodeNames", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType_NodeNames", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NumberOfCores", "FieldType": "int64", "IsNullable": false }, { "FieldName": "WorkingMemorySize", "FieldType": "int64", "IsNullable": false } ] } }, "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "binary", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "missing", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Timestamp": "Wed Jul 27 00:27:47 AST 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [  ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeDataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetType", "FieldType": "string", "IsNullable": false }, { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicyProperties", "FieldType": "DatasetRecordType_CompactionPolicyProperties", "IsNullable": false }, { "FieldName": "InternalDetails", "FieldType": "DatasetRecordType_InternalDetails", "IsNullable": true }, { "FieldName": "ExternalDetails", "FieldType": "DatasetRecordType_ExternalDetails", "IsNullable": true }, { "FieldName": "Hints", "FieldType": "DatasetRecordType_Hints", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetId", "FieldType": "int32", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_CompactionPolicyProperties_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "DatasetRecordType_ExternalDetails_Properties", "IsNullable": false }, { "FieldName": "LastRefreshTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "TransactionState", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_ExternalDetails_Properties_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "DatasetRecordType_Hints_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningStrategy", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningKey", "FieldType": "DatasetRecordType_InternalDetails_PartitioningKey", "IsNullable": false }, { "FieldName": "PrimaryKey", "FieldType": "DatasetRecordType_InternalDetails_PrimaryKey", "IsNullable": false }, { "FieldName": "Autogenerated", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PartitioningKey_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PrimaryKey_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false }, { "FieldName": "Type", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Derived", "FieldType": "DatatypeRecordType_Derived", "IsNullable": true }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsAnonymous", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Record", "FieldType": "DatatypeRecordType_Derived_Record", "IsNullable": true }, { "FieldName": "UnorderedList", "FieldType": "string", "IsNullable": true }, { "FieldName": "OrderedList", "FieldType": "string", "IsNullable": true } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Fields", "FieldType": "DatatypeRecordType_Derived_Record_Fields", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatatypeRecordType_Derived_Record_Fields_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FieldType", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsNullable", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DataFormat", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileNumber", "FieldType": "int32", "IsNullable": false }, { "FieldName": "FileName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileSize", "FieldType": "int64", "IsNullable": false }, { "FieldName": "FileModTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "AppliedFunctions", "FieldType": "FeedConnectionRecordType_AppliedFunctions", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType_AppliedFunctions", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Description", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "FeedPolicyRecordType_Properties", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedPolicyRecordType_Properties_Item" }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "AdapterName", "FieldType": "string", "IsNullable": false }, { "FieldName": "AdapterConfiguration", "FieldType": "FeedRecordType_AdapterConfiguration", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedRecordType_AdapterConfiguration_Item" }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Arity", "FieldType": "string", "IsNullable": false }, { "FieldName": "Params", "FieldType": "FunctionRecordType_Params", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "Definition", "FieldType": "string", "IsNullable": false }, { "FieldName": "Language", "FieldType": "string", "IsNullable": false }, { "FieldName": "Kind", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Params", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "SearchKey", "FieldType": "IndexRecordType_SearchKey", "IsNullable": false }, { "FieldName": "IsPrimary", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "IndexRecordType_SearchKey_Item" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:22 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NodeNames", "FieldType": "NodeGroupRecordType_NodeNames", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType_NodeNames", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NumberOfCores", "FieldType": "int64", "IsNullable": false }, { "FieldName": "WorkingMemorySize", "FieldType": "int64", "IsNullable": false } ] } }, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "binary", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "missing", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Timestamp": "Fri Oct 21 10:29:21 PDT 2016" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
index 3a48c22..0521e37 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
@@ -1,13 +1,14 @@
-{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ [ "NodeName" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ [ "GroupName" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:20 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedConnection", "IndexName": "FeedConnection", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ], [ "DatasetName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ [ "NodeName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ [ "GroupName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm
index da05a15..a5ae0e4 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm
@@ -11,3 +11,4 @@
 { "dv1": "Metadata", "dv2": "Metadata" }
 { "dv1": "Metadata", "dv2": "Metadata" }
 { "dv1": "Metadata", "dv2": "Metadata" }
+{ "dv1": "Metadata", "dv2": "Metadata" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_feed_20M/big_object_feed_20M.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_feed_20M/big_object_feed_20M.2.update.aql
index 14acbe8..e360d8c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_feed_20M/big_object_feed_20M.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_feed_20M/big_object_feed_20M.2.update.aql
@@ -28,3 +28,5 @@
 set wait-for-completion-feed "true";
 
 connect feed BigFeed to dataset testds;
+
+start feed BigFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/typed_adapter/typed_adapter.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/typed_adapter/typed_adapter.4.update.aql
index 8dc330b..3be4db8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/typed_adapter/typed_adapter.4.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/typed_adapter/typed_adapter.4.update.aql
@@ -30,3 +30,5 @@
 set wait-for-completion-feed "true";
 
 connect feed TestTypedAdapterFeed to dataset TweetsTestAdapter;
+
+start feed TestTypedAdapterFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-csv/change-feed-with-meta-csv.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-csv/change-feed-with-meta-csv.2.update.aql
index c83a6c0..da5331a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-csv/change-feed-with-meta-csv.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-csv/change-feed-with-meta-csv.2.update.aql
@@ -24,4 +24,6 @@
  use dataverse test;
 
  set wait-for-completion-feed "true";
- connect feed TestFeed to dataset TestDS;
\ No newline at end of file
+ connect feed TestFeed to dataset TestDS;
+
+ start feed TestFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-after-ingest/change-feed-with-meta-pk-in-meta-index-after-ingest.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-after-ingest/change-feed-with-meta-pk-in-meta-index-after-ingest.2.update.aql
index be3b808..59c04ff 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-after-ingest/change-feed-with-meta-pk-in-meta-index-after-ingest.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-after-ingest/change-feed-with-meta-pk-in-meta-index-after-ingest.2.update.aql
@@ -26,3 +26,5 @@
 
 set wait-for-completion-feed "true";
 connect feed KVChangeStream to dataset KVStore;
+
+start feed KVChangeStream;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-after-ingest/change-feed-with-meta-pk-in-meta-index-after-ingest.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-after-ingest/change-feed-with-meta-pk-in-meta-index-after-ingest.4.query.aql
index 190e472..5ef3a13 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-after-ingest/change-feed-with-meta-pk-in-meta-index-after-ingest.4.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-after-ingest/change-feed-with-meta-pk-in-meta-index-after-ingest.4.query.aql
@@ -26,6 +26,6 @@
 
 count(
     for $d in dataset KVStore
-    where meta($d).vbucket = 5
+    where meta($d).vbucket = 8
     return $d
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-in-meta/change-feed-with-meta-pk-in-meta-index-in-meta.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-in-meta/change-feed-with-meta-pk-in-meta-index-in-meta.2.update.aql
index 81fffc5..6d6c4ef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-in-meta/change-feed-with-meta-pk-in-meta-index-in-meta.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-in-meta/change-feed-with-meta-pk-in-meta-index-in-meta.2.update.aql
@@ -26,3 +26,5 @@
 
 set wait-for-completion-feed "true";
 connect feed KVChangeStream to dataset KVStore;
+
+start feed KVChangeStream;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest/change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest/change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest.2.update.aql
index 37d67d0..11f80f1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest/change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest/change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest.2.update.aql
@@ -26,3 +26,5 @@
 
 set wait-for-completion-feed "true";
 connect feed KVChangeStream to dataset KVStore;
+
+start feed KVChangeStream;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-open-index-in-value/change-feed-with-meta-pk-in-meta-open-index-in-value.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-open-index-in-value/change-feed-with-meta-pk-in-meta-open-index-in-value.2.update.aql
index ae5390b..748af2e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-open-index-in-value/change-feed-with-meta-pk-in-meta-open-index-in-value.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-open-index-in-value/change-feed-with-meta-pk-in-meta-open-index-in-value.2.update.aql
@@ -26,3 +26,5 @@
 
 set wait-for-completion-feed "true";
 connect feed KVChangeStream to dataset KVStore;
+
+start feed KVChangeStream;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-open-index-with-missing/change-feed-with-meta-pk-in-meta-open-index-with-missing.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-open-index-with-missing/change-feed-with-meta-pk-in-meta-open-index-with-missing.2.update.aql
index 9b34e70..00c8645 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-open-index-with-missing/change-feed-with-meta-pk-in-meta-open-index-with-missing.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta-open-index-with-missing/change-feed-with-meta-pk-in-meta-open-index-with-missing.2.update.aql
@@ -26,3 +26,5 @@
 
 set wait-for-completion-feed "true";
 connect feed KVChangeStream to dataset KVStore;
+
+start feed KVChangeStream;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.2.update.aql
index 5951ac8..fc4926b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.2.update.aql
@@ -25,3 +25,5 @@
 
 set wait-for-completion-feed "true";
 connect feed KVChangeStream to dataset KVStore;
+
+start feed KVChangeStream;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-with-mixed-index/change-feed-with-meta-with-mixed-index.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-with-mixed-index/change-feed-with-meta-with-mixed-index.2.update.aql
index 9fa0c3d..03d1ecf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-with-mixed-index/change-feed-with-meta-with-mixed-index.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed-with-meta-with-mixed-index/change-feed-with-meta-with-mixed-index.2.update.aql
@@ -26,3 +26,5 @@
 
 set wait-for-completion-feed "true";
 connect feed KVChangeStream to dataset KVStore;
+
+start feed KVChangeStream;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed/change-feed.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed/change-feed.2.update.aql
index ff6efcb..9134586 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed/change-feed.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/change-feed/change-feed.2.update.aql
@@ -25,3 +25,5 @@
 
 set wait-for-completion-feed "true";
 connect feed ChangeFeed to dataset FeedDataset;
+
+start feed ChangeFeed;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.0.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.0.ddl.aql
new file mode 100644
index 0000000..8a13210
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.0.ddl.aql
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+drop dataverse feeds if exists;
+create dataverse feeds;
+use dataverse feeds;
+
+create type Tweet as closed
+{
+  id: int64
+}
+
+create dataset Tweets0 (Tweet)
+primary key id;
+
+create dataset Tweets1 (Tweet)
+primary key id;
+
+create dataset Tweets2 (Tweet)
+primary key id;
+
+create dataset Tweets3 (Tweet)
+primary key id;
+
+create feed TwitterFeed using push_twitter(
+("type-name"="Tweet"),
+("format"="twitter-status"),
+("consumer.key"="********************"),
+("consumer.secret"="********************"),
+("access.token"="********************"),
+("access.token.secret"="********************"));
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.1.update.aql
similarity index 74%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.1.update.aql
index 37661b7..1870618 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.1.update.aql
@@ -16,10 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
+use dataverse feeds;
+connect feed TwitterFeed to dataset Tweets0;
 
-public interface IFeedWorkManager {
+use dataverse feeds;
+connect feed TwitterFeed to dataset Tweets1;
 
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
+use dataverse feeds;
+connect feed TwitterFeed to dataset Tweets2;
 
-}
+use dataverse feeds;
+connect feed TwitterFeed to dataset Tweets3;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.2.update.aql
similarity index 82%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.2.update.aql
index 37661b7..55e3d23 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.2.update.aql
@@ -16,10 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+use dataverse feeds;
+disconnect feed TwitterFeed from dataset Tweets2;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.3.query.aql
similarity index 82%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.3.query.aql
index 37661b7..b8a70a6 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.3.query.aql
@@ -16,10 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+use dataverse Metadata;
+for $x in dataset Metadata.FeedConnection return $x.DatasetName
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.4.ddl.aql
similarity index 82%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.4.ddl.aql
index 37661b7..67a78cb 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed/connect-feed.4.ddl.aql
@@ -16,10 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+drop dataverse feeds;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.2.update.aql
index 3d7fdbf..c0e069e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.2.update.aql
@@ -28,3 +28,5 @@
 set wait-for-completion-feed "false";
 
 connect feed TweetFeed to dataset Tweets;
+
+start feed TweetFeed;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.3.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.3.server.aql
index 5b610de..066c912 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.3.server.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.3.server.aql
@@ -23,4 +23,4 @@
  * Expected Res : Success
  * Date         : 24th Feb 2016
  */
-start client 10001 file-client localhost ../asterix-app/data/twitter/tw_messages.adm 500 50 1000
\ No newline at end of file
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/twitter/tw_messages.adm 500 50 1000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.5.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.5.update.aql
index 0862bae..7670443 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.5.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-push-socket/feed-push-socket.5.update.aql
@@ -25,4 +25,5 @@
  */
 
 use dataverse experiments;
+stop feed TweetFeed;
 disconnect feed TweetFeed from dataset Tweets;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-function/feed-with-external-function.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-function/feed-with-external-function.1.ddl.aql
index ce1d7fc..58b9174 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-function/feed-with-external-function.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-function/feed-with-external-function.1.ddl.aql
@@ -51,8 +51,7 @@
 using localfs
 (("type-name"="TweetInputType"),
 ("path"="asterix_nc1://data/twitter/obamatweets.adm"),
-("format"="adm"))
-apply function testlib#parseTweet;
+("format"="adm"));
 
 create dataset TweetsFeedIngest(TweetOutputType)
 primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-function/feed-with-external-function.3.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-function/feed-with-external-function.3.update.aql
index ea1a379..7bd3b43 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-function/feed-with-external-function.3.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-function/feed-with-external-function.3.update.aql
@@ -29,4 +29,7 @@
 
 set wait-for-completion-feed "true";
 
-connect feed TweetFeed to dataset TweetsFeedIngest;
+connect feed TweetFeed to dataset TweetsFeedIngest
+apply function testlib#parseTweet;
+
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser-with-open-index/feed-with-external-parser-with-open-index.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser-with-open-index/feed-with-external-parser-with-open-index.4.update.aql
index 5105751..ad54aa5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser-with-open-index/feed-with-external-parser-with-open-index.4.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser-with-open-index/feed-with-external-parser-with-open-index.4.update.aql
@@ -29,3 +29,5 @@
 set wait-for-completion-feed "true";
 
 connect feed CondorFeed to dataset Condor;
+
+start feed CondorFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser-with-two-open-indexes/feed-with-external-parser-with-two-open-indexes.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser-with-two-open-indexes/feed-with-external-parser-with-two-open-indexes.4.update.aql
index a9e3435..40ed9fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser-with-two-open-indexes/feed-with-external-parser-with-two-open-indexes.4.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser-with-two-open-indexes/feed-with-external-parser-with-two-open-indexes.4.update.aql
@@ -29,3 +29,5 @@
 set wait-for-completion-feed "true";
 
 connect feed CondorFeed to dataset Condor;
+
+start feed CondorFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser/feed-with-external-parser.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser/feed-with-external-parser.4.update.aql
index e1e93ca..11428b6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser/feed-with-external-parser.4.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-external-parser/feed-with-external-parser.4.update.aql
@@ -28,3 +28,5 @@
 set wait-for-completion-feed "true";
 
 connect feed CondorFeed to dataset Condor;
+
+start feed CondorFeed;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-filtered-dataset/feed-with-filtered-dataset.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-filtered-dataset/feed-with-filtered-dataset.2.update.aql
index 36564a3..553f09c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-filtered-dataset/feed-with-filtered-dataset.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-filtered-dataset/feed-with-filtered-dataset.2.update.aql
@@ -25,3 +25,5 @@
 use dataverse test;
 set wait-for-completion-feed "true";
 connect feed MessageFeed to dataset FacebookMessages;
+
+start feed MessageFeed;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-meta-pk-in-meta/feed-with-meta-pk-in-meta.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-meta-pk-in-meta/feed-with-meta-pk-in-meta.2.update.aql
index c4748ea..d24d9c0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-meta-pk-in-meta/feed-with-meta-pk-in-meta.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-meta-pk-in-meta/feed-with-meta-pk-in-meta.2.update.aql
@@ -25,3 +25,5 @@
 
 set wait-for-completion-feed "true";
 connect feed KVStream to dataset KVStore;
+
+start feed KVStream;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-multiple-indexes/feed-with-multiple-indexes.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-multiple-indexes/feed-with-multiple-indexes.4.update.aql
index e1e93ca..91bc5dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-multiple-indexes/feed-with-multiple-indexes.4.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feed-with-multiple-indexes/feed-with-multiple-indexes.4.update.aql
@@ -28,3 +28,5 @@
 set wait-for-completion-feed "true";
 
 connect feed CondorFeed to dataset Condor;
+
+start feed CondorFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql
index a83b34a..c3e671b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql
@@ -17,8 +17,8 @@
  * under the License.
  */
 /*
- * Description  : Create a feed dataset that uses the feed simulator adapter. 
-                  Begin ingestion and verify contents of the dataset post completion.  
+ * Description  : Create a feed dataset that uses the feed simulator adapter.
+                  Begin ingestion and verify contents of the dataset post completion.
  * Expected Res : Success
  * Date         : 24th Dec 2012
  */
@@ -28,3 +28,5 @@
 set wait-for-completion-feed "true";
 
 connect feed TweetFeed to dataset Tweets;
+
+start feed TweetFeed;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql
index 5a2a712..68fead0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql
@@ -20,6 +20,7 @@
  * Description  : Create a feed dataset with an associated function and verify contents in Metadata
  * Expected Res : Success
  * Date         : 24th Dec 2012
+ * Update       : 10th Oct 2016
  */
 
 drop dataverse feeds if exists;
@@ -32,7 +33,7 @@
   location : string,
   text : string,
   timestamp : string
-}      
+}
 
 create dataset Tweets(TweetType)
 primary key id;
@@ -43,5 +44,5 @@
 
 create feed TweetFeed
 using localfs
-(("type-name"="TweetType"),("path"="asterix_nc1://data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"))
-apply function feed_processor;
+(("type-name"="TweetType"),("path"="asterix_nc1://data/twitter/obamatweets.adm"),("format"="adm")
+,("tuple-interval"="10"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.2.update.aql
index 0b7156e..343fb52 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.2.update.aql
@@ -20,4 +20,9 @@
  * Description  : Create a feed dataset with an associated function and verify contents in Metadata
  * Expected Res : Success
  * Date         : 24th Dec 2012
+ * Update       : 10th Oct 2016
  */
+use dataverse feeds;
+
+connect feed TweetFeed to dataset Tweets
+apply function feed_processor;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql
index f4da63e..b9a4dea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql
@@ -20,8 +20,9 @@
  * Description  : Create a feed dataset with an associated function and verify contents in Metadata
  * Expected Res : Success
  * Date         : 24th Dec 2012
+ * Update       : 10th Oct 2016
  */
 
-for $x in dataset('Metadata.Feed')
+for $x in dataset('Metadata.FeedConnection')
 where $x.DataverseName='feeds' and $x.FeedName='TweetFeed'
 return $x
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_07/feeds_07.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_07/feeds_07.2.update.aql
index 759f9c0..481b4ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_07/feeds_07.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_07/feeds_07.2.update.aql
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description  : Create a feed using the synthetic feed simulator adapter.                   
+ * Description  : Create a feed using the synthetic feed simulator adapter.
                   Create a dataset that has an associated rtree index.
                   The synthetic feed simulator uses the Social-Data generator to generate data and simulate a feed.
                   The feed lasts a configured duration with data arriving at a configured rate (tweets per second).
@@ -26,9 +26,11 @@
  * Expected Res : Success
  * Date         : 8th Feb 2014
  */
-  
+
 use dataverse feeds;
 
 set wait-for-completion-feed "true";
 
 connect feed SyntheticTweetFeed to dataset SyntheticTweets;
+
+start feed SyntheticTweetFeed;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_08/feeds_08.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_08/feeds_08.2.update.aql
index 4a1a6a4..0a16f64 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_08/feeds_08.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_08/feeds_08.2.update.aql
@@ -28,9 +28,11 @@
  * Expected Res : Success
  * Date         : 8th Feb 2014
  */
-  
+
 use dataverse feeds;
 
 set wait-for-completion-feed "true";
 
 connect feed SyntheticTweetFeed to dataset SyntheticTweets;
+
+start feed SyntheticTweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.2.update.aql
index cab9213..71ba44c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.2.update.aql
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description  : Create a feed using the synthetic feed simulator adapter.                   
+ * Description  : Create a feed using the synthetic feed simulator adapter.
                   Create a dataset that has an associated ngram index.
                   The synthetic feed simulator uses the Social-Data generator to generate data and simulate a feed.
                   The feed lasts a configured duration with data arriving at a configured rate (tweets per second).
@@ -26,9 +26,11 @@
  * Expected Res : Success
  * Date         : 2nd April 2014
  */
-  
+
 use dataverse feeds_09;
 
 set wait-for-completion-feed "true";
 
 connect feed SyntheticTweetFeed to dataset SyntheticTweets;
+
+start feed SyntheticTweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.2.update.aql
index d772dd2..e016ce0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.2.update.aql
@@ -25,9 +25,11 @@
  * Expected Res : Success
  * Date         : 6th Feb 2014
  */
-  
+
 use dataverse feeds_10;
 
 set wait-for-completion-feed "true";
 
 connect feed TweetFeed to dataset Tweets;
+
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.2.update.aql
index 81e2dfe..39bfb31 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.2.update.aql
@@ -17,8 +17,8 @@
  * under the License.
  */
 /*
- * Description  : Create a feed dataset that uses the feed simulator adapter. 
-                  Begin ingestion using a fully qualified name and verify contents of the dataset post completion.  
+ * Description  : Create a feed dataset that uses the feed simulator adapter.
+                  Begin ingestion using a fully qualified name and verify contents of the dataset post completion.
  * Expected Res : Success
  * Date         : 24th Dec 2012
  */
@@ -26,5 +26,7 @@
 use dataverse feeds;
 
 set wait-for-completion-feed "true";
- 
+
 connect feed feeds.TweetFeed to dataset Tweets;
+
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.2.update.aql
index 04f2c7c..ce22ebb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.2.update.aql
@@ -17,8 +17,8 @@
  * under the License.
  */
 /*
- * Description  : Create a feed dataset that uses the feed simulator adapter. 
-                  Begin ingestion using a fully qualified name and verify contents of the dataset post completion.  
+ * Description  : Create a feed dataset that uses the feed simulator adapter.
+                  Begin ingestion using a fully qualified name and verify contents of the dataset post completion.
  * Expected Res : Success
  * Date         : 24th Dec 2012
  */
@@ -26,5 +26,7 @@
 use dataverse feeds_12;
 
 set wait-for-completion-feed "true";
- 
+
 connect feed TweetFeed to dataset Tweets;
+
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql
index 81e2dfe..1a7cd01 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql
@@ -17,8 +17,8 @@
  * under the License.
  */
 /*
- * Description  : Create a feed dataset that uses the feed simulator adapter. 
-                  Begin ingestion using a fully qualified name and verify contents of the dataset post completion.  
+ * Description  : Create a feed dataset that uses the feed simulator adapter.
+                  Begin ingestion using a fully qualified name and verify contents of the dataset post completion.
  * Expected Res : Success
  * Date         : 24th Dec 2012
  */
@@ -26,5 +26,7 @@
 use dataverse feeds;
 
 set wait-for-completion-feed "true";
- 
+
 connect feed feeds.TweetFeed to dataset Tweets;
+
+start feed feeds.TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/revised-tweet-parser/revised-tweet-parser.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/revised-tweet-parser/revised-tweet-parser.2.update.aql
index 66ede3f..c8283f2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/revised-tweet-parser/revised-tweet-parser.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/revised-tweet-parser/revised-tweet-parser.2.update.aql
@@ -20,3 +20,5 @@
 
 use dataverse feeds;
 connect feed TwitterFeed to dataset Tweets;
+
+start feed TwitterFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.1.ddl.aql
new file mode 100644
index 0000000..f14d609
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.1.ddl.aql
@@ -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.
+ */
+/*
+ * Description  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
+ */
+drop dataverse experiments if exists;
+create dataverse experiments;
+use dataverse experiments;
+
+create type TwitterUserType as closed {
+    screen-name: string,
+    lang: string,
+    friends_count: int32,
+    statuses_count: int32,
+    name: string,
+    followers_count: int32
+}
+
+create type TweetMessageType as closed {
+    tweetid: string,
+    tweetid-copy:string,
+    user: TwitterUserType,
+    sender-location: point,
+    send-time: datetime,
+    send-time-copy:datetime,
+    referred-topics: {{ string }},
+    message-text: string
+}
+
+create dataset Tweets1(TweetMessageType) primary key tweetid;
+create dataset Tweets2(TweetMessageType) primary key tweetid;
+
+create feed TweetFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="TweetMessageType"),
+    ("format"="adm")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWork.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.2.update.aql
similarity index 67%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWork.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.2.update.aql
index 2f5379b..a75c4ce 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWork.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.2.update.aql
@@ -16,13 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-/**
- * Represents a feed management task. The task is executed asynchronously.
+/*
+ * Description  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
  */
-public interface IFeedWork {
 
-    public Runnable getRunnable();
+use dataverse experiments;
+set wait-for-completion-feed "false";
 
-}
+connect feed TweetFeed to dataset Tweets1;
+connect feed TweetFeed to dataset Tweets2;
+
+start feed TweetFeed;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.3.server.aql
similarity index 71%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.3.server.aql
index 37661b7..066c912 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.3.server.aql
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+/*
+ * Description  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
+ */
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/twitter/tw_messages.adm 500 50 1000
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWork.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.4.sleep.aql
similarity index 78%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWork.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.4.sleep.aql
index 2f5379b..18bbbbc 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWork.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.4.sleep.aql
@@ -16,13 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-/**
- * Represents a feed management task. The task is executed asynchronously.
+/*
+ * Description  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
  */
-public interface IFeedWork {
-
-    public Runnable getRunnable();
-
-}
+10000
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.5.update.aql
similarity index 68%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.5.update.aql
index 37661b7..7965746 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.5.update.aql
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
+/*
+ * Description  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
+ */
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+use dataverse experiments;
+stop feed TweetFeed;
+disconnect feed TweetFeed from dataset Tweets1;
+disconnect feed TweetFeed from dataset Tweets2;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.6.query.aql
similarity index 70%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.6.query.aql
index 37661b7..1aafc43 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.6.query.aql
@@ -16,10 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
+/*
+ * Description  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
+ */
 
-public interface IFeedWorkManager {
+use dataverse experiments;
 
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+count(for $x in dataset Tweets1
+return $x)
+count(for $x in dataset Tweets2
+return $x)
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.7.server.aql
similarity index 77%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.7.server.aql
index 37661b7..6753868 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.7.server.aql
@@ -16,10 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
+/*
+ * Description  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
+ */
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+stop 10001
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.8.ddl.aql
similarity index 74%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.8.ddl.aql
index 37661b7..1295b97 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-feed/start-feed.8.ddl.aql
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
+/*
+ * Description  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
+ */
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+use dataverse experiments;
+drop dataverse experiments;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.0.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.0.ddl.aql
new file mode 100644
index 0000000..f7e30c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.0.ddl.aql
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+ drop dataverse experiments if exists;
+ create dataverse experiments;
+ use dataverse experiments;
+
+ create type TweetMessageType as closed {
+     tweetid: string,
+     tweetid-copy:string
+ }
+
+create dataset Tweets1(TweetMessageType) primary key tweetid;
+create dataset Tweets2(TweetMessageType) primary key tweetid;
+
+create feed TweetFeed using socket_adapter
+(
+ ("sockets"="127.0.0.1:10001"),
+ ("address-type"="IP"),
+ ("type-name"="TweetMessageType"),
+ ("format"="adm")
+);
+
+connect feed TweetFeed to dataset Tweets1;
+connect feed TweetFeed to dataset Tweets2;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.1.update.aql
similarity index 82%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.1.update.aql
index 37661b7..0a1527f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.1.update.aql
@@ -16,10 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+use dataverse experiments;
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWork.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.2.sleep.aql
similarity index 78%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWork.java
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.2.sleep.aql
index 2f5379b..f487a10 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWork.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.2.sleep.aql
@@ -16,13 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-/**
- * Represents a feed management task. The task is executed asynchronously.
+/*
+ * Description  : Create a socket feed with a client that pushes
+ * 10 records. The feed is connected to a dataset that is then
+ * queried for the data.
+ * Expected Res : Success
+ * Date         : 24th Feb 2016
  */
-public interface IFeedWork {
-
-    public Runnable getRunnable();
-
-}
+1000
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.3.update.aql
similarity index 82%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.3.update.aql
index 37661b7..0a1527f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.3.update.aql
@@ -16,10 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+use dataverse experiments;
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.4.ddl.aql
similarity index 82%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.4.ddl.aql
index 37661b7..e3097c8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/start-started-feed/start-started-feed.4.ddl.aql
@@ -16,10 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+use dataverse experiments;
+drop dataverse experiments;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.0.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.0.ddl.aql
new file mode 100644
index 0000000..00fb12d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.0.ddl.aql
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+drop dataverse new_experiments if exists;
+create dataverse new_experiments;
+use dataverse new_experiments;
+
+create type TweetMessageType as closed {
+ tweetid: string,
+ tweetid-copy:string
+}
+
+create dataset Tweets1(TweetMessageType) primary key tweetid;
+create dataset Tweets2(TweetMessageType) primary key tweetid;
+
+create feed TweetFeed using socket_adapter
+(
+ ("sockets"="127.0.0.1:10001"),
+ ("address-type"="IP"),
+ ("type-name"="TweetMessageType"),
+ ("format"="adm")
+);
+
+connect feed TweetFeed to dataset Tweets1;
+connect feed TweetFeed to dataset Tweets2;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.1.update.aql
similarity index 82%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.1.update.aql
index 37661b7..4f3cb19 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.1.update.aql
@@ -16,10 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+use dataverse new_experiments;
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.2.sleep.aql
similarity index 82%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.2.sleep.aql
index 37661b7..427f2d2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.2.sleep.aql
@@ -16,10 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+3000
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.3.update.aql
similarity index 80%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.3.update.aql
index 37661b7..1e75813 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.3.update.aql
@@ -14,12 +14,8 @@
  * "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.
+ * under the License.1
  */
-package org.apache.asterix.external.feed.api;
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+use dataverse new_experiments;
+stop feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.4.sleep.aql
similarity index 82%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.4.sleep.aql
index 37661b7..98e392b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.4.sleep.aql
@@ -16,10 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+1000
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.5.update.aql
similarity index 80%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.5.update.aql
index 37661b7..8722607 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/stop-stopped-feed/stop-stopped-feed.5.update.aql
@@ -14,12 +14,8 @@
  * "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.
+ * under the License.2
  */
-package org.apache.asterix.external.feed.api;
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
-
-}
+use dataverse new_experiments;
+stop feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/twitter-feed/twitter-feed.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/twitter-feed/twitter-feed.2.update.aql
index 6712969..c3f46e3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/twitter-feed/twitter-feed.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/twitter-feed/twitter-feed.2.update.aql
@@ -22,4 +22,6 @@
  */
 
 use dataverse feeds;
-connect feed TwitterFeed to dataset Tweets;
\ No newline at end of file
+connect feed TwitterFeed to dataset Tweets;
+
+start feed TwitterFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql
index 051f266..bbf20dd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql
@@ -27,3 +27,5 @@
 set wait-for-completion-feed "false";
 
 connect feed UserFeed to dataset TwitterUsers;
+
+start feed UserFeed;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql
index 3da77f7..752ef6e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql
@@ -23,4 +23,4 @@
  * Expected Res : Success
  * Date         : 13th Aug 2016
  */
-start client 10001 file-client localhost ../asterix-app/data/tinysocial/twu_update.adm 500 50 1000
\ No newline at end of file
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/tinysocial/twu_update.adm 500 50 1000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql
index 8ad18ec..30d3274 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql
@@ -23,6 +23,6 @@
  * Expected Res : Success
  * Date         : 13th Aug 2016
  */
-
 use dataverse experiments;
+stop feed UserFeed;
 disconnect feed UserFeed from dataset TwitterUsers;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql
index 73390ff..92a8e39 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql
@@ -17,9 +17,9 @@
  * under the License.
  */
 /*
- * Description  : Create a feed dataset that uses the feed simulator adapter. 
+ * Description  : Create a feed dataset that uses the feed simulator adapter.
                   Use hint (cardinality) for the feed dataset.
-                  Begin ingestion using a fully qualified name and verify contents of the dataset post completion.  
+                  Begin ingestion using a fully qualified name and verify contents of the dataset post completion.
  * Expected Res : Success
  * Date         : 30th Jan 2013
  */
@@ -29,3 +29,5 @@
 set wait-for-completion-feed "true";
 
 connect feed TweetFeed to dataset Tweets;
+
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta_primary_index-1/query_dataset_with_meta_primary_index-1.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta_primary_index-1/query_dataset_with_meta_primary_index-1.2.update.aql
index d67371db..1eae536 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta_primary_index-1/query_dataset_with_meta_primary_index-1.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta_primary_index-1/query_dataset_with_meta_primary_index-1.2.update.aql
@@ -20,4 +20,6 @@
 use dataverse KeyVerse;
 
 set wait-for-completion-feed "true";
-connect feed KVStream to dataset KVStore;
\ No newline at end of file
+connect feed KVStream to dataset KVStore;
+
+start feed KVStream;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_01/feeds_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_01/feeds_01.1.ddl.sqlpp
index da5a20c..40b6ba2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_01/feeds_01.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_01/feeds_01.1.ddl.sqlpp
@@ -39,5 +39,5 @@
 
 create  dataset Tweets(TweetType) primary key id;
 
-create primary feed TweetFeed using localfs
+create feed TweetFeed using localfs
 ((`type-name`=`TweetType`),(`path`=`asterix_nc1://data/twitter/obamatweets.adm`),(`format`=`adm`),(`tuple-interval`=`10`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_02/feeds_02.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_02/feeds_02.1.ddl.sqlpp
index 2657d7a..cbf8b87 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_02/feeds_02.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_02/feeds_02.1.ddl.sqlpp
@@ -39,7 +39,7 @@
 }
 
 create  dataset Tweets(TweetType) primary key id;
-create  primary feed TweetFeed using localfs
+create  feed TweetFeed using localfs
 ((`path`=`asterix_nc1://data/twitter/obamatweets.adm`),
 (`format`=`adm`),
 (`type-name`=`TweetType`),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_02/feeds_02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_02/feeds_02.2.update.sqlpp
index ad7dd1f..f0b8689 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_02/feeds_02.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_02/feeds_02.2.update.sqlpp
@@ -26,3 +26,4 @@
 use feeds;
 set `wait-for-completion-feed` `true`;
 connect feed TweetFeed to  dataset Tweets;
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.1.ddl.sqlpp
index f18b7c0..5bc362f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.1.ddl.sqlpp
@@ -43,5 +43,5 @@
 
 create feed TweetFeed
 using localfs
-((`type-name`=`TweetType`),(`path`=`asterix_nc1://data/twitter/obamatweets.adm`),(`format`=`adm`),(`tuple-interval`=`10`))
-apply function feed_processor;
+((`type-name`=`TweetType`),(`path`=`asterix_nc1://data/twitter/obamatweets.adm`),(`format`=`adm`)
+,(`tuple-interval`=`10`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.2.update.sqlpp
index 63c9909..8b4b5f8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.2.update.sqlpp
@@ -22,3 +22,7 @@
  * Date         : 24th Dec 2012
  */
 
+use feeds;
+
+connect feed TweetFeed to dataset Tweets
+apply function feed_processor;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.3.query.sqlpp
index 61e3bae..755fbf3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_03/feeds_03.3.query.sqlpp
@@ -23,6 +23,6 @@
  */
 
 select element x
-from  `Metadata.Feed` as x
+from  `Metadata.FeedConnection` as x
 where ((x.DataverseName = 'feeds') and (x.FeedName = 'TweetFeed'))
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.1.ddl.sqlpp
index 1f24192..c21dfce 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.1.ddl.sqlpp
@@ -43,9 +43,9 @@
   country : string
 }
 
-create  dataset SyntheticTweets(TweetMessageType) primary key id;
+create dataset SyntheticTweets(TweetMessageType) primary key id;
 
-create  primary feed SyntheticTweetFeed using stream(
+create feed SyntheticTweetFeed using stream(
 (`duration`=`5`),
 (`tps`=`50`),
 (`type-name`=`TweetMessageType`),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.2.update.sqlpp
index db201f0..f2b8e25 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.2.update.sqlpp
@@ -21,4 +21,5 @@
 
 set `wait-for-completion-feed` `true`;
 
-connect  feed SyntheticTweetFeed to  dataset SyntheticTweets;
+connect feed SyntheticTweetFeed to  dataset SyntheticTweets;
+start feed SyntheticTweetFeed;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.1.ddl.sqlpp
index 6311b8b..268a527 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.1.ddl.sqlpp
@@ -45,7 +45,7 @@
 
 create  dataset SyntheticTweets(TweetMessageType) primary key id;
 create  index ngram_index  on SyntheticTweets (message_text) type ngram (3);
-create  primary feed SyntheticTweetFeed using stream (
+create  feed SyntheticTweetFeed using stream (
 (`duration`=`5`),
 (`tps`=`50`),
 (`stream-source`=`org.apache.asterix.external.input.stream.TwitterFirehoseStreamFactory`),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.2.update.sqlpp
index 25c7942..9f60585 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.2.update.sqlpp
@@ -22,3 +22,5 @@
 set `wait-for-completion-feed` `true`;
 
 connect  feed SyntheticTweetFeed to  dataset SyntheticTweets using policy `BasicFT`;
+
+start feed SyntheticTweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.1.ddl.sqlpp
index 71a0ca2..7c71a41 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.1.ddl.sqlpp
@@ -45,7 +45,7 @@
 }
 create  dataset SyntheticTweets(TweetMessageType) primary key id;
 create  index message_text  on SyntheticTweets (message_text) type btree;
-create  primary feed SyntheticTweetFeed using stream ((`duration`=`5`),
+create  feed SyntheticTweetFeed using stream ((`duration`=`5`),
 (`tps`=`50`),(`tput-duration`=`5`),(`type-name`=`TweetMessageType`),
 (`dataverse-dataset`=`feeds:SyntheticTweets`),(`format`=`adm`),
 (`stream-source`=`org.apache.asterix.external.input.stream.TwitterFirehoseStreamFactory`),(`mode`=`controlled`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.2.update.sqlpp
index 004fa33..0a940bd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.2.update.sqlpp
@@ -20,3 +20,4 @@
 use feeds_09;
 set `wait-for-completion-feed` `true`;
 connect  feed SyntheticTweetFeed to  dataset SyntheticTweets using policy `BasicFT`;
+start feed SyntheticTweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.1.ddl.sqlpp
index 2b549d8..5413b55 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.1.ddl.sqlpp
@@ -36,7 +36,7 @@
 
 create  index usernameIdx  on Tweets (username) type btree;
 
-create  primary feed TweetFeed using localfs
+create  feed TweetFeed using localfs
 ((`path`=`asterix_nc1://data/twitter/obamatweets.adm`),
 (`format`=`adm`),
 (`type-name`=`TweetType`),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.2.update.sqlpp
index c4a0fa4..4c5cc66 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.2.update.sqlpp
@@ -20,3 +20,4 @@
 use feeds_10;
 set `wait-for-completion-feed` `true`;
 connect  feed TweetFeed to  dataset Tweets using policy `BasicFT`;
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_11/feeds_11.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_11/feeds_11.1.ddl.sqlpp
index 05cec5d..8e96b78 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_11/feeds_11.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_11/feeds_11.1.ddl.sqlpp
@@ -31,7 +31,7 @@
 }
 
 create  dataset Tweets(TweetType) primary key id;
-create  primary feed TweetFeed using localfs
+create  feed TweetFeed using localfs
 ((`path`=`asterix_nc1://data/twitter/obamatweets.adm`),
 (`format`=`adm`),
 (`type-name`=`TweetType`),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_11/feeds_11.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_11/feeds_11.2.update.sqlpp
index 800fb67..7af2dbd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_11/feeds_11.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_11/feeds_11.2.update.sqlpp
@@ -20,3 +20,4 @@
 use feeds;
 set `wait-for-completion-feed` `true`;
 connect  feed feeds.TweetFeed to  dataset feeds.Tweets using policy `BasicFT`;
+start feed feeds.TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_12/feeds_12.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_12/feeds_12.1.ddl.sqlpp
index dadd92b..0abb2e3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_12/feeds_12.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_12/feeds_12.1.ddl.sqlpp
@@ -34,7 +34,7 @@
 
 create  dataset Tweets(TweetType) primary key id;
 
-create  primary feed TweetFeed using localfs
+create  feed TweetFeed using localfs
 ((`path`=`asterix_nc1://data/twitter/obamatweets_duplicate.adm`),
 (`format`=`adm`),
 (`type-name`=`TweetType`),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_12/feeds_12.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_12/feeds_12.2.update.sqlpp
index 58f8886..a3f0923 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_12/feeds_12.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_12/feeds_12.2.update.sqlpp
@@ -20,3 +20,4 @@
 use feeds_12;
 set `wait-for-completion-feed` `true`;
 connect  feed TweetFeed to  dataset Tweets using policy `BasicFT`;
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/issue_230_feeds/issue_230_feeds.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/issue_230_feeds/issue_230_feeds.1.ddl.sqlpp
index 5a234b5..c102e01 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/issue_230_feeds/issue_230_feeds.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/issue_230_feeds/issue_230_feeds.1.ddl.sqlpp
@@ -34,7 +34,7 @@
 
 create  dataset Tweets(TweetType) primary key id;
 
-create  primary feed TweetFeed using localfs
+create  feed TweetFeed using localfs
 ((`path`=`asterix_nc1://data/twitter/obamatweets.adm`),
 (`format`=`adm`),
 (`type-name`=`TweetType`),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/issue_230_feeds/issue_230_feeds.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/issue_230_feeds/issue_230_feeds.2.update.sqlpp
index 800fb67..7af2dbd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/issue_230_feeds/issue_230_feeds.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/issue_230_feeds/issue_230_feeds.2.update.sqlpp
@@ -20,3 +20,4 @@
 use feeds;
 set `wait-for-completion-feed` `true`;
 connect  feed feeds.TweetFeed to  dataset feeds.Tweets using policy `BasicFT`;
+start feed feeds.TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp
index 32d8389..3bca1a2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp
@@ -27,3 +27,4 @@
 set `wait-for-completion-feed` "false";
 
 connect feed UserFeed to dataset TwitterUsers;
+start feed UserFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp
index 8e2dff8..c19b2f7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp
@@ -25,4 +25,5 @@
  */
 
 use experiments;
+stop feed UserFeed;
 disconnect feed UserFeed from dataset TwitterUsers;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.sqlpp
index e40db6b..44a8f76 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.sqlpp
@@ -41,7 +41,7 @@
 
 create  dataset Tweets(TweetType) primary key id hints (`CARDINALITY`=`200`);
 
-create  primary feed TweetFeed using localfs(
+create feed TweetFeed using localfs(
 (`path`=`asterix_nc1://data/twitter/obamatweets.adm`),
 (`format`=`adm`),
 (`type-name`=`TweetType`),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.sqlpp
index 70725ba..c5f0625 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.sqlpp
@@ -30,3 +30,5 @@
 set `wait-for-completion-feed` `true`;
 
 connect  feed TweetFeed to  dataset Tweets using policy `BasicFT`;
+
+start feed TweetFeed;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.4.adm
index 14b76cb..cbdc907 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.4.adm
@@ -1 +1 @@
-804
+788
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.5.adm
index 14b76cb..cbdc907 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-in-meta/change-feed-with-meta-pk-in-meta.5.adm
@@ -1 +1 @@
-804
+788
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed/connect-feed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed/connect-feed.1.adm
new file mode 100644
index 0000000..f1022a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed/connect-feed.1.adm
@@ -0,0 +1,3 @@
+"Tweets0"
+"Tweets1"
+"Tweets3"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
index 21ad9e0..a6fa97e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
@@ -1 +1 @@
-{ "DataverseName": "feeds", "FeedName": "TweetFeed", "FeedType": "PRIMARY", "PrimaryTypeDetails": { "AdapterName": "localfs", "AdapterConfiguration": {{ { "Name": "type-name", "Value": "TweetType" }, { "Name": "path", "Value": "asterix_nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" }, { "Name": "is-feed", "Value": "true" }, { "Name": "dataverse", "Value": "feeds" }, { "Name": "feed", "Value": "TweetFeed" }, { "Name": "reader", "Value": "localfs" }, { "Name": "parser", "Value": "adm" } }} }, "Timestamp": "Mon Jun 20 17:07:04 PDT 2016" }
+{ "DataverseName": "feeds", "FeedName": "TweetFeed", "AdapterName": "localfs", "AdapterConfiguration": {{ { "Name": "type-name", "Value": "TweetType" }, { "Name": "path", "Value": "asterix_nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" }, { "Name": "is-feed", "Value": "true" }, { "Name": "dataverse", "Value": "feeds" }, { "Name": "feed", "Value": "TweetFeed" }, { "Name": "reader", "Value": "localfs" }, { "Name": "parser", "Value": "adm" } }}, "Timestamp": "Mon Oct 10 14:54:32 PDT 2016" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
index 16c9453..573793f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
@@ -1 +1 @@
-{ "DataverseName": "feeds", "FeedName": "TweetFeed", "Function": "feed_processor", "FeedType": "PRIMARY", "PrimaryTypeDetails": { "AdapterName": "localfs", "AdapterConfiguration": {{ { "Name": "type-name", "Value": "TweetType" }, { "Name": "path", "Value": "asterix_nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" }, { "Name": "is-feed", "Value": "true" }, { "Name": "dataverse", "Value": "feeds" }, { "Name": "feed", "Value": "TweetFeed" }, { "Name": "reader", "Value": "localfs" }, { "Name": "parser", "Value": "adm" } }} }, "Timestamp": "Mon Jun 20 17:07:04 PDT 2016" }
+{ "DataverseName": "feeds", "FeedName": "TweetFeed", "DatasetName": "Tweets", "ReturnType": "TweetType: closed {\n  id: string,\n  username: string,\n  location: string,\n  text: string,\n  timestamp: string\n}\n", "AppliedFunctions": {{ "feed_processor" }}, "PolicyName": "BasicFT" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/start-feed/start-feed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/start-feed/start-feed.1.adm
new file mode 100644
index 0000000..b2f7f08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/start-feed/start-feed.1.adm
@@ -0,0 +1,2 @@
+10
+10
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/any-object/any-object.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/any-object/any-object.2.adm
index 504ba93..91df332 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/any-object/any-object.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/any-object/any-object.2.adm
@@ -1,68 +1,69 @@
-{ "DataverseName": "Metadata", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [  ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeDataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetType", "FieldType": "string", "IsNullable": false }, { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicyProperties", "FieldType": "DatasetRecordType_CompactionPolicyProperties", "IsNullable": false }, { "FieldName": "InternalDetails", "FieldType": "DatasetRecordType_InternalDetails", "IsNullable": true }, { "FieldName": "ExternalDetails", "FieldType": "DatasetRecordType_ExternalDetails", "IsNullable": true }, { "FieldName": "Hints", "FieldType": "DatasetRecordType_Hints", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetId", "FieldType": "int32", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_CompactionPolicyProperties_Item" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "DatasetRecordType_ExternalDetails_Properties", "IsNullable": false }, { "FieldName": "LastRefreshTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "TransactionState", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_ExternalDetails_Properties_Item" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "DatasetRecordType_Hints_Item" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningStrategy", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningKey", "FieldType": "DatasetRecordType_InternalDetails_PartitioningKey", "IsNullable": false }, { "FieldName": "PrimaryKey", "FieldType": "DatasetRecordType_InternalDetails_PrimaryKey", "IsNullable": false }, { "FieldName": "Autogenerated", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PartitioningKey_Item" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PrimaryKey_Item" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false }, { "FieldName": "Type", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Derived", "FieldType": "DatatypeRecordType_Derived", "IsNullable": true }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsAnonymous", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Record", "FieldType": "DatatypeRecordType_Derived_Record", "IsNullable": true }, { "FieldName": "UnorderedList", "FieldType": "string", "IsNullable": true }, { "FieldName": "OrderedList", "FieldType": "string", "IsNullable": true } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Fields", "FieldType": "DatatypeRecordType_Derived_Record_Fields", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatatypeRecordType_Derived_Record_Fields_Item" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FieldType", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsNullable", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DataFormat", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileNumber", "FieldType": "int32", "IsNullable": false }, { "FieldName": "FileName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileSize", "FieldType": "int64", "IsNullable": false }, { "FieldName": "FileModTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Description", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "FeedPolicyRecordType_Properties", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedPolicyRecordType_Properties_Item" }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Function", "FieldType": "string", "IsNullable": true }, { "FieldName": "FeedType", "FieldType": "string", "IsNullable": false }, { "FieldName": "PrimaryTypeDetails", "FieldType": "FeedRecordType_PrimaryTypeDetails", "IsNullable": true }, { "FieldName": "SecondaryTypeDetails", "FieldType": "FeedRecordType_SecondaryTypeDetails", "IsNullable": true }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_PrimaryTypeDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "AdapterName", "FieldType": "string", "IsNullable": false }, { "FieldName": "AdapterConfiguration", "FieldType": "FeedRecordType_PrimaryTypeDetails_AdapterConfiguration", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_PrimaryTypeDetails_AdapterConfiguration", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "DatasetRecordType_ExternalDetails_Properties_Item" }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_SecondaryTypeDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "SourceFeedName", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Arity", "FieldType": "string", "IsNullable": false }, { "FieldName": "Params", "FieldType": "FunctionRecordType_Params", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "Definition", "FieldType": "string", "IsNullable": false }, { "FieldName": "Language", "FieldType": "string", "IsNullable": false }, { "FieldName": "Kind", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Params", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "SearchKey", "FieldType": "IndexRecordType_SearchKey", "IsNullable": false }, { "FieldName": "IsPrimary", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "IndexRecordType_SearchKey_Item" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:44 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NodeNames", "FieldType": "NodeGroupRecordType_NodeNames", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType_NodeNames", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NumberOfCores", "FieldType": "int64", "IsNullable": false }, { "FieldName": "WorkingMemorySize", "FieldType": "int64", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "binary", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "missing", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Timestamp": "Thu Sep 15 14:05:43 EEST 2016" }
-{ "DataverseName": "test", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [  ] } }, "Timestamp": "Thu Sep 15 14:10:05 EEST 2016" }
-{ "DataverseName": "test", "DatatypeName": "kv1", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "val", "FieldType": "AnyObject", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:10:05 EEST 2016" }
-{ "DataverseName": "test", "DatatypeName": "kv2", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "val", "FieldType": "AnyObject", "IsNullable": false } ] } }, "Timestamp": "Thu Sep 15 14:10:05 EEST 2016" }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [  ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeDataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetType", "FieldType": "string", "IsNullable": false }, { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicyProperties", "FieldType": "DatasetRecordType_CompactionPolicyProperties", "IsNullable": false }, { "FieldName": "InternalDetails", "FieldType": "DatasetRecordType_InternalDetails", "IsNullable": true }, { "FieldName": "ExternalDetails", "FieldType": "DatasetRecordType_ExternalDetails", "IsNullable": true }, { "FieldName": "Hints", "FieldType": "DatasetRecordType_Hints", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetId", "FieldType": "int32", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_CompactionPolicyProperties_Item" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "DatasetRecordType_ExternalDetails_Properties", "IsNullable": false }, { "FieldName": "LastRefreshTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "TransactionState", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_ExternalDetails_Properties_Item" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "DatasetRecordType_Hints_Item" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningStrategy", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningKey", "FieldType": "DatasetRecordType_InternalDetails_PartitioningKey", "IsNullable": false }, { "FieldName": "PrimaryKey", "FieldType": "DatasetRecordType_InternalDetails_PrimaryKey", "IsNullable": false }, { "FieldName": "Autogenerated", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PartitioningKey_Item" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PrimaryKey_Item" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false }, { "FieldName": "Type", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Derived", "FieldType": "DatatypeRecordType_Derived", "IsNullable": true }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsAnonymous", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Record", "FieldType": "DatatypeRecordType_Derived_Record", "IsNullable": true }, { "FieldName": "UnorderedList", "FieldType": "string", "IsNullable": true }, { "FieldName": "OrderedList", "FieldType": "string", "IsNullable": true } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Fields", "FieldType": "DatatypeRecordType_Derived_Record_Fields", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatatypeRecordType_Derived_Record_Fields_Item" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FieldType", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsNullable", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DataFormat", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileNumber", "FieldType": "int32", "IsNullable": false }, { "FieldName": "FileName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileSize", "FieldType": "int64", "IsNullable": false }, { "FieldName": "FileModTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "AppliedFunctions", "FieldType": "FeedConnectionRecordType_AppliedFunctions", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType_AppliedFunctions", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Description", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "FeedPolicyRecordType_Properties", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedPolicyRecordType_Properties_Item" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "AdapterName", "FieldType": "string", "IsNullable": false }, { "FieldName": "AdapterConfiguration", "FieldType": "FeedRecordType_AdapterConfiguration", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedRecordType_AdapterConfiguration_Item" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Arity", "FieldType": "string", "IsNullable": false }, { "FieldName": "Params", "FieldType": "FunctionRecordType_Params", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "Definition", "FieldType": "string", "IsNullable": false }, { "FieldName": "Language", "FieldType": "string", "IsNullable": false }, { "FieldName": "Kind", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Params", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "SearchKey", "FieldType": "IndexRecordType_SearchKey", "IsNullable": false }, { "FieldName": "IsPrimary", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "IndexRecordType_SearchKey_Item" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NodeNames", "FieldType": "NodeGroupRecordType_NodeNames", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType_NodeNames", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NumberOfCores", "FieldType": "int64", "IsNullable": false }, { "FieldName": "WorkingMemorySize", "FieldType": "int64", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "binary", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "missing", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Timestamp": "Mon Oct 10 14:53:55 PDT 2016" }
+{ "DataverseName": "test", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [  ] } }, "Timestamp": "Mon Oct 10 15:00:08 PDT 2016" }
+{ "DataverseName": "test", "DatatypeName": "kv1", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "val", "FieldType": "AnyObject", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 15:00:08 PDT 2016" }
+{ "DataverseName": "test", "DatatypeName": "kv2", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "val", "FieldType": "AnyObject", "IsNullable": false } ] } }, "Timestamp": "Mon Oct 10 15:00:08 PDT 2016" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index e4af727..f9242f6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -167,6 +167,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
+      <compilation-unit name="connect-feed">
+        <output-dir compare="Text">connect-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
       <compilation-unit name="feed-with-external-parser">
         <output-dir compare="Text">feed-with-external-parser</output-dir>
       </compilation-unit>
@@ -271,6 +276,23 @@
         <output-dir compare="Text">issue_230_feeds</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="start-feed">
+        <output-dir compare="Text">start-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <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>
+      </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>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="upsert">
     <test-case FilePath="upsert">
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
similarity index 82%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
index 37661b7..311400f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
@@ -16,10 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
+package org.apache.asterix.common.metadata;
 
-public interface IFeedWorkManager {
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
+public interface IDataset {
 
 }
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/client/FileFeedSocketAdapterClient.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/client/FileFeedSocketAdapterClient.java
index 765dc71..852173c 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/client/FileFeedSocketAdapterClient.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/client/FileFeedSocketAdapterClient.java
@@ -25,6 +25,8 @@
 import java.net.Socket;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 public class FileFeedSocketAdapterClient implements ITestClient {
     private final int port;
@@ -35,6 +37,7 @@
     private int batchSize;
     private int maxCount;
     private OutputStream out = null;
+    static final Logger LOGGER = Logger.getLogger(FileFeedSocketAdapterClient.class.getName());
 
     // expected args: url, source-file-path, max-count, batch-size, wait
     public FileFeedSocketAdapterClient(int port, String[] args) throws Exception {
@@ -51,12 +54,15 @@
     }
 
     @Override
-    public void start() {
+    public void start() throws Exception {
+        synchronized (this) {
+            wait(wait);
+        }
         try {
             socket = new Socket(url, port);
         } catch (IOException e) {
-            System.err.println("Problem in creating socket against host " + url + " on the port " + port);
-            e.printStackTrace();
+            LOGGER.log(Level.WARNING, "Problem in creating socket against host " + url + " on the port " + port, e);
+            throw e;
         }
 
         int recordCount = 0;
@@ -72,6 +78,7 @@
                 }
                 out.write(b.array(), 0, b.limit());
                 recordCount++;
+                LOGGER.log(Level.FINE, "One record filed into feed");
                 if (recordCount == maxCount) {
                     break;
                 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/FeedOperationCounter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/FeedOperationCounter.java
deleted file mode 100644
index dac8fbb..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/FeedOperationCounter.java
+++ /dev/null
@@ -1,59 +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.api;
-
-import org.apache.asterix.active.ActiveJob;
-
-public class FeedOperationCounter {
-    private ActiveJob feedJobInfo;
-    private int partitionCount;
-    private boolean failedIngestion = false;
-
-    public FeedOperationCounter(int partitionCount) {
-        this.partitionCount = partitionCount;
-    }
-
-    public int getPartitionCount() {
-        return partitionCount;
-    }
-
-    public void setPartitionCount(int partitionCount) {
-        this.partitionCount = partitionCount;
-    }
-
-    public boolean isFailedIngestion() {
-        return failedIngestion;
-    }
-
-    public void setFailedIngestion(boolean failedIngestion) {
-        this.failedIngestion = failedIngestion;
-    }
-
-    public ActiveJob getFeedJobInfo() {
-        return feedJobInfo;
-    }
-
-    public void setFeedJobInfo(ActiveJob feedJobInfo) {
-        this.feedJobInfo = feedJobInfo;
-    }
-
-    public int decrementAndGet() {
-        return --partitionCount;
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IActiveLifecycleEventSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IActiveLifecycleEventSubscriber.java
deleted file mode 100644
index 1d8c433..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IActiveLifecycleEventSubscriber.java
+++ /dev/null
@@ -1,40 +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.api;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IActiveLifecycleEventSubscriber {
-
-    public enum ActiveLifecycleEvent {
-        FEED_INTAKE_STARTED,
-        FEED_COLLECT_STARTED,
-        FEED_INTAKE_FAILURE,
-        FEED_COLLECT_FAILURE,
-        FEED_INTAKE_ENDED,
-        FEED_COLLECT_ENDED,
-        ACTIVE_JOB_STARTED,
-        ACTIVE_JOB_ENDED,
-        ACTIVE_JOB_FAILED
-    }
-
-    public void assertEvent(ActiveLifecycleEvent event) throws HyracksDataException, InterruptedException;
-
-    public void handleEvent(ActiveLifecycleEvent event);
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
index ce0b3d6..f5c25f7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
@@ -20,9 +20,7 @@
 
 import java.io.Serializable;
 import java.util.Map;
-
 import org.apache.asterix.active.EntityId;
-import org.apache.asterix.common.functions.FunctionSignature;
 
 public interface IFeed extends Serializable {
 
@@ -38,10 +36,6 @@
         SECONDARY
     }
 
-    public FeedType getFeedType();
-
-    public FunctionSignature getAppliedFunction();
-
     public String getFeedName();
 
     public String getDataverseName();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedJoint.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedJoint.java
deleted file mode 100644
index 548be4a..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedJoint.java
+++ /dev/null
@@ -1,122 +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.api;
-
-import java.io.Serializable;
-import java.util.List;
-
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedConnectionRequest;
-import org.apache.asterix.external.feed.management.FeedJointKey;
-import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
-
-public interface IFeedJoint extends Serializable {
-
-    public enum FeedJointType {
-        /** Feed Joint is located at the intake stage of a primary feed **/
-        INTAKE,
-
-        /** Feed Joint is located at the compute stage of a primary/secondary feed **/
-        COMPUTE
-    }
-
-    public enum State {
-        /** Initial state of a feed joint post creation but prior to scheduling of corresponding Hyracks job. **/
-        CREATED,
-
-        /** State acquired post creation of Hyracks job and known physical locations of the joint **/
-        INITIALIZED,
-
-        /** State acquired post starting of Hyracks job at which point, data begins to flow through the joint **/
-        ACTIVE
-    }
-
-    /**
-     * @return the {@link State} associated with the FeedJoint
-     */
-    public State getState();
-
-    /**
-     * @return the {@link FeedJointType} associated with the FeedJoint
-     */
-    public FeedJointType getType();
-
-    /**
-     * @return the list of data receivers that are
-     *         receiving the data flowing through this FeedJoint
-     */
-    public List<FeedConnectionId> getReceivers();
-
-    /**
-     * @return the list of pending subscription request {@link FeedConnectionRequest} submitted for data flowing through the FeedJoint
-     */
-    public List<FeedConnectionRequest> getConnectionRequests();
-
-    /**
-     * @return the subscription location {@link ConnectionLocation} associated with the FeedJoint
-     */
-    public FeedRuntimeType getConnectionLocation();
-
-    /**
-     * @return the unique {@link FeedJointKey} associated with the FeedJoint
-     */
-    public FeedJointKey getFeedJointKey();
-
-    /**
-     * Returns the feed subscriber {@link FeedSubscriber} corresponding to a given feed connection id.
-     *
-     * @param feedConnectionId
-     *            the unique id of a feed connection
-     * @return an instance of feedConnectionId {@link FeedConnectionId}
-     */
-    public FeedConnectionId getReceiver(FeedConnectionId feedConnectionId);
-
-    /**
-     * @param active
-     */
-    public void setState(State active);
-
-    /**
-     * Remove the subscriber from the set of registered subscribers to the FeedJoint
-     *
-     * @param connectionId
-     *            the connectionId that needs to be removed
-     */
-    public void removeReceiver(FeedConnectionId connectionId);
-
-    public EntityId getOwnerFeedId();
-
-    /**
-     * Add a feed connectionId to the set of registered subscribers
-     *
-     * @param connectionId
-     */
-    public void addReceiver(FeedConnectionId connectionId);
-
-    /**
-     * Add a feed subscription request {@link FeedConnectionRequest} for the FeedJoint
-     *
-     * @param request
-     */
-    public void addConnectionRequest(FeedConnectionRequest request);
-
-    public FeedConnectionId getProvider();
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkEventListener.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkEventListener.java
deleted file mode 100644
index e94ca46..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedWorkEventListener.java
+++ /dev/null
@@ -1,41 +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.api;
-
-/**
- * Provides a callback mechanism that in invoked for events related to
- * the execution of a feed management task.
- */
-public interface IFeedWorkEventListener {
-
-    /**
-     * A call back that is invoked after successful completion of a feed
-     * management task.
-     */
-    public void workCompleted(IFeedWork work);
-
-    /**
-     * A call back that is invokved after a failed execution of a feed
-     * management task.
-     *
-     * @param e
-     *            exception encountered during execution of the task.
-     */
-    public void workFailed(IFeedWork work, Exception e);
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java
deleted file mode 100644
index 397f797..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java
+++ /dev/null
@@ -1,42 +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.api;
-
-import org.apache.asterix.active.IActiveRuntime;
-import org.apache.asterix.external.feed.runtime.CollectionRuntime;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * Represent a feed runtime whose output can be routed along other parallel path(s).
- */
-public interface ISubscribableRuntime extends IActiveRuntime {
-
-    /**
-     * @param collectionRuntime
-     * @throws Exception
-     */
-    public void subscribe(CollectionRuntime collectionRuntime) throws HyracksDataException;
-
-    /**
-     * @param collectionRuntime
-     * @throws InterruptedException
-     * @throws Exception
-     */
-    public void unsubscribe(CollectionRuntime collectionRuntime) throws HyracksDataException, InterruptedException;
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ActiveLifecycleEventSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ActiveLifecycleEventSubscriber.java
deleted file mode 100644
index d0eebde..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ActiveLifecycleEventSubscriber.java
+++ /dev/null
@@ -1,69 +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.Iterator;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.external.feed.api.IActiveLifecycleEventSubscriber;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ActiveLifecycleEventSubscriber implements IActiveLifecycleEventSubscriber {
-
-    private LinkedBlockingQueue<ActiveLifecycleEvent> inbox;
-
-    public ActiveLifecycleEventSubscriber() {
-        this.inbox = new LinkedBlockingQueue<>();
-    }
-
-    @Override
-    public void handleEvent(ActiveLifecycleEvent event) {
-        inbox.add(event);
-    }
-
-    @Override
-    public void assertEvent(ActiveLifecycleEvent event) throws HyracksDataException, InterruptedException {
-        boolean eventOccurred = false;
-        ActiveLifecycleEvent e;
-        Iterator<ActiveLifecycleEvent> eventsSoFar = inbox.iterator();
-        while (eventsSoFar.hasNext()) {
-            e = eventsSoFar.next();
-            assertNoFailure(e);
-            eventOccurred = e.equals(event);
-        }
-
-        while (!eventOccurred) {
-            e = inbox.take();
-            eventOccurred = e.equals(event);
-            if (!eventOccurred) {
-                assertNoFailure(e);
-            }
-        }
-    }
-
-    private void assertNoFailure(ActiveLifecycleEvent e) throws HyracksDataException {
-        if (e.equals(ActiveLifecycleEvent.FEED_INTAKE_FAILURE) || e.equals(ActiveLifecycleEvent.FEED_COLLECT_FAILURE)
-                || e.equals(ActiveLifecycleEvent.ACTIVE_JOB_FAILED)) {
-            throw new RuntimeDataException(
-                    ErrorCode.FEED_MANAGEMENT_ACTIVE_LIFE_CYCLE_EVENT_SUBSCRIBER_ACTIVE_JOB_FAILURE);
-        }
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
index e2ab823..47efd1d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
@@ -29,7 +29,7 @@
 public class FeedConnectionId implements Serializable {
 
     private static final long serialVersionUID = 1L;
-    public static final String FEED_EXTENSION_NAME = "Feed";
+    private static final String FEED_EXTENSION_NAME = "Feed";
 
     private final EntityId feedId; // Dataverse - Feed
     private final String datasetName; // Dataset <Dataset is empty in case of no target dataset>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java
index 1106160..6e35841 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java
@@ -23,6 +23,7 @@
 import java.util.Map;
 
 import org.apache.asterix.active.EntityId;
+import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
 import org.apache.commons.lang3.StringUtils;
 
@@ -32,67 +33,23 @@
 public class FeedConnectionRequest implements Serializable {
 
     private static final long serialVersionUID = 1L;
-
-    public enum ConnectionStatus {
-        /** initial state upon creating a connection request **/
-        INITIALIZED,
-
-        /** connection establish; feed is receiving data **/
-        ACTIVE,
-
-        /** connection removed; feed is not receiving data **/
-        INACTIVE,
-
-        /** connection request failed **/
-        FAILED
-    }
-
-    /** Feed joint on the feed pipeline that serves as the source for this subscription **/
-    private final FeedJointKey feedJointKey;
-
     /** Location in the source feed pipeline from where feed tuples are received. **/
     private final FeedRuntimeType connectionLocation;
-
     /** List of functions that need to be applied in sequence after the data hand-off at the source feedPointKey. **/
-    private final List<String> functionsToApply;
-
-    /** Status associated with the subscription. */
-    private ConnectionStatus connectionStatus;
-
+    private final List<FunctionSignature> functionsToApply;
     /** Name of the policy that governs feed ingestion **/
     private final String policy;
-
-    /** Policy associated with a feed connection **/
-    private final Map<String, String> policyParameters;
-
     /** Target dataset associated with the connection request **/
     private final String targetDataset;
-
     private final EntityId receivingFeedId;
 
-    public FeedConnectionRequest(FeedJointKey feedPointKey, FeedRuntimeType connectionLocation,
-            List<String> functionsToApply, String targetDataset, String policy, Map<String, String> policyParameters,
-            EntityId receivingFeedId) {
-        this.feedJointKey = feedPointKey;
+    public FeedConnectionRequest(FeedRuntimeType connectionLocation, List<FunctionSignature> functionsToApply,
+            String targetDataset, String policy, EntityId receivingFeedId) {
         this.connectionLocation = connectionLocation;
         this.functionsToApply = functionsToApply;
         this.targetDataset = targetDataset;
         this.policy = policy;
-        this.policyParameters = policyParameters;
         this.receivingFeedId = receivingFeedId;
-        this.connectionStatus = ConnectionStatus.INITIALIZED;
-    }
-
-    public FeedJointKey getFeedJointKey() {
-        return feedJointKey;
-    }
-
-    public ConnectionStatus getConnectionStatus() {
-        return connectionStatus;
-    }
-
-    public void setSubscriptionStatus(ConnectionStatus connectionStatus) {
-        this.connectionStatus = connectionStatus;
     }
 
     public String getPolicy() {
@@ -111,17 +68,13 @@
         return receivingFeedId;
     }
 
-    public Map<String, String> getPolicyParameters() {
-        return policyParameters;
-    }
-
-    public List<String> getFunctionsToApply() {
+    public List<FunctionSignature> getFunctionsToApply() {
         return functionsToApply;
     }
 
     @Override
     public String toString() {
-        return "Feed Connection Request " + feedJointKey + " [" + connectionLocation + "]" + " Apply ("
+        return "Feed Connection Request " + receivingFeedId + " [" + connectionLocation + "]" + " Apply ("
                 + StringUtils.join(functionsToApply, ",") + ")";
     }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java
index fa48b58..2a87cab 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java
@@ -19,622 +19,103 @@
 package org.apache.asterix.external.feed.management;
 
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import org.apache.asterix.active.ActiveEvent;
-import org.apache.asterix.active.ActiveJob;
 import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.IActiveEntityEventsListener;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.external.feed.api.FeedOperationCounter;
-import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.feed.api.IFeedJoint.State;
-import org.apache.asterix.external.feed.api.IActiveLifecycleEventSubscriber;
-import org.apache.asterix.external.feed.api.IActiveLifecycleEventSubscriber.ActiveLifecycleEvent;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
-import org.apache.asterix.external.feed.watch.FeedIntakeInfo;
-import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
-import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
-import org.apache.asterix.external.operators.FeedMetaOperatorDescriptor;
-import org.apache.asterix.external.util.FeedUtils.JobType;
+import org.apache.asterix.active.IActiveEventSubscriber;
+import org.apache.asterix.active.message.ActivePartitionMessage;
+import org.apache.asterix.common.metadata.IDataset;
+import org.apache.asterix.external.feed.watch.FeedEventSubscriber;
+import org.apache.asterix.external.feed.watch.NoOpSubscriber;
 import org.apache.asterix.runtime.utils.AppContextInfo;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
-import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
-import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
-import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobInfo;
-import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
-import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
-import org.apache.log4j.Logger;
 
 public class FeedEventsListener implements IActiveEntityEventsListener {
-    private static final Logger LOGGER = Logger.getLogger(FeedEventsListener.class);
-    private final Map<EntityId, Pair<FeedOperationCounter, List<IFeedJoint>>> feedPipeline;
-    private final List<IActiveLifecycleEventSubscriber> subscribers;
-    private final Map<Long, ActiveJob> jobs;
-    private final Map<Long, ActiveJob> intakeJobs;
-    private final Map<EntityId, FeedIntakeInfo> entity2Intake;
-    private final Map<FeedConnectionId, FeedConnectJobInfo> connectJobInfos;
-    private EntityId entityId;
-    private IFeedJoint sourceFeedJoint;
+    // constants
+    private static final Logger LOGGER = Logger.getLogger(FeedEventsListener.class.getName());
+    // members
+    private final EntityId entityId;
+    private final List<IDataset> datasets;
+    private final String[] sources;
+    private final List<IActiveEventSubscriber> subscribers;
+    private volatile ActivityState state;
+    private int numRegistered;
+    private JobId jobId;
 
-    public FeedEventsListener(EntityId entityId) {
+    public FeedEventsListener(EntityId entityId, List<IDataset> datasets, String[] sources) {
         this.entityId = entityId;
+        this.datasets = datasets;
+        this.sources = sources;
         subscribers = new ArrayList<>();
-        jobs = new HashMap<>();
-        feedPipeline = new HashMap<>();
-        entity2Intake = new HashMap<>();
-        connectJobInfos = new HashMap<>();
-        intakeJobs = new HashMap<>();
+        state = ActivityState.STOPPED;
     }
 
     @Override
-    public void notify(ActiveEvent event) {
+    public synchronized void notify(ActiveEvent event) {
         try {
             switch (event.getEventKind()) {
-                case JOB_START:
-                    handleJobStartEvent(event);
+                case JOB_STARTED:
+                    start(event);
                     break;
-                case JOB_FINISH:
-                    handleJobFinishEvent(event);
+                case JOB_FINISHED:
+                    finish();
                     break;
                 case PARTITION_EVENT:
-                    handlePartitionStart(event);
+                    partition((ActivePartitionMessage) event.getEventObject());
                     break;
                 default:
-                    LOGGER.warn("Unknown Feed Event" + event);
+                    LOGGER.log(Level.WARNING, "Unhandled feed event notification: " + event);
                     break;
             }
+            notifySubscribers(event);
         } catch (Exception e) {
-            LOGGER.error("Unhandled Exception", e);
+            LOGGER.log(Level.SEVERE, "Unhandled Exception", e);
         }
     }
 
-    private synchronized void handleJobStartEvent(ActiveEvent message) throws Exception {
-        ActiveJob jobInfo = jobs.get(message.getJobId().getId());
-        JobType jobType = (JobType) jobInfo.getJobObject();
-        switch (jobType) {
-            case INTAKE:
-                handleIntakeJobStartMessage((FeedIntakeInfo) jobInfo);
-                break;
-            case FEED_CONNECT:
-                handleCollectJobStartMessage((FeedConnectJobInfo) jobInfo);
-                break;
-            default:
-        }
-    }
-
-    private synchronized void handleJobFinishEvent(ActiveEvent message) throws Exception {
-        ActiveJob jobInfo = jobs.get(message.getJobId().getId());
-        JobType jobType = (JobType) jobInfo.getJobObject();
-        switch (jobType) {
-            case FEED_CONNECT:
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info("Collect Job finished for  " + jobInfo);
-                }
-                handleFeedCollectJobFinishMessage((FeedConnectJobInfo) jobInfo);
-                break;
-            case INTAKE:
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info("Intake Job finished for feed intake " + jobInfo.getJobId());
-                }
-                handleFeedIntakeJobFinishMessage((FeedIntakeInfo) jobInfo, message);
-                break;
-            default:
-                break;
-        }
-    }
-
-    private synchronized void handlePartitionStart(ActiveEvent message) {
-        ActiveJob jobInfo = jobs.get(message.getJobId().getId());
-        JobType jobType = (JobType) jobInfo.getJobObject();
-        switch (jobType) {
-            case FEED_CONNECT:
-                ((FeedConnectJobInfo) jobInfo).partitionStart();
-                if (((FeedConnectJobInfo) jobInfo).collectionStarted()) {
-                    notifyFeedEventSubscribers(ActiveLifecycleEvent.FEED_COLLECT_STARTED);
-                }
-                break;
-            case INTAKE:
-                handleIntakePartitionStarts(message, jobInfo);
-                break;
-            default:
-                break;
-
-        }
-    }
-
-    private void handleIntakePartitionStarts(ActiveEvent message, ActiveJob jobInfo) {
-        if (feedPipeline.get(message.getEntityId()).first.decrementAndGet() == 0) {
-            ((FeedIntakeInfo) jobInfo).getIntakeFeedJoint().setState(State.ACTIVE);
-            jobInfo.setState(ActivityState.ACTIVE);
-            notifyFeedEventSubscribers(ActiveLifecycleEvent.FEED_INTAKE_STARTED);
-        }
-    }
-
-    public synchronized void registerFeedJoint(IFeedJoint feedJoint, int numOfPrividers) throws HyracksDataException {
-        Pair<FeedOperationCounter, List<IFeedJoint>> feedJointsOnPipeline = feedPipeline
-                .get(feedJoint.getOwnerFeedId());
-        if (feedJointsOnPipeline == null) {
-            feedJointsOnPipeline = new Pair<>(new FeedOperationCounter(numOfPrividers), new ArrayList<IFeedJoint>());
-            feedPipeline.put(feedJoint.getOwnerFeedId(), feedJointsOnPipeline);
-            feedJointsOnPipeline.second.add(feedJoint);
-        } else {
-            if (!feedJointsOnPipeline.second.contains(feedJoint)) {
-                feedJointsOnPipeline.second.add(feedJoint);
+    private synchronized void notifySubscribers(ActiveEvent event) {
+        notifyAll();
+        Iterator<IActiveEventSubscriber> it = subscribers.iterator();
+        while (it.hasNext()) {
+            IActiveEventSubscriber subscriber = it.next();
+            if (subscriber.done()) {
+                it.remove();
             } else {
-                throw new RuntimeDataException(
-                        ErrorCode.FEED_MANAGEMENT_FEED_EVENT_LISTENER_FEED_JOINT_REGISTERED, feedJoint);
+                subscriber.notify(event);
+                if (subscriber.done()) {
+                    it.remove();
+                }
             }
         }
     }
 
-    public synchronized void deregisterFeedIntakeJob(JobId jobId) {
-        FeedIntakeInfo info = (FeedIntakeInfo) intakeJobs.remove(jobId.getId());
-        jobs.remove(jobId.getId());
-        entity2Intake.remove(info.getFeedId());
-        List<IFeedJoint> joints = feedPipeline.get(info.getFeedId()).second;
-        joints.remove(info.getIntakeFeedJoint());
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Deregistered feed intake job [" + jobId + "]");
+    private void partition(ActivePartitionMessage message) {
+        if (message.getEvent() == ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED) {
+            numRegistered++;
+            if (numRegistered == getSources().length) {
+                state = ActivityState.STARTED;
+            }
         }
     }
 
-    private static synchronized void handleIntakeJobStartMessage(FeedIntakeInfo intakeJobInfo) throws Exception {
-        List<OperatorDescriptorId> intakeOperatorIds = new ArrayList<>();
-        Map<OperatorDescriptorId, IOperatorDescriptor> operators = intakeJobInfo.getSpec().getOperatorMap();
-        for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operators.entrySet()) {
-            IOperatorDescriptor opDesc = entry.getValue();
-            if (opDesc instanceof FeedIntakeOperatorDescriptor) {
-                intakeOperatorIds.add(opDesc.getOperatorId());
-            }
-        }
-
+    private void finish() throws Exception {
         IHyracksClientConnection hcc = AppContextInfo.INSTANCE.getHcc();
-        JobInfo info = hcc.getJobInfo(intakeJobInfo.getJobId());
-        List<String> intakeLocations = new ArrayList<>();
-        for (OperatorDescriptorId intakeOperatorId : intakeOperatorIds) {
-            Map<Integer, String> operatorLocations = info.getOperatorLocations().get(intakeOperatorId);
-            int nOperatorInstances = operatorLocations.size();
-            for (int i = 0; i < nOperatorInstances; i++) {
-                intakeLocations.add(operatorLocations.get(i));
-            }
-        }
-        // intakeLocations is an ordered list; 
-        // element at position i corresponds to location of i'th instance of operator
-        intakeJobInfo.setIntakeLocation(intakeLocations);
+        JobStatus status = hcc.getJobStatus(jobId);
+        state = status.equals(JobStatus.FAILURE) ? ActivityState.FAILED : ActivityState.STOPPED;
     }
 
-    public IFeedJoint getSourceFeedJoint(FeedConnectionId connectionId) {
-        FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
-        if (cInfo != null) {
-            return cInfo.getSourceFeedJoint();
-        }
-        return null;
-    }
-
-    public synchronized void registerFeedIntakeJob(EntityId feedId, JobId jobId, JobSpecification jobSpec)
-            throws HyracksDataException {
-        if (entity2Intake.get(feedId) != null) {
-            throw new RuntimeDataException(
-                    ErrorCode.FEED_MANAGEMENT_FEED_EVENTS_LISTENER_ALREADY_HAVE_INTAKE_JOB);
-        }
-        if (intakeJobs.get(jobId.getId()) != null) {
-            throw new RuntimeDataException(ErrorCode.FEED_MANAGEMENT_FEED_EVENTS_LISTENER_INTAKE_JOB_REGISTERED);
-        }
-        if (jobs.get(jobId.getId()) != null) {
-            throw new RuntimeDataException(ErrorCode.FEED_MANAGEMENT_FEED_EVENTS_LISTENER_FEED_JOB_REGISTERED);
-        }
-
-        Pair<FeedOperationCounter, List<IFeedJoint>> pair = feedPipeline.get(feedId);
-        sourceFeedJoint = null;
-        for (IFeedJoint joint : pair.second) {
-            if (joint.getType().equals(IFeedJoint.FeedJointType.INTAKE)) {
-                sourceFeedJoint = joint;
-                break;
-            }
-        }
-
-        if (sourceFeedJoint != null) {
-            FeedIntakeInfo intakeJobInfo =
-                    new FeedIntakeInfo(jobId, ActivityState.CREATED, feedId, sourceFeedJoint, jobSpec);
-            pair.first.setFeedJobInfo(intakeJobInfo);
-            entity2Intake.put(feedId, intakeJobInfo);
-            jobs.put(jobId.getId(), intakeJobInfo);
-            intakeJobs.put(jobId.getId(), intakeJobInfo);
-
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.info("Registered feed intake [" + jobId + "]" + " for feed " + feedId);
-            }
-        } else {
-            throw new RuntimeDataException(ErrorCode.FEED_MANAGEMENT_FEED_EVENT_REGISTER_INTAKE_JOB_FAIL, jobId,
-                    feedId);
-        }
-    }
-
-    public synchronized void registerFeedCollectionJob(EntityId sourceFeedId, FeedConnectionId connectionId,
-            JobId jobId, JobSpecification jobSpec, Map<String, String> feedPolicy) throws HyracksDataException {
-        if (jobs.get(jobId.getId()) != null) {
-            throw new RuntimeDataException(ErrorCode.FEED_MANAGEMENT_FEED_EVENTS_LISTENER_FEED_JOB_REGISTERED);
-        }
-        if (connectJobInfos.containsKey(jobId.getId())) {
-            throw new RuntimeDataException(ErrorCode.FEED_MANAGEMENT_FEED_EVENTS_LISTENER_FEED_JOB_REGISTERED);
-        }
-
-        List<IFeedJoint> feedJoints = feedPipeline.get(sourceFeedId).second;
-        FeedConnectionId cid = null;
-        IFeedJoint collectionSourceFeedJoint = null;
-        for (IFeedJoint joint : feedJoints) {
-            cid = joint.getReceiver(connectionId);
-            if (cid != null) {
-                collectionSourceFeedJoint = joint;
-                break;
-            }
-        }
-
-        if (cid != null) {
-            FeedConnectJobInfo cInfo = new FeedConnectJobInfo(sourceFeedId, jobId, ActivityState.CREATED, connectionId,
-                    collectionSourceFeedJoint, null, jobSpec, feedPolicy);
-            jobs.put(jobId.getId(), cInfo);
-            connectJobInfos.put(connectionId, cInfo);
-
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.info("Registered feed connection [" + jobId + "]" + " for feed " + connectionId);
-            }
-        } else {
-            LOGGER.warn(
-                    "Could not register feed collection job [" + jobId + "]" + " for feed connection " + connectionId);
-        }
-    }
-
-    @Override
-    public void notifyJobCreation(JobId jobId, JobSpecification spec) {
-        FeedConnectionId feedConnectionId = null;
-        Map<String, String> feedPolicy = null;
-        try {
-            for (IOperatorDescriptor opDesc : spec.getOperatorMap().values()) {
-                if (opDesc instanceof FeedCollectOperatorDescriptor) {
-                    feedConnectionId = ((FeedCollectOperatorDescriptor) opDesc).getFeedConnectionId();
-                    feedPolicy = ((FeedCollectOperatorDescriptor) opDesc).getFeedPolicyProperties();
-                    registerFeedCollectionJob(((FeedCollectOperatorDescriptor) opDesc).getSourceFeedId(),
-                            feedConnectionId, jobId, spec, feedPolicy);
-                    return;
-                } else if (opDesc instanceof FeedIntakeOperatorDescriptor) {
-                    registerFeedIntakeJob(((FeedIntakeOperatorDescriptor) opDesc).getFeedId(), jobId, spec);
-                    return;
-                }
-            }
-        } catch (Exception e) {
-            LOGGER.error(e);
-        }
-    }
-
-    public synchronized List<String> getConnectionLocations(IFeedJoint feedJoint, final FeedConnectionRequest request)
-            throws Exception {
-        List<String> locations = null;
-        switch (feedJoint.getType()) {
-            case COMPUTE:
-                FeedConnectionId connectionId = feedJoint.getProvider();
-                FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
-                locations = cInfo.getComputeLocations();
-                break;
-            case INTAKE:
-                FeedIntakeInfo intakeInfo = entity2Intake.get(feedJoint.getOwnerFeedId());
-                locations = intakeInfo.getIntakeLocation();
-                break;
-            default:
-                break;
-        }
-        return locations;
-    }
-
-    private synchronized void notifyFeedEventSubscribers(ActiveLifecycleEvent event) {
-        if (subscribers != null && !subscribers.isEmpty()) {
-            for (IActiveLifecycleEventSubscriber subscriber : subscribers) {
-                subscriber.handleEvent(event);
-            }
-        }
-    }
-
-    private synchronized void handleFeedIntakeJobFinishMessage(FeedIntakeInfo intakeInfo, ActiveEvent message)
-            throws Exception {
-        IHyracksClientConnection hcc = AppContextInfo.INSTANCE.getHcc();
-        JobInfo info = hcc.getJobInfo(message.getJobId());
-        JobStatus status = info.getStatus();
-        EntityId feedId = intakeInfo.getFeedId();
-        Pair<FeedOperationCounter, List<IFeedJoint>> pair = feedPipeline.get(feedId);
-        if (status.equals(JobStatus.FAILURE)) {
-            pair.first.setFailedIngestion(true);
-        }
-        // remove feed joints
-        deregisterFeedIntakeJob(message.getJobId());
-        // notify event listeners
-        feedPipeline.remove(feedId);
-        entity2Intake.remove(feedId);
-        notifyFeedEventSubscribers(pair.first.isFailedIngestion() ? ActiveLifecycleEvent.FEED_INTAKE_FAILURE
-                : ActiveLifecycleEvent.FEED_INTAKE_ENDED);
-    }
-
-    private synchronized void handleFeedCollectJobFinishMessage(FeedConnectJobInfo cInfo) throws Exception {
-        FeedConnectionId connectionId = cInfo.getConnectionId();
-
-        IHyracksClientConnection hcc = AppContextInfo.INSTANCE.getHcc();
-        JobInfo info = hcc.getJobInfo(cInfo.getJobId());
-        JobStatus status = info.getStatus();
-        boolean failure = status != null && status.equals(JobStatus.FAILURE);
-        FeedPolicyAccessor fpa = new FeedPolicyAccessor(cInfo.getFeedPolicy());
-        boolean retainSubsription =
-                cInfo.getState().equals(ActivityState.UNDER_RECOVERY) || (failure && fpa.continueOnHardwareFailure());
-
-        if (!retainSubsription) {
-            IFeedJoint feedJoint = cInfo.getSourceFeedJoint();
-            feedJoint.removeReceiver(connectionId);
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.info(
-                        "Subscription " + cInfo.getConnectionId() + " completed successfully. Removed subscription");
-            }
-        }
-
-        connectJobInfos.remove(connectionId);
-        jobs.remove(cInfo.getJobId().getId());
-        // notify event listeners
-        ActiveLifecycleEvent event =
-                failure ? ActiveLifecycleEvent.FEED_COLLECT_FAILURE : ActiveLifecycleEvent.FEED_COLLECT_ENDED;
-        notifyFeedEventSubscribers(event);
-    }
-
-    public List<String> getFeedStorageLocations(FeedConnectionId connectionId) {
-        return connectJobInfos.get(connectionId).getStorageLocations();
-    }
-
-    public List<String> getFeedCollectLocations(FeedConnectionId connectionId) {
-        return connectJobInfos.get(connectionId).getCollectLocations();
-    }
-
-    public List<String> getFeedIntakeLocations(EntityId feedId) {
-        return entity2Intake.get(feedId).getIntakeLocation();
-    }
-
-    public JobId getFeedCollectJobId(FeedConnectionId connectionId) {
-        return connectJobInfos.get(connectionId).getJobId();
-    }
-
-    public boolean isFeedPointAvailable(FeedJointKey feedJointKey) {
-        List<IFeedJoint> joints = feedPipeline.containsKey(feedJointKey.getFeedId())
-                ? feedPipeline.get(feedJointKey.getFeedId()).second : null;
-        if (joints != null && !joints.isEmpty()) {
-            for (IFeedJoint joint : joints) {
-                if (joint.getFeedJointKey().equals(feedJointKey)) {
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-
-    public Collection<IFeedJoint> getFeedIntakeJoints() {
-        List<IFeedJoint> intakeFeedPoints = new ArrayList<>();
-        for (FeedIntakeInfo info : entity2Intake.values()) {
-            intakeFeedPoints.add(info.getIntakeFeedJoint());
-        }
-        return intakeFeedPoints;
-    }
-
-    public IFeedJoint getFeedJoint(FeedJointKey feedPointKey) {
-        List<IFeedJoint> joints = feedPipeline.containsKey(feedPointKey.getFeedId())
-                ? feedPipeline.get(feedPointKey.getFeedId()).second : null;
-        if (joints != null && !joints.isEmpty()) {
-            for (IFeedJoint joint : joints) {
-                if (joint.getFeedJointKey().equals(feedPointKey)) {
-                    return joint;
-                }
-            }
-        }
-        return null;
-    }
-
-    public IFeedJoint getAvailableFeedJoint(FeedJointKey feedJointKey) {
-        IFeedJoint feedJoint = getFeedJoint(feedJointKey);
-        if (feedJoint != null) {
-            return feedJoint;
-        } else {
-            String jointKeyString = feedJointKey.getStringRep();
-            List<IFeedJoint> jointsOnPipeline = feedPipeline.containsKey(feedJointKey.getFeedId())
-                    ? feedPipeline.get(feedJointKey.getFeedId()).second : null;
-            IFeedJoint candidateJoint = null;
-            if (jointsOnPipeline != null) {
-                for (IFeedJoint joint : jointsOnPipeline) {
-                    if (jointKeyString.contains(joint.getFeedJointKey().getStringRep()) && (candidateJoint == null
-                            || /*found feed point is a super set of the earlier find*/joint.getFeedJointKey()
-                                    .getStringRep().contains(candidateJoint.getFeedJointKey().getStringRep()))) {
-                        candidateJoint = joint;
-                    }
-                }
-            }
-            return candidateJoint;
-        }
-    }
-
-    public JobSpecification getCollectJobSpecification(FeedConnectionId connectionId) {
-        return connectJobInfos.get(connectionId).getSpec();
-    }
-
-    public IFeedJoint getFeedPoint(EntityId sourceFeedId, IFeedJoint.FeedJointType type) {
-        List<IFeedJoint> joints = feedPipeline.get(sourceFeedId).second;
-        for (IFeedJoint joint : joints) {
-            if (joint.getType().equals(type)) {
-                return joint;
-            }
-        }
-        return null;
-    }
-
-    private void setLocations(FeedConnectJobInfo cInfo) {
-        JobSpecification jobSpec = cInfo.getSpec();
-
-        List<OperatorDescriptorId> collectOperatorIds = new ArrayList<>();
-        List<OperatorDescriptorId> computeOperatorIds = new ArrayList<>();
-        List<OperatorDescriptorId> storageOperatorIds = new ArrayList<>();
-
-        Map<OperatorDescriptorId, IOperatorDescriptor> operators = jobSpec.getOperatorMap();
-        for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operators.entrySet()) {
-            IOperatorDescriptor opDesc = entry.getValue();
-            IOperatorDescriptor actualOp;
-            if (opDesc instanceof FeedMetaOperatorDescriptor) {
-                actualOp = ((FeedMetaOperatorDescriptor) opDesc).getCoreOperator();
-            } else {
-                actualOp = opDesc;
-            }
-
-            if (actualOp instanceof AlgebricksMetaOperatorDescriptor) {
-                AlgebricksMetaOperatorDescriptor op = (AlgebricksMetaOperatorDescriptor) actualOp;
-                IPushRuntimeFactory[] runtimeFactories = op.getPipeline().getRuntimeFactories();
-                boolean computeOp = false;
-                for (IPushRuntimeFactory rf : runtimeFactories) {
-                    if (rf instanceof AssignRuntimeFactory) {
-                        IConnectorDescriptor connDesc = jobSpec.getOperatorInputMap().get(op.getOperatorId()).get(0);
-                        IOperatorDescriptor sourceOp =
-                                jobSpec.getConnectorOperatorMap().get(connDesc.getConnectorId()).getLeft().getLeft();
-                        if (sourceOp instanceof FeedCollectOperatorDescriptor) {
-                            computeOp = true;
-                            break;
-                        }
-                    }
-                }
-                if (computeOp) {
-                    computeOperatorIds.add(entry.getKey());
-                }
-            } else if (actualOp instanceof LSMTreeIndexInsertUpdateDeleteOperatorDescriptor) {
-                storageOperatorIds.add(entry.getKey());
-            } else if (actualOp instanceof FeedCollectOperatorDescriptor) {
-                collectOperatorIds.add(entry.getKey());
-            }
-        }
-
-        try {
-            IHyracksClientConnection hcc = AppContextInfo.INSTANCE.getHcc();
-            JobInfo info = hcc.getJobInfo(cInfo.getJobId());
-            List<String> collectLocations = new ArrayList<>();
-            for (OperatorDescriptorId collectOpId : collectOperatorIds) {
-                Map<Integer, String> operatorLocations = info.getOperatorLocations().get(collectOpId);
-                int nOperatorInstances = operatorLocations.size();
-                for (int i = 0; i < nOperatorInstances; i++) {
-                    collectLocations.add(operatorLocations.get(i));
-                }
-            }
-
-            List<String> computeLocations = new ArrayList<>();
-            for (OperatorDescriptorId computeOpId : computeOperatorIds) {
-                Map<Integer, String> operatorLocations = info.getOperatorLocations().get(computeOpId);
-                if (operatorLocations != null) {
-                    int nOperatorInstances = operatorLocations.size();
-                    for (int i = 0; i < nOperatorInstances; i++) {
-                        computeLocations.add(operatorLocations.get(i));
-                    }
-                } else {
-                    computeLocations.clear();
-                    computeLocations.addAll(collectLocations);
-                }
-            }
-
-            List<String> storageLocations = new ArrayList<>();
-            for (OperatorDescriptorId storageOpId : storageOperatorIds) {
-                Map<Integer, String> operatorLocations = info.getOperatorLocations().get(storageOpId);
-                if (operatorLocations == null) {
-                    continue;
-                }
-                int nOperatorInstances = operatorLocations.size();
-                for (int i = 0; i < nOperatorInstances; i++) {
-                    storageLocations.add(operatorLocations.get(i));
-                }
-            }
-            cInfo.setCollectLocations(collectLocations);
-            cInfo.setComputeLocations(computeLocations);
-            cInfo.setStorageLocations(storageLocations);
-
-        } catch (Exception e) {
-            LOGGER.error("Error while setting feed active locations", e);
-        }
-
-    }
-
-    public synchronized void registerFeedEventSubscriber(IActiveLifecycleEventSubscriber subscriber) {
-        subscribers.add(subscriber);
-    }
-
-    public void deregisterFeedEventSubscriber(IActiveLifecycleEventSubscriber subscriber) {
-        subscribers.remove(subscriber);
-    }
-
-    public synchronized boolean isFeedConnectionActive(FeedConnectionId connectionId,
-            IActiveLifecycleEventSubscriber eventSubscriber) {
-        boolean active = false;
-        FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
-        if (cInfo != null) {
-            active = cInfo.getState().equals(ActivityState.ACTIVE);
-        }
-        if (active) {
-            registerFeedEventSubscriber(eventSubscriber);
-        }
-        return active;
-    }
-
-    public FeedConnectJobInfo getFeedConnectJobInfo(FeedConnectionId connectionId) {
-        return connectJobInfos.get(connectionId);
-    }
-
-    private void handleCollectJobStartMessage(FeedConnectJobInfo cInfo) throws ACIDException {
-        // set locations of feed sub-operations (intake, compute, store)
-        setLocations(cInfo);
-        Pair<FeedOperationCounter, List<IFeedJoint>> pair = feedPipeline.get(cInfo.getConnectionId().getFeedId());
-        // activate joints
-        List<IFeedJoint> joints = pair.second;
-        for (IFeedJoint joint : joints) {
-            if (joint.getProvider().equals(cInfo.getConnectionId())) {
-                joint.setState(State.ACTIVE);
-                if (joint.getType().equals(IFeedJoint.FeedJointType.COMPUTE)) {
-                    cInfo.setComputeFeedJoint(joint);
-                }
-            }
-        }
-        cInfo.setState(ActivityState.ACTIVE);
-    }
-
-    private synchronized boolean isConnectedToDataset(String datasetName) {
-        for (FeedConnectionId connection : connectJobInfos.keySet()) {
-            if (connection.getDatasetName().equals(datasetName)) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    public FeedConnectionId[] getConnections() {
-        return connectJobInfos.keySet().toArray(new FeedConnectionId[connectJobInfos.size()]);
-    }
-
-    public boolean isFeedJointAvailable(FeedJointKey feedJointKey) {
-        return isFeedPointAvailable(feedJointKey);
-    }
-
-    @Override
-    public boolean isEntityActive() {
-        return !jobs.isEmpty();
+    private void start(ActiveEvent event) {
+        this.jobId = event.getJobId();
+        state = ActivityState.STARTING;
     }
 
     @Override
@@ -642,12 +123,39 @@
         return entityId;
     }
 
-    public IFeedJoint getSourceFeedJoint() {
-        return sourceFeedJoint;
+    @Override
+    public ActivityState getState() {
+        return state;
     }
 
     @Override
-    public boolean isEntityUsingDataset(String dataverseName, String datasetName) {
-        return isConnectedToDataset(datasetName);
+    public IActiveEventSubscriber subscribe(ActivityState state) throws HyracksDataException {
+        if (state != ActivityState.STARTED && state != ActivityState.STOPPED) {
+            throw new HyracksDataException("Can only wait for STARTED or STOPPED state");
+        }
+        synchronized (this) {
+            if (this.state == ActivityState.FAILED) {
+                throw new HyracksDataException("Feed has failed");
+            } else if (this.state == state) {
+                return NoOpSubscriber.INSTANCE;
+            }
+            return doSubscribe(state);
+        }
+    }
+
+    // Called within synchronized block
+    private FeedEventSubscriber doSubscribe(ActivityState state) {
+        FeedEventSubscriber subscriber = new FeedEventSubscriber(this, state);
+        subscribers.add(subscriber);
+        return subscriber;
+    }
+
+    @Override
+    public boolean isEntityUsingDataset(IDataset dataset) {
+        return datasets.contains(dataset);
+    }
+
+    public String[] getSources() {
+        return sources;
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedInfo.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedInfo.java
deleted file mode 100644
index 93f81d9..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedInfo.java
+++ /dev/null
@@ -1,53 +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 org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobInfo;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class FeedInfo {
-    public JobSpecification jobSpec;
-    public JobInfo jobInfo;
-    public JobId jobId;
-    public FeedInfoType infoType;
-    public State state;
-
-    public enum State {
-        ACTIVE,
-        INACTIVE
-    }
-
-    public enum FeedInfoType {
-        INTAKE,
-        COLLECT
-    }
-
-    public FeedInfo(JobSpecification jobSpec, JobId jobId, FeedInfoType infoType) {
-        this.jobSpec = jobSpec;
-        this.jobId = jobId;
-        this.infoType = infoType;
-        this.state = State.INACTIVE;
-    }
-
-    @Override
-    public String toString() {
-        return " job id " + jobId;
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedJointKey.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedJointKey.java
deleted file mode 100644
index 2905bb2..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedJointKey.java
+++ /dev/null
@@ -1,83 +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.io.Serializable;
-import java.util.List;
-
-import org.apache.asterix.active.EntityId;
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * Represents a unique identifier for a Feed Joint. A Feed joint is a logical entity located
- * along a feed ingestion pipeline at a point where the tuples moving as part of data flow
- * constitute the feed. The feed joint acts as a network tap and allows the flowing data to be
- * routed to multiple paths.
- */
-public class FeedJointKey implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-    private final EntityId primaryFeedId;
-    private final List<String> appliedFunctions;
-    private final String stringRep;
-
-    public FeedJointKey(EntityId feedId, List<String> appliedFunctions) {
-        this.primaryFeedId = feedId;
-        this.appliedFunctions = appliedFunctions;
-        StringBuilder builder = new StringBuilder();
-        builder.append(feedId);
-        builder.append(":");
-        builder.append(StringUtils.join(appliedFunctions, ':'));
-        stringRep = builder.toString();
-    }
-
-    public EntityId getFeedId() {
-        return primaryFeedId;
-    }
-
-    public List<String> getAppliedFunctions() {
-        return appliedFunctions;
-    }
-
-    public String getStringRep() {
-        return stringRep;
-    }
-
-    @Override
-    public final String toString() {
-        return stringRep;
-    }
-
-    @Override
-    public int hashCode() {
-        return stringRep.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || !(o instanceof FeedJointKey)) {
-            return false;
-        }
-        return stringRep.equals(((FeedJointKey) o).stringRep);
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedWorkManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedWorkManager.java
deleted file mode 100644
index bab4376..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedWorkManager.java
+++ /dev/null
@@ -1,50 +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.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.asterix.external.feed.api.IFeedWork;
-import org.apache.asterix.external.feed.api.IFeedWorkEventListener;
-import org.apache.asterix.external.feed.api.IFeedWorkManager;
-
-/**
- * Handles asynchronous execution of feed management related tasks.
- */
-public class FeedWorkManager implements IFeedWorkManager {
-
-    public static final FeedWorkManager INSTANCE = new FeedWorkManager();
-
-    private final ExecutorService executorService = Executors.newCachedThreadPool();
-
-    private FeedWorkManager() {
-    }
-
-    public void submitWork(IFeedWork work, IFeedWorkEventListener listener) {
-        Runnable runnable = work.getRunnable();
-        try {
-            executorService.execute(runnable);
-            listener.workCompleted(work);
-        } catch (Exception e) {
-            listener.workFailed(work, e);
-        }
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/EndFeedMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/EndFeedMessage.java
deleted file mode 100644
index 7a3a376..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/EndFeedMessage.java
+++ /dev/null
@@ -1,100 +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.message;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.util.FeedConstants;
-import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
-
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-/**
- * @deprecated A feed control message indicating the need to end the feed. This message is dispatched
- *             to all locations that host an operator involved in the feed pipeline.
- *             Instead, use IMessageBroker messages
- */
-@Deprecated
-public class EndFeedMessage extends FeedMessage {
-
-    private static final long serialVersionUID = 1L;
-
-    private final EntityId sourceFeedId;
-
-    private final FeedConnectionId connectionId;
-
-    private final FeedRuntimeType sourceRuntimeType;
-
-    private final boolean completeDisconnection;
-
-    private final EndMessageType endMessageType;
-
-    public enum EndMessageType {
-        DISCONNECT_FEED,
-        DISCONTINUE_SOURCE
-    }
-
-    public EndFeedMessage(FeedConnectionId connectionId, FeedRuntimeType sourceRuntimeType, EntityId sourceFeedId,
-            boolean completeDisconnection, EndMessageType endMessageType) {
-        super(MessageType.END);
-        this.connectionId = connectionId;
-        this.sourceRuntimeType = sourceRuntimeType;
-        this.sourceFeedId = sourceFeedId;
-        this.completeDisconnection = completeDisconnection;
-        this.endMessageType = endMessageType;
-    }
-
-    @Override
-    public String toString() {
-        return MessageType.END.name() + "  " + connectionId + " [" + sourceRuntimeType + "] ";
-    }
-
-    public FeedRuntimeType getSourceRuntimeType() {
-        return sourceRuntimeType;
-    }
-
-    public EntityId getSourceFeedId() {
-        return sourceFeedId;
-    }
-
-    public boolean isCompleteDisconnection() {
-        return completeDisconnection;
-    }
-
-    public EndMessageType getEndMessageType() {
-        return endMessageType;
-    }
-
-    @Override
-    public ObjectNode toJSON()  {
-        ObjectMapper om = new ObjectMapper();
-        ObjectNode obj = om.createObjectNode();
-        obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
-        obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
-        obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getEntityName());
-        obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
-        return obj;
-    }
-
-    public FeedConnectionId getFeedConnectionId() {
-        return connectionId;
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedMessage.java
deleted file mode 100644
index 4f57fb5..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedMessage.java
+++ /dev/null
@@ -1,42 +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.message;
-
-import org.apache.asterix.active.IActiveMessage;
-import org.apache.hyracks.api.dataflow.value.JSONSerializable;
-
-/**
- * A control message that can be sent to the runtime instance of a
- * feed's adapter.
- */
-public abstract class FeedMessage implements IActiveMessage, JSONSerializable {
-
-    private static final long serialVersionUID = 1L;
-
-    protected final MessageType messageType;
-
-    public FeedMessage(MessageType messageType) {
-        this.messageType = messageType;
-    }
-
-    public MessageType getMessageType() {
-        return messageType;
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java
deleted file mode 100644
index 821a0b1..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java
+++ /dev/null
@@ -1,100 +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.runtime;
-
-import java.util.Map;
-
-import org.apache.asterix.active.ActiveRuntimeId;
-import org.apache.asterix.active.IActiveRuntime;
-import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * Represents the feed runtime that collects feed tuples from another feed.
- * In case of a primary feed, the CollectionRuntime collects tuples from the feed
- * intake job. For a secondary feed, tuples are collected from the intake/compute
- * runtime associated with the source feed.
- */
-public class CollectionRuntime implements IActiveRuntime {
-
-    private final FeedConnectionId connectionId;        // [Dataverse - Feed - Dataset]
-    private final ISubscribableRuntime sourceRuntime;   // Runtime that provides the data
-    private final Map<String, String> feedPolicy;       // Policy associated with the feed
-    private final FeedFrameCollector frameCollector;    // Collector that can be plugged into a frame distributor
-    private final IHyracksTaskContext ctx;
-    private final ActiveRuntimeId runtimeId;
-
-    public CollectionRuntime(FeedConnectionId connectionId, ActiveRuntimeId runtimeId,
-            ISubscribableRuntime sourceRuntime, Map<String, String> feedPolicy, IHyracksTaskContext ctx,
-            FeedFrameCollector frameCollector) {
-        this.runtimeId = runtimeId;
-        this.connectionId = connectionId;
-        this.sourceRuntime = sourceRuntime;
-        this.feedPolicy = feedPolicy;
-        this.ctx = ctx;
-        this.frameCollector = frameCollector;
-    }
-
-    public void waitTillCollectionOver() throws InterruptedException {
-        if (!(isCollectionOver())) {
-            synchronized (frameCollector) {
-                while (!isCollectionOver()) {
-                    frameCollector.wait();
-                }
-            }
-        }
-    }
-
-    private boolean isCollectionOver() {
-        return frameCollector.getState().equals(FeedFrameCollector.State.FINISHED)
-                || frameCollector.getState().equals(FeedFrameCollector.State.HANDOVER);
-    }
-
-    public Map<String, String> getFeedPolicy() {
-        return feedPolicy;
-    }
-
-    public FeedConnectionId getConnectionId() {
-        return connectionId;
-    }
-
-    public ISubscribableRuntime getSourceRuntime() {
-        return sourceRuntime;
-    }
-
-    public FeedFrameCollector getFrameCollector() {
-        return frameCollector;
-    }
-
-    public IHyracksTaskContext getCtx() {
-        return ctx;
-    }
-
-    @Override
-    public ActiveRuntimeId getRuntimeId() {
-        return runtimeId;
-    }
-
-    @Override
-    public void stop() throws HyracksDataException, InterruptedException {
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java
index d32a604..3100704 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java
@@ -19,79 +19,43 @@
 package org.apache.asterix.external.feed.runtime;
 
 import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import org.apache.asterix.active.ActiveRuntimeId;
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.IActiveRuntime;
-import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
-import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.util.HyracksConstants;
-import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 
-public class IngestionRuntime extends SubscribableRuntime {
+public class IngestionRuntime implements IActiveRuntime {
+
+    private static final Logger LOGGER = Logger.getLogger(IngestionRuntime.class.getName());
 
     private final AdapterRuntimeManager adapterRuntimeManager;
-    private final IHyracksTaskContext ctx;
-    private int numSubscribers = 0;
+    private final ActiveRuntimeId runtimeId;
+    private final EntityId feedId;
 
-    public IngestionRuntime(EntityId entityId, ActiveRuntimeId runtimeId, DistributeFeedFrameWriter feedWriter,
-            AdapterRuntimeManager adaptorRuntimeManager, IHyracksTaskContext ctx) {
-        super(entityId, runtimeId, feedWriter);
+    public IngestionRuntime(EntityId entityId, ActiveRuntimeId runtimeId, AdapterRuntimeManager adaptorRuntimeManager) {
+        this.feedId = entityId;
+        this.runtimeId = runtimeId;
         this.adapterRuntimeManager = adaptorRuntimeManager;
-        this.ctx = ctx;
     }
 
     @Override
-    public synchronized void subscribe(CollectionRuntime collectionRuntime) throws HyracksDataException {
-        FeedFrameCollector collector = collectionRuntime.getFrameCollector();
-        dWriter.subscribe(collector);
-        subscribers.add(collectionRuntime);
-        if (numSubscribers == 0) {
-            TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, new VSizeFrame(ctx), ctx);
-            TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE,
-                    TaskUtil.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx), collectionRuntime.getCtx());
-            start();
-        }
-        numSubscribers++;
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Subscribed feed collection [" + collectionRuntime + "] to " + this);
-        }
-    }
-
-    @Override
-    public synchronized void unsubscribe(CollectionRuntime collectionRuntime) throws InterruptedException {
-        numSubscribers--;
-        if (numSubscribers == 0) {
-            stop();
-        }
-        subscribers.remove(collectionRuntime);
-    }
-
-    public AdapterRuntimeManager getAdapterRuntimeManager() {
-        return adapterRuntimeManager;
-    }
-
-    public void terminate() {
-        for (IActiveRuntime subscriber : subscribers) {
-            try {
-                unsubscribe((CollectionRuntime) subscriber);
-            } catch (Exception e) {
-                if (LOGGER.isLoggable(Level.WARNING)) {
-                    LOGGER.warning("Excpetion in unsubscribing " + subscriber + " message " + e.getMessage());
-                }
-            }
-        }
+    public ActiveRuntimeId getRuntimeId() {
+        return this.runtimeId;
     }
 
     public void start() {
         adapterRuntimeManager.start();
+        LOGGER.log(Level.INFO, "Feed " + feedId.getEntityName() + " running on partition " + runtimeId);
     }
 
     @Override
     public void stop() throws InterruptedException {
         adapterRuntimeManager.stop();
+        LOGGER.log(Level.INFO, "Feed " + feedId.getEntityName() + " stopped on partition " + runtimeId);
+    }
+
+    public EntityId getFeedId() {
+        return feedId;
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java
deleted file mode 100644
index fb70fdb..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java
+++ /dev/null
@@ -1,59 +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.runtime;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.logging.Logger;
-
-import org.apache.asterix.active.ActiveRuntimeId;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.active.IActiveRuntime;
-import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
-
-public abstract class SubscribableRuntime implements ISubscribableRuntime {
-
-    protected static final Logger LOGGER = Logger.getLogger(SubscribableRuntime.class.getName());
-    protected final EntityId feedId;
-    protected final List<IActiveRuntime> subscribers;
-    protected final DistributeFeedFrameWriter dWriter;
-    protected final ActiveRuntimeId runtimeId;
-
-    public SubscribableRuntime(EntityId feedId, ActiveRuntimeId runtimeId, DistributeFeedFrameWriter dWriter) {
-        this.runtimeId = runtimeId;
-        this.feedId = feedId;
-        this.dWriter = dWriter;
-        this.subscribers = new ArrayList<>();
-    }
-
-    @Override
-    public ActiveRuntimeId getRuntimeId() {
-        return runtimeId;
-    }
-
-    public EntityId getFeedId() {
-        return feedId;
-    }
-
-    @Override
-    public String toString() {
-        return "SubscribableRuntime" + " [" + feedId + "]" + "(" + runtimeId + ")";
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedConnectJobInfo.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedConnectJobInfo.java
deleted file mode 100644
index 82cdddf..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedConnectJobInfo.java
+++ /dev/null
@@ -1,108 +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.watch;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.active.ActiveJob;
-import org.apache.asterix.active.ActivityState;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.util.FeedUtils.JobType;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class FeedConnectJobInfo extends ActiveJob {
-
-    private static final long serialVersionUID = 1L;
-    private final FeedConnectionId connectionId;
-    private final Map<String, String> feedPolicy;
-    private final IFeedJoint sourceFeedJoint;
-    private IFeedJoint computeFeedJoint;
-
-    private List<String> collectLocations;
-    private List<String> computeLocations;
-    private List<String> storageLocations;
-    private int partitionStarts = 0;
-
-    public FeedConnectJobInfo(EntityId entityId, JobId jobId, ActivityState state, FeedConnectionId connectionId,
-            IFeedJoint sourceFeedJoint, IFeedJoint computeFeedJoint, JobSpecification spec,
-            Map<String, String> feedPolicy) {
-        super(entityId, jobId, state, JobType.FEED_CONNECT, spec);
-        this.connectionId = connectionId;
-        this.sourceFeedJoint = sourceFeedJoint;
-        this.computeFeedJoint = computeFeedJoint;
-        this.feedPolicy = feedPolicy;
-    }
-
-    public FeedConnectionId getConnectionId() {
-        return connectionId;
-    }
-
-    public List<String> getCollectLocations() {
-        return collectLocations;
-    }
-
-    public List<String> getComputeLocations() {
-        return computeLocations;
-    }
-
-    public List<String> getStorageLocations() {
-        return storageLocations;
-    }
-
-    public void setCollectLocations(List<String> collectLocations) {
-        this.collectLocations = collectLocations;
-    }
-
-    public void setComputeLocations(List<String> computeLocations) {
-        this.computeLocations = computeLocations;
-    }
-
-    public void setStorageLocations(List<String> storageLocations) {
-        this.storageLocations = storageLocations;
-    }
-
-    public IFeedJoint getSourceFeedJoint() {
-        return sourceFeedJoint;
-    }
-
-    public IFeedJoint getComputeFeedJoint() {
-        return computeFeedJoint;
-    }
-
-    public Map<String, String> getFeedPolicy() {
-        return feedPolicy;
-    }
-
-    public void setComputeFeedJoint(IFeedJoint computeFeedJoint) {
-        this.computeFeedJoint = computeFeedJoint;
-    }
-
-    public void partitionStart() {
-        partitionStarts++;
-    }
-
-    public boolean collectionStarted() {
-        return partitionStarts == collectLocations.size();
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedEventSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedEventSubscriber.java
new file mode 100644
index 0000000..0e931f7
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedEventSubscriber.java
@@ -0,0 +1,64 @@
+/*
+ * 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.watch;
+
+import org.apache.asterix.active.ActiveEvent;
+import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.IActiveEventSubscriber;
+import org.apache.asterix.external.feed.management.FeedEventsListener;
+
+public class FeedEventSubscriber implements IActiveEventSubscriber {
+
+    private final FeedEventsListener listener;
+    private final ActivityState state;
+    private boolean done = false;
+
+    public FeedEventSubscriber(FeedEventsListener listener, ActivityState state) {
+        this.listener = listener;
+        this.state = state;
+
+    }
+
+    @Override
+    public synchronized void notify(ActiveEvent event) {
+        if (listener.getState() == state || listener.getState() == ActivityState.FAILED
+                || listener.getState() == ActivityState.STOPPED) {
+            done = true;
+            notifyAll();
+        }
+    }
+
+    @Override
+    public synchronized boolean done() {
+        return done;
+    }
+
+    @Override
+    public synchronized void sync() throws InterruptedException {
+        while (!done) {
+            wait();
+        }
+    }
+
+    @Override
+    public synchronized void unsubscribe() {
+        done = true;
+        notifyAll();
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedIntakeInfo.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedIntakeInfo.java
deleted file mode 100644
index 4114e82..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedIntakeInfo.java
+++ /dev/null
@@ -1,68 +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.watch;
-
-import java.util.List;
-
-import org.apache.asterix.active.ActiveJob;
-import org.apache.asterix.active.ActivityState;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.util.FeedUtils.JobType;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class FeedIntakeInfo extends ActiveJob {
-
-    private static final long serialVersionUID = 1L;
-    private final EntityId feedId;
-    private final IFeedJoint intakeFeedJoint;
-    private final JobSpecification spec;
-    private List<String> intakeLocation;
-
-    public FeedIntakeInfo(JobId jobId, ActivityState state, EntityId feedId, IFeedJoint intakeFeedJoint,
-            JobSpecification spec) {
-        super(feedId, jobId, state, JobType.INTAKE, spec);
-        this.feedId = feedId;
-        this.intakeFeedJoint = intakeFeedJoint;
-        this.spec = spec;
-    }
-
-    public EntityId getFeedId() {
-        return feedId;
-    }
-
-    public IFeedJoint getIntakeFeedJoint() {
-        return intakeFeedJoint;
-    }
-
-    @Override
-    public JobSpecification getSpec() {
-        return spec;
-    }
-
-    public List<String> getIntakeLocation() {
-        return intakeLocation;
-    }
-
-    public void setIntakeLocation(List<String> intakeLocation) {
-        this.intakeLocation = intakeLocation;
-    }
-
-}
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
new file mode 100644
index 0000000..9d8c570
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
@@ -0,0 +1,54 @@
+/*
+ * 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.watch;
+
+import org.apache.asterix.active.ActiveEvent;
+import org.apache.asterix.active.IActiveEventSubscriber;
+
+/**
+ * An event subscriber that does not listen to any events
+ */
+public class NoOpSubscriber implements IActiveEventSubscriber {
+
+    public static final NoOpSubscriber INSTANCE = new NoOpSubscriber();
+
+    private NoOpSubscriber() {
+    }
+
+    @Override
+    public void notify(ActiveEvent event) {
+        // do nothing
+    }
+
+    @Override
+    public boolean done() {
+        return true;
+    }
+
+    @Override
+    public void sync() {
+        // do nothing
+    }
+
+    @Override
+    public void unsubscribe() {
+        // do nothing
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java
index 570155c..4d8be98 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java
@@ -48,6 +48,15 @@
     private Map<String, String> configuration;
     private transient AlgebricksAbsolutePartitionConstraint clusterLocations;
 
+    public static boolean isTwitterPull(Map<String, String> configuration) {
+        String reader = configuration.get(ExternalDataConstants.KEY_READER);
+        if (reader.equals(ExternalDataConstants.READER_TWITTER_PULL)
+                || reader.equals(ExternalDataConstants.READER_PULL_TWITTER)) {
+            return true;
+        }
+        return false;
+    }
+
     @Override
     public DataSourceType getDataSourceType() {
         return DataSourceType.RECORDS;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java
index 5337be1..6a581ef 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java
@@ -20,6 +20,7 @@
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.util.ArrayList;
 import java.util.List;
@@ -109,7 +110,8 @@
         try {
             Pair<String, Integer> socket = sockets.get(partition);
             ServerSocket server;
-            server = new ServerSocket(socket.second);
+            server = new ServerSocket();
+            server.bind(new InetSocketAddress(socket.second));
             return new SocketServerInputStream(server);
         } catch (IOException e) {
             throw new HyracksDataException(e);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java
index 4ecb887..c4cb650 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java
@@ -20,11 +20,6 @@
 
 import java.util.Map;
 
-import org.apache.asterix.active.ActiveManager;
-import org.apache.asterix.active.ActiveRuntimeId;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.common.api.IAppRuntimeContext;
-import org.apache.asterix.external.feed.api.ISubscribableRuntime;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
 import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
 import org.apache.asterix.om.types.ARecordType;
@@ -55,21 +50,17 @@
     /** Map representation of policy parameters */
     private final Map<String, String> feedPolicyProperties;
 
-    /** The source feed from which the feed derives its data from. **/
-    private final EntityId sourceFeedId;
-
     /** The subscription location at which the recipient feed receives tuples from the source feed {SOURCE_FEED_INTAKE_STAGE , SOURCE_FEED_COMPUTE_STAGE} **/
     private final FeedRuntimeType subscriptionLocation;
 
-    public FeedCollectOperatorDescriptor(JobSpecification spec, FeedConnectionId feedConnectionId,
-            EntityId sourceFeedId, ARecordType atype, RecordDescriptor rDesc, Map<String, String> feedPolicyProperties,
+    public FeedCollectOperatorDescriptor(JobSpecification spec, FeedConnectionId feedConnectionId, ARecordType atype,
+            RecordDescriptor rDesc, Map<String, String> feedPolicyProperties,
             FeedRuntimeType subscriptionLocation) {
-        super(spec, 0, 1);
+        super(spec, 1, 1);
         this.recordDescriptors[0] = rDesc;
         this.outputType = atype;
         this.connectionId = feedConnectionId;
         this.feedPolicyProperties = feedPolicyProperties;
-        this.sourceFeedId = sourceFeedId;
         this.subscriptionLocation = subscriptionLocation;
     }
 
@@ -77,11 +68,7 @@
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
             throws HyracksDataException {
-        ActiveManager feedManager = (ActiveManager) ((IAppRuntimeContext) ctx.getJobletContext()
-                .getApplicationContext().getApplicationObject()).getActiveManager();
-        ActiveRuntimeId sourceRuntimeId = new ActiveRuntimeId(sourceFeedId, subscriptionLocation.toString(), partition);
-        ISubscribableRuntime sourceRuntime = (ISubscribableRuntime) feedManager.getRuntime(sourceRuntimeId);
-        return new FeedCollectOperatorNodePushable(ctx, connectionId, feedPolicyProperties, partition, sourceRuntime);
+        return new FeedCollectOperatorNodePushable(ctx, connectionId, feedPolicyProperties, partition);
     }
 
     public FeedConnectionId getFeedConnectionId() {
@@ -100,10 +87,6 @@
         return recordDescriptors[0];
     }
 
-    public EntityId getSourceFeedId() {
-        return sourceFeedId;
-    }
-
     public FeedRuntimeType getSubscriptionLocation() {
         return subscriptionLocation;
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java
index d7fa590..384da84 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java
@@ -18,46 +18,38 @@
  */
 package org.apache.asterix.external.operators;
 
+import java.nio.ByteBuffer;
 import java.util.Map;
 
 import org.apache.asterix.active.ActiveManager;
 import org.apache.asterix.active.ActiveRuntimeId;
-import org.apache.asterix.active.message.ActivePartitionMessage;
 import org.apache.asterix.common.api.IAppRuntimeContext;
-import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
 import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
 import org.apache.asterix.external.feed.dataflow.SyncFeedRuntimeInputHandler;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.CollectionRuntime;
 import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
 
 /**
  * The first operator in a collect job in a feed.
  */
-public class FeedCollectOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+public class FeedCollectOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
 
     private final int partition;
     private final FeedConnectionId connectionId;
-    private final Map<String, String> feedPolicy;
     private final FeedPolicyAccessor policyAccessor;
     private final ActiveManager activeManager;
-    private final ISubscribableRuntime sourceRuntime;
     private final IHyracksTaskContext ctx;
-    private CollectionRuntime collectRuntime;
 
     public FeedCollectOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedConnectionId,
-            Map<String, String> feedPolicy, int partition, ISubscribableRuntime sourceRuntime) {
+            Map<String, String> feedPolicy, int partition) {
         this.ctx = ctx;
         this.partition = partition;
         this.connectionId = feedConnectionId;
-        this.sourceRuntime = sourceRuntime;
-        this.feedPolicy = feedPolicy;
         this.policyAccessor = new FeedPolicyAccessor(feedPolicy);
         this.activeManager = (ActiveManager) ((IAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
                 .getApplicationObject()).getActiveManager();
@@ -68,7 +60,6 @@
         try {
             ActiveRuntimeId runtimeId =
                     new ActiveRuntimeId(connectionId.getFeedId(), FeedRuntimeType.COLLECT.toString(), partition);
-            // Does this collector have a handler?
             FrameTupleAccessor tAccessor = new FrameTupleAccessor(recordDesc);
             if (policyAccessor.bufferingEnabled()) {
                 writer = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, writer, policyAccessor, tAccessor,
@@ -76,17 +67,33 @@
             } else {
                 writer = new SyncFeedRuntimeInputHandler(ctx, writer, tAccessor);
             }
-            collectRuntime = new CollectionRuntime(connectionId, runtimeId, sourceRuntime, feedPolicy, ctx,
-                    new FeedFrameCollector(policyAccessor, writer, connectionId));
-            activeManager.registerRuntime(collectRuntime);
-            sourceRuntime.subscribe(collectRuntime);
-            // Notify CC that Collection started
-            ctx.sendApplicationMessageToCC(new ActivePartitionMessage(runtimeId, ctx.getJobletContext().getJobId(),
-                    ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED), null);
-            collectRuntime.waitTillCollectionOver();
-            activeManager.deregisterRuntime(collectRuntime.getRuntimeId());
         } catch (Exception e) {
             throw new HyracksDataException(e);
         }
     }
+
+    @Override
+    public void open() throws HyracksDataException {
+        writer.open();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        writer.nextFrame(buffer);
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        writer.fail();
+    }
+
+    @Override
+    public void flush() throws HyracksDataException {
+        writer.flush();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        writer.close();
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
index 5f478c3..ab8c8f7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
@@ -28,7 +28,6 @@
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -45,6 +44,8 @@
  */
 public class FeedIntakeOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
 
+    private static final String FEED_EXTENSION_NAME = "Feed";
+
     private static final long serialVersionUID = 1L;
 
     private static final Logger LOGGER = Logger.getLogger(FeedIntakeOperatorDescriptor.class.getName());
@@ -53,29 +54,23 @@
     private final EntityId feedId;
 
     private final FeedPolicyAccessor policyAccessor;
-
+    private final ARecordType adapterOutputType;
     /** The adaptor factory that is used to create an instance of the feed adaptor **/
     private IAdapterFactory adaptorFactory;
-
     /** The library that contains the adapter in use. **/
     private String adaptorLibraryName;
-
     /**
      * The adapter factory class that is used to create an instance of the feed adapter.
      * This value is used only in the case of external adapters.
      **/
     private String adaptorFactoryClassName;
-
     /** The configuration parameters associated with the adapter. **/
     private Map<String, String> adaptorConfiguration;
 
-    private final ARecordType adapterOutputType;
-
     public FeedIntakeOperatorDescriptor(JobSpecification spec, IFeed primaryFeed, IAdapterFactory adapterFactory,
             ARecordType adapterOutputType, FeedPolicyAccessor policyAccessor, RecordDescriptor rDesc) {
         super(spec, 0, 1);
-        this.feedId = new EntityId(FeedConnectionId.FEED_EXTENSION_NAME, primaryFeed.getDataverseName(),
-                primaryFeed.getFeedName());
+        this.feedId = new EntityId(FEED_EXTENSION_NAME, primaryFeed.getDataverseName(), primaryFeed.getFeedName());
         this.adaptorFactory = adapterFactory;
         this.adapterOutputType = adapterOutputType;
         this.policyAccessor = policyAccessor;
@@ -86,8 +81,7 @@
             String adapterFactoryClassName, ARecordType adapterOutputType, FeedPolicyAccessor policyAccessor,
             RecordDescriptor rDesc) {
         super(spec, 0, 1);
-        this.feedId = new EntityId(FeedConnectionId.FEED_EXTENSION_NAME, primaryFeed.getDataverseName(),
-                primaryFeed.getFeedName());
+        this.feedId = new EntityId(FEED_EXTENSION_NAME, primaryFeed.getDataverseName(), primaryFeed.getFeedName());
         this.adaptorFactoryClassName = adapterFactoryClassName;
         this.adaptorLibraryName = adapterLibraryName;
         this.adaptorConfiguration = primaryFeed.getAdapterConfiguration();
@@ -108,8 +102,8 @@
 
     private IAdapterFactory createExternalAdapterFactory(IHyracksTaskContext ctx) throws HyracksDataException {
         IAdapterFactory adapterFactory;
-        IAppRuntimeContext runtimeCtx =
-                (IAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
+        IAppRuntimeContext runtimeCtx = (IAppRuntimeContext) ctx.getJobletContext()
+                .getApplicationContext().getApplicationObject();
         ILibraryManager libraryManager = runtimeCtx.getLibraryManager();
         ClassLoader classLoader = libraryManager.getLibraryClassLoader(feedId.getDataverse(), adaptorLibraryName);
         if (classLoader != null) {
@@ -130,8 +124,32 @@
         return adapterFactory;
     }
 
-    public EntityId getFeedId() {
+    public EntityId getEntityId() {
         return feedId;
     }
 
+    public IAdapterFactory getAdaptorFactory() {
+        return this.adaptorFactory;
+    }
+
+    public void setAdaptorFactory(IAdapterFactory factory) {
+        this.adaptorFactory = factory;
+    }
+
+    public ARecordType getAdapterOutputType() {
+        return this.adapterOutputType;
+    }
+
+    public FeedPolicyAccessor getPolicyAccessor() {
+        return this.policyAccessor;
+    }
+
+    public String getAdaptorLibraryName() {
+        return this.adaptorLibraryName;
+    }
+
+    public String getAdaptorFactoryClassName() {
+        return this.adaptorFactoryClassName;
+    }
+
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
index f58e9e5..99fff19 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
@@ -18,84 +18,58 @@
  */
 package org.apache.asterix.external.operators;
 
-import org.apache.asterix.active.ActiveManager;
 import org.apache.asterix.active.ActiveRuntimeId;
+import org.apache.asterix.active.ActiveSourceOperatorNodePushable;
 import org.apache.asterix.active.EntityId;
-import org.apache.asterix.active.message.ActivePartitionMessage;
-import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.dataset.adapter.FeedAdapter;
-import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
 import org.apache.asterix.external.feed.runtime.AdapterRuntimeManager;
-import org.apache.asterix.external.feed.runtime.IngestionRuntime;
-import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
+import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import org.apache.hyracks.api.util.HyracksConstants;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 
 /**
  * The runtime for @see{FeedIntakeOperationDescriptor}.
  * Provides the core functionality to set up the artifacts for ingestion of a feed.
  * The artifacts are lazily activated when a feed receives a subscription request.
  */
-public class FeedIntakeOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+public class FeedIntakeOperatorNodePushable extends ActiveSourceOperatorNodePushable {
 
-    private final EntityId feedId;
     private final int partition;
-    private final IHyracksTaskContext ctx;
     private final IAdapterFactory adapterFactory;
     private final FeedIntakeOperatorDescriptor opDesc;
+    private volatile AdapterRuntimeManager adapterRuntimeManager;
 
     public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, EntityId feedId, IAdapterFactory adapterFactory,
             int partition, FeedPolicyAccessor policyAccessor, IRecordDescriptorProvider recordDescProvider,
             FeedIntakeOperatorDescriptor feedIntakeOperatorDescriptor) {
+        super(ctx, new ActiveRuntimeId(feedId, FeedIntakeOperatorNodePushable.class.getSimpleName(), partition));
         this.opDesc = feedIntakeOperatorDescriptor;
         this.recordDesc = recordDescProvider.getOutputRecordDescriptor(opDesc.getActivityId(), 0);
-        this.ctx = ctx;
-        this.feedId = feedId;
         this.partition = partition;
         this.adapterFactory = adapterFactory;
     }
 
     @Override
-    public void initialize() throws HyracksDataException {
-        ActiveManager feedManager = (ActiveManager) ((IAppRuntimeContext) ctx.getJobletContext()
-                .getApplicationContext().getApplicationObject()).getActiveManager();
-        AdapterRuntimeManager adapterRuntimeManager = null;
-        DistributeFeedFrameWriter frameDistributor = null;
-        IngestionRuntime ingestionRuntime = null;
-        boolean open = false;
+    protected void start() throws HyracksDataException, InterruptedException {
+        writer.open();
         try {
             Thread.currentThread().setName("Intake Thread");
-            // create the adapter
             FeedAdapter adapter = (FeedAdapter) adapterFactory.createAdapter(ctx, partition);
-            // create the distributor
-            frameDistributor = new DistributeFeedFrameWriter(feedId, writer, FeedRuntimeType.INTAKE, partition);
-            // create adapter runtime manager
-            adapterRuntimeManager = new AdapterRuntimeManager(ctx, feedId, adapter, frameDistributor, partition);
-            // create and register the runtime
-            ActiveRuntimeId runtimeId = new ActiveRuntimeId(feedId, FeedRuntimeType.INTAKE.toString(), partition);
-            ingestionRuntime = new IngestionRuntime(feedId, runtimeId, frameDistributor, adapterRuntimeManager, ctx);
-            feedManager.registerRuntime(ingestionRuntime);
-            // Notify FeedJobNotificationHandler that this provider is ready to receive subscription requests.
-            ctx.sendApplicationMessageToCC(new ActivePartitionMessage(runtimeId, ctx.getJobletContext().getJobId(),
-                    ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED), null);
-            // open the distributor
-            open = true;
-            frameDistributor.open();
-            // wait until ingestion is over
+            adapterRuntimeManager = new AdapterRuntimeManager(ctx, runtimeId.getEntityId(), adapter, writer, partition);
+            TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, new VSizeFrame(ctx), ctx);
+            adapterRuntimeManager.start();
             synchronized (adapterRuntimeManager) {
                 while (!adapterRuntimeManager.isDone()) {
                     adapterRuntimeManager.wait();
                 }
             }
-            // The ingestion is over. we need to remove the runtime from the manager
-            feedManager.deregisterRuntime(ingestionRuntime.getRuntimeId());
-            // If there was a failure, we need to throw an exception
             if (adapterRuntimeManager.isFailed()) {
                 throw new RuntimeDataException(
                         ErrorCode.OPERATORS_FEED_INTAKE_OPERATOR_NODE_PUSHABLE_FAIL_AT_INGESTION);
@@ -106,15 +80,16 @@
              * As the Intake job involves only the intake operator, the exception is indicative of a failure at the sibling intake operator location.
              * The surviving intake partitions must continue to live and receive data from the external source.
              */
-            if (ingestionRuntime != null) {
-                ingestionRuntime.terminate();
-                feedManager.deregisterRuntime(ingestionRuntime.getRuntimeId());
-            }
             throw new HyracksDataException(ie);
         } finally {
-            if (open) {
-                frameDistributor.close();
-            }
+                writer.close();
+        }
+    }
+
+    @Override
+    protected void abort() throws HyracksDataException, InterruptedException {
+        if (adapterRuntimeManager != null) {
+            adapterRuntimeManager.stop();
         }
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorDescriptor.java
deleted file mode 100644
index 61451b1..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorDescriptor.java
+++ /dev/null
@@ -1,56 +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.operators;
-
-import org.apache.asterix.active.IActiveMessage;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-
-/**
- * @deprecated
- *             Sends a control message to the registered message queue for feed specified by its feedId.
- *             For messaging, use IMessageBroker interfaces
- */
-@Deprecated
-public class FeedMessageOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    private final FeedConnectionId connectionId;
-    private final IActiveMessage feedMessage;
-
-    public FeedMessageOperatorDescriptor(JobSpecification spec, FeedConnectionId connectionId,
-            IActiveMessage feedMessage) {
-        super(spec, 0, 1);
-        this.connectionId = connectionId;
-        this.feedMessage = feedMessage;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new FeedMessageOperatorNodePushable(ctx, connectionId, feedMessage, partition);
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java
deleted file mode 100644
index b273325..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java
+++ /dev/null
@@ -1,173 +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.operators;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.active.ActiveManager;
-import org.apache.asterix.active.ActiveRuntimeId;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.active.IActiveMessage;
-import org.apache.asterix.common.api.IAppRuntimeContext;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.message.EndFeedMessage;
-import org.apache.asterix.external.feed.runtime.AdapterRuntimeManager;
-import org.apache.asterix.external.feed.runtime.CollectionRuntime;
-import org.apache.asterix.external.feed.runtime.IngestionRuntime;
-import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-
-/**
- * @deprecated
- *             Runtime for the FeedMessageOpertorDescriptor. This operator is responsible for communicating
- *             a feed message to the local feed manager on the host node controller.
- *             For messages, use IMessageBroker interfaces
- * @see FeedMessageOperatorDescriptor
- *      IFeedMessage
- *      IFeedManager
- */
-@Deprecated
-public class FeedMessageOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
-
-    private static final Logger LOGGER = Logger.getLogger(FeedMessageOperatorNodePushable.class.getName());
-
-    private final FeedConnectionId connectionId;
-    private final IActiveMessage message;
-    private final ActiveManager feedManager;
-    private final int partition;
-
-    public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId connectionId,
-            IActiveMessage feedMessage, int partition) {
-        this.connectionId = connectionId;
-        this.message = feedMessage;
-        this.partition = partition;
-        IAppRuntimeContext runtimeCtx =
-                (IAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
-        this.feedManager = (ActiveManager) runtimeCtx.getActiveManager();
-    }
-
-    @Override
-    public void initialize() throws HyracksDataException {
-        try {
-            writer.open();
-            switch (message.getMessageType()) {
-                case END:
-                    EndFeedMessage endFeedMessage = (EndFeedMessage) message;
-                    switch (endFeedMessage.getEndMessageType()) {
-                        case DISCONNECT_FEED:
-                            hanldeDisconnectFeedTypeMessage(endFeedMessage);
-                            break;
-                        case DISCONTINUE_SOURCE:
-                            handleDiscontinueFeedTypeMessage(endFeedMessage);
-                            break;
-                        default:
-                            break;
-                    }
-                    break;
-                default:
-                    break;
-            }
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        } finally {
-            writer.close();
-        }
-    }
-
-    private void handleDiscontinueFeedTypeMessage(EndFeedMessage endFeedMessage) throws Exception {
-        EntityId sourceFeedId = endFeedMessage.getSourceFeedId();
-        ActiveRuntimeId subscribableRuntimeId =
-                new ActiveRuntimeId(sourceFeedId, FeedRuntimeType.INTAKE.toString(), partition);
-        ISubscribableRuntime feedRuntime = (ISubscribableRuntime) feedManager.getRuntime(subscribableRuntimeId);
-        AdapterRuntimeManager adapterRuntimeManager = ((IngestionRuntime) feedRuntime).getAdapterRuntimeManager();
-        adapterRuntimeManager.stop();
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Stopped Adapter " + adapterRuntimeManager);
-        }
-    }
-
-    private void hanldeDisconnectFeedTypeMessage(EndFeedMessage endFeedMessage) throws Exception {
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Ending feed:" + endFeedMessage.getFeedConnectionId());
-        }
-        ActiveRuntimeId runtimeId;
-        FeedRuntimeType subscribableRuntimeType = ((EndFeedMessage) message).getSourceRuntimeType();
-        if (endFeedMessage.isCompleteDisconnection()) {
-            // subscribableRuntimeType represents the location at which the feed connection receives
-            // data
-            FeedRuntimeType runtimeType;
-            switch (subscribableRuntimeType) {
-                case INTAKE:
-                    runtimeType = FeedRuntimeType.COLLECT;
-                    break;
-                case COMPUTE:
-                    runtimeType = FeedRuntimeType.COMPUTE_COLLECT;
-                    break;
-                default:
-                    throw new RuntimeDataException(
-                            ErrorCode.OPERATORS_FEED_MSG_OPERATOR_NODE_PUSHABLE_INVALID_SUBSCRIBABLE_RUNTIME,
-                            subscribableRuntimeType);
-            }
-
-            runtimeId = new ActiveRuntimeId(endFeedMessage.getSourceFeedId(), runtimeType.toString(), partition);
-            CollectionRuntime feedRuntime = (CollectionRuntime) feedManager.getRuntime(runtimeId);
-            if (feedRuntime != null) {
-                feedRuntime.getSourceRuntime().unsubscribe(feedRuntime);
-            }
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Complete Unsubscription of " + endFeedMessage.getFeedConnectionId());
-            }
-        } else {
-            // subscribaleRuntimeType represents the location for data hand-off in presence of
-            // subscribers
-            switch (subscribableRuntimeType) {
-                case INTAKE:
-                    // illegal state as data hand-off from one feed to another does not happen at
-                    // intake
-                    throw new RuntimeDataException(
-                            ErrorCode.OPERATORS_FEED_MSG_OPERATOR_NODE_PUSHABLE_INVALID_SUBSCRIBABLE_RUNTIME,
-                            subscribableRuntimeType);
-                case COMPUTE:
-                    // feed could be primary or secondary, doesn't matter
-                    ActiveRuntimeId feedSubscribableRuntimeId = new ActiveRuntimeId(connectionId.getFeedId(),
-                            FeedRuntimeType.COMPUTE.toString(), partition);
-                    ISubscribableRuntime feedRuntime =
-                            (ISubscribableRuntime) feedManager.getRuntime(feedSubscribableRuntimeId);
-                    runtimeId = new ActiveRuntimeId(endFeedMessage.getSourceFeedId(),
-                            FeedRuntimeType.COMPUTE_COLLECT.toString(), partition);
-                    CollectionRuntime feedCollectionRuntime = (CollectionRuntime) feedManager.getRuntime(runtimeId);
-                    feedRuntime.unsubscribe(feedCollectionRuntime);
-                    break;
-                default:
-                    break;
-            }
-
-        }
-
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Unsubscribed from feed :" + connectionId);
-        }
-    }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
index 5ec0399..b794ee1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
@@ -44,6 +44,8 @@
 
 public class FeedUtils {
 
+    public static final String FEED_EXTENSION_NAME = "Feed";
+
     public enum JobType {
         INTAKE,
         FEED_CONNECT
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java
index 171d271..d407b8a 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java
@@ -31,6 +31,7 @@
 import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
+import org.apache.asterix.external.util.FeedUtils;
 import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.comm.VSizeFrame;
@@ -72,7 +73,7 @@
     private FeedRuntimeInputHandler createInputHandler(IHyracksTaskContext ctx, IFrameWriter writer,
             FeedPolicyAccessor fpa, ConcurrentFramePool framePool) throws HyracksDataException {
         FrameTupleAccessor fta = Mockito.mock(FrameTupleAccessor.class);
-        EntityId feedId = new EntityId(FeedConnectionId.FEED_EXTENSION_NAME, DATAVERSE, FEED);
+        EntityId feedId = new EntityId(FeedUtils.FEED_EXTENSION_NAME, DATAVERSE, FEED);
         FeedConnectionId connectionId = new FeedConnectionId(feedId, DATASET);
         ActiveRuntimeId runtimeId = new ActiveRuntimeId(feedId, FeedRuntimeType.COLLECT.toString(), 0);
         return new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, writer, fpa, fta, framePool);
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-adapters/typed_adapter/typed_adapter.2.update.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-adapters/typed_adapter/typed_adapter.2.update.aql
index 39a6272..40aec53 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-adapters/typed_adapter/typed_adapter.2.update.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-adapters/typed_adapter/typed_adapter.2.update.aql
@@ -30,3 +30,5 @@
 set wait-for-completion-feed "true";
 
 connect feed TestTypedAdapterFeed to dataset TweetsTestAdapter;
+
+start feed TestTypedAdapterFeed;
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.1.ddl.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.1.ddl.aql
index dbe3cfa..5f3d322 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.1.ddl.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.1.ddl.aql
@@ -19,8 +19,8 @@
 /*
  * Description  : Create a feed dataset that uses the feed simulator adapter.
                   The feed simulator simulates feed from a file in the local fs.
-                  Associate with the feed an external user-defined function. The UDF 
-                  finds topics in each tweet. A topic is identified by a #. 
+                  Associate with the feed an external user-defined function. The UDF
+                  finds topics in each tweet. A topic is identified by a #.
                   Begin ingestion and apply external user defined function
  * Expected Res : Success
  * Date         : 23rd Apr 2013
@@ -48,8 +48,7 @@
 using localfs
 (("type-name"="TweetInputType"),
 ("path"="asterix_nc1://../../../../../../asterix-app/data/twitter/obamatweets.adm"),
-("format"="adm"))
-apply function testlib#parseTweet;
+("format"="adm"));
 
 create dataset TweetsFeedIngest(TweetOutputType)
 primary key id;
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.2.update.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.2.update.aql
index d5a6f58..9642992 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.2.update.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.2.update.aql
@@ -19,8 +19,8 @@
 /*
  * Description  : Create a feed dataset that uses the feed simulator adapter.
                   The feed simulator simulates feed from a file in the local fs.
-                  Associate with the feed an external user-defined function. The UDF 
-                  finds topics in each tweet. A topic is identified by a #. 
+                  Associate with the feed an external user-defined function. The UDF
+                  finds topics in each tweet. A topic is identified by a #.
                   Begin ingestion and apply external user defined function
  * Expected Res : Success
  * Date         : 23rd Apr 2013
@@ -29,4 +29,7 @@
 
 set wait-for-completion-feed "true";
 
-connect feed TweetFeed to dataset TweetsFeedIngest;
+connect feed TweetFeed to dataset TweetsFeedIngest
+apply function testlib#parseTweet;
+
+start feed TweetFeed;
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.4.query.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.4.query.aql
index e4e1b45..8879fa8 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.4.query.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/feed_ingest/feed_ingest.4.query.aql
@@ -19,8 +19,8 @@
 /*
  * Description  : Create a feed dataset that uses the feed simulator adapter.
                   The feed simulator simulates feed from a file in the local fs.
-                  Associate with the feed an external user-defined function. The UDF 
-                  finds topics in each tweet. A topic is identified by a #. 
+                  Associate with the feed an external user-defined function. The UDF
+                  finds topics in each tweet. A topic is identified by a #.
                   Begin ingestion and apply external user defined function
  * Expected Res : Success
  * Date         : 23rd Apr 2013
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.02.ddl.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.02.ddl.aql
index 58dea6b..1e05e37 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.02.ddl.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.02.ddl.aql
@@ -25,3 +25,5 @@
 
 use dataverse twitter;
 connect feed MessageFeed to dataset ds_tweet;
+
+start feed MessageFeed;
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.04.ddl.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.04.ddl.aql
index cf520ca..fe30266 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.04.ddl.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.04.ddl.aql
@@ -33,3 +33,4 @@
 );
 set wait-for-completion-feed "true";
 connect feed TweetFeed to dataset ds_tweet;
+start feed TweetFeed;
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.08.ddl.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.08.ddl.aql
index 860b8ed..8cb5e07 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.08.ddl.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/restart/queries/feed-restart/issue-1636/issue-1636.08.ddl.aql
@@ -24,4 +24,4 @@
  */
 use dataverse twitter;
 set wait-for-completion-feed "false";
-connect feed TweetFeed to dataset ds_tweet;
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/query_after_restart/dataset-with-meta-record/dataset-with-meta-record.3.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/query_after_restart/dataset-with-meta-record/dataset-with-meta-record.3.update.aql
index 7faf013..d9b1230 100644
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/query_after_restart/dataset-with-meta-record/dataset-with-meta-record.3.update.aql
+++ b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/query_after_restart/dataset-with-meta-record/dataset-with-meta-record.3.update.aql
@@ -24,4 +24,5 @@
 use dataverse KeyVerse;
 
 set wait-for-completion-feed "true";
-connect feed KVChangeStream to dataset KVStore;
\ No newline at end of file
+connect feed KVChangeStream to dataset KVStore;
+start feed KVChangeStream;
\ No newline at end of file
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/results/query_after_restart/dataset-with-meta-record/dataset-with-meta-record.5.adm b/asterixdb/asterix-installer/src/test/resources/transactionts/results/query_after_restart/dataset-with-meta-record/dataset-with-meta-record.5.adm
index c31da8b..2975e63 100644
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/results/query_after_restart/dataset-with-meta-record/dataset-with-meta-record.5.adm
+++ b/asterixdb/asterix-installer/src/test/resources/transactionts/results/query_after_restart/dataset-with-meta-record/dataset-with-meta-record.5.adm
@@ -1 +1 @@
-804
\ No newline at end of file
+788
\ No newline at end of file
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 1d0d962..4596054 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
@@ -32,6 +32,7 @@
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
 import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.FeedUtils;
 import org.apache.asterix.lang.aql.parser.AQLParserFactory;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.IParserFactory;
@@ -44,6 +45,7 @@
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -54,14 +56,14 @@
  */
 public class SubscribeFeedStatement implements Statement {
 
+    public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
+    private static final Integer INSERT_STATEMENT_POS = 3;
     private static final Logger LOGGER = Logger.getLogger(SubscribeFeedStatement.class.getName());
-    private final FeedConnectionRequest connectionRequest;
-    private Query query;
     private final int varCounter;
     private final String[] locations;
-
-    public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
+    private final FeedConnectionRequest connectionRequest;
     private final IParserFactory parserFactory = new AQLParserFactory();
+    private Query query;
 
     public SubscribeFeedStatement(String[] locations, FeedConnectionRequest subscriptionRequest) {
         this.connectionRequest = subscriptionRequest;
@@ -71,7 +73,7 @@
 
     public void initialize(MetadataTransactionContext mdTxnCtx) throws MetadataException {
         this.query = new Query(false);
-        EntityId sourceFeedId = connectionRequest.getFeedJointKey().getFeedId();
+        EntityId sourceFeedId = connectionRequest.getReceivingFeedId();
         Feed subscriberFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx,
                 connectionRequest.getReceivingFeedId().getDataverse(),
                 connectionRequest.getReceivingFeedId().getEntityName());
@@ -80,18 +82,6 @@
         }
 
         String feedOutputType = getOutputType(mdTxnCtx);
-        FunctionSignature appliedFunction = subscriberFeed.getAppliedFunction();
-        Function function = null;
-        if (appliedFunction != null) {
-            function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, appliedFunction);
-            if (function == null) {
-                throw new MetadataException(" Unknown function " + appliedFunction);
-            } else if (function.getParams().size() > 1) {
-                throw new MetadataException(
-                        " Incompatible function: " + appliedFunction + " Number if arguments must be 1");
-            }
-        }
-
         StringBuilder builder = new StringBuilder();
         builder.append("use dataverse " + sourceFeedId.getDataverse() + ";\n");
         builder.append("set" + " " + FunctionUtil.IMPORT_PRIVATE_FUNCTIONS + " " + "'" + Boolean.TRUE + "'" + ";\n");
@@ -105,14 +95,15 @@
                 + connectionRequest.getSubscriptionLocation().name() + "'" + "," + "'"
                 + connectionRequest.getTargetDataset() + "'" + "," + "'" + feedOutputType + "'" + ")");
 
-        List<String> functionsToApply = connectionRequest.getFunctionsToApply();
+        List<FunctionSignature> functionsToApply = connectionRequest.getFunctionsToApply();
         if ((functionsToApply != null) && functionsToApply.isEmpty()) {
             builder.append(" return $x");
         } else {
+            Function function;
             String rValueName = "x";
             String lValueName = "y";
             int variableIndex = 0;
-            for (String functionName : functionsToApply) {
+            for (FunctionSignature appliedFunction : functionsToApply) {
                 function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, appliedFunction);
                 variableIndex++;
                 switch (function.getLanguage().toUpperCase()) {
@@ -122,8 +113,8 @@
                         builder.append("\n");
                         break;
                     case Function.LANGUAGE_JAVA:
-                        builder.append(" let " + "$" + lValueName + variableIndex + ":=" + functionName + "(" + "$"
-                                + rValueName + ")");
+                        builder.append(" let " + "$" + lValueName + variableIndex + ":=" + function.getName() + "("
+                                + "$" + rValueName + ")");
                         rValueName = lValueName + variableIndex;
                         break;
                 }
@@ -141,7 +132,7 @@
         List<Statement> statements;
         try {
             statements = parser.parse();
-            query = ((InsertStatement) statements.get(3)).getQuery();
+            query = ((InsertStatement) statements.get(INSERT_STATEMENT_POS)).getQuery();
         } catch (CompilationException pe) {
             throw new MetadataException(pe);
         }
@@ -179,21 +170,13 @@
     }
 
     private String getOutputType(MetadataTransactionContext mdTxnCtx) throws MetadataException {
-        String outputType = null;
+        String outputType;
         EntityId feedId = connectionRequest.getReceivingFeedId();
         Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName());
-        FeedPolicyAccessor policyAccessor = new FeedPolicyAccessor(connectionRequest.getPolicyParameters());
         try {
-            switch (feed.getFeedType()) {
-                case PRIMARY:
-                    outputType = FeedMetadataUtil
-                            .getOutputType(feed, feed.getAdapterConfiguration(), ExternalDataConstants.KEY_TYPE_NAME)
-                            .getTypeName();
-                    break;
-                case SECONDARY:
-                    outputType = FeedMetadataUtil.getSecondaryFeedOutput(feed, policyAccessor, mdTxnCtx);
-                    break;
-            }
+            outputType = FeedMetadataUtil
+                    .getOutputType(feed, feed.getAdapterConfiguration(), ExternalDataConstants.KEY_TYPE_NAME)
+                    .getTypeName();
             return outputType;
 
         } catch (AlgebricksException | RemoteException | ACIDException ae) {
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index e5af86f..f4cdfb8 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -98,10 +98,10 @@
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.statement.StopFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
-import org.apache.asterix.lang.common.statement.CreatePrimaryFeedStatement;
-import org.apache.asterix.lang.common.statement.CreateSecondaryFeedStatement;
 import org.apache.asterix.lang.common.statement.DatasetDecl;
 import org.apache.asterix.lang.common.statement.DataverseDecl;
 import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -702,26 +702,15 @@
   boolean ifNotExists = false;
   String adapterName = null;
   Map<String,String> properties = null;
-  FunctionSignature appliedFunction = null;
   CreateFeedStatement cfs = null;
   Pair<Identifier,Identifier> sourceNameComponents = null;
 
 }
 {
-  (
-    <SECONDARY> <FEED>  nameComponents = QualifiedName() ifNotExists = IfNotExists()
-    <FROM> <FEED> sourceNameComponents = QualifiedName() (appliedFunction = ApplyFunction())?
-    {
-      cfs = new CreateSecondaryFeedStatement(nameComponents, sourceNameComponents, appliedFunction, ifNotExists);
-    }
-   |
-    (<PRIMARY>)? <FEED> nameComponents = QualifiedName() ifNotExists = IfNotExists()
-    <USING> adapterName = AdapterName() properties = Configuration() (appliedFunction = ApplyFunction())?
-     {
-      cfs = new CreatePrimaryFeedStatement(nameComponents, adapterName, properties, appliedFunction, ifNotExists);
-     }
-  )
+  <FEED> nameComponents = QualifiedName() ifNotExists = IfNotExists()
+  <USING> adapterName = AdapterName() properties = Configuration()
   {
+    cfs = new CreateFeedStatement(nameComponents, adapterName, properties, ifNotExists);
     return cfs;
   }
 }
@@ -1157,19 +1146,29 @@
   Pair<Identifier,Identifier> datasetNameComponents = null;
 
   Map<String,String> configuration = null;
+  FunctionSignature appliedFunction = null;
   Statement stmt = null;
   String policy = null;
 }
 {
   (
-    <CONNECT> <FEED> feedNameComponents = QualifiedName() <TO> <DATASET> datasetNameComponents = QualifiedName() (policy = GetPolicy())?
+    <CONNECT> <FEED> feedNameComponents = QualifiedName() <TO> <DATASET> datasetNameComponents = QualifiedName()
+    (appliedFunction = ApplyFunction())? (policy = GetPolicy())?
       {
-        stmt = new ConnectFeedStatement(feedNameComponents, datasetNameComponents, policy, getVarCounter());
+        stmt = new ConnectFeedStatement(feedNameComponents, datasetNameComponents, appliedFunction, policy, getVarCounter());
       }
     | <DISCONNECT> <FEED> feedNameComponents = QualifiedName() <FROM> <DATASET> datasetNameComponents = QualifiedName()
       {
         stmt = new DisconnectFeedStatement(feedNameComponents, datasetNameComponents);
       }
+    | <START> <FEED> feedNameComponents = QualifiedName()
+      {
+        stmt = new StartFeedStatement (feedNameComponents);
+      }
+    | <STOP> <FEED> feedNameComponents = QualifiedName()
+      {
+        stmt = new StopFeedStatement (feedNameComponents);
+      }
   )
     {
       return stmt;
@@ -2706,6 +2705,8 @@
   | <SECONDARY : "secondary">
   | <SELECT : "select">
   | <SET : "set">
+  | <START: "start">
+  | <STOP: "stop">
   | <SOME : "some">
   | <TEMPORARY : "temporary">
   | <THEN : "then">
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index 06fbf33..612b230 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -79,12 +79,12 @@
         public static final byte INDEX_DECL = 0x13;
         public static final byte CREATE_DATAVERSE = 0x14;
         public static final byte INDEX_DROP = 0x15;
-        public static final byte CREATE_PRIMARY_FEED = 0x16;
-        public static final byte CREATE_SECONDARY_FEED = 0x17;
-        public static final byte DROP_FEED = 0x18;
-        public static final byte CONNECT_FEED = 0x19;
-        public static final byte DISCONNECT_FEED = 0x1a;
-        public static final byte SUBSCRIBE_FEED = 0x1b;
+        public static final byte CREATE_FEED = 0x16;
+        public static final byte DROP_FEED = 0x17;
+        public static final byte START_FEED = 0x18;
+        public static final byte STOP_FEED = 0x19;
+        public static final byte CONNECT_FEED = 0x1a;
+        public static final byte DISCONNECT_FEED = 0x1b;
         public static final byte CREATE_FEED_POLICY = 0x1c;
         public static final byte DROP_FEED_POLICY = 0x1d;
         public static final byte CREATE_FUNCTION = 0x1e;
@@ -93,6 +93,7 @@
         public static final byte EXTERNAL_DATASET_REFRESH = 0x21;
         public static final byte RUN = 0x22;
         public static final byte EXTENSION = 0x23;
+        public static final byte SUBSCRIBE_FEED = 0x24;
 
         private Kind() {
         }
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 ceab6e9..0bd34ee 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
@@ -19,26 +19,28 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 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;
     private final Identifier datasetName;
     private final String feedName;
     private final String policy;
-    private Query query;
     private int varCounter;
-    private boolean forceConnect = false;
-
-    public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
+    private final ArrayList<FunctionSignature> appliedFunctions;
 
     public ConnectFeedStatement(Pair<Identifier, Identifier> feedNameCmp, Pair<Identifier, Identifier> datasetNameCmp,
-            String policy, int varCounter) {
+            FunctionSignature appliedFunction, String policy, int varCounter) {
+        appliedFunctions = new ArrayList<>();
         if (feedNameCmp.first != null && datasetNameCmp.first != null
                 && !feedNameCmp.first.getValue().equals(datasetNameCmp.first.getValue())) {
             throw new IllegalArgumentException("Dataverse for source feed and target dataset do not match");
@@ -49,15 +51,9 @@
         this.feedName = feedNameCmp.second.getValue();
         this.policy = policy != null ? policy : BuiltinFeedPolicies.DEFAULT_POLICY.getPolicyName();
         this.varCounter = varCounter;
-    }
-
-    public ConnectFeedStatement(Identifier dataverseName, Identifier feedName, Identifier datasetName, String policy,
-            int varCounter) {
-        this.dataverseName = dataverseName;
-        this.datasetName = datasetName;
-        this.feedName = feedName.getValue();
-        this.policy = policy != null ? policy : BuiltinFeedPolicies.DEFAULT_POLICY.getPolicyName();
-        this.varCounter = varCounter;
+        if (appliedFunction != null) {
+            this.appliedFunctions.add(appliedFunction);
+        }
     }
 
     public Identifier getDataverseName() {
@@ -68,10 +64,6 @@
         return datasetName;
     }
 
-    public Query getQuery() {
-        return query;
-    }
-
     public int getVarCounter() {
         return varCounter;
     }
@@ -90,18 +82,14 @@
         return visitor.visit(this, arg);
     }
 
-    public boolean forceConnect() {
-        return forceConnect;
-    }
-
-    public void setForceConnect(boolean forceConnect) {
-        this.forceConnect = forceConnect;
-    }
-
     public String getFeedName() {
         return feedName;
     }
 
+    public List<FunctionSignature> getAppliedFunctions() {
+        return appliedFunctions;
+    }
+
     @Override
     public byte getCategory() {
         return Category.UPDATE;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
index 56e7d33..1e7a182 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
@@ -23,19 +23,28 @@
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.hadoop.io.compress.bzip2.CBZip2InputStream;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public abstract class CreateFeedStatement implements Statement {
+import java.util.Map;
+
+/**
+ * The new create feed statement only concerns the feed adaptor configuration.
+ * All feeds are considered as primary feeds.
+ */
+public class CreateFeedStatement implements Statement {
 
     private final Pair<Identifier, Identifier> qName;
-    private final FunctionSignature appliedFunction;
     private final boolean ifNotExists;
+    private final String adaptorName;
+    private final Map<String, String> adaptorConfiguration;
 
-    public CreateFeedStatement(Pair<Identifier, Identifier> qName, FunctionSignature appliedFunction,
-            boolean ifNotExists) {
+    public CreateFeedStatement(Pair<Identifier, Identifier> qName, String adaptorName,
+            Map<String, String> adaptorConfiguration, boolean ifNotExists) {
         this.qName = qName;
-        this.appliedFunction = appliedFunction;
         this.ifNotExists = ifNotExists;
+        this.adaptorName = adaptorName;
+        this.adaptorConfiguration = adaptorConfiguration;
     }
 
     public Identifier getDataverseName() {
@@ -46,16 +55,27 @@
         return qName.second;
     }
 
-    public FunctionSignature getAppliedFunction() {
-        return appliedFunction;
-    }
-
     public boolean getIfNotExists() {
         return this.ifNotExists;
     }
 
+    public String getAdaptorName() {
+        return adaptorName;
+    }
+
+    public Map<String, String> getAdaptorConfiguration() {
+        return adaptorConfiguration;
+    }
+
     @Override
-    public abstract <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException;
+    public byte getKind() {
+        return Kind.CREATE_FEED;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
 
     @Override
     public byte getCategory() {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateSecondaryFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateSecondaryFeedStatement.java
deleted file mode 100644
index 241bcd8..0000000
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateSecondaryFeedStatement.java
+++ /dev/null
@@ -1,62 +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.lang.common.statement;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.struct.Identifier;
-import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-
-/**
- * Represents the AQL statement for creating a secondary feed.
- * A secondary feed is one that derives its data from another (primary/secondary) feed.
- */
-public class CreateSecondaryFeedStatement extends CreateFeedStatement {
-
-    /** The source feed that provides data for this secondary feed. */
-    private final Pair<Identifier, Identifier> sourceQName;
-
-    public CreateSecondaryFeedStatement(Pair<Identifier, Identifier> qName, Pair<Identifier, Identifier> sourceQName,
-            FunctionSignature appliedFunction, boolean ifNotExists) {
-        super(qName, appliedFunction, ifNotExists);
-        this.sourceQName = sourceQName;
-    }
-
-    public String getSourceFeedDataverse() {
-        return sourceQName.first != null ? sourceQName.first.toString()
-                : getDataverseName() != null ? getDataverseName().getValue() : null;
-    }
-
-    public String getSourceFeedName() {
-        return sourceQName.second != null ? sourceQName.second.toString() : null;
-    }
-
-    @Override
-    public byte getKind() {
-        return Statement.Kind.CREATE_SECONDARY_FEED;
-    }
-
-    @Override
-    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
-        return visitor.visit(this, arg);
-    }
-
-}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreatePrimaryFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
similarity index 62%
rename from asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreatePrimaryFeedStatement.java
rename to asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
index c584ed0..b3452b5 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreatePrimaryFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
@@ -16,44 +16,46 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.asterix.lang.common.statement;
 
-import java.util.Map;
-
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class CreatePrimaryFeedStatement extends CreateFeedStatement {
+public class StartFeedStatement implements Statement {
 
-    private final String adaptorName;
-    private final Map<String, String> adaptorConfiguration;
+    public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
+    private Identifier dataverseName;
+    private Identifier feedName;
 
-    public CreatePrimaryFeedStatement(Pair<Identifier, Identifier> qName, String adaptorName,
-            Map<String, String> adaptorConfiguration, FunctionSignature appliedFunction, boolean ifNotExists) {
-        super(qName, appliedFunction, ifNotExists);
-        this.adaptorName = adaptorName;
-        this.adaptorConfiguration = adaptorConfiguration;
-    }
-
-    public String getAdaptorName() {
-        return adaptorName;
-    }
-
-    public Map<String, String> getAdaptorConfiguration() {
-        return adaptorConfiguration;
+    public StartFeedStatement(Pair<Identifier, Identifier> feedNameComp) {
+        dataverseName = feedNameComp.first;
+        feedName = feedNameComp.second;
     }
 
     @Override
     public byte getKind() {
-        return Statement.Kind.CREATE_PRIMARY_FEED;
+        return Kind.START_FEED;
     }
 
     @Override
     public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
         return visitor.visit(this, arg);
     }
+
+    @Override
+    public byte getCategory() {
+        return Category.UPDATE;
+    }
+
+    public Identifier getDataverseName() {
+        return dataverseName;
+    }
+
+    public Identifier getFeedName() {
+        return feedName;
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreatePrimaryFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
similarity index 62%
copy from asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreatePrimaryFeedStatement.java
copy to asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
index c584ed0..c45933e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreatePrimaryFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
@@ -18,42 +18,42 @@
  */
 package org.apache.asterix.lang.common.statement;
 
-import java.util.Map;
-
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class CreatePrimaryFeedStatement extends CreateFeedStatement {
+public class StopFeedStatement implements Statement {
 
-    private final String adaptorName;
-    private final Map<String, String> adaptorConfiguration;
+    private final Identifier dataverseName;
+    private final Identifier feedName;
 
-    public CreatePrimaryFeedStatement(Pair<Identifier, Identifier> qName, String adaptorName,
-            Map<String, String> adaptorConfiguration, FunctionSignature appliedFunction, boolean ifNotExists) {
-        super(qName, appliedFunction, ifNotExists);
-        this.adaptorName = adaptorName;
-        this.adaptorConfiguration = adaptorConfiguration;
-    }
-
-    public String getAdaptorName() {
-        return adaptorName;
-    }
-
-    public Map<String, String> getAdaptorConfiguration() {
-        return adaptorConfiguration;
+    public StopFeedStatement(Pair<Identifier, Identifier> feedNameComp) {
+        this.dataverseName = feedNameComp.first;
+        this.feedName = feedNameComp.second;
     }
 
     @Override
     public byte getKind() {
-        return Statement.Kind.CREATE_PRIMARY_FEED;
+        return Kind.STOP_FEED;
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.UPDATE;
     }
 
     @Override
     public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
         return visitor.visit(this, arg);
     }
+
+    public Identifier getDataverseName() {
+        return dataverseName;
+    }
+
+    public Identifier getFeedName() {
+        return feedName;
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index eefed9d..35d0a29 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -60,14 +60,14 @@
 import org.apache.asterix.lang.common.expression.UnaryExpr;
 import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
 import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.IntegerLiteral;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
+import org.apache.asterix.lang.common.statement.CreateFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
-import org.apache.asterix.lang.common.statement.CreatePrimaryFeedStatement;
-import org.apache.asterix.lang.common.statement.CreateSecondaryFeedStatement;
 import org.apache.asterix.lang.common.statement.DatasetDecl;
 import org.apache.asterix.lang.common.statement.DataverseDecl;
 import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -87,6 +87,8 @@
 import org.apache.asterix.lang.common.statement.NodegroupDecl;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.statement.SetStatement;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.statement.StopFeedStatement;
 import org.apache.asterix.lang.common.statement.TypeDecl;
 import org.apache.asterix.lang.common.statement.TypeDropStatement;
 import org.apache.asterix.lang.common.statement.UpdateStatement;
@@ -107,14 +109,13 @@
     private final static String CREATE = "create ";
     private final static String FEED = " feed ";
     private final static String DEFAULT_DATAVERSE_FORMAT = "org.apache.asterix.runtime.formats.NonTaggedDataFormat";
+    private final PrintWriter out;
     protected Set<Character> validIdentifierChars = new HashSet<Character>();
     protected Set<Character> validIdentifierStartChars = new HashSet<Character>();
     protected String dataverseSymbol = " dataverse ";
     protected String datasetSymbol = " dataset ";
     protected String assignSymbol = ":=";
 
-    private final PrintWriter out;
-
     public FormatPrintVisitor() {
         this(new PrintWriter(System.out));
     }
@@ -747,35 +748,36 @@
         if (connectFeedStmt.getPolicy() != null) {
             out.print(" using policy " + revertStringToQuoted(connectFeedStmt.getPolicy()));
         }
-        out.println(SEMICOLON);
-        return null;
-    }
-
-    @Override
-    public Void visit(CreatePrimaryFeedStatement cpfs, Integer step) throws CompilationException {
-        out.print(skip(step) + CREATE + " primary feed ");
-        out.print(generateFullName(cpfs.getDataverseName(), cpfs.getFeedName()));
-        out.print(generateIfNotExists(cpfs.getIfNotExists()));
-        out.print(" using " + cpfs.getAdaptorName() + " ");
-        printConfiguration(cpfs.getAdaptorConfiguration());
-        FunctionSignature func = cpfs.getAppliedFunction();
-        if (func != null) {
-            out.print(" apply function " + generateFullName(func.getNamespace(), func.getName()));
+        if (connectFeedStmt.getAppliedFunctions() != null) {
+            out.print(" apply function " + connectFeedStmt.getAppliedFunctions());
         }
         out.println(SEMICOLON);
         return null;
     }
 
     @Override
-    public Void visit(CreateSecondaryFeedStatement csfs, Integer step) throws CompilationException {
-        out.print(skip(step) + CREATE + " secondary feed ");
-        out.print(generateFullName(csfs.getDataverseName(), csfs.getFeedName()));
-        out.print(generateIfNotExists(csfs.getIfNotExists()));
-        out.print(" from feed " + generateFullName(csfs.getSourceFeedDataverse(), csfs.getSourceFeedName()));
-        FunctionSignature func = csfs.getAppliedFunction();
-        if (func != null) {
-            out.print(" apply function " + generateFullName(func.getNamespace(), func.getName()));
-        }
+    public Void visit(CreateFeedStatement cfs, Integer step) throws CompilationException {
+        out.print(skip(step) + "create " + FEED);
+        out.print(generateFullName(cfs.getDataverseName(), cfs.getFeedName()));
+        out.print(generateIfNotExists(cfs.getIfNotExists()));
+        out.print(" using " + cfs.getAdaptorName() + " ");
+        printConfiguration(cfs.getAdaptorConfiguration());
+        out.println(SEMICOLON);
+        return null;
+    }
+
+    @Override
+    public Void visit(StartFeedStatement startFeedStatement, Integer step) throws CompilationException {
+        out.print(skip(step) + "start " + FEED);
+        out.print(generateFullName(startFeedStatement.getDataverseName(), startFeedStatement.getFeedName()));
+        out.println(SEMICOLON);
+        return null;
+    }
+
+    @Override
+    public Void visit(StopFeedStatement stopFeedStatement, Integer step) throws CompilationException {
+        out.print(skip(step) + "stop " + FEED);
+        out.print(generateFullName(stopFeedStatement.getDataverseName(), stopFeedStatement.getFeedName()));
         out.println(SEMICOLON);
         return null;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index a4868d0..117fa77 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
@@ -28,10 +28,9 @@
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
+import org.apache.asterix.lang.common.statement.CreateFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
-import org.apache.asterix.lang.common.statement.CreatePrimaryFeedStatement;
-import org.apache.asterix.lang.common.statement.CreateSecondaryFeedStatement;
 import org.apache.asterix.lang.common.statement.DatasetDecl;
 import org.apache.asterix.lang.common.statement.DataverseDecl;
 import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -47,6 +46,8 @@
 import org.apache.asterix.lang.common.statement.NodeGroupDropStatement;
 import org.apache.asterix.lang.common.statement.NodegroupDecl;
 import org.apache.asterix.lang.common.statement.SetStatement;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.statement.StopFeedStatement;
 import org.apache.asterix.lang.common.statement.TypeDecl;
 import org.apache.asterix.lang.common.statement.TypeDropStatement;
 import org.apache.asterix.lang.common.statement.UpdateStatement;
@@ -180,12 +181,17 @@
     }
 
     @Override
-    public R visit(CreatePrimaryFeedStatement del, T arg) throws CompilationException {
+    public R visit(CreateFeedStatement cfs, T arg) throws CompilationException {
         return null;
     }
 
     @Override
-    public R visit(CreateSecondaryFeedStatement del, T arg) throws CompilationException {
+    public R visit(StartFeedStatement sfs, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
+    public R visit(StopFeedStatement sfs, T arg) throws CompilationException {
         return null;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index 4a5c5ed..cace925 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
@@ -44,10 +44,9 @@
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
+import org.apache.asterix.lang.common.statement.CreateFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
-import org.apache.asterix.lang.common.statement.CreatePrimaryFeedStatement;
-import org.apache.asterix.lang.common.statement.CreateSecondaryFeedStatement;
 import org.apache.asterix.lang.common.statement.DatasetDecl;
 import org.apache.asterix.lang.common.statement.DataverseDecl;
 import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -65,6 +64,8 @@
 import org.apache.asterix.lang.common.statement.NodegroupDecl;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.statement.SetStatement;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.statement.StopFeedStatement;
 import org.apache.asterix.lang.common.statement.TypeDecl;
 import org.apache.asterix.lang.common.statement.TypeDropStatement;
 import org.apache.asterix.lang.common.statement.UpdateStatement;
@@ -152,9 +153,11 @@
 
     R visit(ConnectFeedStatement del, T arg) throws CompilationException;
 
-    R visit(CreatePrimaryFeedStatement cpfs, T arg) throws CompilationException;
+    R visit(StartFeedStatement sfs, T arg) throws CompilationException;
 
-    R visit(CreateSecondaryFeedStatement csfs, T arg) throws CompilationException;
+    R visit(StopFeedStatement sfs, T arg) throws CompilationException;
+
+    R visit(CreateFeedStatement cfs, T arg) throws CompilationException;
 
     R visit(FeedDropStatement del, T arg) throws CompilationException;
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 81f00ee..d791c85 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -89,13 +89,13 @@
 import org.apache.asterix.lang.common.parser.ScopeChecker;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.statement.StopFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
-import org.apache.asterix.lang.common.statement.CreatePrimaryFeedStatement;
-import org.apache.asterix.lang.common.statement.CreateSecondaryFeedStatement;
 import org.apache.asterix.lang.common.statement.DatasetDecl;
 import org.apache.asterix.lang.common.statement.DataverseDecl;
 import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -741,30 +741,16 @@
   boolean ifNotExists = false;
   String adapterName = null;
   Map<String,String> properties = null;
-  FunctionSignature appliedFunction = null;
   CreateFeedStatement cfs = null;
   Pair<Identifier,Identifier> sourceNameComponents = null;
-
 }
 {
-  (
-    <SECONDARY> <FEED>  nameComponents = QualifiedName() ifNotExists = IfNotExists()
-      <FROM> <FEED> sourceNameComponents = QualifiedName() (appliedFunction = ApplyFunction())?
-      {
-        cfs = new CreateSecondaryFeedStatement(nameComponents,
-                                   sourceNameComponents, appliedFunction, ifNotExists);
-      }
-     |
-     (<PRIMARY>)? <FEED> nameComponents = QualifiedName() ifNotExists = IfNotExists()
-      <USING> adapterName = AdapterName() properties = Configuration() (appliedFunction = ApplyFunction())?
-       {
-        cfs = new CreatePrimaryFeedStatement(nameComponents,
-                                    adapterName, properties, appliedFunction, ifNotExists);
-       }
-  )
-    {
-      return cfs;
-    }
+  <FEED> nameComponents = QualifiedName() ifNotExists = IfNotExists()
+  <USING> adapterName = AdapterName() properties = Configuration()
+  {
+    cfs = new CreateFeedStatement(nameComponents, adapterName, properties, ifNotExists);
+    return cfs;
+  }
 }
 
 CreateFeedPolicyStatement FeedPolicySpecification() throws ParseException:
@@ -1185,12 +1171,43 @@
   (
     <CONNECT> stmt = ConnectStatement()
   | <DISCONNECT> stmt = DisconnectStatement()
+  | <START> stmt = StartStatement()
+  | <STOP> stmt = StopStatement()
   )
   {
     return stmt;
   }
 }
 
+Statement StartStatement() throws ParseException:
+{
+  Pair<Identifier,Identifier> feedNameComponents = null;
+
+  Statement stmt = null;
+}
+{
+  <FEED> feedNameComponents = QualifiedName()
+  {
+    stmt = new StartFeedStatement (feedNameComponents);
+    return stmt;
+  }
+}
+
+Statement StopStatement () throws ParseException:
+{
+  Pair<Identifier,Identifier> feedNameComponents = null;
+
+  Statement stmt = null;
+}
+{
+  <FEED> feedNameComponents = QualifiedName()
+  {
+    stmt = new StopFeedStatement (feedNameComponents);
+    return stmt;
+  }
+}
+
+
 Statement DisconnectStatement() throws ParseException:
 {
   Pair<Identifier,Identifier> feedNameComponents = null;
@@ -1218,14 +1235,17 @@
   Pair<Identifier,Identifier> datasetNameComponents = null;
 
   Map<String,String> configuration = null;
+  FunctionSignature appliedFunction = null;
   Statement stmt = null;
   String policy = null;
 }
 {
   (
-    <FEED> feedNameComponents = QualifiedName() <TO> Dataset() datasetNameComponents = QualifiedName() (policy = GetPolicy())?
+    <FEED> feedNameComponents = QualifiedName() <TO> Dataset() datasetNameComponents = QualifiedName()
+    (appliedFunction = ApplyFunction())?  (policy = GetPolicy())?
       {
-        stmt = new ConnectFeedStatement(feedNameComponents, datasetNameComponents, policy, getVarCounter());
+        stmt = new ConnectFeedStatement(feedNameComponents, datasetNameComponents, appliedFunction,
+         policy, getVarCounter());
       }
   )
   {
@@ -3200,6 +3220,8 @@
   | <SELECT : "select">
   | <SET : "set">
   | <SOME : "some">
+  | <START : "start">
+  | <STOP : "stop">
   | <TEMPORARY : "temporary">
   | <THEN : "then">
   | <TYPE : "type">
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 78a7d6f..01ade10 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
@@ -35,6 +35,7 @@
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
@@ -53,28 +54,34 @@
     // Default life time period of a temp dataset. It is 30 days.
     private final static long TEMP_DATASET_INACTIVE_TIME_THRESHOLD = 3600 * 24 * 30 * 1000L;
     // Key is dataverse name.
-    protected final Map<String, Dataverse> dataverses = new HashMap<String, Dataverse>();
+    protected final Map<String, Dataverse> dataverses = new HashMap<>();
     // Key is dataverse name. Key of value map is dataset name.
-    protected final Map<String, Map<String, Dataset>> datasets = new HashMap<String, Map<String, Dataset>>();
+    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<String, Map<String, Map<String, Index>>>();
+    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<String, Map<String, Datatype>>();
+    protected final Map<String, Map<String, Datatype>> datatypes = new HashMap<>();
     // Key is dataverse name.
-    protected final Map<String, NodeGroup> nodeGroups = new HashMap<String, NodeGroup>();
+    protected final Map<String, NodeGroup> nodeGroups = new HashMap<>();
     // Key is function Identifier . Key of value map is function name.
-    protected final Map<FunctionSignature, Function> functions = new HashMap<FunctionSignature, Function>();
+    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<String, Map<String, DatasourceAdapter>>();
+    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<String, Map<String, FeedPolicyEntity>>();
+    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<String, Map<String, Library>>();
+    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<String, Map<String, Feed>>();
+    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<String, Map<String, CompactionPolicy>>();
+    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<>();
 
     // Atomically executes all metadata operations in ctx's log.
     public void commit(MetadataTransactionContext ctx) {
@@ -163,7 +170,7 @@
 
                 Map<String, Dataset> m = datasets.get(dataset.getDataverseName());
                 if (m == null) {
-                    m = new HashMap<String, Dataset>();
+                    m = new HashMap<>();
                     datasets.put(dataset.getDataverseName(), m);
                 }
                 if (!m.containsKey(dataset.getDatasetName())) {
@@ -184,7 +191,7 @@
         synchronized (datatypes) {
             Map<String, Datatype> m = datatypes.get(datatype.getDataverseName());
             if (m == null) {
-                m = new HashMap<String, Datatype>();
+                m = new HashMap<>();
                 datatypes.put(datatype.getDataverseName(), m);
             }
             if (!m.containsKey(datatype.getDatatypeName())) {
@@ -207,7 +214,7 @@
         synchronized (compactionPolicy) {
             Map<String, CompactionPolicy> p = compactionPolicies.get(compactionPolicy.getDataverseName());
             if (p == null) {
-                p = new HashMap<String, CompactionPolicy>();
+                p = new HashMap<>();
                 p.put(compactionPolicy.getPolicyName(), compactionPolicy);
                 compactionPolicies.put(compactionPolicy.getDataverseName(), p);
             } else {
@@ -244,7 +251,8 @@
                                             datatypes.remove(dataverse.getDataverseName());
                                             adapters.remove(dataverse.getDataverseName());
                                             compactionPolicies.remove(dataverse.getDataverseName());
-                                            List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<FunctionSignature>();
+                                            List<FunctionSignature> markedFunctionsForRemoval =
+                                                    new ArrayList<>();
                                             for (FunctionSignature signature : functions.keySet()) {
                                                 if (signature.getNamespace().equals(dataverse.getDataverseName())) {
                                                     markedFunctionsForRemoval.add(signature);
@@ -371,7 +379,7 @@
     }
 
     public List<Dataset> getDataverseDatasets(String dataverseName) {
-        List<Dataset> retDatasets = new ArrayList<Dataset>();
+        List<Dataset> retDatasets = new ArrayList<>();
         synchronized (datasets) {
             Map<String, Dataset> m = datasets.get(dataverseName);
             if (m == null) {
@@ -385,7 +393,7 @@
     }
 
     public List<Index> getDatasetIndexes(String dataverseName, String datasetName) {
-        List<Index> retIndexes = new ArrayList<Index>();
+        List<Index> retIndexes = new ArrayList<>();
         synchronized (datasets) {
             Map<String, Index> map = indexes.get(dataverseName).get(datasetName);
             if (map == null) {
@@ -398,28 +406,13 @@
         }
     }
 
-    /**
-     * Represents a logical operation against the metadata.
-     */
-    protected class MetadataLogicalOperation {
-        // Entity to be added/dropped.
-        public final IMetadataEntity<?> entity;
-        // True for add, false for drop.
-        public final boolean isAdd;
-
-        public MetadataLogicalOperation(IMetadataEntity<?> entity, boolean isAdd) {
-            this.entity = entity;
-            this.isAdd = isAdd;
-        }
-    };
-
     protected void doOperation(MetadataLogicalOperation op) {
         if (op.isAdd) {
             op.entity.addToCache(this);
         } else {
             op.entity.dropFromCache(this);
         }
-    }
+    };
 
     protected void undoOperation(MetadataLogicalOperation op) {
         if (!op.isAdd) {
@@ -431,8 +424,8 @@
 
     public Function addFunctionIfNotExists(Function function) {
         synchronized (functions) {
-            FunctionSignature signature = new FunctionSignature(function.getDataverseName(), function.getName(),
-                    function.getArity());
+            FunctionSignature signature =
+                    new FunctionSignature(function.getDataverseName(), function.getName(), function.getArity());
             Function fun = functions.get(signature);
             if (fun == null) {
                 return functions.put(signature, function);
@@ -443,8 +436,8 @@
 
     public Function dropFunction(Function function) {
         synchronized (functions) {
-            FunctionSignature signature = new FunctionSignature(function.getDataverseName(), function.getName(),
-                    function.getArity());
+            FunctionSignature signature =
+                    new FunctionSignature(function.getDataverseName(), function.getName(), function.getArity());
             Function fun = functions.get(signature);
             if (fun == null) {
                 return null;
@@ -457,7 +450,7 @@
         synchronized (feedPolicy) {
             Map<String, FeedPolicyEntity> p = feedPolicies.get(feedPolicy.getDataverseName());
             if (p == null) {
-                p = new HashMap<String, FeedPolicyEntity>();
+                p = new HashMap<>();
                 p.put(feedPolicy.getPolicyName(), feedPolicy);
                 feedPolicies.put(feedPolicy.getDataverseName(), p);
             } else {
@@ -481,10 +474,10 @@
 
     public DatasourceAdapter addAdapterIfNotExists(DatasourceAdapter adapter) {
         synchronized (adapters) {
-            Map<String, DatasourceAdapter> adaptersInDataverse = adapters
-                    .get(adapter.getAdapterIdentifier().getNamespace());
+            Map<String, DatasourceAdapter> adaptersInDataverse =
+                    adapters.get(adapter.getAdapterIdentifier().getNamespace());
             if (adaptersInDataverse == null) {
-                adaptersInDataverse = new HashMap<String, DatasourceAdapter>();
+                adaptersInDataverse = new HashMap<>();
                 adapters.put(adapter.getAdapterIdentifier().getNamespace(), adaptersInDataverse);
             }
             DatasourceAdapter adapterObject = adaptersInDataverse.get(adapter.getAdapterIdentifier().getName());
@@ -497,8 +490,8 @@
 
     public DatasourceAdapter dropAdapter(DatasourceAdapter adapter) {
         synchronized (adapters) {
-            Map<String, DatasourceAdapter> adaptersInDataverse = adapters
-                    .get(adapter.getAdapterIdentifier().getNamespace());
+            Map<String, DatasourceAdapter> adaptersInDataverse =
+                    adapters.get(adapter.getAdapterIdentifier().getNamespace());
             if (adaptersInDataverse != null) {
                 return adaptersInDataverse.remove(adapter.getAdapterIdentifier().getName());
             }
@@ -512,7 +505,7 @@
             boolean needToAddd = (libsInDataverse == null || libsInDataverse.get(library.getName()) != null);
             if (needToAddd) {
                 if (libsInDataverse == null) {
-                    libsInDataverse = new HashMap<String, Library>();
+                    libsInDataverse = new HashMap<>();
                     libraries.put(library.getDataverseName(), libsInDataverse);
                 }
                 return libsInDataverse.put(library.getDataverseName(), library);
@@ -531,8 +524,37 @@
         }
     }
 
+    public FeedConnection addFeedConnectionIfNotExists(FeedConnection feedConnection) {
+        synchronized (feedConnections) {
+            Map<String, FeedConnection> feedConnsInDataverse = feedConnections.get(feedConnection.getDataverseName());
+            if (feedConnsInDataverse == null) {
+                feedConnections.put(feedConnection.getDataverseName(), new HashMap<>());
+                feedConnsInDataverse = feedConnections.get(feedConnection.getDataverseName());
+            }
+            return feedConnsInDataverse.put(feedConnection.getConnectionId(), feedConnection);
+        }
+    }
+
+    public FeedConnection dropFeedConnection(FeedConnection feedConnection) {
+        synchronized (feedConnections) {
+            Map<String, FeedConnection> feedConnsInDataverse = feedConnections.get(feedConnection.getDataverseName());
+            if (feedConnsInDataverse != null) {
+                return feedConnsInDataverse.remove(feedConnection.getConnectionId());
+            } else {
+                return null;
+            }
+        }
+    }
+
     public Feed addFeedIfNotExists(Feed feed) {
-        return null;
+        synchronized (feeds) {
+            Map<String, Feed> feedsInDataverse = feeds.get(feed.getDataverseName());
+            if (feedsInDataverse == null) {
+                feeds.put(feed.getDataverseName(), new HashMap<>());
+                feedsInDataverse = feeds.get(feed.getDataverseName());
+            }
+            return feedsInDataverse.put(feed.getFeedName(), feed);
+        }
     }
 
     public Feed dropFeed(Feed feed) {
@@ -548,12 +570,12 @@
     private Index addIndexIfNotExistsInternal(Index index) {
         Map<String, Map<String, Index>> datasetMap = indexes.get(index.getDataverseName());
         if (datasetMap == null) {
-            datasetMap = new HashMap<String, Map<String, Index>>();
+            datasetMap = new HashMap<>();
             indexes.put(index.getDataverseName(), datasetMap);
         }
         Map<String, Index> indexMap = datasetMap.get(index.getDatasetName());
         if (indexMap == null) {
-            indexMap = new HashMap<String, Index>();
+            indexMap = new HashMap<>();
             datasetMap.put(index.getDatasetName(), indexMap);
         }
         if (!indexMap.containsKey(index.getIndexName())) {
@@ -583,4 +605,19 @@
             }
         }
     }
+
+    /**
+     * Represents a logical operation against the metadata.
+     */
+    protected class MetadataLogicalOperation {
+        // Entity to be added/dropped.
+        public final IMetadataEntity<?> entity;
+        // True for add, false for drop.
+        public final boolean isAdd;
+
+        public MetadataLogicalOperation(IMetadataEntity<?> entity, boolean isAdd) {
+            this.entity = entity;
+            this.isAdd = isAdd;
+        }
+    }
 }
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 59911d1..5d44d0b 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
@@ -42,6 +42,7 @@
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
@@ -779,10 +780,16 @@
 
     @Override
     public void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException {
-        Feed feed;
+        Feed feed = null;
+        List<FeedConnection> feedConnections = null;
         try {
             feed = metadataNode.getFeed(ctx.getJobId(), dataverse, feedName);
+            feedConnections = metadataNode.getFeedConnections(ctx.getJobId(), dataverse, feedName);
             metadataNode.dropFeed(ctx.getJobId(), dataverse, feedName);
+            for (FeedConnection feedConnection : feedConnections) {
+                metadataNode.dropFeedConnection(ctx.getJobId(), dataverse, feedName, feedConnection.getDatasetName());
+                ctx.dropFeedConnection(dataverse, feedName, feedConnection.getDatasetName());
+            }
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -800,6 +807,48 @@
     }
 
     @Override
+    public void addFeedConnection(MetadataTransactionContext ctx, FeedConnection feedConnection)
+            throws MetadataException {
+        try {
+            metadataNode.addFeedConnection(ctx.getJobId(), feedConnection);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        ctx.addFeedConnection(feedConnection);
+    }
+
+    @Override
+    public void dropFeedConnection(MetadataTransactionContext ctx, String dataverseName, String feedName,
+            String datasetName) throws MetadataException {
+        try {
+            metadataNode.dropFeedConnection(ctx.getJobId(), dataverseName, feedName, datasetName);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        ctx.dropFeedConnection(dataverseName, feedName, datasetName);
+    }
+
+    @Override
+    public FeedConnection getFeedConnection(MetadataTransactionContext ctx, String dataverseName, String feedName,
+            String datasetName) throws MetadataException {
+        try {
+            return metadataNode.getFeedConnection(ctx.getJobId(), dataverseName, feedName, datasetName);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
+    public List<FeedConnection> getFeedConections(MetadataTransactionContext ctx, String dataverseName, String feedName)
+            throws MetadataException {
+        try {
+            return metadataNode.getFeedConnections(ctx.getJobId(), dataverseName, feedName);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
     public List<DatasourceAdapter> getDataverseAdapters(MetadataTransactionContext mdTxnCtx, String dataverse)
             throws MetadataException {
         List<DatasourceAdapter> dataverseAdapters;
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 8ecc0ed..51790e6 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
@@ -60,6 +60,7 @@
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
@@ -73,6 +74,7 @@
 import org.apache.asterix.metadata.entitytupletranslators.DatatypeTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.DataverseTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.ExternalFileTupleTranslator;
+import org.apache.asterix.metadata.entitytupletranslators.FeedConnectionTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.FeedPolicyTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.FeedTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
@@ -487,11 +489,16 @@
             }
 
             List<Feed> dataverseFeeds;
+            List<FeedConnection> feedConnections;
             Feed feed;
             dataverseFeeds = getDataverseFeeds(jobId, dataverseName);
-            // Drop all datasets in this dataverse.
+            // Drop all feeds&connections in this dataverse.
             for (int i = 0; i < dataverseFeeds.size(); i++) {
                 feed = dataverseFeeds.get(i);
+                feedConnections = getFeedConnections(jobId, dataverseName, feed.getFeedName());
+                for (FeedConnection feedConnection : feedConnections) {
+                    dropFeedConnection(jobId, dataverseName, feed.getFeedName(), feedConnection.getDatasetName());
+                }
                 dropFeed(jobId, dataverseName, feed.getFeedName());
             }
 
@@ -1480,6 +1487,63 @@
     }
 
     @Override
+    public void addFeedConnection(JobId jobId, FeedConnection feedConnection) throws MetadataException {
+        try {
+            FeedConnectionTupleTranslator tupleReaderWriter = new FeedConnectionTupleTranslator(true);
+            ITupleReference feedConnTuple = tupleReaderWriter.getTupleFromMetadataEntity(feedConnection);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, feedConnTuple);
+        } catch (IndexException | ACIDException | IOException e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
+    public List<FeedConnection> getFeedConnections(JobId jobId, String dataverseName, String feedName)
+            throws MetadataException {
+        try {
+            ITupleReference searchKey = createTuple(dataverseName, feedName);
+            FeedConnectionTupleTranslator tupleReaderWriter = new FeedConnectionTupleTranslator(false);
+            List<FeedConnection> results = new ArrayList<>();
+            IValueExtractor<FeedConnection> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+            searchIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey, valueExtractor, results);
+            return results;
+        } catch (IndexException | IOException e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
+    public FeedConnection getFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
+            throws MetadataException {
+        try {
+            ITupleReference searchKey = createTuple(dataverseName, feedName, datasetName);
+            FeedConnectionTupleTranslator tupleReaderWriter = new FeedConnectionTupleTranslator(false);
+            List<FeedConnection> results = new ArrayList<>();
+            IValueExtractor<FeedConnection> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+            searchIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey, valueExtractor, results);
+            if (!results.isEmpty()) {
+                return results.get(0);
+            }
+            return null;
+        } catch (IndexException | IOException e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
+    public void dropFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
+            throws MetadataException {
+        try {
+            ITupleReference searchKey = createTuple(dataverseName, feedName, datasetName);
+            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET,
+                    searchKey);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, tuple);
+        } catch (IndexException | IOException | ACIDException e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
     public void addFeed(JobId jobId, Feed feed) throws MetadataException, RemoteException {
         try {
             // Insert into the 'Feed' dataset.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index 87c1c473..b2ec7f2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -24,14 +24,13 @@
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
-import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.api.IFeed.FeedType;
 import org.apache.asterix.metadata.entities.CompactionPolicy;
 import org.apache.asterix.metadata.entities.Dataset;
 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.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
@@ -231,17 +230,26 @@
     public void addFeed(Feed feed) {
         droppedCache.dropFeed(feed);
         logAndApply(new MetadataLogicalOperation(feed, true));
-
     }
 
-    public void dropFeed(String dataverseName, String feedName, IFeed.FeedType feedType) {
+    public void dropFeed(String dataverseName, String feedName) {
         Feed feed = null;
-        feed = new Feed(dataverseName, feedName, null, feedType, (feedType == FeedType.PRIMARY) ? feedName : null,
-                null, null);
+        feed = new Feed(dataverseName, feedName, null, null);
         droppedCache.addFeedIfNotExists(feed);
         logAndApply(new MetadataLogicalOperation(feed, false));
     }
 
+    public void addFeedConnection(FeedConnection feedConnection) {
+        droppedCache.dropFeedConnection(feedConnection);
+        logAndApply(new MetadataLogicalOperation(feedConnection, true));
+    }
+
+    public void dropFeedConnection(String dataverseName, String feedName, String datasetName) {
+        FeedConnection feedConnection = new FeedConnection(dataverseName, feedName, datasetName, null, null, null);
+        droppedCache.addFeedConnectionIfNotExists(feedConnection);
+        logAndApply(new MetadataLogicalOperation(feedConnection, false));
+    }
+
     @Override
     public void clear() {
         super.clear();
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 feb4db0..bd1c7d1 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
@@ -34,6 +34,7 @@
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
@@ -431,7 +432,6 @@
     void dropAdapter(MetadataTransactionContext ctx, String dataverseName, String name) throws MetadataException;
 
     /**
-     *
      * @param ctx
      *            MetadataTransactionContext of an active metadata transaction.
      * @param dataverseName
@@ -691,4 +691,19 @@
      * rebind it
      */
     void rebindMetadataNode();
+
+    /**
+     * Feed Connection Related Metadata operations
+     */
+    void addFeedConnection(MetadataTransactionContext ctx, FeedConnection feedConnection)
+            throws MetadataException;
+
+    void dropFeedConnection(MetadataTransactionContext ctx, String dataverseName, String feedName,
+            String datasetName) throws MetadataException;
+
+    FeedConnection getFeedConnection(MetadataTransactionContext ctx, String dataverseName, String feedName,
+            String datasetName) throws MetadataException;
+
+    List<FeedConnection> getFeedConections(MetadataTransactionContext ctx, String dataverseName, String feedName)
+            throws MetadataException;
 }
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 41d0b6a..21b170d 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
@@ -35,6 +35,7 @@
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
@@ -764,4 +765,14 @@
     <T extends IExtensionMetadataEntity> List<T> getEntities(JobId jobId, IExtensionMetadataSearchKey searchKey)
             throws MetadataException, RemoteException;
 
+    void addFeedConnection(JobId jobId, FeedConnection feedConnection) throws MetadataException, RemoteException;
+
+    FeedConnection getFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
+            throws MetadataException, RemoteException;
+
+    void dropFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
+            throws MetadataException, RemoteException;
+
+    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/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 6cd1f8b..02a092d 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
@@ -123,7 +123,7 @@
                     MetadataPrimaryIndexes.FUNCTION_DATASET, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET,
                     MetadataPrimaryIndexes.FEED_DATASET, MetadataPrimaryIndexes.FEED_POLICY_DATASET,
                     MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET,
-                    MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
+                    MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET };
 
     private MetadataBootstrap() {
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index 833f3e5..d2a4b1c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -53,8 +53,8 @@
             new MetadataIndexImmutableProperties("Library", 9, 9);
     public static final MetadataIndexImmutableProperties PROPERTIES_FEED =
             new MetadataIndexImmutableProperties("Feed", 10, 10);
-    public static final MetadataIndexImmutableProperties PROPERTIES_FEED_ACTIVITY_DATASET_ID =
-            new MetadataIndexImmutableProperties("FeedActivity", 11, 11);
+    public static final MetadataIndexImmutableProperties PROPERTIES_FEED_CONNECTION =
+            new MetadataIndexImmutableProperties("FeedConnection", 11, 11);
     public static final MetadataIndexImmutableProperties PROPERTIES_FEED_POLICY =
             new MetadataIndexImmutableProperties("FeedPolicy", 12, 12);
     public static final MetadataIndexImmutableProperties PROPERTIES_COMPACTION_POLICY =
@@ -129,6 +129,13 @@
                     Arrays.asList(MetadataRecordTypes.FIELD_NAME_FILE_NUMBER)),
             0, MetadataRecordTypes.EXTERNAL_FILE_RECORDTYPE, true, new int[] { 0, 1, 2 });
 
+    public static final IMetadataIndex FEED_CONNECTION_DATASET = new MetadataIndex(PROPERTIES_FEED_CONNECTION, 4,
+            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
+            Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+                    Arrays.asList(MetadataRecordTypes.FIELD_NAME_FEED_NAME),
+                    Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATASET_NAME)),
+            0, MetadataRecordTypes.FEED_CONNECTION_RECORDTYPE, true, new int[] { 0, 1, 2 });
+
     private MetadataPrimaryIndexes() {
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index a783c63..2a04b58 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -61,7 +61,6 @@
     public static final String FIELD_NAME_FILE_NUMBER = "FileNumber";
     public static final String FIELD_NAME_FILE_SIZE = "FileSize";
     public static final String FIELD_NAME_FILE_STRUCTURE = "FileStructure";
-    public static final String FIELD_NAME_FUNCTION = "Function";
     public static final String FIELD_NAME_GROUP_NAME = "GroupName";
     public static final String FIELD_NAME_HINTS = "Hints";
     public static final String FIELD_NAME_INDEX_NAME = "IndexName";
@@ -87,13 +86,10 @@
     public static final String FIELD_NAME_PENDING_OP = "PendingOp";
     public static final String FIELD_NAME_POLICY_NAME = "PolicyName";
     public static final String FIELD_NAME_PRIMARY_KEY = "PrimaryKey";
-    public static final String FIELD_NAME_PRIMARY_TYPE_DETAILS = "PrimaryTypeDetails";
     public static final String FIELD_NAME_PROPERTIES = "Properties";
     public static final String FIELD_NAME_RECORD = "Record";
     public static final String FIELD_NAME_RETURN_TYPE = "ReturnType";
     public static final String FIELD_NAME_SEARCH_KEY = "SearchKey";
-    public static final String FIELD_NAME_SECONDARY_TYPE_DETAILS = "SecondaryTypeDetails";
-    public static final String FIELD_NAME_SOURCE_FEED_NAME = "SourceFeedName";
     public static final String FIELD_NAME_STATUS = "Status";
     public static final String FIELD_NAME_TAG = "Tag";
     public static final String FIELD_NAME_TIMESTAMP = "Timestamp";
@@ -102,6 +98,7 @@
     public static final String FIELD_NAME_UNORDERED_LIST = "UnorderedList";
     public static final String FIELD_NAME_VALUE = "Value";
     public static final String FIELD_NAME_WORKING_MEMORY_SIZE = "WorkingMemorySize";
+    public static final String FIELD_NAME_APPLIED_FUNCTIONS = "AppliedFunctions";
 
     //---------------------------------- Record Types Creation ----------------------------------//
     //--------------------------------------- Properties ----------------------------------------//
@@ -148,35 +145,6 @@
                     BuiltinType.ADATETIME, BuiltinType.AINT32 },
             //IsOpen?
             true);
-    //-------------------------------------- Feed Details ---------------------------------------//
-    public static final int FEED_DETAILS_ARECORD_FILESTRUCTURE_FIELD_INDEX = 0;
-    public static final int FEED_DETAILS_ARECORD_PARTITIONSTRATEGY_FIELD_INDEX = 1;
-    public static final int FEED_DETAILS_ARECORD_PARTITIONKEY_FIELD_INDEX = 2;
-    public static final int FEED_DETAILS_ARECORD_PRIMARYKEY_FIELD_INDEX = 3;
-    public static final int FEED_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX = 4;
-    public static final int FEED_DETAILS_ARECORD_DATASOURCE_ADAPTER_FIELD_INDEX = 5;
-    public static final int FEED_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX = 6;
-    public static final int FEED_DETAILS_ARECORD_FUNCTION_FIELD_INDEX = 7;
-    public static final int FEED_DETAILS_ARECORD_STATE_FIELD_INDEX = 8;
-    public static final int FEED_DETAILS_ARECORD_COMPACTION_POLICY_FIELD_INDEX = 9;
-    public static final int FEED_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX = 10;
-    public static final ARecordType FEED_DETAILS_RECORDTYPE = createRecordType(
-            // RecordTypeName
-            null,
-            // FieldNames
-            new String[] { FIELD_NAME_FILE_STRUCTURE, FIELD_NAME_PARTITIONING_STRATEGY, FIELD_NAME_PARTITIONING_KEY,
-                    FIELD_NAME_PRIMARY_KEY, FIELD_NAME_GROUP_NAME, FIELD_NAME_DATASOURCE_ADAPTER, FIELD_NAME_PROPERTIES,
-                    FIELD_NAME_FUNCTION, FIELD_NAME_STATUS, FIELD_NAME_COMPACTION_POLICY,
-                    FIELD_NAME_COMPACTION_POLICY_PROPERTIES },
-            // FieldTypes
-            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, new AOrderedListType(BuiltinType.ASTRING, null),
-                    new AOrderedListType(BuiltinType.ASTRING, null), BuiltinType.ASTRING, BuiltinType.ASTRING,
-                    new AOrderedListType(DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE, null),
-                    AUnionType.createUnknownableType(BuiltinType.ASTRING), BuiltinType.ASTRING, BuiltinType.ASTRING,
-                    new AOrderedListType(COMPACTION_POLICY_PROPERTIES_RECORDTYPE, null) },
-            //IsOpen?
-            true);
-
     //---------------------------------------- Dataset ------------------------------------------//
     public static final String RECORD_NAME_DATASET = "DatasetRecordType";
     public static final int DATASET_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
@@ -383,60 +351,45 @@
                     BuiltinType.ASTRING },
             //IsOpen?
             true);
-    //---------------------------------- Primary Feed Details -----------------------------------//
-    public static final int FEED_TYPE_PRIMARY_ARECORD_ADAPTER_NAME_FIELD_INDEX = 0;
-    public static final int FEED_TYPE_PRIMARY_ARECORD_ADAPTER_CONFIGURATION_FIELD_INDEX = 1;
-    public static final ARecordType PRIMARY_FEED_DETAILS_RECORDTYPE = createRecordType(
-            // RecordTypeName
-            null,
-            // FieldNames
-            new String[] { FIELD_NAME_ADAPTER_NAME, FIELD_NAME_ADAPTER_CONFIGURATION },
-            // FieldTypes
-            new IAType[] { BuiltinType.ASTRING,
-                    new AUnorderedListType(DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE, null) },
-            //IsOpen?
-            true);
-    //--------------------------------- Secondary Feed Details ----------------------------------//
-    public static final int FEED_TYPE_SECONDARY_ARECORD_SOURCE_FEED_NAME_FIELD_INDEX = 0;
-    public static final ARecordType SECONDARY_FEED_DETAILS_RECORDTYPE = createRecordType(
-            // RecordTypeName
-            null,
-            // FieldNames
-            new String[] { FIELD_NAME_SOURCE_FEED_NAME },
-            // FieldTypes
-            new IAType[] { BuiltinType.ASTRING },
-            //IsOpen?
-            true);
-    //---------------------------------------- Feed Activity ------------------------------------//
+
+    //---------------------------------------- Feed Details ------------------------------------//
     public static final String RECORD_NAME_FEED = "FeedRecordType";
-    public static final int FEED_ACTIVITY_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
-    public static final int FEED_ACTIVITY_ARECORD_FEED_NAME_FIELD_INDEX = 1;
-    public static final int FEED_ACTIVITY_ARECORD_DATASET_NAME_FIELD_INDEX = 2;
-    public static final int FEED_ACTIVITY_ARECORD_ACTIVITY_ID_FIELD_INDEX = 3;
-    public static final int FEED_ACTIVITY_ARECORD_ACTIVITY_TYPE_FIELD_INDEX = 4;
-    public static final int FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX = 5;
-    public static final int FEED_ACTIVITY_ARECORD_LAST_UPDATE_TIMESTAMP_FIELD_INDEX = 6;
     public static final int FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
     public static final int FEED_ARECORD_FEED_NAME_FIELD_INDEX = 1;
-    public static final int FEED_ARECORD_FUNCTION_FIELD_INDEX = 2;
-    public static final int FEED_ARECORD_FEED_TYPE_FIELD_INDEX = 3;
-    public static final int FEED_ARECORD_PRIMARY_TYPE_DETAILS_FIELD_INDEX = 4;
-    public static final int FEED_ARECORD_SECONDARY_TYPE_DETAILS_FIELD_INDEX = 5;
-    public static final int FEED_ARECORD_TIMESTAMP_FIELD_INDEX = 6;
-    public static final int FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_NAME_FIELD_INDEX = 0;
-    public static final int FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_CONFIGURATION_FIELD_INDEX = 1;
-    public static final int FEED_ARECORD_SECONDARY_FIELD_DETAILS_SOURCE_FEED_NAME_FIELD_INDEX = 0;
+    public static final int FEED_ARECORD_ADAPTOR_NAME_INDEX = 2;
+    public static final int FEED_ARECORD_ADAPTOR_CONFIG_INDEX = 3;
+    public static final int FEED_ARECORD_TIMESTAMP_FIELD_INDEX = 4;
     public static final ARecordType FEED_RECORDTYPE = createRecordType(
             // RecordTypeName
             RECORD_NAME_FEED,
             // FieldNames
-            new String[] { FIELD_NAME_DATAVERSE_NAME, FIELD_NAME_FEED_NAME, FIELD_NAME_FUNCTION, FIELD_NAME_FEED_TYPE,
-                    FIELD_NAME_PRIMARY_TYPE_DETAILS, FIELD_NAME_SECONDARY_TYPE_DETAILS, FIELD_NAME_TIMESTAMP },
+            new String[] { FIELD_NAME_DATAVERSE_NAME, FIELD_NAME_FEED_NAME, FIELD_NAME_ADAPTER_NAME,
+                    FIELD_NAME_ADAPTER_CONFIGURATION, FIELD_NAME_TIMESTAMP },
             // FieldTypes
-            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING,
-                    AUnionType.createUnknownableType(BuiltinType.ASTRING), BuiltinType.ASTRING,
-                    AUnionType.createUnknownableType(PRIMARY_FEED_DETAILS_RECORDTYPE),
-                    AUnionType.createUnknownableType(SECONDARY_FEED_DETAILS_RECORDTYPE), BuiltinType.ASTRING },
+            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+                    new AUnorderedListType(FEED_ADAPTER_CONFIGURATION_RECORDTYPE, null), BuiltinType.ASTRING },
+            //IsOpen?
+            true);
+
+    //------------------------------------- Feed Connection ---------------------------------------//
+    public static final String RECORD_NAME_FEED_CONNECTION = "FeedConnectionRecordType";
+    public static final int FEED_CONN_DATAVERSE_NAME_FIELD_INDEX = 0;
+    public static final int FEED_CONN_FEED_NAME_FIELD_INDEX = 1;
+    public static final int FEED_CONN_DATASET_NAME_FIELD_INDEX = 2;
+    public static final int FEED_CONN_OUTPUT_TYPE_INDEX = 3;
+    public static final int FEED_CONN_APPLIED_FUNCTIONS_FIELD_INDEX = 4;
+    public static final int FEED_CONN_POLICY_FIELD_INDEX = 5;
+
+
+    public static final ARecordType FEED_CONNECTION_RECORDTYPE = createRecordType(
+            // RecordTypeName
+            RECORD_NAME_FEED_CONNECTION,
+            // FieldNames
+            new String[] { FIELD_NAME_DATAVERSE_NAME, FIELD_NAME_FEED_NAME, FIELD_NAME_DATASET_NAME,
+                    FIELD_NAME_RETURN_TYPE, FIELD_NAME_APPLIED_FUNCTIONS, FIELD_NAME_POLICY_NAME },
+            // FieldTypes
+            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+                    new AUnorderedListType(BuiltinType.ASTRING, null), BuiltinType.ASTRING},
             //IsOpen?
             true);
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index 0d3d06d..26cec1e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
 import org.apache.asterix.om.types.ARecordType;
@@ -54,29 +55,29 @@
 
     private final Feed feed;
     private final EntityId sourceFeedId;
-    private final IFeed.FeedType sourceFeedType;
     private final FeedRuntimeType location;
     private final String targetDataset;
     private final String[] locations;
     private final int computeCardinality;
     private final List<IAType> pkTypes;
     private final List<ScalarFunctionCallExpression> keyAccessExpression;
+    private final FeedConnection feedConnection;
 
     public FeedDataSource(Feed feed, DataSourceId id, String targetDataset, IAType itemType, IAType metaType,
             List<IAType> pkTypes, List<List<String>> partitioningKeys,
             List<ScalarFunctionCallExpression> keyAccessExpression, EntityId sourceFeedId,
-            IFeed.FeedType sourceFeedType, FeedRuntimeType location, String[] locations, INodeDomain domain)
+            FeedRuntimeType location, String[] locations, INodeDomain domain, FeedConnection feedConnection)
             throws AlgebricksException {
         super(id, itemType, metaType, Type.FEED, domain);
         this.feed = feed;
         this.targetDataset = targetDataset;
         this.sourceFeedId = sourceFeedId;
-        this.sourceFeedType = sourceFeedType;
         this.location = location;
         this.locations = locations;
         this.pkTypes = pkTypes;
         this.keyAccessExpression = keyAccessExpression;
         this.computeCardinality = ClusterStateManager.INSTANCE.getParticipantNodes().size();
+        this.feedConnection = feedConnection;
         initFeedDataSource();
     }
 
@@ -120,10 +121,6 @@
         }
     }
 
-    public IFeed.FeedType getSourceFeedType() {
-        return sourceFeedType;
-    }
-
     public int getComputeCardinality() {
         return computeCardinality;
     }
@@ -196,7 +193,7 @@
             FeedConnectionId feedConnectionId = new FeedConnectionId(getId().getDataverseName(),
                     getId().getDatasourceName(), getTargetDataset());
             FeedCollectOperatorDescriptor feedCollector = new FeedCollectOperatorDescriptor(jobSpec, feedConnectionId,
-                    getSourceFeedId(), feedOutputType, feedDesc, feedPolicy.getProperties(), getLocation());
+                    feedOutputType, feedDesc, feedPolicy.getProperties(), getLocation());
 
             return new Pair<>(feedCollector, new AlgebricksAbsolutePartitionConstraint(getLocations()));
 
@@ -209,4 +206,8 @@
     public boolean isScanAccessPathALeaf() {
         return true;
     }
+
+    public FeedConnection getFeedConnection() {
+        return feedConnection;
+    }
 }
\ No newline at end of file
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 f1a90c7..b647bb7 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
@@ -29,6 +29,7 @@
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.NodeGroup;
@@ -132,6 +133,15 @@
         }
     }
 
+    public static FeedConnection findFeedConnection(MetadataTransactionContext mdTxnCtx, String dataverse,
+            String feedName, String datasetName) throws AlgebricksException {
+        try {
+            return MetadataManager.INSTANCE.getFeedConnection(mdTxnCtx, dataverse, feedName, datasetName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
     public static FeedPolicyEntity findFeedPolicy(MetadataTransactionContext mdTxnCtx, String dataverse,
             String policyName) throws AlgebricksException {
         try {
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 1beaed0..f5c6d9a 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
@@ -68,6 +68,7 @@
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
@@ -329,6 +330,11 @@
         return MetadataManagerUtil.findFeed(mdTxnCtx, dataverse, feedName);
     }
 
+    public FeedConnection findFeedConnection(String dataverseName, String feedName, String datasetName)
+            throws AlgebricksException {
+        return MetadataManagerUtil.findFeedConnection(mdTxnCtx, dataverseName, feedName, datasetName);
+    }
+
     public FeedPolicyEntity findFeedPolicy(String dataverse, String policyName) throws AlgebricksException {
         return MetadataManagerUtil.findFeedPolicy(mdTxnCtx, dataverse, policyName);
     }
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 d55fde5..2e328f9 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
@@ -29,6 +29,7 @@
 import org.apache.asterix.active.ActiveJobNotificationHandler;
 import org.apache.asterix.active.IActiveEntityEventsListener;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.metadata.IDataset;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
@@ -93,7 +94,7 @@
 /**
  * Metadata describing a dataset.
  */
-public class Dataset implements IMetadataEntity<Dataset> {
+public class Dataset implements IMetadataEntity<Dataset>, IDataset {
 
     /*
      * Constants
@@ -278,7 +279,9 @@
             // prepare job spec(s) that would disconnect any active feeds involving the dataset.
             IActiveEntityEventsListener[] activeListeners = ActiveJobNotificationHandler.INSTANCE.getEventListeners();
             for (IActiveEntityEventsListener listener : activeListeners) {
-                if (listener.isEntityActive() && listener.isEntityUsingDataset(dataverseName, datasetName)) {
+                IDataset ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(),
+                        dataverseName, datasetName);
+                if (listener.isEntityUsingDataset(ds)) {
                     throw new CompilationException(ErrorCode.COMPILATION_CANT_DROP_ACTIVE_DATASET,
                             RecordUtil.toFullyQualifiedName(dataverseName, datasetName),
                             listener.getEntityId().toString());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Feed.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Feed.java
index 1343e53..ea0e4eb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Feed.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Feed.java
@@ -22,7 +22,6 @@
 import java.util.Map;
 
 import org.apache.asterix.active.EntityId;
-import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.external.feed.api.IFeed;
 import org.apache.asterix.metadata.MetadataCache;
 import org.apache.asterix.metadata.api.IMetadataEntity;
@@ -36,28 +35,18 @@
 
     /** A unique identifier for the feed */
     private EntityId feedId;
-    /** The function that is to be applied on each incoming feed tuple **/
-    private FunctionSignature appliedFunction;
-    /** The type {@code FeedType} associated with the feed. **/
-    private IFeed.FeedType feedType;
     /** A string representation of the instance **/
     private String displayName;
     /** A string representation of the adapter name **/
     private String adapterName;
     /** Adapter configuration */
     private Map<String, String> adapterConfiguration;
-    /** Source primary feed */
-    private String sourceFeedName;
 
-    public Feed(String dataverseName, String feedName, FunctionSignature appliedFunction, IFeed.FeedType feedType,
-            String sourceFeedName, String adapterName, Map<String, String> configuration) {
+    public Feed(String dataverseName, String feedName,String adapterName, Map<String, String> configuration) {
         this.feedId = new EntityId(EXTENSION_NAME, dataverseName, feedName);
-        this.appliedFunction = appliedFunction;
-        this.feedType = feedType;
-        this.displayName = feedType + "(" + feedId + ")";
+        this.displayName = "(" + feedId + ")";
         this.adapterName = adapterName;
         this.adapterConfiguration = configuration;
-        this.sourceFeedName = sourceFeedName;
     }
 
     @Override
@@ -76,16 +65,6 @@
     }
 
     @Override
-    public FunctionSignature getAppliedFunction() {
-        return appliedFunction;
-    }
-
-    @Override
-    public IFeed.FeedType getFeedType() {
-        return feedType;
-    }
-
-    @Override
     public boolean equals(Object other) {
         if (this == other) {
             return true;
@@ -104,7 +83,7 @@
 
     @Override
     public String toString() {
-        return feedType + "(" + feedId + ")";
+        return feedId.toString();
     }
 
     @Override
@@ -126,8 +105,4 @@
     public Map<String, String> getAdapterConfiguration() {
         return adapterConfiguration;
     }
-
-    public String getSourceFeedName() {
-        return sourceFeedName;
-    }
 }
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
new file mode 100644
index 0000000..b05e61e
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FeedConnection.java
@@ -0,0 +1,117 @@
+/*
+ * 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.entities;
+
+import org.apache.asterix.active.EntityId;
+import org.apache.asterix.common.functions.FunctionSignature;
+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;
+
+/**
+ * Feed connection records the feed --> dataset mapping.
+ */
+public class FeedConnection implements IMetadataEntity<FeedConnection> {
+
+    private static final long serialVersionUID = 1L;
+
+    private EntityId feedId;
+    private String connectionId;
+    private String dataverseName;
+    private String feedName;
+    private String datasetName;
+    private String policyName;
+    private String outputType;
+    private List<FunctionSignature> appliedFunctions;
+
+    public FeedConnection(String dataverseName, String feedName, String datasetName,
+            List<FunctionSignature> appliedFunctions, String policyName, String outputType) {
+        this.dataverseName = dataverseName;
+        this.feedName = feedName;
+        this.datasetName = datasetName;
+        this.appliedFunctions = appliedFunctions;
+        this.connectionId = feedName + ":" + datasetName;
+        this.policyName = policyName;
+        this.outputType = outputType;
+        this.feedId = new EntityId(FeedUtils.FEED_EXTENSION_NAME, dataverseName, feedName);
+    }
+
+    public List<FunctionSignature> getAppliedFunctions() {
+        return appliedFunctions;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof FeedConnection)) {
+            return false;
+        }
+        return ((FeedConnection) other).getConnectionId().equals(connectionId);
+    }
+
+    @Override
+    public int hashCode() {
+        return connectionId.hashCode();
+    }
+
+    @Override
+    public FeedConnection addToCache(MetadataCache cache) {
+        return null;
+    }
+
+    @Override
+    public FeedConnection dropFromCache(MetadataCache cache) {
+        return null;
+    }
+
+    public String getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getDatasetName() {
+        return datasetName;
+    }
+
+    public String getConnectionId() {
+        return connectionId;
+    }
+
+    public String getFeedName() {
+        return feedName;
+    }
+
+    public String getPolicyName() {
+        return policyName;
+    }
+
+    public String getOutputType() {
+        return outputType;
+    }
+
+    public EntityId getFeedId() {
+        return feedId;
+    }
+}
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
new file mode 100644
index 0000000..e7fe5b4
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java
@@ -0,0 +1,179 @@
+/*
+ * 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.entitytupletranslators;
+
+import org.apache.asterix.builders.IARecordBuilder;
+import org.apache.asterix.builders.UnorderedListBuilder;
+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;
+import org.apache.asterix.om.base.*;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class FeedConnectionTupleTranslator extends AbstractTupleTranslator<FeedConnection> {
+
+    public static final int FEED_CONN_DATAVERSE_NAME_FIELD_INDEX = 0;
+    public static final int FEED_CONN_FEED_NAME_FIELD_INDEX = 1;
+    public static final int FEED_CONN_DATASET_NAME_FIELD_INDEX = 2;
+
+    public static final int FEED_CONN_PAYLOAD_TUPLE_FIELD_INDEX = 3;
+
+    private ISerializerDeserializer<ARecord> recordSerDes = SerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(MetadataRecordTypes.FEED_CONNECTION_RECORDTYPE);
+
+    public FeedConnectionTupleTranslator(boolean getTuple) {
+        super(getTuple, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET.getFieldCount());
+    }
+
+    @Override
+    public FeedConnection getMetadataEntityFromTuple(ITupleReference frameTuple) throws MetadataException, IOException {
+        byte[] serRecord = frameTuple.getFieldData(FEED_CONN_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordStartOffset = frameTuple.getFieldStart(FEED_CONN_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordLength = frameTuple.getFieldLength(FEED_CONN_PAYLOAD_TUPLE_FIELD_INDEX);
+        ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+        DataInput in = new DataInputStream(stream);
+        ARecord feedConnRecord = recordSerDes.deserialize(in);
+        return createFeedConnFromRecord(feedConnRecord);
+    }
+
+    private FeedConnection createFeedConnFromRecord(ARecord feedConnRecord) {
+        String dataverseName = ((AString) feedConnRecord
+                .getValueByPos(MetadataRecordTypes.FEED_CONN_DATAVERSE_NAME_FIELD_INDEX)).getStringValue();
+        String feedName = ((AString) feedConnRecord.getValueByPos(MetadataRecordTypes.FEED_CONN_FEED_NAME_FIELD_INDEX))
+                .getStringValue();
+        String datasetName = ((AString) feedConnRecord
+                .getValueByPos(MetadataRecordTypes.FEED_CONN_DATASET_NAME_FIELD_INDEX)).getStringValue();
+        String outputType = ((AString) feedConnRecord.getValueByPos(MetadataRecordTypes.FEED_CONN_OUTPUT_TYPE_INDEX))
+                .getStringValue();
+        String policyName = ((AString) feedConnRecord.getValueByPos(MetadataRecordTypes.FEED_CONN_POLICY_FIELD_INDEX))
+                .getStringValue();
+        ArrayList<FunctionSignature> appliedFunctions = null;
+        Object o = feedConnRecord.getValueByPos(MetadataRecordTypes.FEED_CONN_APPLIED_FUNCTIONS_FIELD_INDEX);
+        IACursor cursor;
+
+        if (!(o instanceof ANull) && !(o instanceof AMissing)) {
+            appliedFunctions = new ArrayList<>();
+            FunctionSignature functionSignature;
+            cursor = ((AUnorderedList) feedConnRecord
+                    .getValueByPos(MetadataRecordTypes.FEED_CONN_APPLIED_FUNCTIONS_FIELD_INDEX)).getCursor();
+            while (cursor.next()) {
+                //TODO: allow different arity
+                functionSignature = new FunctionSignature(dataverseName, ((AString) cursor.get()).getStringValue(), 1);
+                appliedFunctions.add(functionSignature);
+            }
+        }
+
+        return new FeedConnection(dataverseName, feedName, datasetName, appliedFunctions, policyName, outputType);
+    }
+
+    @Override
+    public ITupleReference getTupleFromMetadataEntity(FeedConnection me) throws MetadataException, IOException {
+        tupleBuilder.reset();
+
+        // key: dataverse
+        aString.setValue(me.getDataverseName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        // key: feedName
+        aString.setValue(me.getFeedName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        // key: dataset
+        aString.setValue(me.getDatasetName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        recordBuilder.reset(MetadataRecordTypes.FEED_CONNECTION_RECORDTYPE);
+        // field dataverse
+        fieldValue.reset();
+        aString.setValue(me.getDataverseName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_CONN_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+        // field: feedId
+        fieldValue.reset();
+        aString.setValue(me.getFeedName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_CONN_FEED_NAME_FIELD_INDEX, fieldValue);
+
+        // field: dataset
+        fieldValue.reset();
+        aString.setValue(me.getDatasetName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_CONN_DATASET_NAME_FIELD_INDEX, fieldValue);
+
+        // field: outputType
+        fieldValue.reset();
+        aString.setValue(me.getOutputType());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_CONN_OUTPUT_TYPE_INDEX, fieldValue);
+
+        // field: appliedFunctions
+        fieldValue.reset();
+        writeAppliedFunctionsField(recordBuilder, me, fieldValue);
+
+        // field: policyName
+        fieldValue.reset();
+        aString.setValue(me.getPolicyName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.FEED_CONN_POLICY_FIELD_INDEX, fieldValue);
+
+        recordBuilder.write(tupleBuilder.getDataOutput(), true);
+        tupleBuilder.addFieldEndOffset();
+
+        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+        return tuple;
+    }
+
+    private void writeAppliedFunctionsField(IARecordBuilder rb, FeedConnection fc, ArrayBackedValueStorage buffer)
+            throws HyracksDataException {
+        UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+        ArrayBackedValueStorage listEleBuffer = new ArrayBackedValueStorage();
+
+        listBuilder.reset((AUnorderedListType) MetadataRecordTypes.FEED_CONNECTION_RECORDTYPE
+                .getFieldTypes()[MetadataRecordTypes.FEED_CONN_APPLIED_FUNCTIONS_FIELD_INDEX]);
+        if (fc.getAppliedFunctions() != null) {
+            List<FunctionSignature> appliedFunctions = fc.getAppliedFunctions();
+            for (FunctionSignature af : appliedFunctions) {
+                aString.setValue(af.getName());
+                stringSerde.serialize(aString, listEleBuffer.getDataOutput());
+                listBuilder.addItem(listEleBuffer);
+            }
+        }
+        listBuilder.write(buffer.getDataOutput(), true);
+        rb.addField(MetadataRecordTypes.FEED_CONN_APPLIED_FUNCTIONS_FIELD_INDEX, buffer);
+    }
+}
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 dc0b9c9..4503e09 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
@@ -29,19 +29,14 @@
 import java.util.Map;
 
 import org.apache.asterix.builders.IARecordBuilder;
-import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.api.IFeed.FeedType;
+import org.apache.asterix.builders.UnorderedListBuilder;
 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;
-import org.apache.asterix.om.base.AMissing;
 import org.apache.asterix.om.base.AMutableString;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.ARecord;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.AUnorderedList;
@@ -67,8 +62,8 @@
     public static final int FEED_PAYLOAD_TUPLE_FIELD_INDEX = 2;
 
     @SuppressWarnings("unchecked")
-    private ISerializerDeserializer<ARecord> recordSerDes =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(MetadataRecordTypes.FEED_RECORDTYPE);
+    private ISerializerDeserializer<ARecord> recordSerDes = SerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(MetadataRecordTypes.FEED_RECORDTYPE);
 
     protected FeedTupleTranslator(boolean getTuple) {
         super(getTuple, MetadataPrimaryIndexes.FEED_DATASET.getFieldCount());
@@ -86,65 +81,30 @@
     }
 
     private Feed createFeedFromARecord(ARecord feedRecord) {
-        Feed feed = null;
-        String dataverseName =
-                ((AString) feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX))
-                        .getStringValue();
+        Feed feed;
+        String dataverseName = ((AString) feedRecord
+                .getValueByPos(MetadataRecordTypes.FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX)).getStringValue();
         String feedName = ((AString) feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_FEED_NAME_FIELD_INDEX))
                 .getStringValue();
 
-        Object o = feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_FUNCTION_FIELD_INDEX);
-        FunctionSignature signature = null;
-        if (!(o instanceof ANull) && !(o instanceof AMissing)) {
-            String functionName = ((AString) o).getStringValue();
-            signature = new FunctionSignature(dataverseName, functionName, 1);
+        AUnorderedList feedConfig = (AUnorderedList) feedRecord
+                .getValueByPos(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_CONFIG_INDEX);
+        String adapterName = ((AString) feedRecord
+                .getValueByPos(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_NAME_INDEX)).getStringValue();
+
+        IACursor cursor = feedConfig.getCursor();
+
+        // restore configurations
+        String key;
+        String value;
+        Map<String, String> adaptorConfiguration = new HashMap<>();
+        while (cursor.next()) {
+            ARecord field = (ARecord) cursor.get();
+            key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX)).getStringValue();
+            value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX)).getStringValue();
+            adaptorConfiguration.put(key, value);
         }
-
-        String feedType = ((AString) feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_FEED_TYPE_FIELD_INDEX))
-                .getStringValue();
-
-        IFeed.FeedType feedTypeEnum = IFeed.FeedType.valueOf(feedType.toUpperCase());
-        switch (feedTypeEnum) {
-            case PRIMARY: {
-                ARecord feedTypeDetailsRecord = (ARecord) feedRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_ARECORD_PRIMARY_TYPE_DETAILS_FIELD_INDEX);
-                String adapterName = ((AString) feedTypeDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_NAME_FIELD_INDEX))
-                                .getStringValue();
-
-                IACursor cursor = ((AUnorderedList) feedTypeDetailsRecord.getValueByPos(
-                        MetadataRecordTypes.FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_CONFIGURATION_FIELD_INDEX))
-                                .getCursor();
-                String key;
-                String value;
-                Map<String, String> adaptorConfiguration = new HashMap<String, String>();
-                while (cursor.next()) {
-                    ARecord field = (ARecord) cursor.get();
-                    key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX))
-                            .getStringValue();
-                    value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX))
-                            .getStringValue();
-                    adaptorConfiguration.put(key, value);
-                }
-                feed = new Feed(dataverseName, feedName, signature, FeedType.PRIMARY, feedName, adapterName,
-                        adaptorConfiguration);
-
-            }
-                break;
-            case SECONDARY: {
-                ARecord feedTypeDetailsRecord = (ARecord) feedRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_ARECORD_SECONDARY_TYPE_DETAILS_FIELD_INDEX);
-
-                String sourceFeedName = ((AString) feedTypeDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.FEED_TYPE_SECONDARY_ARECORD_SOURCE_FEED_NAME_FIELD_INDEX))
-                                .getStringValue();
-
-                feed = new Feed(dataverseName, feedName, signature, FeedType.SECONDARY, sourceFeedName, null, null);
-
-            }
-                break;
-        }
-
+        feed = new Feed(dataverseName, feedName, adapterName, adaptorConfiguration);
         return feed;
     }
 
@@ -162,37 +122,29 @@
 
         recordBuilder.reset(MetadataRecordTypes.FEED_RECORDTYPE);
 
-        // write field 0
+        // write dataverse name
         fieldValue.reset();
         aString.setValue(feed.getDataverseName());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
 
-        // write field 1
+        // write feed name
         fieldValue.reset();
         aString.setValue(feed.getFeedName());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_FEED_NAME_FIELD_INDEX, fieldValue);
 
-        // write field 2
+        // adaptor name
         fieldValue.reset();
-        if (feed.getAppliedFunction() != null) {
-            aString.setValue(feed.getAppliedFunction().getName());
-            stringSerde.serialize(aString, fieldValue.getDataOutput());
-            recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_FUNCTION_FIELD_INDEX, fieldValue);
-        }
-
-        // write field 3
-        fieldValue.reset();
-        aString.setValue(feed.getFeedType().name().toUpperCase());
+        aString.setValue(feed.getAdapterName());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
-        recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_FEED_TYPE_FIELD_INDEX, fieldValue);
+        recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_NAME_INDEX, fieldValue);
 
-        // write field 4/5
+        // write adaptor configuration
         fieldValue.reset();
-        writeFeedTypeDetailsRecordType(recordBuilder, feed, fieldValue);
+        writeFeedAdaptorField(recordBuilder, feed, fieldValue);
 
-        // write field 6
+        // write timestamp
         fieldValue.reset();
         aString.setValue(Calendar.getInstance().getTime().toString());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
@@ -206,81 +158,32 @@
         return tuple;
     }
 
-    @SuppressWarnings("unchecked")
-    private void writeFeedTypeDetailsRecordType(IARecordBuilder recordBuilder, Feed feed,
-            ArrayBackedValueStorage fieldValue) throws HyracksDataException {
+    private void writeFeedAdaptorField(IARecordBuilder recordBuilder, Feed feed,
+            ArrayBackedValueStorage fieldValueBuffer) throws HyracksDataException {
+        UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+        ArrayBackedValueStorage listEleBuffer = new ArrayBackedValueStorage();
 
-        switch (feed.getFeedType()) {
-            case PRIMARY: {
-
-                IARecordBuilder primaryDetailsRecordBuilder = new RecordBuilder();
-                OrderedListBuilder listBuilder = new OrderedListBuilder();
-                ArrayBackedValueStorage primaryRecordfieldValue = new ArrayBackedValueStorage();
-                ArrayBackedValueStorage primaryRecordItemValue = new ArrayBackedValueStorage();
-                primaryDetailsRecordBuilder.reset(MetadataRecordTypes.PRIMARY_FEED_DETAILS_RECORDTYPE);
-
-                AMutableString aString = new AMutableString("");
-                ISerializerDeserializer<AString> stringSerde =
-                        SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
-
-                // write field 0
-                fieldValue.reset();
-                aString.setValue(feed.getAdapterName());
-                stringSerde.serialize(aString, primaryRecordfieldValue.getDataOutput());
-                primaryDetailsRecordBuilder.addField(
-                        MetadataRecordTypes.FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_NAME_FIELD_INDEX,
-                        primaryRecordfieldValue);
-
-                // write field 1
-                listBuilder.reset((AUnorderedListType) MetadataRecordTypes.PRIMARY_FEED_DETAILS_RECORDTYPE
-                        .getFieldTypes()[MetadataRecordTypes.FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_CONFIGURATION_FIELD_INDEX]);
-                for (Map.Entry<String, String> property : feed.getAdapterConfiguration().entrySet()) {
-                    String name = property.getKey();
-                    String value = property.getValue();
-                    primaryRecordItemValue.reset();
-                    writePropertyTypeRecord(name, value, primaryRecordItemValue.getDataOutput());
-                    listBuilder.addItem(primaryRecordItemValue);
-                }
-                primaryRecordfieldValue.reset();
-                listBuilder.write(primaryRecordfieldValue.getDataOutput(), true);
-                primaryDetailsRecordBuilder.addField(
-                        MetadataRecordTypes.FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_CONFIGURATION_FIELD_INDEX,
-                        primaryRecordfieldValue);
-
-                primaryDetailsRecordBuilder.write(fieldValue.getDataOutput(), true);
-
-                recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_PRIMARY_TYPE_DETAILS_FIELD_INDEX, fieldValue);
-            }
-                break;
-
-            case SECONDARY:
-                IARecordBuilder secondaryDetailsRecordBuilder = new RecordBuilder();
-                ArrayBackedValueStorage secondaryFieldValue = new ArrayBackedValueStorage();
-                secondaryDetailsRecordBuilder.reset(MetadataRecordTypes.SECONDARY_FEED_DETAILS_RECORDTYPE);
-
-                // write field 0
-                fieldValue.reset();
-                aString.setValue(feed.getSourceFeedName());
-                stringSerde.serialize(aString, secondaryFieldValue.getDataOutput());
-                secondaryDetailsRecordBuilder.addField(
-                        MetadataRecordTypes.FEED_ARECORD_SECONDARY_FIELD_DETAILS_SOURCE_FEED_NAME_FIELD_INDEX,
-                        secondaryFieldValue);
-
-                secondaryDetailsRecordBuilder.write(fieldValue.getDataOutput(), true);
-                recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_SECONDARY_TYPE_DETAILS_FIELD_INDEX, fieldValue);
-                break;
+        listBuilder.reset((AUnorderedListType) MetadataRecordTypes.FEED_RECORDTYPE
+                .getFieldTypes()[MetadataRecordTypes.FEED_ARECORD_ADAPTOR_CONFIG_INDEX]);
+        for (Map.Entry<String, String> property : feed.getAdapterConfiguration().entrySet()) {
+            String name = property.getKey();
+            String value = property.getValue();
+            listEleBuffer.reset();
+            writePropertyTypeRecord(name, value, listEleBuffer.getDataOutput());
+            listBuilder.addItem(listEleBuffer);
         }
-
+        listBuilder.write(fieldValueBuffer.getDataOutput(), true);
+        recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_CONFIG_INDEX, fieldValueBuffer);
     }
 
     @SuppressWarnings("unchecked")
     public void writePropertyTypeRecord(String name, String value, DataOutput out) throws HyracksDataException {
         IARecordBuilder propertyRecordBuilder = new RecordBuilder();
         ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
-        propertyRecordBuilder.reset(MetadataRecordTypes.FEED_ADAPTER_CONFIGURATION_RECORDTYPE);
+        propertyRecordBuilder.reset(MetadataRecordTypes.DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE);
         AMutableString aString = new AMutableString("");
-        ISerializerDeserializer<AString> stringSerde =
-                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
+        ISerializerDeserializer<AString> stringSerde = SerializerDeserializerProvider.INSTANCE
+                .getSerializerDeserializer(BuiltinType.ASTRING);
 
         // write field 0
         fieldValue.reset();
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 21db749..385f2bd 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
@@ -19,34 +19,21 @@
 package org.apache.asterix.metadata.feeds;
 
 import java.rmi.RemoteException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
 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.functions.FunctionSignature;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.api.IDataSourceAdapter;
 import org.apache.asterix.external.api.IDataSourceAdapter.AdapterType;
 import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
-import org.apache.asterix.external.operators.FeedMetaOperatorDescriptor;
 import org.apache.asterix.external.provider.AdapterFactoryProvider;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
-import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
@@ -56,34 +43,14 @@
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
-import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Triple;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
-import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
-import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
-import org.apache.hyracks.api.constraints.Constraint;
-import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
-import org.apache.hyracks.api.constraints.expressions.ConstantExpression;
-import org.apache.hyracks.api.constraints.expressions.ConstraintExpression;
-import org.apache.hyracks.api.constraints.expressions.LValueConstraintExpression;
-import org.apache.hyracks.api.constraints.expressions.PartitionCountExpression;
-import org.apache.hyracks.api.constraints.expressions.PartitionLocationExpression;
-import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
-import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
-import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningWithMessageConnectorDescriptor;
 
 /**
  * A utility class for providing helper functions for feeds
@@ -91,8 +58,6 @@
  */
 public class FeedMetadataUtil {
 
-    private static final Logger LOGGER = Logger.getLogger(FeedMetadataUtil.class.getName());
-
     public static Dataset validateIfDatasetExists(String dataverse, String datasetName, MetadataTransactionContext ctx)
             throws CompilationException {
         Dataset dataset = MetadataManager.INSTANCE.getDataset(ctx, dataverse, datasetName);
@@ -130,193 +95,6 @@
         return feedPolicy;
     }
 
-    public static JobSpecification alterJobSpecificationForFeed(JobSpecification spec,
-            FeedConnectionId feedConnectionId, Map<String, String> feedPolicyProperties) {
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Original Job Spec:" + spec);
-        }
-
-        JobSpecification altered = new JobSpecification(spec.getFrameSize());
-        Map<OperatorDescriptorId, IOperatorDescriptor> operatorMap = spec.getOperatorMap();
-        boolean preProcessingRequired = preProcessingRequired(feedConnectionId);
-        // copy operators
-        Map<OperatorDescriptorId, OperatorDescriptorId> oldNewOID = new HashMap<>();
-        FeedMetaOperatorDescriptor metaOp;
-        for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operatorMap.entrySet()) {
-            String operandId = null;
-            IOperatorDescriptor opDesc = entry.getValue();
-            if (opDesc instanceof FeedCollectOperatorDescriptor) {
-                FeedCollectOperatorDescriptor orig = (FeedCollectOperatorDescriptor) opDesc;
-                FeedCollectOperatorDescriptor fiop = new FeedCollectOperatorDescriptor(altered,
-                        orig.getFeedConnectionId(), orig.getSourceFeedId(), (ARecordType) orig.getOutputType(),
-                        orig.getRecordDescriptor(), orig.getFeedPolicyProperties(), orig.getSubscriptionLocation());
-                oldNewOID.put(opDesc.getOperatorId(), fiop.getOperatorId());
-            } else if ((opDesc instanceof LSMTreeInsertDeleteOperatorDescriptor)
-                    && ((LSMTreeInsertDeleteOperatorDescriptor) opDesc).isPrimary()) {
-                // only introduce store before primary index
-                operandId = ((LSMTreeInsertDeleteOperatorDescriptor) opDesc).getIndexName();
-                metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc, feedPolicyProperties,
-                        FeedRuntimeType.STORE, false, operandId);
-                oldNewOID.put(opDesc.getOperatorId(), metaOp.getOperatorId());
-            } else {
-                FeedRuntimeType runtimeType;
-                boolean enableSubscriptionMode;
-                OperatorDescriptorId opId = null;
-                if (opDesc instanceof AlgebricksMetaOperatorDescriptor) {
-                    IPushRuntimeFactory[] runtimeFactories = ((AlgebricksMetaOperatorDescriptor) opDesc).getPipeline()
-                            .getRuntimeFactories();
-                    if (runtimeFactories[0] instanceof AssignRuntimeFactory && runtimeFactories.length > 1) {
-                        IConnectorDescriptor connectorDesc = spec.getOperatorInputMap().get(opDesc.getOperatorId())
-                                .get(0);
-                        IOperatorDescriptor sourceOp = spec.getProducer(connectorDesc);
-                        if (sourceOp instanceof FeedCollectOperatorDescriptor) {
-                            runtimeType = FeedRuntimeType.COMPUTE;
-                            enableSubscriptionMode = preProcessingRequired;
-                            metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc,
-                                    feedPolicyProperties, runtimeType, enableSubscriptionMode, operandId);
-                            opId = metaOp.getOperatorId();
-                        }
-                    }
-                }
-                if (opId == null) {
-                    opId = altered.createOperatorDescriptorId(opDesc);
-                }
-                oldNewOID.put(opDesc.getOperatorId(), opId);
-            }
-        }
-
-        // copy connectors
-        Map<ConnectorDescriptorId, ConnectorDescriptorId> connectorMapping = new HashMap<>();
-        for (Entry<ConnectorDescriptorId, IConnectorDescriptor> entry : spec.getConnectorMap().entrySet()) {
-            IConnectorDescriptor connDesc = entry.getValue();
-            ConnectorDescriptorId newConnId;
-            if (connDesc instanceof MToNPartitioningConnectorDescriptor) {
-                MToNPartitioningConnectorDescriptor m2nConn = (MToNPartitioningConnectorDescriptor) connDesc;
-                connDesc = new MToNPartitioningWithMessageConnectorDescriptor(altered,
-                        m2nConn.getTuplePartitionComputerFactory());
-                newConnId = connDesc.getConnectorId();
-            } else {
-                newConnId = altered.createConnectorDescriptor(connDesc);
-            }
-            connectorMapping.put(entry.getKey(), newConnId);
-        }
-
-        // make connections between operators
-        for (Entry<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>,
-                Pair<IOperatorDescriptor, Integer>>> entry : spec.getConnectorOperatorMap().entrySet()) {
-            IConnectorDescriptor connDesc = altered.getConnectorMap().get(connectorMapping.get(entry.getKey()));
-            Pair<IOperatorDescriptor, Integer> leftOp = entry.getValue().getLeft();
-            Pair<IOperatorDescriptor, Integer> rightOp = entry.getValue().getRight();
-
-            IOperatorDescriptor leftOpDesc = altered.getOperatorMap()
-                    .get(oldNewOID.get(leftOp.getLeft().getOperatorId()));
-            IOperatorDescriptor rightOpDesc = altered.getOperatorMap()
-                    .get(oldNewOID.get(rightOp.getLeft().getOperatorId()));
-
-            altered.connect(connDesc, leftOpDesc, leftOp.getRight(), rightOpDesc, rightOp.getRight());
-        }
-
-        // prepare for setting partition constraints
-        Map<OperatorDescriptorId, List<LocationConstraint>> operatorLocations = new HashMap<>();
-        Map<OperatorDescriptorId, Integer> operatorCounts = new HashMap<>();
-
-        for (Constraint constraint : spec.getUserConstraints()) {
-            LValueConstraintExpression lexpr = constraint.getLValue();
-            ConstraintExpression cexpr = constraint.getRValue();
-            OperatorDescriptorId opId;
-            switch (lexpr.getTag()) {
-                case PARTITION_COUNT:
-                    opId = ((PartitionCountExpression) lexpr).getOperatorDescriptorId();
-                    operatorCounts.put(opId, (int) ((ConstantExpression) cexpr).getValue());
-                    break;
-                case PARTITION_LOCATION:
-                    opId = ((PartitionLocationExpression) lexpr).getOperatorDescriptorId();
-
-                    IOperatorDescriptor opDesc = altered.getOperatorMap().get(oldNewOID.get(opId));
-                    List<LocationConstraint> locations = operatorLocations.get(opDesc.getOperatorId());
-                    if (locations == null) {
-                        locations = new ArrayList<>();
-                        operatorLocations.put(opDesc.getOperatorId(), locations);
-                    }
-                    String location = (String) ((ConstantExpression) cexpr).getValue();
-                    LocationConstraint lc = new LocationConstraint(location,
-                            ((PartitionLocationExpression) lexpr).getPartition());
-                    locations.add(lc);
-                    break;
-                default:
-                    break;
-            }
-        }
-
-        // set absolute location constraints
-        for (Entry<OperatorDescriptorId, List<LocationConstraint>> entry : operatorLocations.entrySet()) {
-            IOperatorDescriptor opDesc = altered.getOperatorMap().get(oldNewOID.get(entry.getKey()));
-            Collections.sort(entry.getValue(), (LocationConstraint o1, LocationConstraint o2) -> {
-                return o1.partition - o2.partition;
-            });
-            String[] locations = new String[entry.getValue().size()];
-            for (int i = 0; i < locations.length; ++i) {
-                locations[i] = entry.getValue().get(i).location;
-            }
-            PartitionConstraintHelper.addAbsoluteLocationConstraint(altered, opDesc, locations);
-        }
-
-        // set count constraints
-        for (Entry<OperatorDescriptorId, Integer> entry : operatorCounts.entrySet()) {
-            IOperatorDescriptor opDesc = altered.getOperatorMap().get(oldNewOID.get(entry.getKey()));
-            if (!operatorLocations.keySet().contains(entry.getKey())) {
-                PartitionConstraintHelper.addPartitionCountConstraint(altered, opDesc, entry.getValue());
-            }
-        }
-
-        // useConnectorSchedulingPolicy
-        altered.setUseConnectorPolicyForScheduling(spec.isUseConnectorPolicyForScheduling());
-
-        // connectorAssignmentPolicy
-        altered.setConnectorPolicyAssignmentPolicy(spec.getConnectorPolicyAssignmentPolicy());
-
-        // roots
-        for (OperatorDescriptorId root : spec.getRoots()) {
-            altered.addRoot(altered.getOperatorMap().get(oldNewOID.get(root)));
-        }
-
-        // jobEventListenerFactory
-        altered.setJobletEventListenerFactory(spec.getJobletEventListenerFactory());
-
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("New Job Spec:" + altered);
-        }
-
-        return altered;
-
-    }
-
-    private static boolean preProcessingRequired(FeedConnectionId connectionId) {
-        MetadataTransactionContext ctx = null;
-        Feed feed = null;
-        boolean preProcessingRequired = false;
-        try {
-            MetadataManager.INSTANCE.acquireReadLatch();
-            ctx = MetadataManager.INSTANCE.beginTransaction();
-            feed = MetadataManager.INSTANCE.getFeed(ctx, connectionId.getFeedId().getDataverse(),
-                    connectionId.getFeedId().getEntityName());
-            preProcessingRequired = feed.getAppliedFunction() != null;
-            MetadataManager.INSTANCE.commitTransaction(ctx);
-        } catch (Exception e) {
-            if (ctx != null) {
-                try {
-                    MetadataManager.INSTANCE.abortTransaction(ctx);
-                } catch (Exception abortException) {
-                    e.addSuppressed(abortException);
-                    throw new IllegalStateException(e);
-                }
-            }
-        } finally {
-            MetadataManager.INSTANCE.releaseReadLatch();
-        }
-        return preProcessingRequired;
-    }
-
     public static void validateFeed(Feed feed, MetadataTransactionContext mdTxnCtx, ILibraryManager libraryManager)
             throws AsterixException {
         try {
@@ -537,32 +315,4 @@
         }
         return outputType;
     }
-
-    public static String getSecondaryFeedOutput(Feed feed, FeedPolicyAccessor policyAccessor,
-            MetadataTransactionContext mdTxnCtx)
-            throws AlgebricksException, MetadataException, RemoteException, ACIDException {
-        String outputType = null;
-        String primaryFeedName = feed.getSourceFeedName();
-        Feed primaryFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feed.getDataverseName(), primaryFeedName);
-        FunctionSignature appliedFunction = primaryFeed.getAppliedFunction();
-        if (appliedFunction == null) {
-            outputType = getOutputType(feed, feed.getAdapterConfiguration(), ExternalDataConstants.KEY_TYPE_NAME)
-                    .getDisplayName();
-        } else {
-            Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, appliedFunction);
-            if (function != null) {
-                if (function.getLanguage().equals(Function.LANGUAGE_AQL)) {
-                    throw new NotImplementedException(
-                            "Secondary feeds derived from a source feed that has an applied AQL function"
-                                    + " are not supported yet.");
-                } else {
-                    outputType = function.getReturnType();
-                }
-            } else {
-                throw new IllegalArgumentException(
-                        "Function " + appliedFunction + " associated with source feed not found in Metadata.");
-            }
-        }
-        return outputType;
-    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedOperations.java
deleted file mode 100644
index 00c9010..0000000
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedOperations.java
+++ /dev/null
@@ -1,182 +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.feeds;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.asterix.active.ActiveJobNotificationHandler;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.active.IActiveMessage;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.external.api.IAdapterFactory;
-import org.apache.asterix.external.feed.api.IFeedJoint;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedEventsListener;
-import org.apache.asterix.external.feed.message.EndFeedMessage;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
-import org.apache.asterix.external.operators.FeedMessageOperatorDescriptor;
-import org.apache.asterix.external.util.FeedConstants;
-import org.apache.asterix.external.util.FeedUtils;
-import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
-import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.asterix.metadata.entities.Feed;
-import org.apache.asterix.runtime.utils.ClusterStateManager;
-import org.apache.asterix.runtime.utils.RuntimeUtils;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.common.utils.Triple;
-import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.io.FileSplit;
-import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.file.FileRemoveOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-import org.apache.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
-
-/**
- * Provides helper method(s) for creating JobSpec for operations on a feed.
- */
-public class FeedOperations {
-
-    private FeedOperations() {
-    }
-
-    /**
-     * Builds the job spec for ingesting a (primary) feed from its external source via the feed adaptor.
-     *
-     * @param primaryFeed
-     * @param metadataProvider
-     * @return JobSpecification the Hyracks job specification for receiving data from external source
-     * @throws Exception
-     */
-    public static Pair<JobSpecification, IAdapterFactory> buildFeedIntakeJobSpec(Feed primaryFeed,
-            MetadataProvider metadataProvider, FeedPolicyAccessor policyAccessor) throws Exception {
-        JobSpecification spec = RuntimeUtils.createJobSpecification();
-        spec.setFrameSize(FeedConstants.JobConstants.DEFAULT_FRAME_SIZE);
-        IAdapterFactory adapterFactory;
-        IOperatorDescriptor feedIngestor;
-        AlgebricksPartitionConstraint ingesterPc;
-        Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> t =
-                metadataProvider.buildFeedIntakeRuntime(spec, primaryFeed, policyAccessor);
-        feedIngestor = t.first;
-        ingesterPc = t.second;
-        adapterFactory = t.third;
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, feedIngestor, ingesterPc);
-        NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, nullSink, ingesterPc);
-        spec.connect(new OneToOneConnectorDescriptor(spec), feedIngestor, 0, nullSink, 0);
-        spec.addRoot(nullSink);
-        return new Pair<>(spec, adapterFactory);
-    }
-
-    /**
-     * Builds the job spec for sending message to an active feed to disconnect it from the
-     * its source.
-     */
-    public static Pair<JobSpecification, Boolean> buildDisconnectFeedJobSpec(FeedConnectionId connectionId)
-            throws AlgebricksException {
-
-        JobSpecification spec = RuntimeUtils.createJobSpecification();
-        IOperatorDescriptor feedMessenger;
-        AlgebricksPartitionConstraint messengerPc;
-        List<String> locations = null;
-        FeedRuntimeType sourceRuntimeType;
-        try {
-            FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
-                    .getActiveEntityListener(connectionId.getFeedId());
-            FeedConnectJobInfo cInfo = listener.getFeedConnectJobInfo(connectionId);
-            IFeedJoint sourceFeedJoint = cInfo.getSourceFeedJoint();
-            IFeedJoint computeFeedJoint = cInfo.getComputeFeedJoint();
-
-            boolean terminateIntakeJob = false;
-            boolean completeDisconnect = computeFeedJoint == null || computeFeedJoint.getReceivers().isEmpty();
-            if (completeDisconnect) {
-                sourceRuntimeType = FeedRuntimeType.INTAKE;
-                locations = cInfo.getCollectLocations();
-                terminateIntakeJob = sourceFeedJoint.getReceivers().size() == 1;
-            } else {
-                locations = cInfo.getComputeLocations();
-                sourceRuntimeType = FeedRuntimeType.COMPUTE;
-            }
-
-            Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = buildDisconnectFeedMessengerRuntime(spec,
-                    connectionId, locations, sourceRuntimeType, completeDisconnect, sourceFeedJoint.getOwnerFeedId());
-
-            feedMessenger = p.first;
-            messengerPc = p.second;
-
-            AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, feedMessenger, messengerPc);
-            NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
-            AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, nullSink, messengerPc);
-            spec.connect(new OneToOneConnectorDescriptor(spec), feedMessenger, 0, nullSink, 0);
-            spec.addRoot(nullSink);
-            return new Pair<>(spec, terminateIntakeJob);
-
-        } catch (AlgebricksException e) {
-            throw new AsterixException(e);
-        }
-
-    }
-
-    private static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildSendFeedMessageRuntime(
-            JobSpecification jobSpec, FeedConnectionId feedConenctionId, IActiveMessage feedMessage,
-            Collection<String> locations) throws AlgebricksException {
-        AlgebricksPartitionConstraint partitionConstraint =
-                new AlgebricksAbsolutePartitionConstraint(locations.toArray(new String[] {}));
-        FeedMessageOperatorDescriptor feedMessenger =
-                new FeedMessageOperatorDescriptor(jobSpec, feedConenctionId, feedMessage);
-        return new Pair<>(feedMessenger, partitionConstraint);
-    }
-
-    private static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDisconnectFeedMessengerRuntime(
-            JobSpecification jobSpec, FeedConnectionId feedConenctionId, List<String> locations,
-            FeedRuntimeType sourceFeedRuntimeType, boolean completeDisconnection, EntityId sourceFeedId)
-            throws AlgebricksException {
-        IActiveMessage feedMessage = new EndFeedMessage(feedConenctionId, sourceFeedRuntimeType, sourceFeedId,
-                completeDisconnection, EndFeedMessage.EndMessageType.DISCONNECT_FEED);
-        return buildSendFeedMessageRuntime(jobSpec, feedConenctionId, feedMessage, locations);
-    }
-
-    public static JobSpecification buildRemoveFeedStorageJob(Feed feed) throws AsterixException {
-        JobSpecification spec = RuntimeUtils.createJobSpecification();
-        AlgebricksAbsolutePartitionConstraint allCluster = ClusterStateManager.INSTANCE.getClusterLocations();
-        Set<String> nodes = new TreeSet<>();
-        for (String node : allCluster.getLocations()) {
-            nodes.add(node);
-        }
-        AlgebricksAbsolutePartitionConstraint locations =
-                new AlgebricksAbsolutePartitionConstraint(nodes.toArray(new String[nodes.size()]));
-        FileSplit[] feedLogFileSplits =
-                FeedUtils.splitsForAdapter(feed.getDataverseName(), feed.getFeedName(), locations);
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
-                StoragePathUtil.splitProviderAndPartitionConstraints(feedLogFileSplits);
-        FileRemoveOperatorDescriptor frod = new FileRemoveOperatorDescriptor(spec, splitsAndConstraint.first, true);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, frod, splitsAndConstraint.second);
-        spec.addRoot(frod);
-        return spec;
-    }
-}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java
index a9c721a..81eaa9b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java
@@ -24,6 +24,8 @@
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.entities.Feed;
+import org.apache.asterix.metadata.entities.FeedConnection;
 
 public class MetadataLockManager {
 
@@ -480,6 +482,33 @@
         releaseDataverseReadLock(dataverseName);
     }
 
+    public void startFeedBegin(String dataverseName, String feedName, List<FeedConnection> feedConnections) {
+        acquireDataverseReadLock(dataverseName);
+        acquireFeedReadLock(feedName);
+        for (FeedConnection feedConnection : feedConnections) {
+            acquireDatasetReadLock(dataverseName + "." + feedConnection.getDatasetName());
+        }
+    }
+
+    public void startFeedEnd(String dataverseName, String feedName, List<FeedConnection> feedConnections) {
+        releaseDataverseReadLock(dataverseName);
+        releaseFeedReadLock(feedName);
+        for (FeedConnection feedConnection : feedConnections) {
+            releaseDatasetReadLock(dataverseName + "." + feedConnection.getDatasetName());
+        }
+    }
+
+    public void StopFeedBegin(String dataverseName, String feedName) {
+        // TODO: dataset lock?
+        acquireDataverseReadLock(dataverseName);
+        acquireFeedReadLock(feedName);
+    }
+
+    public void StopFeedEnd(String dataverseName, String feedName) {
+        releaseDataverseReadLock(dataverseName);
+        releaseFeedReadLock(feedName);
+    }
+
     public void createFeedBegin(String dataverseName, String feedFullyQualifiedName) {
         acquireDataverseReadLock(dataverseName);
         acquireFeedWriteLock(feedFullyQualifiedName);
diff --git a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.1.ddl.aql b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.1.ddl.aql
index df6c9b6..565d61b 100644
--- a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.1.ddl.aql
+++ b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.1.ddl.aql
@@ -19,8 +19,8 @@
 /*
  * Description  : Create a feed dataset that uses the feed simulator adapter.
                   The feed simulator simulates feed from a file in the local fs.
-                  Associate with the feed an external user-defined function. The UDF 
-                  finds topics in each tweet. A topic is identified by a #. 
+                  Associate with the feed an external user-defined function. The UDF
+                  finds topics in each tweet. A topic is identified by a #.
                   Begin ingestion and apply external user defined function
  * Expected Res : Success
  * Date         : 23rd Apr 2013
diff --git a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.2.update.aql b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.2.update.aql
index 39a6272..3be4db8 100644
--- a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.2.update.aql
+++ b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.2.update.aql
@@ -19,8 +19,8 @@
 /*
  * Description  : Create a feed dataset that uses the feed simulator adapter.
                   The feed simulator simulates feed from a file in the local fs.
-                  Associate with the feed an external user-defined function. The UDF 
-                  finds topics in each tweet. A topic is identified by a #. 
+                  Associate with the feed an external user-defined function. The UDF
+                  finds topics in each tweet. A topic is identified by a #.
                   Begin ingestion and apply external user defined function
  * Expected Res : Success
  * Date         : 23rd Apr 2013
@@ -30,3 +30,5 @@
 set wait-for-completion-feed "true";
 
 connect feed TestTypedAdapterFeed to dataset TweetsTestAdapter;
+
+start feed TestTypedAdapterFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.3.query.aql b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.3.query.aql
index 3df9d2b..2860f17 100644
--- a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.3.query.aql
+++ b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-adapters/typed_adapter/typed_adapter.3.query.aql
@@ -19,8 +19,8 @@
 /*
  * Description  : Create a feed dataset that uses the feed simulator adapter.
                   The feed simulator simulates feed from a file in the local fs.
-                  Associate with the feed an external user-defined function. The UDF 
-                  finds topics in each tweet. A topic is identified by a #. 
+                  Associate with the feed an external user-defined function. The UDF
+                  finds topics in each tweet. A topic is identified by a #.
                   Begin ingestion and apply external user defined function
  * Expected Res : Success
  * Date         : 23rd Apr 2013
diff --git a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.1.ddl.aql b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.1.ddl.aql
index 35125a1..6e84ea3 100644
--- a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.1.ddl.aql
+++ b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.1.ddl.aql
@@ -19,8 +19,8 @@
 /*
  * Description  : Create a feed dataset that uses the feed simulator adapter.
                   The feed simulator simulates feed from a file in the local fs.
-                  Associate with the feed an external user-defined function. The UDF 
-                  finds topics in each tweet. A topic is identified by a #. 
+                  Associate with the feed an external user-defined function. The UDF
+                  finds topics in each tweet. A topic is identified by a #.
                   Begin ingestion and apply external user defined function
  * Expected Res : Success
  * Date         : 23rd Apr 2013
@@ -46,8 +46,9 @@
 
 create feed TweetFeed
 using file_feed
-(("type-name"="TweetInputType"),("fs"="localfs"),("path"="127.0.0.1://../../../../../../asterix-app/data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"))
-apply function testlib#parseTweet;
+(("type-name"="TweetInputType"),("fs"="localfs"),
+("path"="127.0.0.1://../../../../../../asterix-app/data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"));
 
-create dataset TweetsFeedIngest(TweetOutputType) 
+
+create dataset TweetsFeedIngest(TweetOutputType)
 primary key id;
diff --git a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.2.update.aql b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.2.update.aql
index d5a6f58..db68138 100644
--- a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.2.update.aql
+++ b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.2.update.aql
@@ -19,8 +19,8 @@
 /*
  * Description  : Create a feed dataset that uses the feed simulator adapter.
                   The feed simulator simulates feed from a file in the local fs.
-                  Associate with the feed an external user-defined function. The UDF 
-                  finds topics in each tweet. A topic is identified by a #. 
+                  Associate with the feed an external user-defined function. The UDF
+                  finds topics in each tweet. A topic is identified by a #.
                   Begin ingestion and apply external user defined function
  * Expected Res : Success
  * Date         : 23rd Apr 2013
@@ -29,4 +29,5 @@
 
 set wait-for-completion-feed "true";
 
-connect feed TweetFeed to dataset TweetsFeedIngest;
+connect feed TweetFeed to dataset TweetsFeedIngesta pply function testlib#parseTweet;
+start feed TweetFeed;
diff --git a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.3.query.aql b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.3.query.aql
index b188b52..22d1d27 100644
--- a/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.3.query.aql
+++ b/asterixdb/asterix-yarn/src/test/resources/library/queries/library-feeds/feed_ingest/feed_ingest.3.query.aql
@@ -19,8 +19,8 @@
 /*
  * Description  : Create a feed dataset that uses the feed simulator adapter.
                   The feed simulator simulates feed from a file in the local fs.
-                  Associate with the feed an external user-defined function. The UDF 
-                  finds topics in each tweet. A topic is identified by a #. 
+                  Associate with the feed an external user-defined function. The UDF
+                  finds topics in each tweet. A topic is identified by a #.
                   Begin ingestion and apply external user defined function
  * Expected Res : Success
  * Date         : 23rd Apr 2013