Repository: asterixdb
Updated Branches:
  refs/heads/master f86a25b86 -> 604c921ae


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 6f3b667..0000000
--- 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.feed.management;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.active.ActiveEvent;
-import org.apache.asterix.active.ActiveLifecycleListener;
-import org.apache.asterix.active.ActivityState;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.active.IActiveEventSubscriber;
-import org.apache.asterix.active.message.ActivePartitionMessage;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-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.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobStatus;
-
-public class FeedEventsListener extends ActiveEntityEventsListener {
-    // constants
-    private static final Logger LOGGER = 
Logger.getLogger(FeedEventsListener.class.getName());
-    // members
-    private final ICcApplicationContext appCtx;
-    private final String[] sources;
-    private final List<IActiveEventSubscriber> subscribers;
-    private int numRegistered;
-
-    public FeedEventsListener(ICcApplicationContext appCtx, EntityId entityId, 
List<IDataset> datasets,
-            String[] sources) {
-        this.appCtx = appCtx;
-        this.entityId = entityId;
-        this.datasets = datasets;
-        this.sources = sources;
-        subscribers = new ArrayList<>();
-        state = ActivityState.STOPPED;
-    }
-
-    @Override
-    public synchronized void notify(ActiveEvent event) {
-        try {
-            switch (event.getEventKind()) {
-                case JOB_STARTED:
-                    start(event);
-                    break;
-                case JOB_FINISHED:
-                    finish();
-                    break;
-                case PARTITION_EVENT:
-                    partition((ActivePartitionMessage) event.getEventObject());
-                    break;
-                default:
-                    LOGGER.log(Level.WARNING, "Unhandled feed event 
notification: " + event);
-                    break;
-            }
-            notifySubscribers(event);
-        } catch (Exception e) {
-            LOGGER.log(Level.SEVERE, "Unhandled Exception", e);
-        }
-    }
-
-    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 {
-                subscriber.notify(event);
-                if (subscriber.done()) {
-                    it.remove();
-                }
-            }
-        }
-    }
-
-    private void partition(ActivePartitionMessage message) {
-        if (message.getEvent() == 
ActivePartitionMessage.ACTIVE_RUNTIME_REGISTERED) {
-            numRegistered++;
-            if (numRegistered == getSources().length) {
-                state = ActivityState.STARTED;
-            }
-        }
-    }
-
-    private void finish() throws Exception {
-        IHyracksClientConnection hcc = appCtx.getHcc();
-        JobStatus status = hcc.getJobStatus(jobId);
-        state = status.equals(JobStatus.FAILURE) ? ActivityState.FAILED : 
ActivityState.STOPPED;
-        ActiveLifecycleListener activeLcListener = (ActiveLifecycleListener) 
appCtx.getActiveLifecycleListener();
-        activeLcListener.getNotificationHandler().removeListener(this);
-    }
-
-    private void start(ActiveEvent event) {
-        this.jobId = event.getJobId();
-        state = ActivityState.STARTING;
-    }
-
-    @Override
-    public 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;
-    }
-
-    public String[] getSources() {
-        return sources;
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
----------------------------------------------------------------------
diff --git 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
index c71b8a2..e6ac265 100644
--- 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
+++ 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
@@ -34,6 +34,7 @@ public class AdapterExecutor implements Runnable {
     // increase or decrease at any time)
     private final FeedAdapter adapter; // The adapter
     private final AdapterRuntimeManager adapterManager;// The runtime manager 
<-- two way visibility -->
+    private int restartCount = 0;
 
     public AdapterExecutor(IFrameWriter writer, FeedAdapter adapter, 
AdapterRuntimeManager adapterManager) {
         this.writer = writer;
@@ -81,8 +82,13 @@ public class AdapterExecutor implements Runnable {
                 LOGGER.error("Exception during feed ingestion ", e);
                 continueIngestion = adapter.handleException(e);
                 failedIngestion = !continueIngestion;
+                restartCount++;
             }
         }
         return failedIngestion;
     }
+
+    public String getStats() {
+        return "{\"adapter-stats\": " + adapter.getStats() + ", 
\"executor-restart-times\": " + restartCount + "}";
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java
----------------------------------------------------------------------
diff --git 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java
 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java
index 6214d9f..1b5eeac 100644
--- 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java
+++ 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java
@@ -137,4 +137,8 @@ public class AdapterRuntimeManager {
     public void setDone(boolean done) {
         this.done = done;
     }
+
+    public String getStats() {
+        return adapterExecutor.getStats();
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 590af01..0000000
--- 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.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.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.hyracks.api.exceptions.HyracksDataException;
-
-public class IngestionRuntime implements IActiveRuntime {
-
-    private static final Logger LOGGER = 
Logger.getLogger(IngestionRuntime.class.getName());
-
-    private final AdapterRuntimeManager adapterRuntimeManager;
-    private final ActiveRuntimeId runtimeId;
-    private final EntityId feedId;
-
-    public IngestionRuntime(EntityId entityId, ActiveRuntimeId runtimeId, 
AdapterRuntimeManager adaptorRuntimeManager) {
-        this.feedId = entityId;
-        this.runtimeId = runtimeId;
-        this.adapterRuntimeManager = adaptorRuntimeManager;
-    }
-
-    @Override
-    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, HyracksDataException {
-        adapterRuntimeManager.stop();
-        LOGGER.log(Level.INFO, "Feed " + feedId.getEntityName() + " stopped on 
partition " + runtimeId);
-    }
-
-    public EntityId getFeedId() {
-        return feedId;
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
----------------------------------------------------------------------
diff --git 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
new file mode 100644
index 0000000..822d725
--- /dev/null
+++ 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
@@ -0,0 +1,56 @@
+/*
+ * 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.IActiveEntityEventsListener;
+import org.apache.asterix.active.IActiveEventSubscriber;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractSubscriber implements IActiveEventSubscriber {
+
+    protected final IActiveEntityEventsListener listener;
+    private boolean done = false;
+
+    public AbstractSubscriber(IActiveEntityEventsListener listener) {
+        this.listener = listener;
+    }
+
+    @Override
+    public synchronized boolean isDone() {
+        return done;
+    }
+
+    public synchronized void complete() throws HyracksDataException {
+        done = true;
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void sync() throws InterruptedException {
+        while (!done) {
+            wait();
+        }
+    }
+
+    @Override
+    public synchronized void unsubscribe() {
+        done = true;
+        notifyAll();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedEventSubscriber.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 0e931f7..0000000
--- 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedEventSubscriber.java
+++ /dev/null
@@ -1,64 +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 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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
----------------------------------------------------------------------
diff --git 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
index 9d8c570..42f7a74 100644
--- 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
+++ 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
@@ -19,7 +19,9 @@
 package org.apache.asterix.external.feed.watch;
 
 import org.apache.asterix.active.ActiveEvent;
+import org.apache.asterix.active.IActiveEntityEventsListener;
 import org.apache.asterix.active.IActiveEventSubscriber;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * An event subscriber that does not listen to any events
@@ -33,22 +35,26 @@ public class NoOpSubscriber implements 
IActiveEventSubscriber {
 
     @Override
     public void notify(ActiveEvent event) {
-        // do nothing
+        // no op
     }
 
     @Override
-    public boolean done() {
+    public boolean isDone() {
         return true;
     }
 
     @Override
     public void sync() {
-        // do nothing
+        // no op
     }
 
     @Override
     public void unsubscribe() {
-        // do nothing
+        // no op
     }
 
+    @Override
+    public void subscribed(IActiveEntityEventsListener eventsListener) throws 
HyracksDataException {
+        // no op
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java
----------------------------------------------------------------------
diff --git 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java
 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java
new file mode 100644
index 0000000..fa2fa7f
--- /dev/null
+++ 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java
@@ -0,0 +1,43 @@
+/*
+ * 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.IActiveEntityEventsListener;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class StatsSubscriber extends AbstractSubscriber {
+
+    public StatsSubscriber(IActiveEntityEventsListener listener) throws 
HyracksDataException {
+        super(listener);
+        listener.subscribe(this);
+    }
+
+    @Override
+    public void notify(ActiveEvent event) throws HyracksDataException {
+        if (event.getEventKind() == ActiveEvent.Kind.STATS_UPDATED) {
+            complete();
+        }
+    }
+
+    @Override
+    public void subscribed(IActiveEntityEventsListener eventsListener) throws 
HyracksDataException {
+        //Does nothing upon subscription
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
----------------------------------------------------------------------
diff --git 
a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
new file mode 100644
index 0000000..ea7e3ae
--- /dev/null
+++ 
b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
@@ -0,0 +1,58 @@
+/*
+ * 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.IActiveEntityEventsListener;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class WaitForStateSubscriber extends AbstractSubscriber {
+
+    private final ActivityState targetState;
+
+    public WaitForStateSubscriber(IActiveEntityEventsListener listener, 
ActivityState targetState)
+            throws HyracksDataException {
+        super(listener);
+        this.targetState = targetState;
+        if (targetState != ActivityState.STARTED && targetState != 
ActivityState.STOPPED) {
+            throw new RuntimeDataException(ErrorCode.CANNOT_WAIT_FOR_STATE, 
targetState);
+        }
+        listener.subscribe(this);
+    }
+
+    @Override
+    public void notify(ActiveEvent event) throws HyracksDataException {
+        if (listener.getState() == targetState) {
+            complete();
+        }
+    }
+
+    @Override
+    public void subscribed(IActiveEntityEventsListener eventsListener) throws 
HyracksDataException {
+        if (eventsListener.getState() == ActivityState.FAILED) {
+            throw new 
RuntimeDataException(ErrorCode.CANNOT_SUBSCRIBE_TO_FAILED_ACTIVE_ENTITY);
+        }
+        if (listener.getState() == targetState) {
+            complete();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
----------------------------------------------------------------------
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 2876ea6..8c6a420 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
@@ -105,4 +105,13 @@ public class FeedIntakeOperatorNodePushable extends 
ActiveSourceOperatorNodePush
             adapterRuntimeManager.stop();
         }
     }
+
+    @Override
+    public String getStats() {
+        if (adapterRuntimeManager != null) {
+            return adapterRuntimeManager.getStats();
+        } else {
+            return "\"Runtime stats is not available.\"";
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/604c921a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
 
b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
index 3c41165..4a4c25a 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
@@ -91,8 +91,7 @@ public abstract class AbstractServlet implements IServlet {
         }
     }
 
-    protected void sendError(IServletResponse response, HttpResponseStatus 
status, String message)
-            throws IOException {
+    protected void sendError(IServletResponse response, HttpResponseStatus 
status, String message) throws IOException {
         response.setStatus(status);
         HttpUtil.setContentType(response, HttpUtil.ContentType.TEXT_PLAIN, 
HttpUtil.Encoding.UTF8);
         if (message != null) {
@@ -154,7 +153,7 @@ public abstract class AbstractServlet implements IServlet {
 
     public String localPath(IServletRequest request) {
         final String uri = request.getHttpRequest().uri();
-        int queryStart = uri.indexOf("?");
+        int queryStart = uri.indexOf('?');
         return queryStart == -1 ? uri.substring(trim(uri)) : 
uri.substring(trim(uri), queryStart);
     }
 

Reply via email to