You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by ti...@apache.org on 2017/01/26 15:29:28 UTC

[1/5] asterixdb git commit: Replace Servlets with Netty Based HTTP Servers

Repository: asterixdb
Updated Branches:
  refs/heads/master 9d30640f2 -> 60e7f12b4


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
index 80e8d09..ff30df3 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
@@ -48,7 +48,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.utils.ServletUtil.Servlets;
+import org.apache.asterix.common.utils.LetUtil.Lets;
 import org.apache.asterix.test.base.ComparisonException;
 import org.apache.asterix.test.server.ITestServer;
 import org.apache.asterix.test.server.TestServerProvider;
@@ -603,7 +603,7 @@ public class TestExecutor {
     }
 
     private InputStream getHandleResult(String handle, OutputFormat fmt) throws Exception {
-        final String url = getEndpoint(Servlets.QUERY_RESULT);
+        final String url = getEndpoint(Lets.QUERY_RESULT);
 
         // Create a method instance.
         HttpUriRequest request = RequestBuilder.get(url).addParameter("handle", handle)
@@ -744,9 +744,9 @@ public class TestExecutor {
         switch (ctx.getType()) {
             case "ddl":
                 if (ctx.getFile().getName().endsWith("aql")) {
-                    executeDDL(statement, getEndpoint(Servlets.AQL_DDL));
+                    executeDDL(statement, getEndpoint(Lets.AQL_DDL));
                 } else {
-                    InputStream resultStream = executeQueryService(statement, getEndpoint(Servlets.QUERY_SERVICE));
+                    InputStream resultStream = executeQueryService(statement, getEndpoint(Lets.QUERY_SERVICE));
                     ResultExtractor.extract(resultStream);
                 }
                 break;
@@ -756,9 +756,9 @@ public class TestExecutor {
                     statement = statement.replaceAll("nc1://", "127.0.0.1://../../../../../../asterix-app/");
                 }
                 if (ctx.getFile().getName().endsWith("aql")) {
-                    executeUpdate(statement, getEndpoint(Servlets.AQL_UPDATE));
+                    executeUpdate(statement, getEndpoint(Lets.AQL_UPDATE));
                 } else {
-                    InputStream resultStream = executeQueryService(statement, getEndpoint(Servlets.QUERY_SERVICE));
+                    InputStream resultStream = executeQueryService(statement, getEndpoint(Lets.QUERY_SERVICE));
                     ResultExtractor.extract(resultStream);
                 }
                 break;
@@ -812,16 +812,16 @@ public class TestExecutor {
                 OutputFormat fmt = OutputFormat.forCompilationUnit(cUnit);
                 if (ctx.getFile().getName().endsWith("aql")) {
                     if (ctx.getType().equalsIgnoreCase("query")) {
-                        resultStream = executeQuery(statement, fmt, getEndpoint(Servlets.AQL_QUERY),
+                        resultStream = executeQuery(statement, fmt, getEndpoint(Lets.AQL_QUERY),
                                 cUnit.getParameter());
                     } else if (ctx.getType().equalsIgnoreCase("async")) {
-                        resultStream = executeAnyAQLAsync(statement, false, fmt, getEndpoint(Servlets.AQL));
+                        resultStream = executeAnyAQLAsync(statement, false, fmt, getEndpoint(Lets.AQL));
                     } else if (ctx.getType().equalsIgnoreCase("asyncdefer")) {
-                        resultStream = executeAnyAQLAsync(statement, true, fmt, getEndpoint(Servlets.AQL));
+                        resultStream = executeAnyAQLAsync(statement, true, fmt, getEndpoint(Lets.AQL));
                     }
                 } else {
                     final String reqType = ctx.getType();
-                    final String url = getEndpoint(Servlets.QUERY_SERVICE);
+                    final String url = getEndpoint(Lets.QUERY_SERVICE);
                     final List<CompilationUnit.Parameter> params = cUnit.getParameter();
                     if (reqType.equalsIgnoreCase("query")) {
                         resultStream = executeQueryService(statement, fmt, url, params, true);
@@ -854,13 +854,13 @@ public class TestExecutor {
                 break;
             case "txnqbc": // qbc represents query before crash
                 resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
-                        getEndpoint(Servlets.AQL_QUERY), cUnit.getParameter());
+                        getEndpoint(Lets.AQL_QUERY), cUnit.getParameter());
                 qbcFile = getTestCaseQueryBeforeCrashFile(actualPath, testCaseCtx, cUnit);
                 writeOutputToFile(qbcFile, resultStream);
                 break;
             case "txnqar": // qar represents query after recovery
                 resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
-                        getEndpoint(Servlets.AQL_QUERY), cUnit.getParameter());
+                        getEndpoint(Lets.AQL_QUERY), cUnit.getParameter());
                 File qarFile = new File(actualPath + File.separator
                         + testCaseCtx.getTestCase().getFilePath().replace(File.separator, "_") + "_" + cUnit.getName()
                         + "_qar.adm");
@@ -870,7 +870,7 @@ public class TestExecutor {
                 break;
             case "txneu": // eu represents erroneous update
                 try {
-                    executeUpdate(statement, getEndpoint(Servlets.AQL_UPDATE));
+                    executeUpdate(statement, getEndpoint(Lets.AQL_UPDATE));
                 } catch (Exception e) {
                     // An exception is expected.
                     failed = true;
@@ -898,7 +898,7 @@ public class TestExecutor {
                 break;
             case "errddl": // a ddlquery that expects error
                 try {
-                    executeDDL(statement, getEndpoint(Servlets.AQL_DDL));
+                    executeDDL(statement, getEndpoint(Lets.AQL_DDL));
                 } catch (Exception e) {
                     // expected error happens
                     failed = true;
@@ -1136,11 +1136,11 @@ public class TestExecutor {
                         + cUnit.getName() + "_qbc.adm");
     }
 
-    protected String getPath(Servlets servlet) {
+    protected String getPath(Lets servlet) {
         return servlet.getPath();
     }
 
-    protected String getEndpoint(Servlets servlet) {
+    protected String getEndpoint(Lets servlet) {
         return "http://" + host + ":" + port + getPath(servlet).replaceAll("/\\*$", "");
     }
 
@@ -1152,7 +1152,7 @@ public class TestExecutor {
         try {
             ArrayList<String> toBeDropped = new ArrayList<>();
             InputStream resultStream = executeQueryService("select dv.DataverseName from Metadata.`Dataverse` as dv;",
-                    getEndpoint(Servlets.QUERY_SERVICE));
+                    getEndpoint(Lets.QUERY_SERVICE));
             String out = IOUtils.toString(resultStream);
             ObjectMapper om = new ObjectMapper();
             om.setConfig(om.getDeserializationConfig().with(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT));
@@ -1181,7 +1181,7 @@ public class TestExecutor {
                     dropStatement.append(dv);
                     dropStatement.append(";\n");
                 }
-                resultStream = executeQueryService(dropStatement.toString(), getEndpoint(Servlets.QUERY_SERVICE));
+                resultStream = executeQueryService(dropStatement.toString(), getEndpoint(Lets.QUERY_SERVICE));
                 ResultExtractor.extract(resultStream);
             }
         } catch (Throwable th) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/SampleLocalClusterIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/SampleLocalClusterIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/SampleLocalClusterIT.java
index 83d421f..0991d4f 100644
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/SampleLocalClusterIT.java
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/SampleLocalClusterIT.java
@@ -27,7 +27,7 @@ import java.io.StringWriter;
 import java.net.URL;
 import java.util.Collections;
 
-import org.apache.asterix.common.utils.ServletUtil.Servlets;
+import org.apache.asterix.common.utils.LetUtil.Lets;
 import org.apache.asterix.test.aql.TestExecutor;
 import org.apache.asterix.test.base.TestMethodTracer;
 import org.apache.asterix.test.common.TestHelper;
@@ -89,7 +89,7 @@ public class SampleLocalClusterIT {
     public void test1_sanityQuery() throws Exception {
         TestExecutor testExecutor = new TestExecutor();
         InputStream resultStream = testExecutor.executeQuery("1+1", OutputFormat.ADM,
-                "http://127.0.0.1:19002" + Servlets.AQL_QUERY.getPath(), Collections.emptyList());
+                "http://127.0.0.1:19002" + Lets.AQL_QUERY.getPath(), Collections.emptyList());
         StringWriter sw = new StringWriter();
         IOUtils.copy(resultStream, sw);
         Assert.assertEquals("2", sw.toString().trim());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index ea80127..3133231 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -825,6 +825,11 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hyracks</groupId>
+        <artifactId>hyracks-http</artifactId>
+        <version>${hyracks.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hyracks</groupId>
         <artifactId>hyracks-util</artifactId>
         <version>${hyracks.version}</version>
       </dependency>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/pom.xml b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
new file mode 100644
index 0000000..8dc57a2
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
@@ -0,0 +1,34 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.3.1-SNAPSHOT</version>
+  </parent>
+  <artifactId>hyracks-http</artifactId>
+  <dependencies>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-all</artifactId>
+      <version>4.1.6.Final</version>
+    </dependency>
+  </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/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
new file mode 100644
index 0000000..22bbc50
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.util.concurrent.ConcurrentMap;
+
+public abstract class AbstractServlet implements IServlet {
+    protected final String[] paths;
+    protected final ConcurrentMap<String, Object> ctx;
+    private final int[] trims;
+
+    public AbstractServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        this.paths = paths;
+        this.ctx = ctx;
+        trims = new int[paths.length];
+        for (int i = 0; i < paths.length; i++) {
+            String path = paths[i];
+            if (path.endsWith("/*")) {
+                trims[i] = path.indexOf("/*");
+            } else if (path.endsWith("/")) {
+                trims[i] = path.length() - 1;
+            } else {
+                trims[i] = path.length();
+            }
+        }
+    }
+
+    @Override
+    public String[] getPaths() {
+        return paths;
+    }
+
+    @Override
+    public ConcurrentMap<String, Object> ctx() {
+        return ctx;
+    }
+
+    public String path(IServletRequest request) {
+        int trim = -1;
+        if (paths.length > 1) {
+            for (int i = 0; i < paths.length; i++) {
+                String path = paths[i].indexOf('*') >= 0 ? paths[i].substring(0, paths[i].indexOf('*')) : paths[0];
+                if (request.getHttpRequest().uri().indexOf(path) == 0) {
+                    trim = trims[i];
+                    break;
+                }
+            }
+        } else {
+            trim = trims[0];
+        }
+        return request.getHttpRequest().uri().substring(trim);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
new file mode 100644
index 0000000..984122b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.http.DefaultHttpContent;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class ChunkedNettyOutputStream extends OutputStream {
+
+    private final ChannelHandlerContext ctx;
+    private final ChunkedResponse response;
+    private ByteBuf buffer;
+
+    public ChunkedNettyOutputStream(ChannelHandlerContext ctx, int chunkSize,
+            ChunkedResponse response) {
+        this.response = response;
+        this.ctx = ctx;
+        buffer = ctx.alloc().buffer(chunkSize);
+    }
+
+    @Override
+    public synchronized void write(byte[] b, int off, int len) {
+        if ((off < 0) || (off > b.length) || (len < 0) ||
+                ((off + len) > b.length)) {
+            throw new IndexOutOfBoundsException();
+        } else if (len == 0) {
+            return;
+        }
+
+        if (len > buffer.capacity()) {
+            flush();
+            flush(b, off, len);
+        } else {
+            int space = buffer.writableBytes();
+            if (space >= len) {
+                buffer.writeBytes(b, off, len);
+            } else {
+                buffer.writeBytes(b, off, space);
+                flush();
+                buffer.writeBytes(b, off + space, len - space);
+            }
+        }
+    }
+
+    @Override
+    public synchronized void write(int b) {
+        if (buffer.writableBytes() > 0) {
+            buffer.writeByte(b);
+        } else {
+            flush();
+            buffer.writeByte(b);
+        }
+    }
+
+    @Override
+    public synchronized void close() throws IOException {
+        flush();
+        buffer.release();
+        super.close();
+    }
+
+    @Override
+    public synchronized void flush() {
+        if (buffer.readableBytes() > 0) {
+            int size = buffer.capacity();
+            if (response.status() == HttpResponseStatus.OK) {
+                response.flush();
+                DefaultHttpContent content = new DefaultHttpContent(buffer);
+                ctx.write(content);
+            } else {
+                response.error(buffer);
+            }
+            buffer = ctx.alloc().buffer(size);
+        }
+    }
+
+    private synchronized void flush(byte[] buf, int offset, int len) {
+        ByteBuf aBuffer = ctx.alloc().buffer(len);
+        aBuffer.writeBytes(buf, offset, len);
+        if (response.status() == HttpResponseStatus.OK) {
+            response.flush();
+            ctx.write(new DefaultHttpContent(aBuffer));
+        } else {
+            response.error(aBuffer);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
new file mode 100644
index 0000000..19c2664
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.FullHttpRequest;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpUtil;
+import io.netty.handler.codec.http.LastHttpContent;
+
+public class ChunkedResponse implements IServletResponse {
+    private final ChannelHandlerContext ctx;
+    private final ChunkedNettyOutputStream outputStream;
+    private final PrintWriter writer;
+    private HttpResponse response;
+    private boolean headerSent;
+    private ByteBuf error;
+    private ChannelFuture future;
+
+    public ChunkedResponse(ChannelHandlerContext ctx, FullHttpRequest request) {
+        this.ctx = ctx;
+        outputStream = new ChunkedNettyOutputStream(ctx, 4096, this);
+        writer = new PrintWriter(outputStream);
+        response = new DefaultHttpResponse(HTTP_1_1, OK);
+        response.headers().set(HttpHeaderNames.TRANSFER_ENCODING, HttpHeaderValues.CHUNKED);
+        if (HttpUtil.isKeepAlive(request)) {
+            response.headers().set(HttpHeaderNames.CONNECTION, HttpHeaderValues.KEEP_ALIVE);
+        }
+    }
+
+    @Override
+    public IServletResponse setHeader(CharSequence name, Object value) throws IOException {
+        if (headerSent) {
+            throw new IOException("Can't add more headers since the initial response was sent");
+        }
+        response.headers().set(name, value);
+        return this;
+    }
+
+    @Override
+    public ChannelFuture future() {
+        return future;
+    }
+
+    @Override
+    public PrintWriter writer() {
+        return writer;
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (error == null) {
+            writer.close();
+            future = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT);
+        }
+    }
+
+    public HttpResponseStatus status() {
+        return response.status();
+    }
+
+    public void flush() {
+        if (!headerSent && response.status() == HttpResponseStatus.OK) {
+            ctx.writeAndFlush(response);
+            headerSent = true;
+        }
+    }
+
+    public void error(ByteBuf error) {
+        this.error = error;
+    }
+
+    @Override
+    public OutputStream outputStream() {
+        return outputStream;
+    }
+
+    @Override
+    public void setStatus(HttpResponseStatus status) {
+        // update the response
+        // close the stream
+        // write the response
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FullResponse.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FullResponse.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FullResponse.java
new file mode 100644
index 0000000..245f28a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FullResponse.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.FullHttpRequest;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpUtil;
+import io.netty.handler.codec.http.HttpVersion;
+
+public class FullResponse implements IServletResponse {
+    private final ChannelHandlerContext ctx;
+    private final ByteArrayOutputStream baos;
+    private final PrintWriter writer;
+    private final FullHttpResponse response;
+    private final boolean keepAlive;
+    private ChannelFuture future;
+
+    public FullResponse(ChannelHandlerContext ctx, FullHttpRequest request) {
+        this.ctx = ctx;
+        baos = new ByteArrayOutputStream();
+        writer = new PrintWriter(baos);
+        response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR);
+        keepAlive = HttpUtil.isKeepAlive(request);
+        if (keepAlive) {
+            response.headers().set(HttpHeaderNames.CONNECTION, HttpHeaderValues.KEEP_ALIVE);
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        writer.close();
+        FullHttpResponse fullResponse = response.replace(Unpooled.copiedBuffer(baos.toByteArray()));
+        if (keepAlive && response.status() == HttpResponseStatus.OK) {
+            fullResponse.headers().setInt(HttpHeaderNames.CONTENT_LENGTH, fullResponse.content().readableBytes());
+        }
+        future = ctx.writeAndFlush(fullResponse);
+        if (response.status() != HttpResponseStatus.OK) {
+            future.addListener(ChannelFutureListener.CLOSE);
+        }
+    }
+
+    @Override
+    public IServletResponse setHeader(CharSequence name, Object value) throws IOException {
+        response.headers().set(name, value);
+        return this;
+    }
+
+    @Override
+    public PrintWriter writer() {
+        return writer;
+    }
+
+    @Override
+    public ChannelFuture future() throws IOException {
+        return future;
+    }
+
+    @Override
+    public OutputStream outputStream() {
+        return baos;
+    }
+
+    @Override
+    public void setStatus(HttpResponseStatus status) {
+        response.setStatus(status);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/GetRequest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/GetRequest.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/GetRequest.java
new file mode 100644
index 0000000..0b80a78
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/GetRequest.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.util.List;
+import java.util.Map;
+
+import io.netty.handler.codec.http.FullHttpRequest;
+
+public class GetRequest implements IServletRequest {
+    private final FullHttpRequest request;
+    private final Map<String, List<String>> parameters;
+
+    public GetRequest(FullHttpRequest request, Map<String, List<String>> parameters) {
+        this.request = request;
+        this.parameters = parameters;
+    }
+
+    @Override
+    public FullHttpRequest getHttpRequest() {
+        return request;
+    }
+
+    @Override
+    public String getParameter(CharSequence name) {
+        return IServletRequest.getParameter(parameters, name);
+    }
+
+    @Override
+    public String getHeader(CharSequence name) {
+        return request.headers().get(name);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
new file mode 100644
index 0000000..f7f55bd
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.handler.codec.http.FullHttpRequest;
+import io.netty.handler.logging.LogLevel;
+import io.netty.handler.logging.LoggingHandler;
+
+public class HttpServer {
+    // Constants
+    private static final Logger LOGGER = Logger.getLogger(HttpServer.class.getName());
+    private static final int FAILED = -1;
+    private static final int STOPPED = 0;
+    private static final int STARTING = 1;
+    private static final int STARTED = 2;
+    private static final int STOPPING = 3;
+    // Final members
+    private final Object lock = new Object();
+    private final ConcurrentMap<String, Object> ctx;
+    private final List<IServlet> lets;
+    private final EventLoopGroup bossGroup;
+    private final EventLoopGroup workerGroup;
+    private final int port;
+    // Mutable members
+    private volatile int state = STOPPED;
+    private Channel channel;
+    private Throwable cause;
+
+    public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup,
+            int port) {
+        this.bossGroup = bossGroup;
+        this.workerGroup = workerGroup;
+        this.port = port;
+        ctx = new ConcurrentHashMap<>();
+        lets = new ArrayList<>();
+    }
+
+    public final void start() throws Exception { // NOSONAR
+        synchronized (lock) {
+            try {
+                if (state == STARTED || state == STARTING) {
+                    return;
+                }
+                setStarting();
+                doStart();
+                setStarted();
+            } catch (Throwable e) { // NOSONAR
+                LOGGER.log(Level.SEVERE, "Failure starting an Http Server", e);
+                setFailed(e);
+                throw e;
+            }
+        }
+    }
+
+    public final void stop() throws Exception { // NOSONAR
+        synchronized (lock) {
+            try {
+                if (state == STOPPING || state == STOPPED) {
+                    return;
+                }
+                setStopping();
+                doStop();
+                setStopped();
+            } catch (Throwable e) { // NOSONAR
+                LOGGER.log(Level.SEVERE, "Failure stopping an Http Server", e);
+                setFailed(e);
+                throw e;
+            }
+        }
+    }
+
+    /**
+     * @return String representation of the State for this server
+     */
+    public String getState() {
+        switch (state) {
+            case FAILED:
+                return "FAILED";
+            case STARTING:
+                return "STARTING";
+            case STARTED:
+                return "STARTED";
+            case STOPPING:
+                return "STOPPING";
+            case STOPPED:
+                return "STOPPED";
+            default:
+                return "UNKNOWN";
+        }
+    }
+
+    private void setStarting() {
+        state = STARTING;
+    }
+
+    private void setStarted() {
+        state = STARTED;
+    }
+
+    private void setStopping() {
+        state = STOPPING;
+    }
+
+    private void setStopped() {
+        state = STOPPED;
+    }
+
+    private void setFailed(Throwable th) {
+        state = FAILED;
+        cause = th;
+    }
+
+    public Throwable getCause() {
+        return cause;
+    }
+
+    public void setAttribute(String name, Object value) {
+        ctx.put(name, value);
+    }
+
+    public Object getAttribute(String name) {
+        return ctx.get(name);
+    }
+
+    public ConcurrentMap<String, Object> ctx() {
+        return ctx;
+    }
+
+    public void addLet(IServlet let) {
+        lets.add(let);
+    }
+
+
+    protected void doStart() throws InterruptedException {
+        /*
+         * This is a hacky way to ensure that ILets with more specific paths are checked first.
+         * For example:
+         * "/path/to/resource/"
+         * is checked before
+         * "/path/to/"
+         * which in turn is checked before
+         * "/path/"
+         * Note that it doesn't work for the case where multiple paths map to a single ILet
+         */
+        Collections.sort(lets, (l1, l2) -> l2.getPaths()[0].length() - l1.getPaths()[0].length());
+        ServerBootstrap b = new ServerBootstrap();
+        b.group(bossGroup, workerGroup)
+                .channel(NioServerSocketChannel.class)
+                .handler(new LoggingHandler(LogLevel.INFO))
+                .childHandler(new HttpServerInitializer(this));
+        channel = b.bind(port).sync().channel();
+    }
+
+
+    protected void doStop() throws InterruptedException {
+        channel.close();
+        channel.closeFuture().sync();
+    }
+
+    public IServlet getServlet(FullHttpRequest request) {
+        String uri = request.uri();
+        int i = uri.indexOf('?');
+        if (i >= 0) {
+            uri = uri.substring(0, i);
+        }
+        for (IServlet let : lets) {
+            for (String path : let.getPaths()) {
+                if (match(path, uri)) {
+                    return let;
+                }
+            }
+        }
+        return null;
+    }
+
+    private static boolean match(String pathSpec, String path) {
+        char c = pathSpec.charAt(0);
+        if (c == '/') {
+            if (pathSpec.length() == 1 || pathSpec.equals(path)) {
+                return true;
+            }
+
+            if (isPathWildcardMatch(pathSpec, path)) {
+                return true;
+            }
+        } else if (c == '*') {
+            return path.regionMatches(path.length() - pathSpec.length() + 1,
+                    pathSpec, 1, pathSpec.length() - 1);
+        }
+        return false;
+    }
+    private static boolean isPathWildcardMatch(String pathSpec, String path) {
+        int cpl = pathSpec.length() - 2;
+        return (pathSpec.endsWith("/*") && path.regionMatches(0, pathSpec, 0, cpl))
+                && (path.length() == cpl || '/' == path.charAt(cpl));
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java
new file mode 100644
index 0000000..c8ed937
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.FullHttpRequest;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpUtil;
+import io.netty.handler.codec.http.QueryStringDecoder;
+import io.netty.handler.codec.http.multipart.Attribute;
+import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
+import io.netty.handler.codec.http.multipart.InterfaceHttpData;
+import io.netty.handler.codec.http.multipart.InterfaceHttpData.HttpDataType;
+import io.netty.handler.codec.http.multipart.MixedAttribute;
+
+public class HttpServerHandler extends SimpleChannelInboundHandler<Object> {
+
+    private static final Logger LOGGER = Logger.getLogger(HttpServerHandler.class.getName());
+    protected final HttpServer server;
+
+    public HttpServerHandler(HttpServer server) {
+        this.server = server;
+    }
+
+    @Override
+    public void channelReadComplete(ChannelHandlerContext ctx) {
+        ctx.flush();
+    }
+
+    @Override
+    protected void channelRead0(ChannelHandlerContext ctx, Object msg) {
+        try {
+            FullHttpRequest http = (FullHttpRequest) msg;
+            IServlet servlet = server.getServlet(http);
+            if (servlet == null) {
+                DefaultHttpResponse response = new DefaultHttpResponse(http.protocolVersion(),
+                        HttpResponseStatus.NOT_FOUND);
+                ctx.write(response).addListener(ChannelFutureListener.CLOSE);
+            } else {
+                if (http.method() != HttpMethod.GET && http.method() != HttpMethod.POST) {
+                    DefaultHttpResponse response = new DefaultHttpResponse(http.protocolVersion(),
+                            HttpResponseStatus.METHOD_NOT_ALLOWED);
+                    ctx.write(response).addListener(ChannelFutureListener.CLOSE);
+                    return;
+                }
+                IServletRequest request = http.method() == HttpMethod.GET ? get(http) : post(http);
+                IServletResponse response = new FullResponse(ctx, http);
+                try {
+                    servlet.handle(request, response);
+                } catch (Throwable th) { // NOSONAR
+                    LOGGER.log(Level.WARNING, "Failure during handling of an IServLetRequest", th);
+                    response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+                } finally {
+                    response.close();
+                }
+                ChannelFuture lastContentFuture = response.future();
+                if (!HttpUtil.isKeepAlive(http)) {
+                    lastContentFuture.addListener(ChannelFutureListener.CLOSE);
+                }
+            }
+        } catch (Exception e) {
+            LOGGER.log(Level.SEVERE, "Failure handling HTTP Request", e);
+            ctx.close();
+        }
+    }
+
+    public static IServletRequest post(FullHttpRequest request) throws IOException {
+        List<String> names = new ArrayList<>();
+        List<String> values = new ArrayList<>();
+        HttpPostRequestDecoder decoder = null;
+        try {
+            decoder = new HttpPostRequestDecoder(request);
+        } catch (Exception e) {
+            //ignore. this means that the body of the POST request does not have key value pairs
+            LOGGER.log(Level.WARNING, "Failed to decode a post message. Fix the API not to have queries as POST body",
+                    e);
+        }
+        if (decoder != null) {
+            try {
+                List<InterfaceHttpData> bodyHttpDatas = decoder.getBodyHttpDatas();
+                for (InterfaceHttpData data : bodyHttpDatas) {
+                    if (data.getHttpDataType().equals(HttpDataType.Attribute)) {
+                        Attribute attr = (MixedAttribute) data;
+                        names.add(data.getName());
+                        values.add(attr.getValue());
+                    }
+                }
+            } finally {
+                decoder.destroy();
+            }
+        }
+        return new PostRequest(request, new QueryStringDecoder(request.uri()).parameters(), names, values);
+    }
+
+    public static IServletRequest get(FullHttpRequest request) throws IOException {
+        return new GetRequest(request, new QueryStringDecoder(request.uri()).parameters());
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+        LOGGER.log(Level.SEVERE, "Failure handling HTTP Request", cause);
+        ctx.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
new file mode 100644
index 0000000..3b32ee6
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.handler.codec.http.HttpObjectAggregator;
+import io.netty.handler.codec.http.HttpRequestDecoder;
+import io.netty.handler.codec.http.HttpResponseEncoder;
+
+public class HttpServerInitializer extends ChannelInitializer<SocketChannel> {
+
+    private static final int MAX_CHUNK_SIZE = 262144;
+    private static final int MAX_HEADER_SIZE = 262144;
+    private static final int MAX_INITIAL_LINE_LENGTH = 131072;
+    private HttpServer server;
+
+    public HttpServerInitializer(HttpServer server) {
+        this.server = server;
+    }
+
+    @Override
+    public void initChannel(SocketChannel ch) {
+        ChannelPipeline p = ch.pipeline();
+        p.addLast(new HttpRequestDecoder(MAX_INITIAL_LINE_LENGTH, MAX_HEADER_SIZE, MAX_CHUNK_SIZE));
+        p.addLast(new HttpResponseEncoder());
+        p.addLast(new HttpObjectAggregator(Integer.MAX_VALUE));
+        p.addLast(new HttpServerHandler(server));
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServlet.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServlet.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServlet.java
new file mode 100644
index 0000000..5691fd9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServlet.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Represents a component that handles IServLet requests
+ */
+public interface IServlet {
+
+    public class Encoding {
+        public static final String UTF8 = "utf-8";
+
+        private Encoding() {
+        }
+    }
+
+    public class ContentType {
+        public static final String APPLICATION_ADM = "application/x-adm";
+        public static final String APPLICATION_JSON = "application/json";
+        public static final String CSV = "text/csv";
+        public static final String IMG_PNG = "image/png";
+        public static final String TEXT_HTML = "text/html";
+        public static final String TEXT_PLAIN = "text/plain";
+
+        private ContentType() {
+        }
+
+        /**
+         * Get the mime string representation from the extension
+         * @param extension
+         * @return
+         */
+        public static String mime(String extension) {
+            switch (extension) {
+                case ".png":
+                    return "image/png";
+                case ".eot":
+                    return "application/vnd.ms-fontobject";
+                case ".svg":
+                    return "image/svg+xml\t";
+                case ".ttf":
+                    return "application/x-font-ttf";
+                case ".woff":
+                case ".woff2":
+                    return "application/x-font-woff";
+                case ".html":
+                    return "text/html";
+                case ".css":
+                    return "text/css";
+                case ".js":
+                    return "application/javascript";
+                default:
+                    return null;
+            }
+        }
+    }
+
+    /**
+     * @return an array of paths associated with this IServLet
+     */
+    String[] getPaths();
+
+    /**
+     * @return the context associated with this IServlet
+     */
+    ConcurrentMap<String, Object> ctx();
+
+    /**
+     * handle the IServLetRequest writing the response in the passed IServLetResponse
+     * @param request
+     * @param response
+     */
+    void handle(IServletRequest request, IServletResponse response);
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServletRequest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServletRequest.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServletRequest.java
new file mode 100644
index 0000000..8aebd07
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServletRequest.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.util.List;
+import java.util.Map;
+
+import io.netty.handler.codec.http.FullHttpRequest;
+
+/**
+ * An Http Request instance
+ */
+public interface IServletRequest {
+    /**
+     * @return the full http request
+     */
+    FullHttpRequest getHttpRequest();
+
+    /**
+     * Get a request parameter
+     * @param name
+     * @return the parameter or null if not found
+     */
+    String getParameter(CharSequence name);
+
+    /**
+     * Get a request header
+     * @param name
+     * @return the header or null if not found
+     */
+    String getHeader(CharSequence name);
+
+    static String getParameter(Map<String, List<String>> parameters, CharSequence name) {
+        List<String> parameter = parameters.get(name);
+        if (parameter == null) {
+            return null;
+        } else if (parameter.size() == 1) {
+            return parameter.get(0);
+        } else {
+            StringBuilder aString = new StringBuilder(parameter.get(0));
+            for (int i = 1; i < parameter.size(); i++) {
+                aString.append(",").append(parameter.get(i));
+            }
+            return aString.toString();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServletResponse.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServletResponse.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServletResponse.java
new file mode 100644
index 0000000..342e643
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/IServletResponse.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+
+import io.netty.channel.ChannelFuture;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+/**
+ * A response to an instance of IServLetRequest
+ */
+public interface IServletResponse extends Closeable {
+
+    /**
+     * Set a response header
+     * @param name
+     * @param value
+     * @return
+     * @throws Exception
+     */
+    IServletResponse setHeader(CharSequence name, Object value) throws IOException;
+
+    /**
+     * Get the output writer for the response
+     * @return
+     * @throws Exception
+     */
+    PrintWriter writer();
+
+    /**
+     * Send the last http response if any and return the future
+     * @return
+     * @throws Exception
+     */
+    ChannelFuture future() throws IOException;
+
+    /**
+     * Set the status of the http response
+     * @param status
+     */
+    void setStatus(HttpResponseStatus status);
+
+    /**
+     * Get the output stream for the response
+     * @return
+     */
+    OutputStream outputStream();
+
+    public static void setContentType(IServletResponse response, String type, String charset) throws IOException {
+        response.setHeader(HttpHeaderNames.CONTENT_TYPE, type + "; charset=" + charset);
+    }
+
+    public static void setContentType(IServletResponse response, String type) throws IOException {
+        response.setHeader(HttpHeaderNames.CONTENT_TYPE, type);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/PostRequest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/PostRequest.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/PostRequest.java
new file mode 100644
index 0000000..99f338c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/PostRequest.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.util.List;
+import java.util.Map;
+
+import io.netty.handler.codec.http.FullHttpRequest;
+
+public class PostRequest implements IServletRequest {
+    private final FullHttpRequest request;
+    private final List<String> names;
+    private final List<String> values;
+    private final Map<String, List<String>> parameters;
+
+    public PostRequest(FullHttpRequest request, Map<String, List<String>> parameters, List<String> names,
+            List<String> values) {
+        this.request = request;
+        this.parameters = parameters;
+        this.names = names;
+        this.values = values;
+    }
+
+    @Override
+    public FullHttpRequest getHttpRequest() {
+        return request;
+    }
+
+    @Override
+    public String getParameter(CharSequence name) {
+        for (int i = 0; i < names.size(); i++) {
+            if (name.equals(names.get(i))) {
+                return values.get(i);
+            }
+        }
+        return IServletRequest.getParameter(parameters, name);
+    }
+
+    @Override
+    public String getHeader(CharSequence name) {
+        return request.headers().get(name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/hyracks-fullstack/hyracks/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/pom.xml b/hyracks-fullstack/hyracks/pom.xml
index e2d5a95..fc7d8b5 100644
--- a/hyracks-fullstack/hyracks/pom.xml
+++ b/hyracks-fullstack/hyracks/pom.xml
@@ -130,5 +130,6 @@
     <module>hyracks-maven-plugins</module>
     <module>hyracks-hdfs</module>
     <module>hyracks-dist</module>
+    <module>hyracks-http</module>
   </modules>
 </project>


[3/5] asterixdb git commit: Replace Servlets with Netty Based HTTP Servers

Posted by ti...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterNodeDetailsAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterNodeDetailsAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterNodeDetailsAPIServlet.java
deleted file mode 100644
index d737ad8..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterNodeDetailsAPIServlet.java
+++ /dev/null
@@ -1,215 +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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import javax.servlet.ServletContext;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.SerializationFeature;
-import org.apache.asterix.runtime.util.ClusterStateManager;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-public class ClusterNodeDetailsAPIServlet extends ClusterAPIServlet {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(ClusterNodeDetailsAPIServlet.class.getName());
-    private final ObjectMapper om = new ObjectMapper();
-
-    @Override
-    public void init() throws ServletException{
-        om.enable(SerializationFeature.INDENT_OUTPUT);
-    }
-
-    @Override
-    protected void getUnsafe(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        PrintWriter responseWriter = response.getWriter();
-        ServletContext context = getServletContext();
-        IHyracksClientConnection hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
-        try {
-            ObjectNode json;
-            if (request.getPathInfo() == null) {
-                json = om.createObjectNode();
-                json.set("ncs", getClusterStateJSON(request, "../").get("ncs"));
-            } else {
-                json = processNode(request, hcc);
-            }
-            response.setStatus(HttpServletResponse.SC_OK);
-            response.setContentType("application/json");
-            response.setCharacterEncoding("utf-8");
-            responseWriter.write(om.writerWithDefaultPrettyPrinter().writeValueAsString(json));
-        } catch (IllegalStateException e) { // NOSONAR - exception not logged or rethrown
-            response.sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
-        } catch (IllegalArgumentException e) { // NOSONAR - exception not logged or rethrown
-            response.sendError(HttpServletResponse.SC_NOT_FOUND);
-        } catch (Exception e) {
-            LOGGER.log(Level.INFO, "exception thrown for " + request, e);
-            response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, e.toString());
-        }
-        responseWriter.flush();
-    }
-
-    private ObjectNode processNode(HttpServletRequest request, IHyracksClientConnection hcc)
-            throws Exception {
-        String pathInfo = request.getPathInfo();
-        if (pathInfo.endsWith("/")) {
-            throw new IllegalArgumentException();
-        }
-        String[] parts = pathInfo.substring(1).split("/");
-        final String node = parts[0];
-
-        if (parts.length == 1) {
-            ArrayNode ncs = (ArrayNode) getClusterStateJSON(request, "../../").get("ncs");
-            for (int i = 0; i < ncs.size(); i++) {
-                if (node.equals(ncs.get(i).get("node_id").asText())) {
-                    return (ObjectNode) ncs.get(i);
-                }
-            }
-            if ("cc".equals(node)) {
-                return om.createObjectNode();
-            }
-            throw new IllegalArgumentException();
-        } else if (parts.length == 2) {
-            ObjectNode json;
-
-            switch (parts[1]) {
-                case "config":
-                    json = processNodeConfig(hcc, node);
-                    break;
-
-                case "stats":
-                    json = processNodeStats(hcc, node);
-                    break;
-
-                case "threaddump":
-                    return processNodeThreadDump(hcc, node);
-
-                default:
-                    throw new IllegalArgumentException();
-            }
-            fixupKeys(json);
-
-            return json;
-        } else {
-            throw new IllegalArgumentException();
-        }
-    }
-
-    protected ObjectNode fixupKeys(ObjectNode json)  {
-        // TODO (mblow): generate the keys with _ to begin with
-        List<String> keys = new ArrayList<>();
-        for (Iterator iter = json.fieldNames(); iter.hasNext(); ) {
-            keys.add((String) iter.next());
-        }
-        for (String key : keys) {
-            String newKey = key.replace('-', '_');
-            if (!newKey.equals(key)) {
-                json.set(newKey, json.remove(key));
-            }
-        }
-        return json;
-    }
-
-    protected ObjectNode processNodeStats(IHyracksClientConnection hcc, String node) throws Exception {
-        final String details = hcc.getNodeDetailsJSON(node, true, false);
-        if (details == null) {
-            throw new IllegalArgumentException();
-        }
-        ObjectNode json = (ObjectNode) om.readTree(details);
-        int index = json.get("rrd-ptr").asInt() - 1;
-        json.remove("rrd-ptr");
-
-        List<String> keys = new ArrayList<>();
-        for (Iterator iter = json.fieldNames(); iter.hasNext(); ) {
-            keys.add((String) iter.next());
-        }
-
-        final ArrayNode gcNames = (ArrayNode) json.get("gc-names");
-        final ArrayNode gcCollectionTimes  = (ArrayNode) json.get("gc-collection-times");
-        final ArrayNode gcCollectionCounts = (ArrayNode) json.get("gc-collection-counts");
-
-        for (String key : keys) {
-            if (key.startsWith("gc-")) {
-                json.remove(key);
-            } else {
-                final JsonNode keyNode = json.get(key);
-                if (keyNode instanceof ArrayNode) {
-                    final ArrayNode valueArray = (ArrayNode) keyNode;
-                    // fixup an index of -1 to the final element in the array (i.e. RRD_SIZE)
-                    if (index == -1) {
-                        index = valueArray.size() - 1;
-                    }
-                    final JsonNode value = valueArray.get(index);
-                    json.remove(key);
-                    json.set(key.replaceAll("s$",""), value);
-                }
-            }
-        }
-        ArrayNode gcs = om.createArrayNode();
-
-        for (int i = 0; i < gcNames.size(); i++) {
-            ObjectNode gc = om.createObjectNode();
-            gc.set("name", gcNames.get(i));
-            gc.set("collection-time", ((ArrayNode)gcCollectionTimes.get(i)).get(index));
-            gc.set("collection-count", ((ArrayNode)gcCollectionCounts.get(i)).get(index));
-            gcs.add(gc);
-        }
-        json.set("gcs", gcs);
-
-        return json;
-    }
-
-    private ObjectNode processNodeConfig(IHyracksClientConnection hcc, String node) throws Exception {
-        String config = hcc.getNodeDetailsJSON(node, false, true);
-        if (config == null) {
-            throw new IllegalArgumentException();
-        }
-        return (ObjectNode) om.readTree(config);
-    }
-
-    private ObjectNode processNodeThreadDump(IHyracksClientConnection hcc, String node) throws Exception {
-        if ("cc".equals(node)) {
-            return om.createObjectNode();
-        }
-        String dump = hcc.getThreadDump(node);
-        if (dump == null) {
-            // check to see if this is a node that is simply down
-            throw ClusterStateManager.INSTANCE.getNodePartitions(node) != null
-                    ? new IllegalStateException()
-                    : new IllegalArgumentException();
-        }
-        return (ObjectNode) om.readTree(dump);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java
deleted file mode 100644
index 78801b8..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java
+++ /dev/null
@@ -1,181 +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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.List;
-import java.util.Map;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.metadata.MetadataTransactionContext;
-import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.utils.DatasetUtils;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.util.FlushDatasetUtils;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.client.NodeControllerInfo;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.hyracks.api.io.FileSplit;
-
-/***
- * The REST API that takes a dataverse name and a dataset name as the input
- * and returns an array of file splits (IP, file-path) of the dataset in LOSSLESS_JSON.
- * It is mostly used by external runtime, e.g., Pregelix or IMRU to pull data
- * in parallel from existing AsterixDB datasets.
- *
- * @author yingyi
- */
-public class ConnectorAPIServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        response.setContentType("text/html");
-        response.setCharacterEncoding("utf-8");
-        PrintWriter out = response.getWriter();
-        try {
-            ObjectMapper om = new ObjectMapper();
-            ObjectNode jsonResponse = om.createObjectNode();
-            String dataverseName = request.getParameter("dataverseName");
-            String datasetName = request.getParameter("datasetName");
-            if (dataverseName == null || datasetName == null) {
-                jsonResponse.put("error", "Parameter dataverseName or datasetName is null,");
-                out.write(jsonResponse.toString());
-                out.flush();
-                return;
-            }
-            ServletContext context = getServletContext();
-
-            IHyracksClientConnection hcc = null;
-            synchronized (context) {
-                hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
-            }
-
-            // Metadata transaction begins.
-            MetadataManager.INSTANCE.init();
-            MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-
-            // Retrieves file splits of the dataset.
-            MetadataProvider metadataProvider = new MetadataProvider(null);
-            metadataProvider.setMetadataTxnContext(mdTxnCtx);
-            Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
-            if (dataset == null) {
-                jsonResponse.put("error",
-                        "Dataset " + datasetName + " does not exist in " + "dataverse " + dataverseName);
-                out.write(jsonResponse.toString());
-                out.flush();
-                return;
-            }
-            boolean temp = dataset.getDatasetDetails().isTemp();
-            FileSplit[] fileSplits = metadataProvider.splitsForDataset(mdTxnCtx, dataverseName, datasetName,
-                    datasetName, temp);
-            ARecordType recordType = (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(),
-                    dataset.getItemTypeName());
-            List<List<String>> primaryKeys = DatasetUtils.getPartitioningKeys(dataset);
-            StringBuilder pkStrBuf = new StringBuilder();
-            for (List<String> keys : primaryKeys) {
-                for (String key : keys) {
-                    pkStrBuf.append(key).append(",");
-                }
-            }
-            pkStrBuf.delete(pkStrBuf.length() - 1, pkStrBuf.length());
-
-            // Constructs the returned json object.
-            formResponseObject(jsonResponse, fileSplits, recordType, pkStrBuf.toString(), temp,
-                    hcc.getNodeControllerInfos());
-
-            // Flush the cached contents of the dataset to file system.
-            FlushDatasetUtils.flushDataset(hcc, metadataProvider, mdTxnCtx, dataverseName, datasetName, datasetName);
-
-            // Metadata transaction commits.
-            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-            // Writes file splits.
-            out.write(jsonResponse.toString());
-            out.flush();
-        } catch (Exception e) {
-            e.printStackTrace();
-            out.println(e.getMessage());
-            out.flush();
-            e.printStackTrace(out);
-        }
-    }
-
-    private void formResponseObject(ObjectNode jsonResponse, FileSplit[] fileSplits, ARecordType recordType,
-            String primaryKeys, boolean temp, Map<String, NodeControllerInfo> nodeMap) throws Exception {
-        ObjectMapper om = new ObjectMapper();
-        ArrayNode partititons = om.createArrayNode();
-        // Whether the dataset is temp or not
-        jsonResponse.put("temp", temp);
-        // Adds a primary key.
-        jsonResponse.put("keys", primaryKeys);
-        // Adds record type.
-        jsonResponse.set("type", recordType.toJSON());
-        // Generates file partitions.
-        for (FileSplit split : fileSplits) {
-            String ipAddress = nodeMap.get(split.getNodeName()).getNetworkAddress().getAddress().toString();
-            String path = split.getPath();
-            FilePartition partition = new FilePartition(ipAddress, path);
-            partititons.add(partition.toObjectNode());
-        }
-        // Generates the response object which contains the splits.
-        jsonResponse.set("splits", partititons);
-    }
-}
-
-class FilePartition {
-    private final String ipAddress;
-    private final String path;
-
-    public FilePartition(String ipAddress, String path) {
-        this.ipAddress = ipAddress;
-        this.path = path;
-    }
-
-    public String getIPAddress() {
-        return ipAddress;
-    }
-
-    public String getPath() {
-        return path;
-    }
-
-    @Override
-    public String toString() {
-        return ipAddress + ":" + path;
-    }
-
-    public ObjectNode toObjectNode() {
-        ObjectMapper om = new ObjectMapper();
-        ObjectNode partition = om.createObjectNode();
-        partition.put("ip", ipAddress);
-        partition.put("path", path);
-        return partition;
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DDLAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DDLAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DDLAPIServlet.java
deleted file mode 100644
index 19a7609..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DDLAPIServlet.java
+++ /dev/null
@@ -1,51 +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.api.http.servlet;
-
-import javax.servlet.http.HttpServletRequest;
-
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-
-public class DDLAPIServlet extends RESTAPIServlet {
-    private static final long serialVersionUID = 1L;
-    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE
-            | Statement.Category.DDL;
-
-    public DDLAPIServlet(ILangCompilationProvider compilationProvider,
-            IStatementExecutorFactory statementExecutorFactory) {
-        super(compilationProvider, statementExecutorFactory);
-    }
-
-    @Override
-    protected String getQueryParameter(HttpServletRequest request) {
-        return request.getParameter("ddl");
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        return "Invalid statement: Non-DDL statement %s to the DDL API.";
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DiagnosticsAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DiagnosticsAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DiagnosticsAPIServlet.java
deleted file mode 100644
index 4416d5c..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/DiagnosticsAPIServlet.java
+++ /dev/null
@@ -1,118 +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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.SerializationFeature;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-public class DiagnosticsAPIServlet extends ClusterNodeDetailsAPIServlet {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(DiagnosticsAPIServlet.class.getName());
-
-    @Override
-    protected void getUnsafe(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        response.setContentType("application/json");
-        response.setCharacterEncoding("utf-8");
-        PrintWriter responseWriter = response.getWriter();
-        ObjectNode json;
-        ObjectMapper om = new ObjectMapper();
-        om.enable(SerializationFeature.INDENT_OUTPUT);
-        try {
-            if (request.getPathInfo() != null) {
-                throw new IllegalArgumentException();
-            }
-            json = getClusterDiagnosticsJSON();
-            response.setStatus(HttpServletResponse.SC_OK);
-            responseWriter.write(om.writerWithDefaultPrettyPrinter().writeValueAsString(json));
-        } catch (IllegalStateException e) { // NOSONAR - exception not logged or rethrown
-            response.sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
-        } catch (IllegalArgumentException e) { // NOSONAR - exception not logged or rethrown
-            response.sendError(HttpServletResponse.SC_NOT_FOUND);
-        } catch (Exception e) {
-            LOGGER.log(Level.INFO, "exception thrown for " + request, e);
-            response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, e.toString());
-        }
-        responseWriter.flush();
-    }
-
-    private ObjectNode getClusterDiagnosticsJSON() throws Exception {
-        ObjectMapper om = new ObjectMapper();
-        final ServletContext context = getServletContext();
-        IHyracksClientConnection hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
-        ExecutorService executor = (ExecutorService) context.getAttribute(ServletConstants.EXECUTOR_SERVICE);
-
-        Map<String, Future<ObjectNode>> ccFutureData = new HashMap<>();
-        ccFutureData.put("threaddump",
-                executor.submit(() -> fixupKeys((ObjectNode) om.readTree(hcc.getThreadDump(null)))));
-        ccFutureData.put("config",
-                executor.submit(() -> fixupKeys((ObjectNode) om.readTree(hcc.getNodeDetailsJSON(null, false, true)))));
-        ccFutureData.put("stats",
-                executor.submit(() -> fixupKeys((ObjectNode) om.readTree(hcc.getNodeDetailsJSON(null, true, false)))));
-
-        Map<String, Map<String, Future<ObjectNode>>> ncDataMap = new HashMap<>();
-        for (String nc : AppContextInfo.INSTANCE.getMetadataProperties().getNodeNames()) {
-            Map<String, Future<ObjectNode>> ncData = new HashMap<>();
-            ncData.put("threaddump", executor.submit(() -> fixupKeys((ObjectNode) om.readTree(hcc.getThreadDump(nc)))));
-            ncData.put("config", executor
-                    .submit(() -> fixupKeys((ObjectNode) om.readTree(hcc.getNodeDetailsJSON(nc, false, true)))));
-            ncData.put("stats", executor.submit(() -> fixupKeys(processNodeStats(hcc, nc))));
-            ncDataMap.put(nc, ncData);
-        }
-        ObjectNode result = om.createObjectNode();
-        result.putPOJO("cc", resolveFutures(ccFutureData));
-        List<Map<String, ?>> ncList = new ArrayList<>();
-        for (Map.Entry<String, Map<String, Future<ObjectNode>>> entry : ncDataMap.entrySet()) {
-            final Map<String, Object> ncMap = resolveFutures(entry.getValue());
-            ncMap.put("node_id", entry.getKey());
-            ncList.add(ncMap);
-        }
-        result.putPOJO("ncs", ncList);
-        result.putPOJO("date", new Date());
-        return result;
-    }
-
-    private Map<String, Object> resolveFutures(Map<String, Future<ObjectNode>> futureMap)
-            throws ExecutionException, InterruptedException {
-        Map<String, Object> result = new HashMap<>();
-        for (Map.Entry<String, Future<ObjectNode>> entry : futureMap.entrySet()) {
-            result.put(entry.getKey(), entry.getValue().get());
-        }
-        return result;
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java
deleted file mode 100644
index 6c8019d..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java
+++ /dev/null
@@ -1,85 +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.api.http.servlet;
-
-import java.awt.image.BufferedImage;
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.PrintWriter;
-
-import javax.imageio.ImageIO;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-public class FeedServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        String resourcePath = null;
-        String requestURI = request.getRequestURI();
-
-        if (requestURI.equals("/")) {
-            response.setContentType("text/html");
-            resourcePath = "/feed/home.html";
-        } else {
-            resourcePath = requestURI;
-        }
-
-        InputStream is = FeedServlet.class.getResourceAsStream(resourcePath);
-        if (is == null) {
-            response.sendError(HttpServletResponse.SC_NOT_FOUND);
-            return;
-        }
-
-        // Special handler for font files and .png resources
-        if (resourcePath.endsWith(".png")) {
-
-            BufferedImage img = ImageIO.read(is);
-            OutputStream outputStream = response.getOutputStream();
-            String formatName = "png";
-            response.setContentType("image/png");
-            ImageIO.write(img, formatName, outputStream);
-            outputStream.close();
-            return;
-
-        }
-
-        response.setCharacterEncoding("utf-8");
-        InputStreamReader isr = new InputStreamReader(is);
-        StringBuilder sb = new StringBuilder();
-        BufferedReader br = new BufferedReader(isr);
-        String line = br.readLine();
-
-        while (line != null) {
-            sb.append(line + "\n");
-            line = br.readLine();
-        }
-
-        String outStr = null;
-        outStr = sb.toString();
-
-        PrintWriter out = response.getWriter();
-        out.println(outStr);
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FullAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FullAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FullAPIServlet.java
deleted file mode 100644
index 0f86ec6..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FullAPIServlet.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.api.http.servlet;
-
-import javax.servlet.http.HttpServletRequest;
-
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-
-public class FullAPIServlet extends RESTAPIServlet {
-
-    private static final long serialVersionUID = 1L;
-    private static final String AQL_STMT_PARAM_NAME = "aql";
-    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE
-            | Statement.Category.DDL | Statement.Category.PROCEDURE;
-
-    public FullAPIServlet(ILangCompilationProvider compilationProvider,
-            IStatementExecutorFactory statementExecutorFactory) {
-        super(compilationProvider, statementExecutorFactory);
-    }
-
-    @Override
-    protected String getQueryParameter(HttpServletRequest request) {
-        return request.getParameter(AQL_STMT_PARAM_NAME);
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        throw new IllegalStateException();
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryAPIServlet.java
deleted file mode 100644
index cc70324..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryAPIServlet.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.api.http.servlet;
-
-import javax.servlet.http.HttpServletRequest;
-
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-
-public class QueryAPIServlet extends RESTAPIServlet {
-    private static final long serialVersionUID = 1L;
-    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY;
-
-    public QueryAPIServlet(ILangCompilationProvider compilationProvider,
-            IStatementExecutorFactory queryTranslatorFactory) {
-        super(compilationProvider, queryTranslatorFactory);
-    }
-
-    @Override
-    protected String getQueryParameter(HttpServletRequest request) {
-        return request.getParameter("query");
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        return "Invalid statement: Non-query statement %s to the query API.";
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryResultAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryResultAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryResultAPIServlet.java
deleted file mode 100644
index b250690..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryResultAPIServlet.java
+++ /dev/null
@@ -1,114 +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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.asterix.app.result.ResultReader;
-import org.apache.asterix.app.result.ResultUtil;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.translator.IStatementExecutor.Stats;
-import org.apache.asterix.translator.SessionConfig;
-import org.apache.hyracks.api.client.HyracksConnection;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.client.dataset.HyracksDataset;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-public class QueryResultAPIServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(QueryResultAPIServlet.class.getName());
-
-    @Override
-    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        int respCode = HttpServletResponse.SC_OK;
-        response.setContentType("text/html"); // TODO this seems wrong ...
-        response.setCharacterEncoding("utf-8");
-        String strHandle = request.getParameter("handle");
-        PrintWriter out = response.getWriter();
-        ServletContext context = getServletContext();
-        IHyracksClientConnection hcc;
-        IHyracksDataset hds;
-
-        try {
-            if (strHandle == null || strHandle.isEmpty()) {
-                throw new AsterixException("Empty request, no handle provided");
-            }
-
-            HyracksProperties hp = new HyracksProperties();
-            String strIP = hp.getHyracksIPAddress();
-            int port = hp.getHyracksPort();
-
-            synchronized (context) {
-                hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
-                if (hcc == null) {
-                    hcc = new HyracksConnection(strIP, port);
-                    context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
-                }
-
-                hds = (IHyracksDataset) context.getAttribute(HYRACKS_DATASET_ATTR);
-                if (hds == null) {
-                    hds = new HyracksDataset(hcc, ResultReader.FRAME_SIZE, ResultReader.NUM_READERS);
-                    context.setAttribute(HYRACKS_DATASET_ATTR, hds);
-                }
-            }
-            ObjectMapper om = new ObjectMapper();
-            ObjectNode handleObj = (ObjectNode) om.readTree(strHandle);
-            ArrayNode handle = (ArrayNode) handleObj.get("handle");
-            JobId jobId = new JobId(handle.get(0).asLong());
-            ResultSetId rsId = new ResultSetId(handle.get(1).asLong());
-
-            ResultReader resultReader = new ResultReader(hds);
-            resultReader.open(jobId, rsId);
-
-            // QQQ The output format is determined by the initial
-            // query and cannot be modified here, so calling back to
-            // initResponse() is really an error. We need to find a
-            // way to send the same OutputFormat value here as was
-            // originally determined there. Need to save this value on
-            // some object that we can obtain here.
-            SessionConfig sessionConfig = RESTAPIServlet.initResponse(request, response);
-            ResultUtil.printResults(resultReader, sessionConfig, new Stats(), null);
-
-        } catch (Exception e) {
-            respCode = HttpServletResponse.SC_BAD_REQUEST;
-            out.println(e.getMessage());
-            LOGGER.log(Level.WARNING, "Error retrieving result", e);
-        }
-        response.setStatus(respCode);
-        if (out.checkError()) {
-            LOGGER.warning("Error flushing output writer");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryServiceServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryServiceServlet.java
deleted file mode 100644
index a0454fd..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryServiceServlet.java
+++ /dev/null
@@ -1,571 +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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-import java.util.UUID;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import javax.servlet.ServletContext;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.core.JsonParseException;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.JsonMappingException;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.asterix.app.result.ResultReader;
-import org.apache.asterix.app.result.ResultUtil;
-import org.apache.asterix.app.translator.QueryTranslator;
-import org.apache.asterix.common.api.IClusterManagementWork;
-import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.utils.JSONUtil;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.aql.parser.TokenMgrError;
-import org.apache.asterix.lang.common.base.IParser;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.runtime.util.ClusterStateManager;
-import org.apache.asterix.translator.IStatementExecutor;
-import org.apache.asterix.translator.IStatementExecutor.Stats;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.asterix.translator.SessionConfig;
-import org.apache.commons.io.IOUtils;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.client.dataset.HyracksDataset;
-
-public class QueryServiceServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-
-    private static final Logger LOGGER = Logger.getLogger(QueryServiceServlet.class.getName());
-    private final transient ILangCompilationProvider compilationProvider;
-    private final transient IStatementExecutorFactory statementExecutorFactory;
-
-    public QueryServiceServlet(ILangCompilationProvider compilationProvider,
-            IStatementExecutorFactory statementExecutorFactory) {
-        this.compilationProvider = compilationProvider;
-        this.statementExecutorFactory = statementExecutorFactory;
-    }
-
-    public enum Parameter {
-        STATEMENT("statement"),
-        FORMAT("format"),
-        CLIENT_ID("client_context_id"),
-        PRETTY("pretty"),
-        MODE("mode");
-
-        private final String str;
-
-        Parameter(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
-    private enum MediaType {
-        CSV("text/csv"),
-        JSON("application/json"),
-        ADM("application/x-adm");
-
-        private final String str;
-
-        MediaType(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
-    private enum Attribute {
-        HEADER("header"),
-        LOSSLESS("lossless");
-
-        private final String str;
-
-        Attribute(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
-    public enum ResultFields {
-        REQUEST_ID("requestID"),
-        CLIENT_ID("clientContextID"),
-        SIGNATURE("signature"),
-        TYPE("type"),
-        STATUS("status"),
-        RESULTS("results"),
-        HANDLE("handle"),
-        ERRORS("errors"),
-        METRICS("metrics");
-
-        private final String str;
-
-        ResultFields(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
-    public enum ResultStatus {
-        SUCCESS("success"),
-        TIMEOUT("timeout"),
-        ERRORS("errors"),
-        FATAL("fatal");
-
-        private final String str;
-
-        ResultStatus(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
-    private enum ErrorField {
-        CODE("code"),
-        MSG("msg"),
-        STACK("stack");
-
-        private final String str;
-
-        ErrorField(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
-    private enum Metrics {
-        ELAPSED_TIME("elapsedTime"),
-        EXECUTION_TIME("executionTime"),
-        RESULT_COUNT("resultCount"),
-        RESULT_SIZE("resultSize");
-
-        private final String str;
-
-        Metrics(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
-    enum TimeUnit {
-        SEC("s", 9),
-        MILLI("ms", 6),
-        MICRO("�s", 3),
-        NANO("ns", 0);
-
-        String unit;
-        int nanoDigits;
-
-        TimeUnit(String unit, int nanoDigits) {
-            this.unit = unit;
-            this.nanoDigits = nanoDigits;
-        }
-
-        static String formatNanos(long nanoTime) {
-            final String strTime = String.valueOf(nanoTime);
-            final int len = strTime.length();
-            for (TimeUnit tu : TimeUnit.values()) {
-                if (len > tu.nanoDigits) {
-                    final String integer = strTime.substring(0, len - tu.nanoDigits);
-                    final String fractional = strTime.substring(len - tu.nanoDigits);
-                    return integer + (fractional.length() > 0 ? "." + fractional : "") + tu.unit;
-                }
-            }
-            return "illegal string value: " + strTime;
-        }
-    }
-
-    static class RequestParameters {
-        String statement;
-        String format;
-        boolean pretty;
-        String clientContextID;
-        String mode;
-
-        @Override
-        public String toString() {
-            return append(new StringBuilder()).toString();
-        }
-
-        public StringBuilder append(final StringBuilder sb) {
-            sb.append("{ ");
-            sb.append("\"statement\": \"");
-            JSONUtil.escape(sb, statement);
-            sb.append("\", ");
-            sb.append("\"format\": \"").append(format).append("\", ");
-            sb.append("\"pretty\": ").append(pretty).append(", ");
-            sb.append("\"mode\": ").append(mode).append(", ");
-            sb.append("\"clientContextID\": \"").append(clientContextID).append("\" ");
-            sb.append('}');
-            return sb;
-        }
-    }
-
-    private static String getParameterValue(String content, String attribute) {
-        if (content == null || attribute == null) {
-            return null;
-        }
-        int sc = content.indexOf(';');
-        if (sc < 0) {
-            return null;
-        }
-        int eq = content.indexOf('=', sc + 1);
-        if (eq < 0) {
-            return null;
-        }
-        if (content.substring(sc + 1, eq).trim().equalsIgnoreCase(attribute)) {
-            return content.substring(eq + 1).trim().toLowerCase();
-        }
-        return null;
-    }
-
-    private static String toLower(String s) {
-        return s != null ? s.toLowerCase() : s;
-    }
-
-    private static SessionConfig.OutputFormat getFormat(String format) {
-        if (format != null) {
-            if (format.startsWith(MediaType.CSV.str())) {
-                return SessionConfig.OutputFormat.CSV;
-            }
-            if (format.equals(MediaType.ADM.str())) {
-                return SessionConfig.OutputFormat.ADM;
-            }
-            if (format.startsWith(MediaType.JSON.str())) {
-                return Boolean.parseBoolean(getParameterValue(format, Attribute.LOSSLESS.str()))
-                        ? SessionConfig.OutputFormat.LOSSLESS_JSON : SessionConfig.OutputFormat.CLEAN_JSON;
-            }
-        }
-        return SessionConfig.OutputFormat.CLEAN_JSON;
-    }
-
-    private static SessionConfig createSessionConfig(RequestParameters param, PrintWriter resultWriter) {
-        SessionConfig.ResultDecorator resultPrefix = new SessionConfig.ResultDecorator() {
-            int resultNo = -1;
-
-            @Override
-            public AlgebricksAppendable append(AlgebricksAppendable app) throws AlgebricksException {
-                app.append("\t\"");
-                app.append(ResultFields.RESULTS.str());
-                if (resultNo >= 0) {
-                    app.append('-').append(String.valueOf(resultNo));
-                }
-                ++resultNo;
-                app.append("\": ");
-                return app;
-            }
-        };
-
-
-        SessionConfig.ResultDecorator resultPostfix = (AlgebricksAppendable app) -> app.append("\t,\n");
-
-        SessionConfig.ResultDecorator handlePrefix = new SessionConfig.ResultDecorator() {
-            @Override
-            public AlgebricksAppendable append(AlgebricksAppendable app) throws AlgebricksException {
-                app.append("\t\"");
-                app.append(ResultFields.HANDLE.str());
-                app.append("\": ");
-                return app;
-            }
-        };
-
-        SessionConfig.ResultDecorator handlePostfix = (AlgebricksAppendable app) -> app.append(",\n");
-
-        SessionConfig.OutputFormat format = getFormat(param.format);
-        SessionConfig sessionConfig = new SessionConfig(resultWriter, format, resultPrefix, resultPostfix, handlePrefix,
-                handlePostfix);
-        sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, true);
-        sessionConfig.set(SessionConfig.FORMAT_INDENT_JSON, param.pretty);
-        sessionConfig.set(SessionConfig.FORMAT_QUOTE_RECORD,
-                format != SessionConfig.OutputFormat.CLEAN_JSON && format != SessionConfig.OutputFormat.LOSSLESS_JSON);
-        sessionConfig.set(SessionConfig.FORMAT_CSV_HEADER, format == SessionConfig.OutputFormat.CSV
-                && "present".equals(getParameterValue(param.format, Attribute.HEADER.str())));
-        return sessionConfig;
-    }
-
-    private static void printField(PrintWriter pw, String name, String value) {
-        printField(pw, name, value, true);
-    }
-
-    private static void printField(PrintWriter pw, String name, String value, boolean comma) {
-        pw.print("\t\"");
-        pw.print(name);
-        pw.print("\": \"");
-        pw.print(value);
-        pw.print('"');
-        if (comma) {
-            pw.print(',');
-        }
-        pw.print('\n');
-    }
-
-    private static UUID printRequestId(PrintWriter pw) {
-        UUID requestId = UUID.randomUUID();
-        printField(pw, ResultFields.REQUEST_ID.str(), requestId.toString());
-        return requestId;
-    }
-
-    private static void printClientContextID(PrintWriter pw, RequestParameters params) {
-        if (params.clientContextID != null && !params.clientContextID.isEmpty()) {
-            printField(pw, ResultFields.CLIENT_ID.str(), params.clientContextID);
-        }
-    }
-
-    private static void printSignature(PrintWriter pw) {
-        printField(pw, ResultFields.SIGNATURE.str(), "*");
-    }
-
-    private static void printType(PrintWriter pw, SessionConfig sessionConfig) {
-        switch (sessionConfig.fmt()) {
-            case ADM:
-                printField(pw, ResultFields.TYPE.str(), MediaType.ADM.str());
-                break;
-            case CSV:
-                String contentType = MediaType.CSV.str() + "; header="
-                        + (sessionConfig.is(SessionConfig.FORMAT_CSV_HEADER) ? "present" : "absent");
-                printField(pw, ResultFields.TYPE.str(), contentType);
-                break;
-            default:
-                break;
-        }
-    }
-
-    private static void printStatus(PrintWriter pw, ResultStatus rs) {
-        printField(pw, ResultFields.STATUS.str(), rs.str());
-    }
-
-    private static void printError(PrintWriter pw, Throwable e) throws JsonProcessingException {
-        Throwable rootCause = ResultUtil.getRootCause(e);
-        if (rootCause == null) {
-            rootCause = e;
-        }
-        final boolean addStack = false;
-        pw.print("\t\"");
-        pw.print(ResultFields.ERRORS.str());
-        pw.print("\": [{ \n");
-        printField(pw, ErrorField.CODE.str(), "1");
-
-        final String msg = rootCause.getMessage();
-        printField(pw, ErrorField.MSG.str(), JSONUtil.escape(msg != null ? msg : rootCause.getClass().getSimpleName()),
-                addStack);
-        pw.print("\t}],\n");
-    }
-
-    private static void printMetrics(PrintWriter pw, long elapsedTime, long executionTime, long resultCount,
-            long resultSize) {
-        pw.print("\t\"");
-        pw.print(ResultFields.METRICS.str());
-        pw.print("\": {\n");
-        pw.print("\t");
-        printField(pw, Metrics.ELAPSED_TIME.str(), TimeUnit.formatNanos(elapsedTime));
-        pw.print("\t");
-        printField(pw, Metrics.EXECUTION_TIME.str(), TimeUnit.formatNanos(executionTime));
-        pw.print("\t");
-        printField(pw, Metrics.RESULT_COUNT.str(), String.valueOf(resultCount));
-        pw.print("\t");
-        printField(pw, Metrics.RESULT_SIZE.str(), String.valueOf(resultSize), false);
-        pw.print("\t}\n");
-    }
-
-    @Override
-    protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException {
-        try {
-            handleRequest(getRequestParameters(request), response);
-        } catch (IOException e) {
-            // Servlet methods should not throw exceptions
-            // http://cwe.mitre.org/data/definitions/600.html
-            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
-        }
-    }
-
-    private String getOptText(JsonNode node, String fieldName) {
-        final JsonNode value = node.get(fieldName);
-        return value != null ? value.asText() : null;
-    }
-
-    private boolean getOptBoolean(JsonNode node, String fieldName, boolean defaultValue) {
-        final JsonNode value = node.get(fieldName);
-        return value != null ? value.asBoolean() : defaultValue;
-    }
-
-    private RequestParameters getRequestParameters(HttpServletRequest request) throws IOException {
-        final String contentTypeParam = request.getContentType();
-        int sep = contentTypeParam.indexOf(';');
-        final String contentType = sep < 0 ? contentTypeParam.trim() : contentTypeParam.substring(0, sep).trim();
-        RequestParameters param = new RequestParameters();
-        if (MediaType.JSON.str().equals(contentType)) {
-            try {
-                JsonNode jsonRequest = new ObjectMapper().readTree(getRequestBody(request));
-                param.statement = jsonRequest.get(Parameter.STATEMENT.str()).asText();
-                param.format = toLower(getOptText(jsonRequest, Parameter.FORMAT.str()));
-                param.pretty = getOptBoolean(jsonRequest, Parameter.PRETTY.str(), false);
-                param.mode = toLower(getOptText(jsonRequest, Parameter.MODE.str()));
-                param.clientContextID = getOptText(jsonRequest, Parameter.CLIENT_ID.str());
-            } catch (JsonParseException | JsonMappingException e) {
-                // if the JSON parsing fails, the statement is empty and we get an empty statement error
-                GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
-            }
-        } else {
-            param.statement = request.getParameter(Parameter.STATEMENT.str());
-            if (param.statement == null) {
-                param.statement = getRequestBody(request);
-            }
-            param.format = toLower(request.getParameter(Parameter.FORMAT.str()));
-            param.pretty = Boolean.parseBoolean(request.getParameter(Parameter.PRETTY.str()));
-            param.mode = toLower(request.getParameter(Parameter.MODE.str()));
-            param.clientContextID = request.getParameter(Parameter.CLIENT_ID.str());
-        }
-        return param;
-    }
-
-    private static String getRequestBody(HttpServletRequest request) throws IOException {
-        StringWriter sw = new StringWriter();
-        IOUtils.copy(request.getInputStream(), sw, StandardCharsets.UTF_8.name());
-        return sw.toString();
-    }
-
-    private static QueryTranslator.ResultDelivery parseResultDelivery(String mode) {
-        if ("async".equals(mode)) {
-            return QueryTranslator.ResultDelivery.ASYNC;
-        } else if ("deferred".equals(mode)) {
-            return QueryTranslator.ResultDelivery.DEFERRED;
-        } else {
-            return QueryTranslator.ResultDelivery.IMMEDIATE;
-        }
-    }
-
-    private void handleRequest(RequestParameters param, HttpServletResponse response) throws IOException {
-        LOGGER.info(param.toString());
-        long elapsedStart = System.nanoTime();
-        final StringWriter stringWriter = new StringWriter();
-        final PrintWriter resultWriter = new PrintWriter(stringWriter);
-
-        QueryTranslator.ResultDelivery delivery = parseResultDelivery(param.mode);
-
-        SessionConfig sessionConfig = createSessionConfig(param, resultWriter);
-        response.setCharacterEncoding("utf-8");
-        response.setContentType(MediaType.JSON.str());
-
-        int respCode = HttpServletResponse.SC_OK;
-        Stats stats = new Stats();
-        long execStart = -1;
-        long execEnd = -1;
-
-        resultWriter.print("{\n");
-        printRequestId(resultWriter);
-        printClientContextID(resultWriter, param);
-        printSignature(resultWriter);
-        printType(resultWriter, sessionConfig);
-        try {
-            final IClusterManagementWork.ClusterState clusterState = ClusterStateManager.INSTANCE.getState();
-            if (clusterState != IClusterManagementWork.ClusterState.ACTIVE) {
-                // using a plain IllegalStateException here to get into the right catch clause for a 500
-                throw new IllegalStateException("Cannot execute request, cluster is " + clusterState);
-            }
-            if (param.statement == null || param.statement.isEmpty()) {
-                throw new AsterixException("Empty request, no statement provided");
-            }
-            IHyracksClientConnection hcc;
-            IHyracksDataset hds;
-            ServletContext context = getServletContext();
-            synchronized (context) {
-                hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
-                hds = (IHyracksDataset) context.getAttribute(HYRACKS_DATASET_ATTR);
-                if (hds == null) {
-                    hds = new HyracksDataset(hcc, ResultReader.FRAME_SIZE, ResultReader.NUM_READERS);
-                    context.setAttribute(HYRACKS_DATASET_ATTR, hds);
-                }
-            }
-            IParser parser = compilationProvider.getParserFactory().createParser(param.statement);
-            List<Statement> statements = parser.parse();
-            MetadataManager.INSTANCE.init();
-            IStatementExecutor translator = statementExecutorFactory.create(statements, sessionConfig,
-                    compilationProvider);
-            execStart = System.nanoTime();
-            translator.compileAndExecute(hcc, hds, delivery, stats);
-            execEnd = System.nanoTime();
-            printStatus(resultWriter, ResultStatus.SUCCESS);
-        } catch (AsterixException | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
-            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, pe.getMessage(), pe);
-            printError(resultWriter, pe);
-            printStatus(resultWriter, ResultStatus.FATAL);
-            respCode = HttpServletResponse.SC_BAD_REQUEST;
-        } catch (Exception e) {
-            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
-            printError(resultWriter, e);
-            printStatus(resultWriter, ResultStatus.FATAL);
-            respCode = HttpServletResponse.SC_INTERNAL_SERVER_ERROR;
-        } finally {
-            if (execStart == -1) {
-                execEnd = -1;
-            } else if (execEnd == -1) {
-                execEnd = System.nanoTime();
-            }
-        }
-        printMetrics(resultWriter, System.nanoTime() - elapsedStart, execEnd - execStart, stats.getCount(),
-                stats.getSize());
-        resultWriter.print("}\n");
-        resultWriter.flush();
-        String result = stringWriter.toString();
-
-        GlobalConfig.ASTERIX_LOGGER.log(Level.FINE, result);
-
-        response.setStatus(respCode);
-        response.getWriter().print(result);
-        if (response.getWriter().checkError()) {
-            LOGGER.warning("Error flushing output writer");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryStatusAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryStatusAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryStatusAPIServlet.java
deleted file mode 100644
index 6d1b1b2..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryStatusAPIServlet.java
+++ /dev/null
@@ -1,101 +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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.asterix.app.result.ResultReader;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.client.dataset.HyracksDataset;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-public class QueryStatusAPIServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        response.setContentType("text/html");
-        response.setCharacterEncoding("utf-8");
-        String strHandle = request.getParameter("handle");
-        PrintWriter out = response.getWriter();
-        ServletContext context = getServletContext();
-        IHyracksClientConnection hcc;
-        IHyracksDataset hds;
-
-        try {
-            synchronized (context) {
-                hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
-
-                hds = (IHyracksDataset) context.getAttribute(HYRACKS_DATASET_ATTR);
-                if (hds == null) {
-                    hds = new HyracksDataset(hcc, ResultReader.FRAME_SIZE, ResultReader.NUM_READERS);
-                    context.setAttribute(HYRACKS_DATASET_ATTR, hds);
-                }
-            }
-            ObjectMapper om = new ObjectMapper();
-            JsonNode handleObj = om.readTree(strHandle);
-            JsonNode handle = handleObj.get("handle");
-            JobId jobId = new JobId(handle.get(0).asLong());
-            ResultSetId rsId = new ResultSetId(handle.get(1).asLong());
-
-            /* TODO(madhusudancs): We need to find a way to LOSSLESS_JSON serialize default format obtained from
-             * metadataProvider in the AQLTranslator and store it as part of the result handle.
-             */
-            ResultReader resultReader = new ResultReader(hds);
-            resultReader.open(jobId, rsId);
-
-            ObjectNode jsonResponse = om.createObjectNode();
-            String status;
-            switch (resultReader.getStatus()) {
-                case RUNNING:
-                    status = "RUNNING";
-                    break;
-                case FAILED:
-                    status = "ERROR";
-                    break;
-                case SUCCESS:
-                    status = "SUCCESS";
-                    break;
-                default:
-                    status = "ERROR";
-                    break;
-            }
-            jsonResponse.put("status", status);
-            out.write(jsonResponse.toString());
-
-        } catch (Exception e) {
-            out.println(e.getMessage());
-            e.printStackTrace(out);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryWebInterfaceServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryWebInterfaceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryWebInterfaceServlet.java
deleted file mode 100644
index d3e5556..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/QueryWebInterfaceServlet.java
+++ /dev/null
@@ -1,132 +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.api.http.servlet;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PrintWriter;
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.asterix.common.config.ExternalProperties;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-public class QueryWebInterfaceServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-    private static final HashMap<String, String> fileMimePair = new HashMap<>();
-    private static final Log LOG = LogFactory.getLog(QueryWebInterfaceServlet.class);
-
-    public QueryWebInterfaceServlet() {
-        fileMimePair.put("png", "image/png");
-        fileMimePair.put("eot", "application/vnd.ms-fontobject");
-        fileMimePair.put("svg", "image/svg+xml\t");
-        fileMimePair.put("ttf", "application/x-font-ttf");
-        fileMimePair.put("woff", "application/x-font-woff");
-        fileMimePair.put("woff2", "application/x-font-woff");
-        fileMimePair.put("html", "text/html");
-        fileMimePair.put("css", "text/css");
-        fileMimePair.put("js", "application/javascript");
-    }
-
-    @Override
-    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
-        String resourcePath = null;
-        String requestURI = request.getRequestURI();
-
-        if ("/".equals(requestURI)) {
-            response.setContentType("text/html");
-            resourcePath = "/queryui/queryui.html";
-        } else {
-            resourcePath = requestURI;
-        }
-
-        try (InputStream is = APIServlet.class.getResourceAsStream(resourcePath)) {
-            if (is == null) {
-                try {
-                    response.sendError(HttpServletResponse.SC_NOT_FOUND);
-                } catch (IllegalStateException | IOException e) {
-                    LOG.error(e);
-                }
-                return;
-            }
-            // Multiple MIME type support
-            for (Map.Entry<String, String> entry : fileMimePair.entrySet()) {
-                OutputStream out = null;
-                if (resourcePath.endsWith(entry.getKey())) {
-                    response.setContentType(entry.getValue());
-                    try {
-                        out = response.getOutputStream();
-                        IOUtils.copy(is, out);
-
-                    } catch (IOException e) {
-                        LOG.info(e);
-                    } finally {
-
-                        if (out != null) {
-                            IOUtils.closeQuietly(out);
-                        }
-                        IOUtils.closeQuietly(is);
-
-                    }
-                    return;
-                }
-            }
-            try {
-                response.sendError(HttpServletResponse.SC_BAD_REQUEST);
-            } catch (IllegalStateException | IOException e) {
-                LOG.error(e);
-            }
-        }
-    }
-
-    @Override
-    public void doPost(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        response.setCharacterEncoding("utf-8");
-        response.setContentType("application/json");
-        ExternalProperties externalProperties = AppContextInfo.INSTANCE.getExternalProperties();
-        ObjectMapper om = new ObjectMapper();
-        ObjectNode obj = om.createObjectNode();
-        try {
-            PrintWriter out = response.getWriter();
-            obj.put("api_port", String.valueOf(externalProperties.getAPIServerPort()));
-            out.println(obj.toString());
-            return;
-        } catch (Exception e) {
-            LOG.error(e);
-        }
-
-        try {
-            response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
-        } catch (IllegalStateException | IOException e) {
-            LOG.error(e);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/RESTAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/RESTAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/RESTAPIServlet.java
deleted file mode 100644
index dd030b0..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/RESTAPIServlet.java
+++ /dev/null
@@ -1,252 +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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-import java.util.logging.Level;
-
-import javax.servlet.ServletContext;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.asterix.app.result.ResultReader;
-import org.apache.asterix.app.result.ResultUtil;
-import org.apache.asterix.app.translator.QueryTranslator;
-import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.aql.parser.TokenMgrError;
-import org.apache.asterix.lang.common.base.IParser;
-import org.apache.asterix.lang.common.base.IParserFactory;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.translator.IStatementExecutor;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.asterix.translator.SessionConfig;
-import org.apache.asterix.translator.SessionConfig.OutputFormat;
-import org.apache.commons.io.IOUtils;
-import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.client.dataset.HyracksDataset;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-abstract class RESTAPIServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-
-    private final ILangCompilationProvider compilationProvider;
-    private final IParserFactory parserFactory;
-    private final transient IStatementExecutorFactory statementExecutorFactory;
-
-    public RESTAPIServlet(ILangCompilationProvider compilationProvider,
-            IStatementExecutorFactory statementExecutorFactory) {
-        this.compilationProvider = compilationProvider;
-        this.parserFactory = compilationProvider.getParserFactory();
-        this.statementExecutorFactory = statementExecutorFactory;
-    }
-
-    /**
-     * Initialize the Content-Type of the response, and construct a
-     * SessionConfig with the appropriate output writer and output-format
-     * based on the Accept: header and other servlet parameters.
-     */
-    static SessionConfig initResponse(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        response.setCharacterEncoding("utf-8");
-
-        // CLEAN_JSON output is the default; most generally useful for a
-        // programmatic HTTP API
-        OutputFormat format = OutputFormat.CLEAN_JSON;
-
-        // First check the "output" servlet parameter.
-        String output = request.getParameter("output");
-        String accept = request.getHeader("Accept");
-        if (accept == null) {
-            accept = "";
-        }
-        if (output != null) {
-            if (output.equals("CSV")) {
-                format = OutputFormat.CSV;
-            } else if (output.equals("ADM")) {
-                format = OutputFormat.ADM;
-            }
-        } else {
-            // Second check the Accept: HTTP header.
-            if (accept.contains("application/x-adm")) {
-                format = OutputFormat.ADM;
-            } else if (accept.contains("text/csv")) {
-                format = OutputFormat.CSV;
-            }
-        }
-
-        // If it's JSON, check for the "lossless" flag
-
-        if (format == OutputFormat.CLEAN_JSON
-                && ("true".equals(request.getParameter("lossless")) || accept.contains("lossless=true"))) {
-            format = OutputFormat.LOSSLESS_JSON;
-        }
-
-        SessionConfig.ResultDecorator handlePrefix = (AlgebricksAppendable app) -> app.append("{ \"").append("handle")
-                .append("\": ");
-        SessionConfig.ResultDecorator handlePostfix = (AlgebricksAppendable app) -> app.append(" }");
-
-        SessionConfig sessionConfig = new SessionConfig(response.getWriter(), format, null, null, handlePrefix,
-                handlePostfix);
-
-        // If it's JSON or ADM, check for the "wrapper-array" flag. Default is
-        // "true" for JSON and "false" for ADM. (Not applicable for CSV.)
-        boolean wrapper_array;
-        switch (format) {
-            case CLEAN_JSON:
-            case LOSSLESS_JSON:
-                wrapper_array = true;
-                break;
-            default:
-                wrapper_array = false;
-                break;
-        }
-        String wrapper_param = request.getParameter("wrapper-array");
-        if (wrapper_param != null) {
-            wrapper_array = Boolean.valueOf(wrapper_param);
-        } else if (accept.contains("wrap-array=true")) {
-            wrapper_array = true;
-        } else if (accept.contains("wrap-array=false")) {
-            wrapper_array = false;
-        }
-        sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, wrapper_array);
-
-        // Now that format is set, output the content-type
-        switch (format) {
-            case ADM:
-                response.setContentType("application/x-adm");
-                break;
-            case CLEAN_JSON:
-                // No need to reflect "clean-ness" in output type; fall through
-            case LOSSLESS_JSON:
-                response.setContentType("application/json");
-                break;
-            case CSV: {
-                // Check for header parameter or in Accept:.
-                if ("present".equals(request.getParameter("header")) || accept.contains("header=present")) {
-                    response.setContentType("text/csv; header=present");
-                    sessionConfig.set(SessionConfig.FORMAT_CSV_HEADER, true);
-                } else {
-                    response.setContentType("text/csv; header=absent");
-                }
-            }
-        }
-
-        return sessionConfig;
-    }
-
-    @Override
-    protected void doPost(HttpServletRequest request, HttpServletResponse response)
-            throws ServletException, IOException {
-        StringWriter sw = new StringWriter();
-        IOUtils.copy(request.getInputStream(), sw, StandardCharsets.UTF_8.name());
-        String query = sw.toString();
-        handleRequest(request, response, query);
-    }
-
-    @Override
-    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        String query = getQueryParameter(request);
-        handleRequest(request, response, query);
-    }
-
-    public void handleRequest(HttpServletRequest request, HttpServletResponse response, String query)
-            throws IOException {
-        // enable cross-origin resource sharing
-        response.addHeader("Access-Control-Allow-Origin", "*");
-        response.addHeader("Access-Control-Allow-Headers", "Origin, X-Requested-With, Content-Type, Accept");
-
-        SessionConfig sessionConfig = initResponse(request, response);
-        QueryTranslator.ResultDelivery resultDelivery = whichResultDelivery(request);
-
-        ServletContext context = getServletContext();
-        IHyracksClientConnection hcc;
-        IHyracksDataset hds;
-
-        try {
-            synchronized (context) {
-                hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
-                hds = (IHyracksDataset) context.getAttribute(HYRACKS_DATASET_ATTR);
-                if (hds == null) {
-                    hds = new HyracksDataset(hcc, ResultReader.FRAME_SIZE, ResultReader.NUM_READERS);
-                    context.setAttribute(HYRACKS_DATASET_ATTR, hds);
-                }
-            }
-
-            IParser parser = parserFactory.createParser(query);
-            List<Statement> aqlStatements = parser.parse();
-            if (!containsForbiddenStatements(aqlStatements)) {
-                MetadataManager.INSTANCE.init();
-                IStatementExecutor translator =
-                        statementExecutorFactory.create(aqlStatements, sessionConfig, compilationProvider);
-                translator.compileAndExecute(hcc, hds, resultDelivery);
-            }
-        } catch (AsterixException | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
-            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, pe.getMessage(), pe);
-            String errorMessage = ResultUtil.buildParseExceptionMessage(pe, query);
-            ObjectNode errorResp =
-                    ResultUtil.getErrorResponse(2, errorMessage, "", ResultUtil.extractFullStackTrace(pe));
-            sessionConfig.out().write(new ObjectMapper().writeValueAsString(errorResp));
-            response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
-        } catch (Exception e) {
-            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
-            ResultUtil.apiErrorHandler(sessionConfig.out(), e);
-            response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
-        }
-    }
-
-    private boolean containsForbiddenStatements(List<Statement> aqlStatements) throws AsterixException {
-        for (Statement st : aqlStatements) {
-            if ((st.getCategory() & getAllowedCategories()) == 0) {
-                throw new AsterixException(String.format(getErrorMessage(), st.getKind()));
-            }
-        }
-        return false;
-    }
-
-    protected QueryTranslator.ResultDelivery whichResultDelivery(HttpServletRequest request) {
-        String mode = request.getParameter("mode");
-        if (mode != null) {
-            if ("asynchronous".equals(mode) || "async".equals(mode)) {
-                return QueryTranslator.ResultDelivery.ASYNC;
-            } else if ("asynchronous-deferred".equals(mode) || "deferred".equals(mode)) {
-                return QueryTranslator.ResultDelivery.DEFERRED;
-            }
-        }
-        return QueryTranslator.ResultDelivery.IMMEDIATE;
-    }
-
-    protected abstract String getQueryParameter(HttpServletRequest request);
-
-    protected abstract byte getAllowedCategories();
-
-    protected abstract String getErrorMessage();
-}


[5/5] asterixdb git commit: Replace Servlets with Netty Based HTTP Servers

Posted by ti...@apache.org.
Replace Servlets with Netty Based HTTP Servers

Change-Id: I3d552d4eb8c868535ca4c41cbcf7e352217b18ae
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1429
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
BAD: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/60e7f12b
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/60e7f12b
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/60e7f12b

Branch: refs/heads/master
Commit: 60e7f12b47fa7e8e31d817ce2d46999908efb9d4
Parents: 9d30640
Author: Abdullah Alamoudi <ba...@gmail.com>
Authored: Wed Jan 25 19:16:46 2017 -0800
Committer: Till Westmann <ti...@apache.org>
Committed: Thu Jan 26 07:28:19 2017 -0800

----------------------------------------------------------------------
 asterixdb/asterix-app/pom.xml                   |  71 +--
 .../asterix/api/http/server/ApiServlet.java     | 237 ++++++++
 .../api/http/server/ClusterApiServlet.java      | 190 ++++++
 .../ClusterControllerDetailsApiServlet.java     | 107 ++++
 .../api/http/server/ConnectorApiServlet.java    | 194 +++++++
 .../asterix/api/http/server/DdlApiServlet.java  |  52 ++
 .../api/http/server/DiagnosticsApiServlet.java  | 125 ++++
 .../asterix/api/http/server/FeedServlet.java    |  98 ++++
 .../asterix/api/http/server/FullApiServlet.java |  53 ++
 .../server/NodeControllerDetailsApiServlet.java | 214 +++++++
 .../api/http/server/QueryApiServlet.java        |  50 ++
 .../api/http/server/QueryResultApiServlet.java  | 133 +++++
 .../api/http/server/QueryServiceServlet.java    | 557 ++++++++++++++++++
 .../api/http/server/QueryStatusApiServlet.java  | 118 ++++
 .../http/server/QueryWebInterfaceServlet.java   | 133 +++++
 .../asterix/api/http/server/RestApiServlet.java | 248 ++++++++
 .../api/http/server/ShutdownApiServlet.java     | 107 ++++
 .../api/http/server/UpdateApiServlet.java       |  50 ++
 .../api/http/server/VersionApiServlet.java      |  75 +++
 .../asterix/api/http/servlet/APIServlet.java    | 205 -------
 .../api/http/servlet/ClusterAPIServlet.java     | 178 ------
 .../servlet/ClusterCCDetailsAPIServlet.java     | 101 ----
 .../servlet/ClusterNodeDetailsAPIServlet.java   | 215 -------
 .../api/http/servlet/ConnectorAPIServlet.java   | 181 ------
 .../asterix/api/http/servlet/DDLAPIServlet.java |  51 --
 .../api/http/servlet/DiagnosticsAPIServlet.java | 118 ----
 .../asterix/api/http/servlet/FeedServlet.java   |  85 ---
 .../api/http/servlet/FullAPIServlet.java        |  53 --
 .../api/http/servlet/QueryAPIServlet.java       |  50 --
 .../api/http/servlet/QueryResultAPIServlet.java | 114 ----
 .../api/http/servlet/QueryServiceServlet.java   | 571 -------------------
 .../api/http/servlet/QueryStatusAPIServlet.java | 101 ----
 .../http/servlet/QueryWebInterfaceServlet.java  | 132 -----
 .../api/http/servlet/RESTAPIServlet.java        | 252 --------
 .../api/http/servlet/ShutdownAPIServlet.java    |  92 ---
 .../api/http/servlet/UpdateAPIServlet.java      |  50 --
 .../api/http/servlet/VersionAPIServlet.java     |  55 --
 .../app/external/FeedWorkCollection.java        |   2 +-
 .../apache/asterix/app/result/ResultUtil.java   |   7 +-
 .../asterix/app/translator/QueryTranslator.java |   4 +-
 .../asterix/drivers/AsterixWebServer.java       |  41 --
 .../bootstrap/CCApplicationEntryPoint.java      | 234 ++++----
 .../asterix/hyracks/bootstrap/WebManager.java   |  69 +++
 .../http/servlet/ConnectorAPIServletTest.java   | 177 ------
 .../api/http/servlet/ConnectorApiLetTest.java   | 178 ++++++
 .../api/http/servlet/QueryServiceLetTest.java   |  46 ++
 .../http/servlet/QueryServiceServletTest.java   |  45 --
 .../api/http/servlet/VersionAPIServletTest.java | 120 ----
 .../api/http/servlet/VersionApiLetTest.java     | 120 ++++
 .../apache/asterix/common/utils/LetUtil.java    |  58 ++
 .../asterix/common/utils/ServletUtil.java       |  58 --
 .../apache/asterix/test/aql/TestExecutor.java   |  36 +-
 .../server/test/SampleLocalClusterIT.java       |   4 +-
 asterixdb/pom.xml                               |   5 +
 hyracks-fullstack/hyracks/hyracks-http/pom.xml  |  34 ++
 .../hyracks/http/server/AbstractServlet.java    |  69 +++
 .../http/server/ChunkedNettyOutputStream.java   | 108 ++++
 .../hyracks/http/server/ChunkedResponse.java    | 113 ++++
 .../hyracks/http/server/FullResponse.java       |  96 ++++
 .../apache/hyracks/http/server/GetRequest.java  |  50 ++
 .../apache/hyracks/http/server/HttpServer.java  | 225 ++++++++
 .../hyracks/http/server/HttpServerHandler.java  | 131 +++++
 .../http/server/HttpServerInitializer.java      |  47 ++
 .../apache/hyracks/http/server/IServlet.java    |  92 +++
 .../hyracks/http/server/IServletRequest.java    |  63 ++
 .../hyracks/http/server/IServletResponse.java   |  77 +++
 .../apache/hyracks/http/server/PostRequest.java |  59 ++
 hyracks-fullstack/hyracks/pom.xml               |   1 +
 68 files changed, 4540 insertions(+), 3245 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index f4667d0..77ca6ef 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -32,12 +32,10 @@
       <comments>A business-friendly OSS license</comments>
     </license>
   </licenses>
-
   <properties>
     <appendedResourcesDirectory>${basedir}/src/main/appended-resources</appendedResourcesDirectory>
     <sonar.sources>pom.xml,src/main/java,src/main/resources</sonar.sources>
   </properties>
-
   <build>
     <plugins>
       <plugin>
@@ -62,7 +60,6 @@
             </goals>
           </execution>
         </executions>
-
         <configuration>
           <!--
               If you'd like to tell the plugin where your .git directory is,
@@ -80,7 +77,6 @@
                at source assembly time) is used in this case -->
           <failOnNoGitDirectory>false</failOnNoGitDirectory>
         </configuration>
-
       </plugin>
       <plugin>
         <artifactId>maven-resources-plugin</artifactId>
@@ -165,33 +161,33 @@
       </plugin>
     </plugins>
     <pluginManagement>
-        <plugins>
+      <plugins>
             <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
-            <plugin>
-                <groupId>org.eclipse.m2e</groupId>
-                <artifactId>lifecycle-mapping</artifactId>
-                <version>1.0.0</version>
-                <configuration>
-                    <lifecycleMappingMetadata>
-                        <pluginExecutions>
-                            <pluginExecution>
-                                <pluginExecutionFilter>
-                                    <groupId>org.apache.asterix</groupId>
-                                    <artifactId>asterix-test-datagenerator-maven-plugin</artifactId>
-                                    <versionRange>[0.8.9-SNAPSHOT,)</versionRange>
-                                    <goals>
-                                        <goal>generate-testdata</goal>
-                                    </goals>
-                                </pluginExecutionFilter>
-                                <action>
-                                    <ignore />
-                                </action>
-                            </pluginExecution>
-                        </pluginExecutions>
-                    </lifecycleMappingMetadata>
-                </configuration>
-            </plugin>
-        </plugins>
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <version>1.0.0</version>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.asterix</groupId>
+                    <artifactId>asterix-test-datagenerator-maven-plugin</artifactId>
+                    <versionRange>[0.8.9-SNAPSHOT,)</versionRange>
+                    <goals>
+                      <goal>generate-testdata</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore />
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
     </pluginManagement>
   </build>
   <dependencies>
@@ -201,10 +197,6 @@
     </dependency>
     <dependency>
       <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-server</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-servlet</artifactId>
     </dependency>
     <dependency>
@@ -228,6 +220,10 @@
       <artifactId>hyracks-api</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-http</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.asterix</groupId>
       <artifactId>asterix-algebra</artifactId>
       <version>${project.version}</version>
@@ -484,5 +480,10 @@
       <classifier>tests</classifier>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-all</artifactId>
+      <version>4.1.6.Final</version>
+    </dependency>
   </dependencies>
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
new file mode 100644
index 0000000..c38e0a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
@@ -0,0 +1,237 @@
+/*
+ * 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.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+
+import java.awt.image.BufferedImage;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.util.List;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.imageio.ImageIO;
+
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResultUtil;
+import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.aql.parser.TokenMgrError;
+import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.IParserFactory;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.asterix.translator.SessionConfig;
+import org.apache.asterix.translator.SessionConfig.OutputFormat;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.dataset.IHyracksDataset;
+import org.apache.hyracks.client.dataset.HyracksDataset;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class ApiServlet extends AbstractServlet {
+
+    private static final Logger LOGGER = Logger.getLogger(ApiServlet.class.getName());
+    public static final String HTML_STATEMENT_SEPARATOR = "<!-- BEGIN -->";
+
+    private final ILangCompilationProvider aqlCompilationProvider;
+    private final ILangCompilationProvider sqlppCompilationProvider;
+    private final IStatementExecutorFactory statementExectorFactory;
+
+    public ApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
+            ILangCompilationProvider aqlCompilationProvider, ILangCompilationProvider sqlppCompilationProvider,
+            IStatementExecutorFactory statementExecutorFactory) {
+        super(ctx, paths);
+        this.aqlCompilationProvider = aqlCompilationProvider;
+        this.sqlppCompilationProvider = sqlppCompilationProvider;
+        this.statementExectorFactory = statementExecutorFactory;
+    }
+
+    public void doPost(IServletRequest request, IServletResponse response) {
+        // Query language
+        ILangCompilationProvider compilationProvider = "AQL".equals(request.getParameter("query-language"))
+                ? aqlCompilationProvider : sqlppCompilationProvider;
+        IParserFactory parserFactory = compilationProvider.getParserFactory();
+
+        // Output format.
+        PrintWriter out = response.writer();
+        OutputFormat format;
+        boolean csvAndHeader = false;
+        String output = request.getParameter("output-format");
+        try {
+            format = OutputFormat.valueOf(output);
+        } catch (IllegalArgumentException e) {
+            LOGGER.log(Level.INFO,
+                    output + ": unsupported output-format, using " + OutputFormat.CLEAN_JSON + " instead", e);
+            // Default output format
+            format = OutputFormat.CLEAN_JSON;
+        }
+
+        String query = request.getParameter("query");
+        String wrapperArray = request.getParameter("wrapper-array");
+        String printExprParam = request.getParameter("print-expr-tree");
+        String printRewrittenExprParam = request.getParameter("print-rewritten-expr-tree");
+        String printLogicalPlanParam = request.getParameter("print-logical-plan");
+        String printOptimizedLogicalPlanParam = request.getParameter("print-optimized-logical-plan");
+        String printJob = request.getParameter("print-job");
+        String executeQuery = request.getParameter("execute-query");
+        try {
+            IServletResponse.setContentType(response, IServlet.ContentType.TEXT_HTML, IServlet.Encoding.UTF8);
+        } catch (IOException e) {
+            LOGGER.log(Level.WARNING, "Failure setting content type", e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        }
+        try {
+            IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
+            IHyracksDataset hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
+            if (hds == null) {
+                synchronized (ctx) {
+                    hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
+                    if (hds == null) {
+                        hds = new HyracksDataset(hcc, ResultReader.FRAME_SIZE, ResultReader.NUM_READERS);
+                        ctx.put(HYRACKS_DATASET_ATTR, hds);
+                    }
+                }
+            }
+            IParser parser = parserFactory.createParser(query);
+            List<Statement> aqlStatements = parser.parse();
+            SessionConfig sessionConfig = new SessionConfig(out, format, true, isSet(executeQuery), true);
+            sessionConfig.set(SessionConfig.FORMAT_HTML, true);
+            sessionConfig.set(SessionConfig.FORMAT_CSV_HEADER, csvAndHeader);
+            sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, isSet(wrapperArray));
+            sessionConfig.setOOBData(isSet(printExprParam), isSet(printRewrittenExprParam),
+                    isSet(printLogicalPlanParam), isSet(printOptimizedLogicalPlanParam), isSet(printJob));
+            MetadataManager.INSTANCE.init();
+            IStatementExecutor translator =
+                    statementExectorFactory.create(aqlStatements, sessionConfig, compilationProvider);
+            double duration;
+            long startTime = System.currentTimeMillis();
+            translator.compileAndExecute(hcc, hds, IStatementExecutor.ResultDelivery.IMMEDIATE);
+            long endTime = System.currentTimeMillis();
+            duration = (endTime - startTime) / 1000.00;
+            out.println(HTML_STATEMENT_SEPARATOR);
+            out.println("<PRE>Duration of all jobs: " + duration + " sec</PRE>");
+        } catch (AsterixException | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
+            GlobalConfig.ASTERIX_LOGGER.log(Level.INFO, pe.toString(), pe);
+            ResultUtil.webUIParseExceptionHandler(out, pe, query);
+        } catch (Exception e) {
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
+            ResultUtil.webUIErrorHandler(out, e);
+        }
+    }
+
+    public void doGet(IServletRequest request, IServletResponse response) {
+        String resourcePath = null;
+        String requestURI = request.getHttpRequest().uri();
+
+        if ("/".equals(requestURI)) {
+            try {
+                IServletResponse.setContentType(response, IServlet.ContentType.TEXT_HTML, IServlet.Encoding.UTF8);
+            } catch (IOException e) {
+                LOGGER.log(Level.WARNING, "Failure setting content type", e);
+                response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+                return;
+            }
+            resourcePath = "/webui/querytemplate.html";
+        } else {
+            resourcePath = requestURI;
+        }
+
+        try (InputStream is = ApiServlet.class.getResourceAsStream(resourcePath)) {
+            if (is == null) {
+                response.setStatus(HttpResponseStatus.NOT_FOUND);
+                return;
+            }
+            // Special handler for font files and .png resources
+            if (resourcePath.endsWith(".png")) {
+                BufferedImage img = ImageIO.read(is);
+                IServletResponse.setContentType(response, IServlet.ContentType.IMG_PNG);
+                OutputStream outputStream = response.outputStream();
+                String formatName = "png";
+                ImageIO.write(img, formatName, outputStream);
+                outputStream.close();
+                return;
+            }
+            String type = IServlet.ContentType.mime(QueryWebInterfaceServlet.extension(resourcePath));
+            IServletResponse.setContentType(response, "".equals(type) ? IServlet.ContentType.TEXT_PLAIN : type,
+                    IServlet.Encoding.UTF8);
+            writeOutput(response, is, resourcePath);
+        } catch (IOException e) {
+            LOGGER.log(Level.WARNING, "Failure handling request", e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        }
+    }
+
+    private void writeOutput(IServletResponse response, InputStream is, String resourcePath) throws IOException {
+        try (InputStreamReader isr = new InputStreamReader(is); BufferedReader br = new BufferedReader(isr)) {
+            StringBuilder sb = new StringBuilder();
+            String line;
+            try {
+                line = br.readLine();
+            } catch (NullPointerException e) {
+                LOGGER.log(Level.WARNING,
+                        "NPE reading resource " + resourcePath + ", assuming JDK-8080094; returning 404", e);
+                // workaround lame JDK bug where a broken InputStream is returned in case the resourcePath is a
+                // directory; see https://bugs.openjdk.java.net/browse/JDK-8080094
+                response.setStatus(HttpResponseStatus.NOT_FOUND);
+                return;
+            }
+            while (line != null) {
+                sb.append(line);
+                line = br.readLine();
+            }
+            PrintWriter out = response.writer();
+            out.println(sb.toString());
+        }
+    }
+
+    private static boolean isSet(String requestParameter) {
+        return requestParameter != null && "true".equals(requestParameter);
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        response.setStatus(HttpResponseStatus.OK);
+        if (request.getHttpRequest().method() == HttpMethod.GET) {
+            doGet(request, response);
+        } else if (request.getHttpRequest().method() == HttpMethod.POST) {
+            doPost(request, response);
+        } else {
+            response.setStatus(HttpResponseStatus.METHOD_NOT_ALLOWED);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
new file mode 100644
index 0000000..6fd6c47
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
@@ -0,0 +1,190 @@
+/*
+ * 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.api.http.server;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+import java.util.regex.Pattern;
+
+import org.apache.asterix.common.config.AbstractProperties;
+import org.apache.asterix.common.config.ReplicationProperties;
+import org.apache.asterix.common.utils.JSONUtil;
+import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class ClusterApiServlet extends AbstractServlet {
+
+    private static final Logger LOGGER = Logger.getLogger(ClusterApiServlet.class.getName());
+    private static final Pattern PARENT_DIR = Pattern.compile("/[^./]+/\\.\\./");
+    private static final Pattern REPLICATION_PROPERTY = Pattern.compile("^replication\\.");
+    protected static final String NODE_ID_KEY = "node_id";
+    protected static final String CONFIG_URI_KEY = "configUri";
+    protected static final String STATS_URI_KEY = "statsUri";
+    protected static final String THREAD_DUMP_URI_KEY = "threadDumpUri";
+    protected static final String SHUTDOWN_URI_KEY = "shutdownUri";
+    protected static final String FULL_SHUTDOWN_URI_KEY = "fullShutdownUri";
+    protected static final String VERSION_URI_KEY = "versionUri";
+    protected static final String DIAGNOSTICS_URI_KEY = "diagnosticsUri";
+    protected static final String REPLICATION_URI_KEY = "replicationUri";
+    private final ObjectMapper om = new ObjectMapper();
+
+    public ClusterApiServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+    }
+
+    protected void getUnsafe(IServletRequest request, IServletResponse response) throws IOException {
+        IServletResponse.setContentType(response, IServlet.ContentType.APPLICATION_JSON, IServlet.Encoding.UTF8);
+        PrintWriter responseWriter = response.writer();
+        try {
+            ObjectNode json;
+            response.setStatus(HttpResponseStatus.OK);
+            switch (path(request)) {
+                case "":
+                    json = getClusterStateJSON(request, "");
+                    break;
+                case "/replication":
+                    json = getReplicationJSON();
+                    break;
+                case "/summary":
+                    json = getClusterStateSummaryJSON();
+                    break;
+                default:
+                    throw new IllegalArgumentException();
+            }
+            responseWriter.write(JSONUtil.convertNode(json));
+        } catch (IllegalArgumentException e) { // NOSONAR - exception not logged or rethrown
+            response.setStatus(HttpResponseStatus.NOT_FOUND);
+        } catch (Exception e) {
+            LOGGER.log(Level.INFO, "exception thrown for " + request, e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            responseWriter.write(e.toString());
+        }
+        responseWriter.flush();
+    }
+
+    protected ObjectNode getClusterStateSummaryJSON() {
+        return ClusterStateManager.INSTANCE.getClusterStateSummary();
+    }
+
+    protected ObjectNode getReplicationJSON() {
+        for (AbstractProperties props : getPropertiesInstances()) {
+            if (props instanceof ReplicationProperties) {
+                ObjectNode json = om.createObjectNode();
+                json.putPOJO("config", props.getProperties(key -> REPLICATION_PROPERTY.matcher(key).replaceFirst("")));
+                return json;
+            }
+        }
+        throw new IllegalStateException("ERROR: replication properties not found");
+    }
+
+    protected Map<String, Object> getAllClusterProperties() {
+        Map<String, Object> allProperties = new HashMap<>();
+        for (AbstractProperties properties : getPropertiesInstances()) {
+            if (!(properties instanceof ReplicationProperties)) {
+                allProperties.putAll(properties.getProperties());
+            }
+        }
+        return allProperties;
+    }
+
+    protected List<AbstractProperties> getPropertiesInstances() {
+        return AbstractProperties.getImplementations();
+    }
+
+    protected ObjectNode getClusterStateJSON(IServletRequest request, String pathToNode) {
+        ObjectNode json = ClusterStateManager.INSTANCE.getClusterStateDescription();
+        Map<String, Object> allProperties = getAllClusterProperties();
+        json.putPOJO("config", allProperties);
+
+        ArrayNode ncs = (ArrayNode) json.get("ncs");
+        final StringBuilder requestURL = new StringBuilder("http://");
+        requestURL.append(request.getHeader(HttpHeaderNames.HOST));
+        requestURL.append(request.getHttpRequest().uri());
+        if (requestURL.charAt(requestURL.length() - 1) != '/') {
+            requestURL.append('/');
+        }
+        requestURL.append(pathToNode);
+        String clusterURL = canonicalize(requestURL);
+        String adminURL = canonicalize(clusterURL + "../");
+        String nodeURL = clusterURL + "node/";
+        for (int i = 0; i < ncs.size(); i++) {
+            ObjectNode nc = (ObjectNode) ncs.get(i);
+            nc.put(CONFIG_URI_KEY, nodeURL + nc.get(NODE_ID_KEY).asText() + "/config");
+            nc.put(STATS_URI_KEY, nodeURL + nc.get(NODE_ID_KEY).asText() + "/stats");
+            nc.put(THREAD_DUMP_URI_KEY, nodeURL + nc.get(NODE_ID_KEY).asText() + "/threaddump");
+        }
+        ObjectNode cc;
+        if (json.has("cc")) {
+            cc = (ObjectNode) json.get("cc");
+        } else {
+            cc = om.createObjectNode();
+            json.set("cc", cc);
+        }
+        cc.put(CONFIG_URI_KEY, clusterURL + "cc/config");
+        cc.put(STATS_URI_KEY, clusterURL + "cc/stats");
+        cc.put(THREAD_DUMP_URI_KEY, clusterURL + "cc/threaddump");
+        json.put(REPLICATION_URI_KEY, clusterURL + "replication");
+        json.put(SHUTDOWN_URI_KEY, adminURL + "shutdown");
+        json.put(FULL_SHUTDOWN_URI_KEY, adminURL + "shutdown?all=true");
+        json.put(VERSION_URI_KEY, adminURL + "version");
+        json.put(DIAGNOSTICS_URI_KEY, adminURL + "diagnostics");
+        return json;
+    }
+
+    private String canonicalize(CharSequence requestURL) {
+        String clusterURL = "";
+        String newClusterURL = requestURL.toString();
+        while (!clusterURL.equals(newClusterURL)) {
+            clusterURL = newClusterURL;
+            newClusterURL = PARENT_DIR.matcher(clusterURL).replaceAll("/");
+        }
+        return clusterURL;
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        if (request.getHttpRequest().method() != HttpMethod.GET) {
+            response.setStatus(HttpResponseStatus.METHOD_NOT_ALLOWED);
+            return;
+        }
+        try {
+            getUnsafe(request, response);
+        } catch (IOException e) {
+            LOGGER.log(Level.WARNING, "Unhandled IOException thrown from " + getClass().getName() + " get impl", e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterControllerDetailsApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterControllerDetailsApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterControllerDetailsApiServlet.java
new file mode 100644
index 0000000..4419e8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterControllerDetailsApiServlet.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class ClusterControllerDetailsApiServlet extends ClusterApiServlet {
+
+    private static final Logger LOGGER = Logger.getLogger(ClusterControllerDetailsApiServlet.class.getName());
+    private final ObjectMapper om = new ObjectMapper();
+
+    public ClusterControllerDetailsApiServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+    }
+
+    @Override
+    protected void getUnsafe(IServletRequest request, IServletResponse response) throws IOException {
+        PrintWriter responseWriter = response.writer();
+        IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
+        try {
+            ObjectNode json;
+            response.setStatus(HttpResponseStatus.OK);
+            if ("".equals(path(request))) {
+                json = (ObjectNode) getClusterStateJSON(request, "../").get("cc");
+            } else {
+                json = processNode(request, hcc);
+            }
+            IServletResponse.setContentType(response, IServlet.ContentType.APPLICATION_JSON, IServlet.Encoding.UTF8);
+            responseWriter.write(new ObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(json));
+        } catch (IllegalArgumentException e) { // NOSONAR - exception not logged or rethrown
+            response.setStatus(HttpResponseStatus.NOT_FOUND);
+        } catch (Exception e) {
+            LOGGER.log(Level.INFO, "exception thrown for " + request, e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            responseWriter.write(e.toString());
+        }
+        responseWriter.flush();
+    }
+
+    private ObjectNode processNode(IServletRequest request, IHyracksClientConnection hcc)
+            throws Exception {
+        String pathInfo = path(request);
+        if (pathInfo.endsWith("/")) {
+            throw new IllegalArgumentException();
+        }
+        String[] parts = pathInfo.substring(1).split("/");
+
+        if ("".equals(pathInfo)) {
+            return (ObjectNode) getClusterStateJSON(request, "../../").get("cc");
+        } else if (parts.length == 1) {
+            switch (parts[0]) {
+                case "config":
+                    return om.readValue(hcc.getNodeDetailsJSON(null, false, true), ObjectNode.class);
+                case "stats":
+                    return om.readValue(hcc.getNodeDetailsJSON(null, true, false), ObjectNode.class);
+                case "threaddump":
+                    return processCCThreadDump(hcc);
+
+                default:
+                    throw new IllegalArgumentException();
+            }
+
+        } else {
+            throw new IllegalArgumentException();
+        }
+    }
+
+    private ObjectNode processCCThreadDump(IHyracksClientConnection hcc) throws Exception {
+        String dump = hcc.getThreadDump(null);
+        if (dump == null) {
+            throw new IllegalArgumentException();
+        }
+        return (ObjectNode) om.readTree(dump);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
new file mode 100644
index 0000000..d832672
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
@@ -0,0 +1,194 @@
+/*
+ * 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.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.MetadataTransactionContext;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.util.FlushDatasetUtils;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.client.NodeControllerInfo;
+import org.apache.hyracks.api.io.FileSplit;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+/**
+ * The REST API that takes a dataverse name and a dataset name as the input
+ * and returns an array of file splits (IP, file-path) of the dataset in LOSSLESS_JSON.
+ * It is mostly used by external runtime, e.g., Pregelix or IMRU to pull data
+ * in parallel from existing AsterixDB datasets.
+ */
+public class ConnectorApiServlet extends AbstractServlet {
+    private static final Logger LOGGER = Logger.getLogger(ConnectorApiServlet.class.getName());
+
+    public ConnectorApiServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        if (request.getHttpRequest().method() != HttpMethod.GET) {
+            response.setStatus(HttpResponseStatus.METHOD_NOT_ALLOWED);
+            return;
+        }
+        response.setStatus(HttpResponseStatus.OK);
+        try {
+            IServletResponse.setContentType(response, IServlet.ContentType.TEXT_HTML, IServlet.Encoding.UTF8);
+        } catch (IOException e) {
+            LOGGER.log(Level.WARNING, "Failure setting content type", e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        }
+        PrintWriter out = response.writer();
+        try {
+            ObjectMapper om = new ObjectMapper();
+            ObjectNode jsonResponse = om.createObjectNode();
+            String dataverseName = request.getParameter("dataverseName");
+            String datasetName = request.getParameter("datasetName");
+            if (dataverseName == null || datasetName == null) {
+                jsonResponse.put("error", "Parameter dataverseName or datasetName is null,");
+                out.write(jsonResponse.toString());
+                out.flush();
+                return;
+            }
+
+            IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
+            // Metadata transaction begins.
+            MetadataManager.INSTANCE.init();
+            MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+
+            // Retrieves file splits of the dataset.
+            MetadataProvider metadataProvider = new MetadataProvider(null);
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
+            if (dataset == null) {
+                jsonResponse.put("error",
+                        "Dataset " + datasetName + " does not exist in " + "dataverse " + dataverseName);
+                out.write(jsonResponse.toString());
+                out.flush();
+                return;
+            }
+            boolean temp = dataset.getDatasetDetails().isTemp();
+            FileSplit[] fileSplits = metadataProvider.splitsForDataset(mdTxnCtx, dataverseName, datasetName,
+                    datasetName, temp);
+            ARecordType recordType = (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(),
+                    dataset.getItemTypeName());
+            List<List<String>> primaryKeys = DatasetUtils.getPartitioningKeys(dataset);
+            StringBuilder pkStrBuf = new StringBuilder();
+            for (List<String> keys : primaryKeys) {
+                for (String key : keys) {
+                    pkStrBuf.append(key).append(",");
+                }
+            }
+            pkStrBuf.delete(pkStrBuf.length() - 1, pkStrBuf.length());
+
+            // Constructs the returned json object.
+            formResponseObject(jsonResponse, fileSplits, recordType, pkStrBuf.toString(), temp,
+                    hcc.getNodeControllerInfos());
+
+            // Flush the cached contents of the dataset to file system.
+            FlushDatasetUtils.flushDataset(hcc, metadataProvider, mdTxnCtx, dataverseName, datasetName, datasetName);
+
+            // Metadata transaction commits.
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            // Writes file splits.
+            out.write(jsonResponse.toString());
+            out.flush();
+        } catch (Exception e) {
+            LOGGER.log(Level.WARNING, "Failure handling a request", e);
+            out.println(e.getMessage());
+            out.flush();
+            e.printStackTrace(out);
+        }
+    }
+
+    private void formResponseObject(ObjectNode jsonResponse, FileSplit[] fileSplits, ARecordType recordType,
+            String primaryKeys, boolean temp, Map<String, NodeControllerInfo> nodeMap) {
+        ObjectMapper om = new ObjectMapper();
+        ArrayNode partititons = om.createArrayNode();
+        // Whether the dataset is temp or not
+        jsonResponse.put("temp", temp);
+        // Adds a primary key.
+        jsonResponse.put("keys", primaryKeys);
+        // Adds record type.
+        jsonResponse.set("type", recordType.toJSON());
+        // Generates file partitions.
+        for (FileSplit split : fileSplits) {
+            String ipAddress = nodeMap.get(split.getNodeName()).getNetworkAddress().getAddress();
+            String path = split.getPath();
+            FilePartition partition = new FilePartition(ipAddress, path);
+            partititons.add(partition.toObjectNode());
+        }
+        // Generates the response object which contains the splits.
+        jsonResponse.set("splits", partititons);
+    }
+}
+
+class FilePartition {
+    private final String ipAddress;
+    private final String path;
+
+    public FilePartition(String ipAddress, String path) {
+        this.ipAddress = ipAddress;
+        this.path = path;
+    }
+
+    public String getIPAddress() {
+        return ipAddress;
+    }
+
+    public String getPath() {
+        return path;
+    }
+
+    @Override
+    public String toString() {
+        return ipAddress + ":" + path;
+    }
+
+    public ObjectNode toObjectNode() {
+        ObjectMapper om = new ObjectMapper();
+        ObjectNode partition = om.createObjectNode();
+        partition.put("ip", ipAddress);
+        partition.put("path", path);
+        return partition;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
new file mode 100644
index 0000000..bcc6914
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
@@ -0,0 +1,52 @@
+/*
+ * 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.api.http.server;
+
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.hyracks.http.server.IServletRequest;
+
+public class DdlApiServlet extends RestApiServlet {
+    private static final byte ALLOWED_CATEGORIES =
+            Statement.Category.QUERY | Statement.Category.UPDATE | Statement.Category.DDL;
+
+    public DdlApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
+        super(ctx, paths, compilationProvider, statementExecutorFactory);
+    }
+
+    @Override
+    protected String getQueryParameter(IServletRequest request) {
+        return request.getParameter("ddl");
+    }
+
+    @Override
+    protected byte getAllowedCategories() {
+        return ALLOWED_CATEGORIES;
+    }
+
+    @Override
+    protected String getErrorMessage() {
+        return "Invalid statement: Non-DDL statement %s to the DDL API.";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DiagnosticsApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DiagnosticsApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DiagnosticsApiServlet.java
new file mode 100644
index 0000000..d91352d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DiagnosticsApiServlet.java
@@ -0,0 +1,125 @@
+/*
+ * 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.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.api.http.servlet.ServletConstants;
+import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class DiagnosticsApiServlet extends NodeControllerDetailsApiServlet {
+    private static final Logger LOGGER = Logger.getLogger(DiagnosticsApiServlet.class.getName());
+
+    public DiagnosticsApiServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+    }
+
+    @Override
+    protected void getUnsafe(IServletRequest request, IServletResponse response) throws IOException {
+        IServletResponse.setContentType(response, IServlet.ContentType.APPLICATION_JSON, IServlet.Encoding.UTF8);
+        PrintWriter responseWriter = response.writer();
+        ObjectNode json;
+        ObjectMapper om = new ObjectMapper();
+        response.setStatus(HttpResponseStatus.OK);
+        om.enable(SerializationFeature.INDENT_OUTPUT);
+        try {
+            if (!"".equals(path(request))) {
+                throw new IllegalArgumentException();
+            }
+            json = getClusterDiagnosticsJSON();
+            responseWriter.write(om.writerWithDefaultPrettyPrinter().writeValueAsString(json));
+        } catch (IllegalStateException e) { // NOSONAR - exception not logged or rethrown
+            response.setStatus(HttpResponseStatus.SERVICE_UNAVAILABLE);
+        } catch (IllegalArgumentException e) { // NOSONAR - exception not logged or rethrown
+            response.setStatus(HttpResponseStatus.NOT_FOUND);
+        } catch (Exception e) {
+            LOGGER.log(Level.INFO, "exception thrown for " + request, e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            responseWriter.write(e.toString());
+        }
+        responseWriter.flush();
+    }
+
+    private ObjectNode getClusterDiagnosticsJSON() throws Exception {
+        ObjectMapper om = new ObjectMapper();
+        IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
+        ExecutorService executor = (ExecutorService) ctx.get(ServletConstants.EXECUTOR_SERVICE);
+        Map<String, Future<ObjectNode>> ccFutureData = new HashMap<>();
+        ccFutureData.put("threaddump",
+                executor.submit(() -> fixupKeys((ObjectNode) om.readTree(hcc.getThreadDump(null)))));
+        ccFutureData.put("config",
+                executor.submit(() -> fixupKeys((ObjectNode) om.readTree(hcc.getNodeDetailsJSON(null, false, true)))));
+        ccFutureData.put("stats",
+                executor.submit(() -> fixupKeys((ObjectNode) om.readTree(hcc.getNodeDetailsJSON(null, true, false)))));
+
+        Map<String, Map<String, Future<ObjectNode>>> ncDataMap = new HashMap<>();
+        for (String nc : AppContextInfo.INSTANCE.getMetadataProperties().getNodeNames()) {
+            Map<String, Future<ObjectNode>> ncData = new HashMap<>();
+            ncData.put("threaddump", executor.submit(() -> fixupKeys((ObjectNode) om.readTree(hcc.getThreadDump(nc)))));
+            ncData.put("config", executor
+                    .submit(() -> fixupKeys((ObjectNode) om.readTree(hcc.getNodeDetailsJSON(nc, false, true)))));
+            ncData.put("stats", executor.submit(() -> fixupKeys(processNodeStats(hcc, nc))));
+            ncDataMap.put(nc, ncData);
+        }
+        ObjectNode result = om.createObjectNode();
+        result.putPOJO("cc", resolveFutures(ccFutureData));
+        List<Map<String, ?>> ncList = new ArrayList<>();
+        for (Map.Entry<String, Map<String, Future<ObjectNode>>> entry : ncDataMap.entrySet()) {
+            final Map<String, Object> ncMap = resolveFutures(entry.getValue());
+            ncMap.put("node_id", entry.getKey());
+            ncList.add(ncMap);
+        }
+        result.putPOJO("ncs", ncList);
+        result.putPOJO("date", new Date());
+        return result;
+    }
+
+    private Map<String, Object> resolveFutures(Map<String, Future<ObjectNode>> futureMap)
+            throws ExecutionException, InterruptedException {
+        Map<String, Object> result = new HashMap<>();
+        for (Map.Entry<String, Future<ObjectNode>> entry : futureMap.entrySet()) {
+            result.put(entry.getKey(), entry.getValue().get());
+        }
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FeedServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FeedServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FeedServlet.java
new file mode 100644
index 0000000..ac79088
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FeedServlet.java
@@ -0,0 +1,98 @@
+/*
+ * 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.api.http.server;
+
+import java.awt.image.BufferedImage;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.imageio.ImageIO;
+
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class FeedServlet extends AbstractServlet {
+    private static final Logger LOGGER = Logger.getLogger(FeedServlet.class.getName());
+
+    public FeedServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        try {
+            response.setStatus(HttpResponseStatus.OK);
+            String resourcePath;
+            String requestURI = request.getHttpRequest().uri();
+
+            if ("/".equals(requestURI)) {
+                IServletResponse.setContentType(response, IServlet.ContentType.TEXT_HTML);
+                resourcePath = "/feed/home.html";
+            } else {
+                resourcePath = requestURI;
+            }
+
+            InputStream is = FeedServlet.class.getResourceAsStream(resourcePath);
+            if (is == null) {
+                response.setStatus(HttpResponseStatus.NOT_FOUND);
+                return;
+            }
+
+            // Special handler for font files and .png resources
+            if (resourcePath.endsWith(".png")) {
+
+                BufferedImage img = ImageIO.read(is);
+                OutputStream outputStream = response.outputStream();
+                String formatName = "png";
+                IServletResponse.setContentType(response, IServlet.ContentType.IMG_PNG);
+                ImageIO.write(img, formatName, outputStream);
+                return;
+            }
+
+            IServletResponse.setContentType(response, IServlet.ContentType.TEXT_HTML, IServlet.Encoding.UTF8);
+            InputStreamReader isr = new InputStreamReader(is);
+            StringBuilder sb = new StringBuilder();
+            BufferedReader br = new BufferedReader(isr);
+            String line = br.readLine();
+
+            while (line != null) {
+                sb.append(line + "\n");
+                line = br.readLine();
+            }
+
+            PrintWriter out = response.writer();
+            out.println(sb.toString());
+        } catch (IOException e) {
+            LOGGER.log(Level.WARNING, "Failure handling request", e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
new file mode 100644
index 0000000..7788136
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
@@ -0,0 +1,53 @@
+/*
+ * 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.api.http.server;
+
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.hyracks.http.server.IServletRequest;
+
+public class FullApiServlet extends RestApiServlet {
+
+    private static final String AQL_STMT_PARAM_NAME = "aql";
+    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE
+            | Statement.Category.DDL | Statement.Category.PROCEDURE;
+
+    public FullApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
+        super(ctx, paths, compilationProvider, statementExecutorFactory);
+    }
+
+    @Override
+    protected byte getAllowedCategories() {
+        return ALLOWED_CATEGORIES;
+    }
+
+    @Override
+    protected String getErrorMessage() {
+        throw new IllegalStateException();
+    }
+
+    @Override
+    protected String getQueryParameter(IServletRequest request) {
+        return request.getParameter(AQL_STMT_PARAM_NAME);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NodeControllerDetailsApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NodeControllerDetailsApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NodeControllerDetailsApiServlet.java
new file mode 100644
index 0000000..c1423e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NodeControllerDetailsApiServlet.java
@@ -0,0 +1,214 @@
+/*
+ * 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.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class NodeControllerDetailsApiServlet extends ClusterApiServlet {
+
+    private static final Logger LOGGER = Logger.getLogger(NodeControllerDetailsApiServlet.class.getName());
+    private final ObjectMapper om = new ObjectMapper();
+
+    public NodeControllerDetailsApiServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+        om.enable(SerializationFeature.INDENT_OUTPUT);
+    }
+
+    @Override
+    protected void getUnsafe(IServletRequest request, IServletResponse response) throws IOException {
+        PrintWriter responseWriter = response.writer();
+        IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
+        try {
+            ObjectNode json;
+            response.setStatus(HttpResponseStatus.OK);
+            if ("".equals(path(request))) {
+                json = om.createObjectNode();
+                json.set("ncs", getClusterStateJSON(request, "../").get("ncs"));
+            } else {
+                json = processNode(request, hcc);
+            }
+            IServletResponse.setContentType(response, IServlet.ContentType.APPLICATION_JSON, IServlet.Encoding.UTF8);
+            responseWriter.write(om.writerWithDefaultPrettyPrinter().writeValueAsString(json));
+        } catch (IllegalStateException e) { // NOSONAR - exception not logged or rethrown
+            response.setStatus(HttpResponseStatus.SERVICE_UNAVAILABLE);
+        } catch (IllegalArgumentException e) { // NOSONAR - exception not logged or rethrown
+            response.setStatus(HttpResponseStatus.NOT_FOUND);
+        } catch (Exception e) {
+            LOGGER.log(Level.INFO, "exception thrown for " + request, e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            responseWriter.write(e.toString());
+        }
+        responseWriter.flush();
+    }
+
+    private ObjectNode processNode(IServletRequest request, IHyracksClientConnection hcc)
+            throws Exception {
+        String pathInfo = path(request);
+        if (pathInfo.endsWith("/")) {
+            throw new IllegalArgumentException();
+        }
+        String[] parts = pathInfo.substring(1).split("/");
+        final String node = parts[0];
+
+        if (parts.length == 1) {
+            ArrayNode ncs = (ArrayNode) getClusterStateJSON(request, "../../").get("ncs");
+            for (int i = 0; i < ncs.size(); i++) {
+                if (node.equals(ncs.get(i).get("node_id").asText())) {
+                    return (ObjectNode) ncs.get(i);
+                }
+            }
+            if ("cc".equals(node)) {
+                return om.createObjectNode();
+            }
+            throw new IllegalArgumentException();
+        } else if (parts.length == 2) {
+            ObjectNode json;
+
+            switch (parts[1]) {
+                case "config":
+                    json = processNodeConfig(hcc, node);
+                    break;
+
+                case "stats":
+                    json = processNodeStats(hcc, node);
+                    break;
+
+                case "threaddump":
+                    return processNodeThreadDump(hcc, node);
+
+                default:
+                    throw new IllegalArgumentException();
+            }
+            fixupKeys(json);
+
+            return json;
+        } else {
+            throw new IllegalArgumentException();
+        }
+    }
+
+    protected ObjectNode fixupKeys(ObjectNode json) {
+        // TODO (mblow): generate the keys with _ to begin with
+        List<String> keys = new ArrayList<>();
+        for (Iterator<String> iter = json.fieldNames(); iter.hasNext();) {
+            keys.add(iter.next());
+        }
+        for (String key : keys) {
+            String newKey = key.replace('-', '_');
+            if (!newKey.equals(key)) {
+                json.set(newKey, json.remove(key));
+            }
+        }
+        return json;
+    }
+
+    protected ObjectNode processNodeStats(IHyracksClientConnection hcc, String node) throws Exception {
+        final String details = hcc.getNodeDetailsJSON(node, true, false);
+        if (details == null) {
+            throw new IllegalArgumentException();
+        }
+        ObjectNode json = (ObjectNode) om.readTree(details);
+        int index = json.get("rrd-ptr").asInt() - 1;
+        json.remove("rrd-ptr");
+
+        List<String> keys = new ArrayList<>();
+        for (Iterator<String> iter = json.fieldNames(); iter.hasNext();) {
+            keys.add(iter.next());
+        }
+
+        final ArrayNode gcNames = (ArrayNode) json.get("gc-names");
+        final ArrayNode gcCollectionTimes = (ArrayNode) json.get("gc-collection-times");
+        final ArrayNode gcCollectionCounts = (ArrayNode) json.get("gc-collection-counts");
+
+        for (String key : keys) {
+            if (key.startsWith("gc-")) {
+                json.remove(key);
+            } else {
+                final JsonNode keyNode = json.get(key);
+                if (keyNode instanceof ArrayNode) {
+                    final ArrayNode valueArray = (ArrayNode) keyNode;
+                    // fixup an index of -1 to the final element in the array (i.e. RRD_SIZE)
+                    if (index == -1) {
+                        index = valueArray.size() - 1;
+                    }
+                    final JsonNode value = valueArray.get(index);
+                    json.remove(key);
+                    json.set(key.replaceAll("s$",""), value);
+                }
+            }
+        }
+        ArrayNode gcs = om.createArrayNode();
+
+        for (int i = 0; i < gcNames.size(); i++) {
+            ObjectNode gc = om.createObjectNode();
+            gc.set("name", gcNames.get(i));
+            gc.set("collection-time", ((ArrayNode) gcCollectionTimes.get(i)).get(index));
+            gc.set("collection-count", ((ArrayNode) gcCollectionCounts.get(i)).get(index));
+            gcs.add(gc);
+        }
+        json.set("gcs", gcs);
+
+        return json;
+    }
+
+    private ObjectNode processNodeConfig(IHyracksClientConnection hcc, String node) throws Exception {
+        String config = hcc.getNodeDetailsJSON(node, false, true);
+        if (config == null) {
+            throw new IllegalArgumentException();
+        }
+        return (ObjectNode) om.readTree(config);
+    }
+
+    private ObjectNode processNodeThreadDump(IHyracksClientConnection hcc, String node) throws Exception {
+        if ("cc".equals(node)) {
+            return om.createObjectNode();
+        }
+        String dump = hcc.getThreadDump(node);
+        if (dump == null) {
+            // check to see if this is a node that is simply down
+            throw ClusterStateManager.INSTANCE.getNodePartitions(node) != null
+                    ? new IllegalStateException()
+                    : new IllegalArgumentException();
+        }
+        return (ObjectNode) om.readTree(dump);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
new file mode 100644
index 0000000..917d9a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
@@ -0,0 +1,50 @@
+/*
+ * 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.api.http.server;
+
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.hyracks.http.server.IServletRequest;
+
+public class QueryApiServlet extends RestApiServlet {
+    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY;
+
+    public QueryApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
+        super(ctx, paths, compilationProvider, statementExecutorFactory);
+    }
+
+    @Override
+    protected String getQueryParameter(IServletRequest request) {
+        return request.getParameter("query");
+    }
+
+    @Override
+    protected byte getAllowedCategories() {
+        return ALLOWED_CATEGORIES;
+    }
+
+    @Override
+    protected String getErrorMessage() {
+        return "Invalid statement: Non-query statement %s to the query API.";
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
new file mode 100644
index 0000000..6240f51
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
@@ -0,0 +1,133 @@
+/*
+ * 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.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.api.http.servlet.HyracksProperties;
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResultUtil;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.translator.IStatementExecutor.Stats;
+import org.apache.asterix.translator.SessionConfig;
+import org.apache.hyracks.api.client.HyracksConnection;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.dataset.IHyracksDataset;
+import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.client.dataset.HyracksDataset;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class QueryResultApiServlet extends AbstractServlet {
+    private static final Logger LOGGER = Logger.getLogger(QueryResultApiServlet.class.getName());
+
+    public QueryResultApiServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        if (request.getHttpRequest().method() != HttpMethod.GET) {
+            response.setStatus(HttpResponseStatus.METHOD_NOT_ALLOWED);
+            return;
+        }
+        response.setStatus(HttpResponseStatus.OK);
+        // TODO this seems wrong ...
+        try {
+            IServletResponse.setContentType(response, IServlet.ContentType.TEXT_HTML, IServlet.Encoding.UTF8);
+        } catch (IOException e) {
+            LOGGER.log(Level.WARNING, "Failure setting content type", e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        }
+        String strHandle = request.getParameter("handle");
+        PrintWriter out = response.writer();
+        IHyracksClientConnection hcc;
+        IHyracksDataset hds;
+
+        try {
+            if (strHandle == null || strHandle.isEmpty()) {
+                throw new AsterixException("Empty request, no handle provided");
+            }
+
+            HyracksProperties hp = new HyracksProperties();
+            String strIP = hp.getHyracksIPAddress();
+            int port = hp.getHyracksPort();
+
+            hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
+            hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
+            if (hcc == null || hds == null) {
+                synchronized (ctx) {
+                    hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
+                    hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
+                    if (hcc == null) {
+                        hcc = new HyracksConnection(strIP, port);
+                        ctx.put(HYRACKS_CONNECTION_ATTR, hcc);
+                    }
+                    if (hds == null) {
+                        hds = new HyracksDataset(hcc, ResultReader.FRAME_SIZE, ResultReader.NUM_READERS);
+                        ctx.put(HYRACKS_DATASET_ATTR, hds);
+                    }
+                }
+            }
+            ObjectMapper om = new ObjectMapper();
+            ObjectNode handleObj = (ObjectNode) om.readTree(strHandle);
+            ArrayNode handle = (ArrayNode) handleObj.get("handle");
+            JobId jobId = new JobId(handle.get(0).asLong());
+            ResultSetId rsId = new ResultSetId(handle.get(1).asLong());
+            ResultReader resultReader = new ResultReader(hds);
+            resultReader.open(jobId, rsId);
+
+            // QQQ The output format is determined by the initial
+            // query and cannot be modified here, so calling back to
+            // initResponse() is really an error. We need to find a
+            // way to send the same OutputFormat value here as was
+            // originally determined there. Need to save this value on
+            // some object that we can obtain here.
+            SessionConfig sessionConfig = RestApiServlet.initResponse(request, response);
+            ResultUtil.printResults(resultReader, sessionConfig, new Stats(), null);
+
+        } catch (Exception e) {
+            response.setStatus(HttpResponseStatus.BAD_REQUEST);
+            out.println(e.getMessage());
+            LOGGER.log(Level.WARNING, "Error retrieving result", e);
+        }
+        if (out.checkError()) {
+            LOGGER.warning("Error flushing output writer");
+        }
+    }
+
+}


[2/5] asterixdb git commit: Replace Servlets with Netty Based HTTP Servers

Posted by ti...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ShutdownAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ShutdownAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ShutdownAPIServlet.java
deleted file mode 100644
index d07166e..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ShutdownAPIServlet.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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.Date;
-import java.util.logging.Level;
-
-import javax.servlet.ServletContext;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.runtime.util.ClusterStateManager;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-public class ShutdownAPIServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-    public static final String NODE_ID_KEY = "node_id";
-    public static final String NCSERVICE_PID = "ncservice_pid";
-    public static final String INI = "ini";
-    public static final String PID = "pid";
-
-    @Override
-    protected void doPost(HttpServletRequest request, HttpServletResponse response)
-            throws ServletException, IOException {
-
-        ServletContext context = getServletContext();
-        IHyracksClientConnection hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
-        boolean terminateNCServices = "true".equalsIgnoreCase(request.getParameter("all"));
-        Thread t = new Thread(() -> {
-            try {
-                hcc.stopCluster(terminateNCServices);
-            } catch (Exception e) {
-                GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, "Exception stopping cluster", e);
-            }
-        }, "Shutdown Servlet Worker");
-
-        response.setContentType("application/json");
-        response.setCharacterEncoding("utf-8");
-        response.setStatus(HttpServletResponse.SC_ACCEPTED);
-        ObjectMapper om = new ObjectMapper();
-        ObjectNode jsonObject = om.createObjectNode();
-        try {
-            jsonObject.put("status", "SHUTTING_DOWN");
-            jsonObject.putPOJO("date", new Date());
-            ObjectNode clusterState = ClusterStateManager.INSTANCE.getClusterStateDescription();
-            ArrayNode ncs = (ArrayNode) clusterState.get("ncs");
-            for (int i = 0; i < ncs.size(); i++) {
-                ObjectNode nc = (ObjectNode) ncs.get(i);
-                String node = nc.get(NODE_ID_KEY).asText();
-                ObjectNode details = (ObjectNode) om.readTree(hcc.getNodeDetailsJSON(node, false, true));
-                nc.set(PID, details.get(PID));
-                if (details.has(INI) && details.get(INI).has(NCSERVICE_PID)) {
-                    nc.put(NCSERVICE_PID, details.get(INI).get(NCSERVICE_PID).asInt());
-                }
-            }
-            jsonObject.set("cluster", clusterState);
-            final PrintWriter writer = response.getWriter();
-            writer.print(om.writeValueAsString(jsonObject));
-            writer.close();
-        } catch (Exception e) {
-            GlobalConfig.ASTERIX_LOGGER.log(Level.INFO, "Exception writing response", e);
-        }
-        t.start();
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/UpdateAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/UpdateAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/UpdateAPIServlet.java
deleted file mode 100644
index 1c62cdb..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/UpdateAPIServlet.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.api.http.servlet;
-
-import javax.servlet.http.HttpServletRequest;
-
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-
-public class UpdateAPIServlet extends RESTAPIServlet {
-    private static final long serialVersionUID = 1L;
-    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE;
-
-    public UpdateAPIServlet(ILangCompilationProvider compilationProvider,
-            IStatementExecutorFactory statementExecutorFactory) {
-        super(compilationProvider, statementExecutorFactory);
-    }
-
-    @Override
-    protected String getQueryParameter(HttpServletRequest request) {
-        return request.getParameter("statements");
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        return "Invalid statement: Non-Update statement %s to the Update API.";
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/VersionAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/VersionAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/VersionAPIServlet.java
deleted file mode 100644
index 398a355..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/VersionAPIServlet.java
+++ /dev/null
@@ -1,55 +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.api.http.servlet;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.Map;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.asterix.runtime.util.AppContextInfo;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.ASTERIX_BUILD_PROP_ATTR;
-
-public class VersionAPIServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        ServletContext context = getServletContext();
-        AppContextInfo props = (AppContextInfo) context.getAttribute(ASTERIX_BUILD_PROP_ATTR);
-        Map<String, String> buildProperties = props.getBuildProperties().getAllProps();
-        ObjectMapper om = new ObjectMapper();
-        ObjectNode responseObject = om.createObjectNode();
-        for (Map.Entry<String, String> e : buildProperties.entrySet()) {
-            responseObject.put(e.getKey(), e.getValue());
-        }
-        response.setCharacterEncoding("utf-8");
-        PrintWriter responseWriter = response.getWriter();
-        responseWriter.write(responseObject.toString());
-        response.setStatus(HttpServletResponse.SC_OK);
-        responseWriter.flush();
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
----------------------------------------------------------------------
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
index f42beb3..6be7af9 100644
--- 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
@@ -88,7 +88,7 @@ public class FeedWorkCollection {
                     DataverseDecl dataverseDecl = new DataverseDecl(
                             new Identifier(request.getReceivingFeedId().getDataverse()));
                     SubscribeFeedStatement subscribeStmt = new SubscribeFeedStatement(locations, request);
-                    List<Statement> statements = new ArrayList<Statement>();
+                    List<Statement> statements = new ArrayList<>();
                     statements.add(dataverseDecl);
                     statements.add(subscribeStmt);
                     IStatementExecutor translator = qtFactory.create(statements, pc, compilationProvider);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultUtil.java
index 80b8ffe..3afb653 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultUtil.java
@@ -33,8 +33,6 @@ import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.asterix.api.http.servlet.APIServlet;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionConfig;
@@ -43,8 +41,9 @@ import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.log4j.Logger;
-import com.fasterxml.jackson.databind.node.ArrayNode;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class ResultUtil {
@@ -244,7 +243,7 @@ public class ResultUtil {
         String errorTemplate = defaultTemplate;
         try {
             String resourcePath = "/webui/errortemplate_message.html";
-            InputStream is = APIServlet.class.getResourceAsStream(resourcePath);
+            InputStream is = ResultUtil.class.getResourceAsStream(resourcePath);
             InputStreamReader isr = new InputStreamReader(is);
             StringBuilder sb = new StringBuilder();
             BufferedReader br = new BufferedReader(isr);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
----------------------------------------------------------------------
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 f039271..d60ed37 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
@@ -46,7 +46,7 @@ import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.IActiveEntityEventsListener;
 import org.apache.asterix.algebra.extension.IExtensionStatement;
 import org.apache.asterix.api.common.APIFramework;
-import org.apache.asterix.api.http.servlet.APIServlet;
+import org.apache.asterix.api.http.server.ApiServlet;
 import org.apache.asterix.app.external.ExternalIndexingOperations;
 import org.apache.asterix.app.external.FeedJoint;
 import org.apache.asterix.app.external.FeedOperations;
@@ -284,7 +284,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         try {
             for (Statement stmt : statements) {
                 if (sessionConfig.is(SessionConfig.FORMAT_HTML)) {
-                    sessionConfig.out().println(APIServlet.HTML_STATEMENT_SEPARATOR);
+                    sessionConfig.out().println(ApiServlet.HTML_STATEMENT_SEPARATOR);
                 }
                 validateOperation(activeDefaultDataverse, stmt);
                 rewriteStatement(stmt); // Rewrite the statement's AST.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixWebServer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixWebServer.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixWebServer.java
deleted file mode 100644
index f0eac39..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixWebServer.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.drivers;
-
-import org.apache.asterix.api.http.servlet.APIServlet;
-import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
-import org.apache.asterix.compiler.provider.AqlCompilationProvider;
-import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.ServletHolder;
-
-public class AsterixWebServer {
-    public static void main(String[] args) throws Exception {
-        Server server = new Server(8080);
-        ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
-        context.setContextPath("/");
-        server.setHandler(context);
-
-        context.addServlet(new ServletHolder(new APIServlet(new AqlCompilationProvider(),
-                new SqlppCompilationProvider(), new DefaultStatementExecutorFactory())), "/*");
-        server.start();
-        server.join();
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 5756e7d..3a8852e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -21,7 +21,6 @@ package org.apache.asterix.hyracks.bootstrap;
 import static org.apache.asterix.api.http.servlet.ServletConstants.ASTERIX_BUILD_PROP_ATTR;
 import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -29,24 +28,24 @@ import java.util.logging.Logger;
 import javax.servlet.Servlet;
 
 import org.apache.asterix.active.ActiveLifecycleListener;
-import org.apache.asterix.api.http.servlet.APIServlet;
-import org.apache.asterix.api.http.servlet.ClusterAPIServlet;
-import org.apache.asterix.api.http.servlet.ClusterCCDetailsAPIServlet;
-import org.apache.asterix.api.http.servlet.ClusterNodeDetailsAPIServlet;
-import org.apache.asterix.api.http.servlet.ConnectorAPIServlet;
-import org.apache.asterix.api.http.servlet.DDLAPIServlet;
-import org.apache.asterix.api.http.servlet.DiagnosticsAPIServlet;
-import org.apache.asterix.api.http.servlet.FeedServlet;
-import org.apache.asterix.api.http.servlet.FullAPIServlet;
-import org.apache.asterix.api.http.servlet.QueryAPIServlet;
-import org.apache.asterix.api.http.servlet.QueryResultAPIServlet;
-import org.apache.asterix.api.http.servlet.QueryServiceServlet;
-import org.apache.asterix.api.http.servlet.QueryStatusAPIServlet;
-import org.apache.asterix.api.http.servlet.QueryWebInterfaceServlet;
+import org.apache.asterix.api.http.server.ApiServlet;
+import org.apache.asterix.api.http.server.FullApiServlet;
+import org.apache.asterix.api.http.server.ClusterApiServlet;
+import org.apache.asterix.api.http.server.ClusterControllerDetailsApiServlet;
+import org.apache.asterix.api.http.server.ConnectorApiServlet;
+import org.apache.asterix.api.http.server.DdlApiServlet;
+import org.apache.asterix.api.http.server.DiagnosticsApiServlet;
+import org.apache.asterix.api.http.server.FeedServlet;
+import org.apache.asterix.api.http.server.NodeControllerDetailsApiServlet;
+import org.apache.asterix.api.http.server.QueryApiServlet;
+import org.apache.asterix.api.http.server.QueryResultApiServlet;
+import org.apache.asterix.api.http.server.QueryServiceServlet;
+import org.apache.asterix.api.http.server.QueryStatusApiServlet;
+import org.apache.asterix.api.http.server.QueryWebInterfaceServlet;
+import org.apache.asterix.api.http.server.ShutdownApiServlet;
+import org.apache.asterix.api.http.server.UpdateApiServlet;
+import org.apache.asterix.api.http.server.VersionApiServlet;
 import org.apache.asterix.api.http.servlet.ServletConstants;
-import org.apache.asterix.api.http.servlet.ShutdownAPIServlet;
-import org.apache.asterix.api.http.servlet.UpdateAPIServlet;
-import org.apache.asterix.api.http.servlet.VersionAPIServlet;
 import org.apache.asterix.app.cc.CompilerExtensionManager;
 import org.apache.asterix.app.cc.ResourceIdManager;
 import org.apache.asterix.app.external.ExternalLibraryUtils;
@@ -55,7 +54,7 @@ import org.apache.asterix.common.config.AsterixExtension;
 import org.apache.asterix.common.config.ExternalProperties;
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.common.utils.ServletUtil.Servlets;
+import org.apache.asterix.common.utils.LetUtil.Lets;
 import org.apache.asterix.external.library.ExternalLibraryManager;
 import org.apache.asterix.messaging.CCMessageBroker;
 import org.apache.asterix.metadata.MetadataManager;
@@ -73,7 +72,8 @@ import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
 import org.apache.hyracks.api.messages.IMessageBroker;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.common.controllers.CCConfig;
-import org.eclipse.jetty.server.Server;
+import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.IServlet;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.servlet.ServletMapping;
@@ -81,13 +81,11 @@ import org.eclipse.jetty.servlet.ServletMapping;
 public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
 
     private static final Logger LOGGER = Logger.getLogger(CCApplicationEntryPoint.class.getName());
-
-    private List<Server> servers;
-
     private static IAsterixStateProxy proxy;
     protected ICCApplicationContext appCtx;
     protected CompilerExtensionManager ccExtensionManager;
     private IJobCapacityController jobCapacityController;
+    protected WebManager webManager;
 
     @Override
     public void start(ICCApplicationContext ccAppCtx, String[] args) throws Exception {
@@ -124,14 +122,11 @@ public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
         AppContextInfo.INSTANCE.getCCApplicationContext()
                 .addJobLifecycleListener(ActiveLifecycleListener.INSTANCE);
 
-        servers = configureServers();
-
-        for (Server server : servers) {
-            server.start();
-        }
-
+        // create event loop groups
+        webManager = new WebManager();
+        configureServers();
+        webManager.start();
         ClusterManagerProvider.getClusterManager().registerSubscriber(GlobalRecoveryManager.instance());
-
         ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
         ccAppCtx.setMessageBroker(messageBroker);
 
@@ -142,15 +137,11 @@ public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
         return AppContextInfo.INSTANCE.getExtensionProperties().getExtensions();
     }
 
-    protected List<Server> configureServers() throws Exception {
-        ExternalProperties externalProperties = AppContextInfo.INSTANCE.getExternalProperties();
-
-        List<Server> serverList = new ArrayList<>();
-        serverList.add(setupWebServer(externalProperties));
-        serverList.add(setupJSONAPIServer(externalProperties));
-        serverList.add(setupFeedServer(externalProperties));
-        serverList.add(setupQueryWebServer(externalProperties));
-        return serverList;
+    protected void configureServers() throws Exception {
+        webManager.add(setupWebServer(AppContextInfo.INSTANCE.getExternalProperties()));
+        webManager.add(setupJSONAPIServer(AppContextInfo.INSTANCE.getExternalProperties()));
+        webManager.add(setupFeedServer(AppContextInfo.INSTANCE.getExternalProperties()));
+        webManager.add(setupQueryWebServer(AppContextInfo.INSTANCE.getExternalProperties()));
     }
 
     @Override
@@ -160,14 +151,7 @@ public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
             LOGGER.info("Stopping Asterix cluster controller");
         }
         AsterixStateProxy.unregisterRemoteObject();
-        // Stop servers
-        for (Server server : servers) {
-            server.stop();
-        }
-        // Make sure servers are stopped before proceeding
-        for (Server server : servers) {
-            server.join();
-        }
+        webManager.stop();
     }
 
     protected IHyracksClientConnection getNewHyracksClientConnection() throws Exception {
@@ -176,77 +160,62 @@ public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
         return new HyracksConnection(strIP, port);
     }
 
-    protected Server setupWebServer(ExternalProperties externalProperties) throws Exception {
-
-        Server webServer = new Server(externalProperties.getWebInterfacePort());
-
-        ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
-        context.setContextPath("/");
-
+    protected HttpServer setupWebServer(ExternalProperties externalProperties) throws Exception {
+        HttpServer webServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties
+                .getWebInterfacePort());
         IHyracksClientConnection hcc = getNewHyracksClientConnection();
-        context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
-
-        webServer.setHandler(context);
-        context.addServlet(new ServletHolder(new APIServlet(ccExtensionManager.getAqlCompilationProvider(),
-                ccExtensionManager.getSqlppCompilationProvider(), ccExtensionManager.getQueryTranslatorFactory())),
-                "/*");
-
+        webServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
+        webServer.addLet(new ApiServlet(webServer.ctx(), new String[] { "/*" }, ccExtensionManager
+                .getAqlCompilationProvider(), ccExtensionManager.getSqlppCompilationProvider(), ccExtensionManager
+                        .getQueryTranslatorFactory()));
         return webServer;
     }
 
-    protected Server setupJSONAPIServer(ExternalProperties externalProperties) throws Exception {
-        Server jsonAPIServer = new Server(externalProperties.getAPIServerPort());
-
-        ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
-        context.setContextPath("/");
-
+    protected HttpServer setupJSONAPIServer(ExternalProperties externalProperties) throws Exception {
+        HttpServer jsonAPIServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties
+                .getAPIServerPort());
         IHyracksClientConnection hcc = getNewHyracksClientConnection();
-        context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
-        context.setAttribute(ASTERIX_BUILD_PROP_ATTR, AppContextInfo.INSTANCE);
-        context.setAttribute(ServletConstants.EXECUTOR_SERVICE,
+        jsonAPIServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
+        jsonAPIServer.setAttribute(ASTERIX_BUILD_PROP_ATTR, AppContextInfo.INSTANCE);
+        jsonAPIServer.setAttribute(ServletConstants.EXECUTOR_SERVICE,
                 ((ClusterControllerService) appCtx.getControllerService()).getExecutor());
 
-        jsonAPIServer.setHandler(context);
-
         // AQL rest APIs.
-        addServlet(context, Servlets.AQL_QUERY);
-        addServlet(context, Servlets.AQL_UPDATE);
-        addServlet(context, Servlets.AQL_DDL);
-        addServlet(context, Servlets.AQL);
+        addLet(jsonAPIServer, Lets.AQL_QUERY);
+        addLet(jsonAPIServer, Lets.AQL_UPDATE);
+        addLet(jsonAPIServer, Lets.AQL_DDL);
+        addLet(jsonAPIServer, Lets.AQL);
 
         // SQL+x+ rest APIs.
-        addServlet(context, Servlets.SQLPP_QUERY);
-        addServlet(context, Servlets.SQLPP_UPDATE);
-        addServlet(context, Servlets.SQLPP_DDL);
-        addServlet(context, Servlets.SQLPP);
+        addLet(jsonAPIServer, Lets.SQLPP_QUERY);
+        addLet(jsonAPIServer, Lets.SQLPP_UPDATE);
+        addLet(jsonAPIServer, Lets.SQLPP_DDL);
+        addLet(jsonAPIServer, Lets.SQLPP);
 
         // Other APIs.
-        addServlet(context, Servlets.QUERY_STATUS);
-        addServlet(context, Servlets.QUERY_RESULT);
-        addServlet(context, Servlets.QUERY_SERVICE);
-        addServlet(context, Servlets.CONNECTOR);
-        addServlet(context, Servlets.SHUTDOWN);
-        addServlet(context, Servlets.VERSION);
-        addServlet(context, Servlets.CLUSTER_STATE);
-        addServlet(context, Servlets.CLUSTER_STATE_NODE_DETAIL); // this must not precede add of CLUSTER_STATE
-        addServlet(context, Servlets.CLUSTER_STATE_CC_DETAIL); // this must not precede add of CLUSTER_STATE
-        addServlet(context, Servlets.DIAGNOSTICS);
-
+        addLet(jsonAPIServer, Lets.QUERY_STATUS);
+        addLet(jsonAPIServer, Lets.QUERY_RESULT);
+        addLet(jsonAPIServer, Lets.QUERY_SERVICE);
+        addLet(jsonAPIServer, Lets.CONNECTOR);
+        addLet(jsonAPIServer, Lets.SHUTDOWN);
+        addLet(jsonAPIServer, Lets.VERSION);
+        addLet(jsonAPIServer, Lets.CLUSTER_STATE);
+        addLet(jsonAPIServer, Lets.CLUSTER_STATE_NODE_DETAIL); // this must not precede add of CLUSTER_STATE
+        addLet(jsonAPIServer, Lets.CLUSTER_STATE_CC_DETAIL); // this must not precede add of CLUSTER_STATE
+        addLet(jsonAPIServer, Lets.DIAGNOSTICS);
         return jsonAPIServer;
     }
 
-    protected Server setupQueryWebServer(ExternalProperties externalProperties) throws Exception {
-
-        Server queryWebServer = new Server(externalProperties.getQueryWebInterfacePort());
-
-        ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
-        context.setContextPath("/");
+    protected void addLet(HttpServer server, Lets let) {
+        server.addLet(createServLet(server, let, let.getPath()));
+    }
 
+    protected HttpServer setupQueryWebServer(ExternalProperties externalProperties) throws Exception {
+        HttpServer queryWebServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties
+                .getQueryWebInterfacePort());
         IHyracksClientConnection hcc = getNewHyracksClientConnection();
-        context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
-
-        queryWebServer.setHandler(context);
-        context.addServlet(new ServletHolder(new QueryWebInterfaceServlet()), "/*");
+        queryWebServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
+        queryWebServer.addLet(new QueryWebInterfaceServlet(queryWebServer.ctx(), new String[] { "/*" }));
         return queryWebServer;
     }
 
@@ -259,77 +228,66 @@ public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
         context.getServletHandler().addServletMapping(mapping);
     }
 
-    protected void addServlet(ServletContextHandler context, Servlets key) {
-        addServlet(context, createServlet(key), key.getPath());
+    protected HttpServer setupFeedServer(ExternalProperties externalProperties) throws Exception {
+        HttpServer feedServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties
+                .getFeedServerPort());
+        feedServer.setAttribute(HYRACKS_CONNECTION_ATTR, getNewHyracksClientConnection());
+        feedServer.addLet(new FeedServlet(feedServer.ctx(), new String[] { "/" }));
+        return feedServer;
     }
 
-    protected Servlet createServlet(Servlets key) {
+    protected IServlet createServLet(HttpServer server, Lets key, String... paths) {
         switch (key) {
             case AQL:
-                return new FullAPIServlet(ccExtensionManager.getAqlCompilationProvider(),
+                return new FullApiServlet(server.ctx(), paths, ccExtensionManager.getAqlCompilationProvider(),
                         ccExtensionManager.getQueryTranslatorFactory());
             case AQL_QUERY:
-                return new QueryAPIServlet(ccExtensionManager.getAqlCompilationProvider(),
+                return new QueryApiServlet(server.ctx(), paths, ccExtensionManager.getAqlCompilationProvider(),
                         ccExtensionManager.getQueryTranslatorFactory());
             case AQL_UPDATE:
-                return new UpdateAPIServlet(ccExtensionManager.getAqlCompilationProvider(),
+                return new UpdateApiServlet(server.ctx(), paths, ccExtensionManager.getAqlCompilationProvider(),
                         ccExtensionManager.getQueryTranslatorFactory());
             case AQL_DDL:
-                return new DDLAPIServlet(ccExtensionManager.getAqlCompilationProvider(),
+                return new DdlApiServlet(server.ctx(), paths, ccExtensionManager.getAqlCompilationProvider(),
                         ccExtensionManager.getQueryTranslatorFactory());
             case SQLPP:
-                return new FullAPIServlet(ccExtensionManager.getSqlppCompilationProvider(),
+                return new FullApiServlet(server.ctx(), paths, ccExtensionManager.getSqlppCompilationProvider(),
                         ccExtensionManager.getQueryTranslatorFactory());
             case SQLPP_QUERY:
-                return new QueryAPIServlet(ccExtensionManager.getSqlppCompilationProvider(),
+                return new QueryApiServlet(server.ctx(), paths, ccExtensionManager.getSqlppCompilationProvider(),
                         ccExtensionManager.getQueryTranslatorFactory());
             case SQLPP_UPDATE:
-                return new UpdateAPIServlet(ccExtensionManager.getSqlppCompilationProvider(),
+                return new UpdateApiServlet(server.ctx(), paths, ccExtensionManager.getSqlppCompilationProvider(),
                         ccExtensionManager.getQueryTranslatorFactory());
             case SQLPP_DDL:
-                return new DDLAPIServlet(ccExtensionManager.getSqlppCompilationProvider(),
+                return new DdlApiServlet(server.ctx(), paths, ccExtensionManager.getSqlppCompilationProvider(),
                         ccExtensionManager.getQueryTranslatorFactory());
             case QUERY_STATUS:
-                return new QueryStatusAPIServlet();
+                return new QueryStatusApiServlet(server.ctx(), paths);
             case QUERY_RESULT:
-                return new QueryResultAPIServlet();
+                return new QueryResultApiServlet(server.ctx(), paths);
             case QUERY_SERVICE:
-                return new QueryServiceServlet(ccExtensionManager.getSqlppCompilationProvider(),
-                        ccExtensionManager.getQueryTranslatorFactory());
+                return new QueryServiceServlet(server.ctx(), paths, ccExtensionManager
+                        .getSqlppCompilationProvider(), ccExtensionManager.getQueryTranslatorFactory());
             case CONNECTOR:
-                return new ConnectorAPIServlet();
+                return new ConnectorApiServlet(server.ctx(), paths);
             case SHUTDOWN:
-                return new ShutdownAPIServlet();
+                return new ShutdownApiServlet(server.ctx(), paths);
             case VERSION:
-                return new VersionAPIServlet();
+                return new VersionApiServlet(server.ctx(), paths);
             case CLUSTER_STATE:
-                return new ClusterAPIServlet();
+                return new ClusterApiServlet(server.ctx(), paths);
             case CLUSTER_STATE_NODE_DETAIL:
-                return new ClusterNodeDetailsAPIServlet();
+                return new NodeControllerDetailsApiServlet(server.ctx(), paths);
             case CLUSTER_STATE_CC_DETAIL:
-                return new ClusterCCDetailsAPIServlet();
+                return new ClusterControllerDetailsApiServlet(server.ctx(), paths);
             case DIAGNOSTICS:
-                return new DiagnosticsAPIServlet();
+                return new DiagnosticsApiServlet(server.ctx(), paths);
             default:
                 throw new IllegalStateException(String.valueOf(key));
         }
     }
 
-    protected Server setupFeedServer(ExternalProperties externalProperties) throws Exception {
-        Server feedServer = new Server(externalProperties.getFeedServerPort());
-
-        ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
-        context.setContextPath("/");
-
-        IHyracksClientConnection hcc = getNewHyracksClientConnection();
-        context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
-
-        feedServer.setHandler(context);
-        context.addServlet(new ServletHolder(new FeedServlet()), "/");
-
-        return feedServer;
-    }
-
     @Override
     public void startupCompleted() throws Exception {
         ClusterManagerProvider.getClusterManager().notifyStartupCompleted();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/WebManager.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/WebManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/WebManager.java
new file mode 100644
index 0000000..070a530
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/WebManager.java
@@ -0,0 +1,69 @@
+/*
+ * 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.hyracks.bootstrap;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.http.server.HttpServer;
+
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+
+public class WebManager {
+    private final List<HttpServer> servers;
+    private final EventLoopGroup bosses;
+    private final EventLoopGroup workers;
+
+    public WebManager() {
+        servers = new ArrayList<>();
+        bosses = new NioEventLoopGroup(1);
+        workers = new NioEventLoopGroup();
+    }
+
+    public List<HttpServer> getServers() {
+        return servers;
+    }
+
+    public EventLoopGroup getBosses() {
+        return bosses;
+    }
+
+    public EventLoopGroup getWorkers() {
+        return workers;
+    }
+
+    public void start() throws Exception {
+        for (HttpServer server : servers) {
+            server.start();
+        }
+    }
+
+    public void stop() throws Exception {
+        for (HttpServer server : servers) {
+            server.stop();
+        }
+        workers.shutdownGracefully().sync();
+        bosses.shutdownGracefully().sync();
+    }
+
+    public void add(HttpServer server) {
+        servers.add(server);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java
deleted file mode 100644
index 41785e5..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java
+++ /dev/null
@@ -1,177 +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.api.http.servlet;
-
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
-import java.io.ByteArrayOutputStream;
-import java.io.PrintWriter;
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.servlet.ServletConfig;
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import junit.extensions.PA;
-import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.metadata.MetadataTransactionContext;
-import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.JSONDeserializerForTypes;
-import org.apache.asterix.test.runtime.SqlppExecutionTest;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.client.NodeControllerInfo;
-import org.apache.hyracks.api.comm.NetworkAddress;
-import org.apache.hyracks.api.io.FileSplit;
-import org.apache.hyracks.api.io.ManagedFileSplit;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class ConnectorAPIServletTest {
-
-    @Test
-    public void testGet() throws Exception {
-        // Starts test asterixdb cluster.
-        SqlppExecutionTest.setUp();
-
-        // Configures a test connector api servlet.
-        ConnectorAPIServlet servlet = spy(new ConnectorAPIServlet());
-        ServletConfig mockServletConfig = mock(ServletConfig.class);
-        servlet.init(mockServletConfig);
-        Map<String, NodeControllerInfo> nodeMap = new HashMap<>();
-        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        PrintWriter outputWriter = new PrintWriter(outputStream);
-
-        // Creates mocks.
-        ServletContext mockContext = mock(ServletContext.class);
-        IHyracksClientConnection mockHcc = mock(IHyracksClientConnection.class);
-        NodeControllerInfo mockInfo1 = mock(NodeControllerInfo.class);
-        NodeControllerInfo mockInfo2 = mock(NodeControllerInfo.class);
-        HttpServletRequest mockRequest = mock(HttpServletRequest.class);
-        HttpServletResponse mockResponse = mock(HttpServletResponse.class);
-
-        // Sets up mock returns.
-        when(servlet.getServletContext()).thenReturn(mockContext);
-        when(mockRequest.getParameter("dataverseName")).thenReturn("Metadata");
-        when(mockRequest.getParameter("datasetName")).thenReturn("Dataset");
-        when(mockResponse.getWriter()).thenReturn(outputWriter);
-        when(mockContext.getAttribute(anyString())).thenReturn(mockHcc);
-        when(mockHcc.getNodeControllerInfos()).thenReturn(nodeMap);
-        when(mockInfo1.getNetworkAddress()).thenReturn(new NetworkAddress("127.0.0.1", 3099));
-        when(mockInfo2.getNetworkAddress()).thenReturn(new NetworkAddress("127.0.0.2", 3099));
-
-        // Calls ConnectorAPIServlet.formResponseObject.
-        nodeMap.put("asterix_nc1", mockInfo1);
-        nodeMap.put("asterix_nc2", mockInfo2);
-        servlet.doGet(mockRequest, mockResponse);
-
-        // Constructs the actual response.
-        ObjectMapper om = new ObjectMapper();
-        ObjectNode actualResponse = (ObjectNode) om.readTree(outputStream.toString());
-
-        // Checks the temp-or-not, primary key, data type of the dataset.
-        boolean temp = actualResponse.get("temp").asBoolean();
-        Assert.assertFalse(temp);
-        String primaryKey = actualResponse.get("keys").asText();
-        Assert.assertEquals("DataverseName,DatasetName", primaryKey);
-        ARecordType recordType = (ARecordType) JSONDeserializerForTypes
-                .convertFromJSON((ObjectNode) actualResponse.get("type"));
-        Assert.assertEquals(getMetadataRecordType("Metadata", "Dataset"), recordType);
-
-        // Checks the correctness of results.
-        ArrayNode splits = (ArrayNode) actualResponse.get("splits");
-        String path = (splits.get(0)).get("path").asText();
-        Assert.assertTrue(path.endsWith("Metadata/Dataset_idx_Dataset"));
-
-        // Tears down the asterixdb cluster.
-        SqlppExecutionTest.tearDown();
-    }
-
-    @Test
-    public void testFormResponseObject() throws Exception {
-        ConnectorAPIServlet servlet = new ConnectorAPIServlet();
-        ObjectMapper om = new ObjectMapper();
-        ObjectNode actualResponse = om.createObjectNode();
-        FileSplit[] splits = new FileSplit[2];
-        splits[0] = new ManagedFileSplit("asterix_nc1", "foo1");
-        splits[1] = new ManagedFileSplit("asterix_nc2", "foo2");
-        Map<String, NodeControllerInfo> nodeMap = new HashMap<>();
-        NodeControllerInfo mockInfo1 = mock(NodeControllerInfo.class);
-        NodeControllerInfo mockInfo2 = mock(NodeControllerInfo.class);
-
-        // Sets up mock returns.
-        when(mockInfo1.getNetworkAddress()).thenReturn(new NetworkAddress("127.0.0.1", 3099));
-        when(mockInfo2.getNetworkAddress()).thenReturn(new NetworkAddress("127.0.0.2", 3099));
-
-        String[] fieldNames = new String[] { "a1", "a2" };
-        IAType[] fieldTypes = new IAType[] { BuiltinType.ABOOLEAN, BuiltinType.ADAYTIMEDURATION };
-        ARecordType recordType = new ARecordType("record", fieldNames, fieldTypes, true);
-        String primaryKey = "a1";
-
-        // Calls ConnectorAPIServlet.formResponseObject.
-        nodeMap.put("asterix_nc1", mockInfo1);
-        nodeMap.put("asterix_nc2", mockInfo2);
-        PA.invokeMethod(servlet, "formResponseObject(" + ObjectNode.class.getName() + ", " + FileSplit.class.getName()
-                + "[], " + ARecordType.class.getName() + ", " + String.class.getName() + ", boolean, " + Map.class
-                        .getName() + ")", actualResponse, splits, recordType, primaryKey, true, nodeMap);
-        // Constructs expected response.
-        ObjectNode expectedResponse = om.createObjectNode();
-        expectedResponse.put("temp", true);
-        expectedResponse.put("keys", primaryKey);
-        expectedResponse.set("type", recordType.toJSON());
-        ArrayNode splitsArray = om.createArrayNode();
-        ObjectNode element1 = om.createObjectNode();
-        element1.put("ip", "127.0.0.1");
-        element1.put("path", splits[0].getPath());
-        ObjectNode element2 = om.createObjectNode();
-        element2.put("ip", "127.0.0.2");
-        element2.put("path", splits[1].getPath());
-        splitsArray.add(element1);
-        splitsArray.add(element2);
-        expectedResponse.set("splits", splitsArray);
-
-        // Checks results.
-        Assert.assertEquals(actualResponse.toString(), expectedResponse.toString());
-    }
-
-    private ARecordType getMetadataRecordType(String dataverseName, String datasetName) throws Exception {
-        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        // Retrieves file splits of the dataset.
-        MetadataProvider metadataProvider = new MetadataProvider(null);
-        metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
-        ARecordType recordType = (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(),
-                dataset.getItemTypeName());
-        // Metadata transaction commits.
-        MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-        return recordType;
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiLetTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiLetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiLetTest.java
new file mode 100644
index 0000000..b1fdab5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiLetTest.java
@@ -0,0 +1,178 @@
+/*
+ * 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.api.http.servlet;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.asterix.api.http.server.ConnectorApiServlet;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.MetadataTransactionContext;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.util.JSONDeserializerForTypes;
+import org.apache.asterix.test.runtime.SqlppExecutionTest;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.client.NodeControllerInfo;
+import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.io.FileSplit;
+import org.apache.hyracks.api.io.ManagedFileSplit;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.FullHttpRequest;
+import io.netty.handler.codec.http.HttpMethod;
+import junit.extensions.PA;
+
+public class ConnectorApiLetTest {
+
+    @Test
+    public void testGet() throws Exception {
+        // Starts test asterixdb cluster.
+        SqlppExecutionTest.setUp();
+
+        // Configures a test connector api servlet.
+        ConnectorApiServlet let = new ConnectorApiServlet(new ConcurrentHashMap<>(), new String[] { "/" });
+        Map<String, NodeControllerInfo> nodeMap = new HashMap<>();
+        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        PrintWriter outputWriter = new PrintWriter(outputStream);
+
+        // Creates mocks.
+        IHyracksClientConnection mockHcc = mock(IHyracksClientConnection.class);
+        NodeControllerInfo mockInfo1 = mock(NodeControllerInfo.class);
+        NodeControllerInfo mockInfo2 = mock(NodeControllerInfo.class);
+        IServletRequest mockRequest = mock(IServletRequest.class);
+        IServletResponse mockResponse = mock(IServletResponse.class);
+        FullHttpRequest mockHttpRequest = mock(FullHttpRequest.class);
+
+        // Put stuff in let map
+        let.ctx().put(ServletConstants.HYRACKS_CONNECTION_ATTR, mockHcc);
+        // Sets up mock returns.
+        when(mockRequest.getHttpRequest()).thenReturn(mockHttpRequest);
+        when(mockHttpRequest.method()).thenReturn(HttpMethod.GET);
+        when(mockRequest.getParameter("dataverseName")).thenReturn("Metadata");
+        when(mockRequest.getParameter("datasetName")).thenReturn("Dataset");
+        when(mockResponse.writer()).thenReturn(outputWriter);
+        when(mockHcc.getNodeControllerInfos()).thenReturn(nodeMap);
+        when(mockInfo1.getNetworkAddress()).thenReturn(new NetworkAddress("127.0.0.1", 3099));
+        when(mockInfo2.getNetworkAddress()).thenReturn(new NetworkAddress("127.0.0.2", 3099));
+
+        // Calls ConnectorAPIServlet.formResponseObject.
+        nodeMap.put("asterix_nc1", mockInfo1);
+        nodeMap.put("asterix_nc2", mockInfo2);
+        let.handle(mockRequest, mockResponse);
+
+        // Constructs the actual response.
+        ObjectMapper om = new ObjectMapper();
+        ObjectNode actualResponse = (ObjectNode) om.readTree(outputStream.toString());
+
+        // Checks the temp-or-not, primary key, data type of the dataset.
+        boolean temp = actualResponse.get("temp").asBoolean();
+        Assert.assertFalse(temp);
+        String primaryKey = actualResponse.get("keys").asText();
+        Assert.assertEquals("DataverseName,DatasetName", primaryKey);
+        ARecordType recordType = (ARecordType) JSONDeserializerForTypes
+                .convertFromJSON(actualResponse.get("type"));
+        Assert.assertEquals(getMetadataRecordType("Metadata", "Dataset"), recordType);
+
+        // Checks the correctness of results.
+        ArrayNode splits = (ArrayNode) actualResponse.get("splits");
+        String path = (splits.get(0)).get("path").asText();
+        Assert.assertTrue(path.endsWith("Metadata/Dataset_idx_Dataset"));
+
+        // Tears down the asterixdb cluster.
+        SqlppExecutionTest.tearDown();
+    }
+
+    @Test
+    public void testFormResponseObject() throws Exception {
+        ConnectorApiServlet let = new ConnectorApiServlet(new ConcurrentHashMap<>(), new String[] { "/" });
+        ObjectMapper om = new ObjectMapper();
+        ObjectNode actualResponse = om.createObjectNode();
+        FileSplit[] splits = new FileSplit[2];
+        splits[0] = new ManagedFileSplit("asterix_nc1", "foo1");
+        splits[1] = new ManagedFileSplit("asterix_nc2", "foo2");
+        Map<String, NodeControllerInfo> nodeMap = new HashMap<>();
+        NodeControllerInfo mockInfo1 = mock(NodeControllerInfo.class);
+        NodeControllerInfo mockInfo2 = mock(NodeControllerInfo.class);
+
+        // Sets up mock returns.
+        when(mockInfo1.getNetworkAddress()).thenReturn(new NetworkAddress("127.0.0.1", 3099));
+        when(mockInfo2.getNetworkAddress()).thenReturn(new NetworkAddress("127.0.0.2", 3099));
+
+        String[] fieldNames = new String[] { "a1", "a2" };
+        IAType[] fieldTypes = new IAType[] { BuiltinType.ABOOLEAN, BuiltinType.ADAYTIMEDURATION };
+        ARecordType recordType = new ARecordType("record", fieldNames, fieldTypes, true);
+        String primaryKey = "a1";
+
+        // Calls ConnectorAPIServlet.formResponseObject.
+        nodeMap.put("asterix_nc1", mockInfo1);
+        nodeMap.put("asterix_nc2", mockInfo2);
+        PA.invokeMethod(let, "formResponseObject(" + ObjectNode.class.getName() + ", " + FileSplit.class.getName()
+                + "[], " + ARecordType.class.getName() + ", " + String.class.getName() + ", boolean, " + Map.class
+                        .getName() + ")", actualResponse, splits, recordType, primaryKey, true, nodeMap);
+        // Constructs expected response.
+        ObjectNode expectedResponse = om.createObjectNode();
+        expectedResponse.put("temp", true);
+        expectedResponse.put("keys", primaryKey);
+        expectedResponse.set("type", recordType.toJSON());
+        ArrayNode splitsArray = om.createArrayNode();
+        ObjectNode element1 = om.createObjectNode();
+        element1.put("ip", "127.0.0.1");
+        element1.put("path", splits[0].getPath());
+        ObjectNode element2 = om.createObjectNode();
+        element2.put("ip", "127.0.0.2");
+        element2.put("path", splits[1].getPath());
+        splitsArray.add(element1);
+        splitsArray.add(element2);
+        expectedResponse.set("splits", splitsArray);
+
+        // Checks results.
+        Assert.assertEquals(actualResponse.toString(), expectedResponse.toString());
+    }
+
+    private ARecordType getMetadataRecordType(String dataverseName, String datasetName) throws Exception {
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        // Retrieves file splits of the dataset.
+        MetadataProvider metadataProvider = new MetadataProvider(null);
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
+        ARecordType recordType = (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(),
+                dataset.getItemTypeName());
+        // Metadata transaction commits.
+        MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        return recordType;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceLetTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceLetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceLetTest.java
new file mode 100644
index 0000000..7b7c0d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceLetTest.java
@@ -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.
+ */
+
+package org.apache.asterix.api.http.servlet;
+
+import org.apache.asterix.api.http.server.QueryServiceServlet;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class QueryServiceLetTest {
+
+    @Test
+    public void testTimeUnitFormatNanos() throws Exception {
+        Assert.assertEquals("123.456789012s", QueryServiceServlet.TimeUnit.formatNanos(123456789012l));
+        Assert.assertEquals("12.345678901s", QueryServiceServlet.TimeUnit.formatNanos(12345678901l));
+        Assert.assertEquals("1.234567890s", QueryServiceServlet.TimeUnit.formatNanos(1234567890l));
+        Assert.assertEquals("123.456789ms", QueryServiceServlet.TimeUnit.formatNanos(123456789l));
+        Assert.assertEquals("12.345678ms", QueryServiceServlet.TimeUnit.formatNanos(12345678l));
+        Assert.assertEquals("1.234567ms", QueryServiceServlet.TimeUnit.formatNanos(1234567l));
+        Assert.assertEquals("123.456�s", QueryServiceServlet.TimeUnit.formatNanos(123456l));
+        Assert.assertEquals("12.345�s", QueryServiceServlet.TimeUnit.formatNanos(12345l));
+        Assert.assertEquals("1.234�s", QueryServiceServlet.TimeUnit.formatNanos(1234l));
+        Assert.assertEquals("123ns", QueryServiceServlet.TimeUnit.formatNanos(123l));
+        Assert.assertEquals("12ns", QueryServiceServlet.TimeUnit.formatNanos(12l));
+        Assert.assertEquals("1ns", QueryServiceServlet.TimeUnit.formatNanos(1l));
+        Assert.assertEquals("-123.456789012s", QueryServiceServlet.TimeUnit.formatNanos(-123456789012l));
+        Assert.assertEquals("120.000000000s", QueryServiceServlet.TimeUnit.formatNanos(120000000000l));
+        Assert.assertEquals("-12ns", QueryServiceServlet.TimeUnit.formatNanos(-12l));
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceServletTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceServletTest.java
deleted file mode 100644
index f4889a0..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceServletTest.java
+++ /dev/null
@@ -1,45 +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.api.http.servlet;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class QueryServiceServletTest {
-
-    @Test
-    public void testTimeUnitFormatNanos() throws Exception {
-        Assert.assertEquals("123.456789012s", QueryServiceServlet.TimeUnit.formatNanos(123456789012l));
-        Assert.assertEquals("12.345678901s", QueryServiceServlet.TimeUnit.formatNanos(12345678901l));
-        Assert.assertEquals("1.234567890s", QueryServiceServlet.TimeUnit.formatNanos(1234567890l));
-        Assert.assertEquals("123.456789ms", QueryServiceServlet.TimeUnit.formatNanos(123456789l));
-        Assert.assertEquals("12.345678ms", QueryServiceServlet.TimeUnit.formatNanos(12345678l));
-        Assert.assertEquals("1.234567ms", QueryServiceServlet.TimeUnit.formatNanos(1234567l));
-        Assert.assertEquals("123.456�s", QueryServiceServlet.TimeUnit.formatNanos(123456l));
-        Assert.assertEquals("12.345�s", QueryServiceServlet.TimeUnit.formatNanos(12345l));
-        Assert.assertEquals("1.234�s", QueryServiceServlet.TimeUnit.formatNanos(1234l));
-        Assert.assertEquals("123ns", QueryServiceServlet.TimeUnit.formatNanos(123l));
-        Assert.assertEquals("12ns", QueryServiceServlet.TimeUnit.formatNanos(12l));
-        Assert.assertEquals("1ns", QueryServiceServlet.TimeUnit.formatNanos(1l));
-        Assert.assertEquals("-123.456789012s", QueryServiceServlet.TimeUnit.formatNanos(-123456789012l));
-        Assert.assertEquals("120.000000000s", QueryServiceServlet.TimeUnit.formatNanos(120000000000l));
-        Assert.assertEquals("-12ns", QueryServiceServlet.TimeUnit.formatNanos(-12l));
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionAPIServletTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionAPIServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionAPIServletTest.java
deleted file mode 100644
index da0ec0c..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionAPIServletTest.java
+++ /dev/null
@@ -1,120 +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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.ASTERIX_BUILD_PROP_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.InputStreamReader;
-import java.io.PrintWriter;
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.servlet.ServletConfig;
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.asterix.common.config.BuildProperties;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.test.runtime.SqlppExecutionTest;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class VersionAPIServletTest {
-
-    @Test
-    public void testGet() throws Exception {
-        // Starts test asterixdb cluster.
-        SqlppExecutionTest.setUp();
-
-        // Configures a test version api servlet.
-        VersionAPIServlet servlet = spy(new VersionAPIServlet());
-        ServletConfig mockServletConfig = mock(ServletConfig.class);
-        servlet.init(mockServletConfig);
-        Map<String, String> propMap = new HashMap<String, String>();
-        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        PrintWriter outputWriter = new PrintWriter(outputStream);
-
-        // Creates mocks.
-        ServletContext mockContext = mock(ServletContext.class);
-        AppContextInfo mockCtx = mock(AppContextInfo.class);
-        HttpServletRequest mockRequest = mock(HttpServletRequest.class);
-        IHyracksClientConnection mockHcc = mock(IHyracksClientConnection.class);
-        HttpServletResponse mockResponse = mock(HttpServletResponse.class);
-        BuildProperties mockProperties = mock(BuildProperties.class);
-
-        // Sets up mock returns.
-        when(servlet.getServletContext()).thenReturn(mockContext);
-        when(mockResponse.getWriter()).thenReturn(outputWriter);
-        when(mockContext.getAttribute(HYRACKS_CONNECTION_ATTR)).thenReturn(mockHcc);
-        when(mockContext.getAttribute(ASTERIX_BUILD_PROP_ATTR)).thenReturn(mockCtx);
-        when(mockCtx.getBuildProperties()).thenReturn(mockProperties);
-        when(mockProperties.getAllProps()).thenReturn(propMap);
-
-        propMap.put("git.build.user.email", "foo@bar.baz");
-        propMap.put("git.build.host", "fulliautomatix");
-        propMap.put("git.dirty", "true");
-        propMap.put("git.remote.origin.url", "git@github.com:apache/incubator-asterixdb.git");
-        propMap.put("git.closest.tag.name", "asterix-0.8.7-incubating");
-        propMap.put("git.commit.id.describe-short", "asterix-0.8.7-incubating-19-dirty");
-        propMap.put("git.commit.user.email", "foo@bar.baz");
-        propMap.put("git.commit.time", "21.10.2015 @ 23:36:41 PDT");
-        propMap.put("git.commit.message.full",
-                "ASTERIXDB-1045: fix log file reading during recovery\n\nChange-Id: Ic83ee1dd2d7ba88180c25f4ec6c7aa8d0a5a7162\nReviewed-on: https://asterix-gerrit.ics.uci.edu/465\nTested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>");
-        propMap.put("git.build.version", "0.8.8-SNAPSHOT");
-        propMap.put("git.commit.message.short", "ASTERIXDB-1045: fix log file reading during recovery");
-        propMap.put("git.commit.id.abbrev", "e1dad19");
-        propMap.put("git.branch", "foo/bar");
-        propMap.put("git.build.user.name", "Asterix");
-        propMap.put("git.closest.tag.commit.count", "19");
-        propMap.put("git.commit.id.describe", "asterix-0.8.7-incubating-19-ge1dad19-dirty");
-        propMap.put("git.commit.id", "e1dad1984640517366a7e73e323c9de27b0676f7");
-        propMap.put("git.tags", "");
-        propMap.put("git.build.time", "22.10.2015 @ 17:11:07 PDT");
-        propMap.put("git.commit.user.name", "Obelix");
-
-        // Calls VersionAPIServlet.formResponseObject.
-        servlet.doGet(mockRequest, mockResponse);
-
-        // Constructs the actual response.
-
-        ObjectMapper om = new ObjectMapper();
-        ObjectNode actualResponse = (ObjectNode) om.readTree(outputStream.toByteArray());
-        ObjectNode expectedResponse = om.createObjectNode();
-        for (Map.Entry<String, String> e : propMap.entrySet()) {
-            expectedResponse.put(e.getKey(), e.getValue());
-        }
-
-        // Checks the response contains all the expected keys.
-        Assert.assertEquals(actualResponse.toString(), expectedResponse.toString());
-
-        // Tears down the asterixdb cluster.
-        SqlppExecutionTest.tearDown();
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiLetTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiLetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiLetTest.java
new file mode 100644
index 0000000..936b717
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiLetTest.java
@@ -0,0 +1,120 @@
+/*
+ * 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.api.http.servlet;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.ASTERIX_BUILD_PROP_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.asterix.api.http.server.VersionApiServlet;
+import org.apache.asterix.common.config.BuildProperties;
+import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.test.runtime.SqlppExecutionTest;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.FullHttpRequest;
+import io.netty.handler.codec.http.HttpMethod;
+
+public class VersionApiLetTest {
+
+    @Test
+    public void testGet() throws Exception {
+        // Starts test asterixdb cluster.
+        SqlppExecutionTest.setUp();
+
+        // Configures a test version api servlet.
+        VersionApiServlet servlet = new VersionApiServlet(new ConcurrentHashMap<>(), new String[] { "/" });
+        Map<String, String> propMap = new HashMap<>();
+        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        PrintWriter outputWriter = new PrintWriter(outputStream);
+
+        // Creates mocks.
+        AppContextInfo mockCtx = mock(AppContextInfo.class);
+        IServletRequest mockRequest = mock(IServletRequest.class);
+        IHyracksClientConnection mockHcc = mock(IHyracksClientConnection.class);
+        IServletResponse mockResponse = mock(IServletResponse.class);
+        BuildProperties mockProperties = mock(BuildProperties.class);
+        FullHttpRequest mockHttpRequest = mock(FullHttpRequest.class);
+
+        // Put stuff in let map
+        servlet.ctx().put(HYRACKS_CONNECTION_ATTR, mockHcc);
+        servlet.ctx().put(ASTERIX_BUILD_PROP_ATTR, mockCtx);
+        // Sets up mock returns.
+        when(mockResponse.writer()).thenReturn(outputWriter);
+        when(mockRequest.getHttpRequest()).thenReturn(mockHttpRequest);
+        when(mockHttpRequest.method()).thenReturn(HttpMethod.GET);
+        when(mockCtx.getBuildProperties()).thenReturn(mockProperties);
+        when(mockProperties.getAllProps()).thenReturn(propMap);
+
+        propMap.put("git.build.user.email", "foo@bar.baz");
+        propMap.put("git.build.host", "fulliautomatix");
+        propMap.put("git.dirty", "true");
+        propMap.put("git.remote.origin.url", "git@github.com:apache/incubator-asterixdb.git");
+        propMap.put("git.closest.tag.name", "asterix-0.8.7-incubating");
+        propMap.put("git.commit.id.describe-short", "asterix-0.8.7-incubating-19-dirty");
+        propMap.put("git.commit.user.email", "foo@bar.baz");
+        propMap.put("git.commit.time", "21.10.2015 @ 23:36:41 PDT");
+        propMap.put("git.commit.message.full",
+                "ASTERIXDB-1045: fix log file reading during recovery\n\nChange-Id: Ic83ee1dd2d7ba88180c25f4ec6c7aa8d0a5a7162\nReviewed-on: https://asterix-gerrit.ics.uci.edu/465\nTested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>");
+        propMap.put("git.build.version", "0.8.8-SNAPSHOT");
+        propMap.put("git.commit.message.short", "ASTERIXDB-1045: fix log file reading during recovery");
+        propMap.put("git.commit.id.abbrev", "e1dad19");
+        propMap.put("git.branch", "foo/bar");
+        propMap.put("git.build.user.name", "Asterix");
+        propMap.put("git.closest.tag.commit.count", "19");
+        propMap.put("git.commit.id.describe", "asterix-0.8.7-incubating-19-ge1dad19-dirty");
+        propMap.put("git.commit.id", "e1dad1984640517366a7e73e323c9de27b0676f7");
+        propMap.put("git.tags", "");
+        propMap.put("git.build.time", "22.10.2015 @ 17:11:07 PDT");
+        propMap.put("git.commit.user.name", "Obelix");
+
+        // Calls VersionAPIServlet.formResponseObject.
+        servlet.handle(mockRequest, mockResponse);
+
+        // Constructs the actual response.
+
+        ObjectMapper om = new ObjectMapper();
+        ObjectNode actualResponse = (ObjectNode) om.readTree(outputStream.toByteArray());
+        ObjectNode expectedResponse = om.createObjectNode();
+        for (Map.Entry<String, String> e : propMap.entrySet()) {
+            expectedResponse.put(e.getKey(), e.getValue());
+        }
+
+        // Checks the response contains all the expected keys.
+        Assert.assertEquals(actualResponse.toString(), expectedResponse.toString());
+
+        // Tears down the asterixdb cluster.
+        SqlppExecutionTest.tearDown();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/LetUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/LetUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/LetUtil.java
new file mode 100644
index 0000000..2fe9380
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/LetUtil.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.common.utils;
+
+public class LetUtil {
+
+    public enum Lets {
+        AQL("/aql"),
+        AQL_QUERY("/query"),
+        AQL_UPDATE("/update"),
+        AQL_DDL("/ddl"),
+        SQLPP("/sqlpp"),
+        SQLPP_QUERY("/query/sqlpp"),
+        SQLPP_UPDATE("/update/sqlpp"),
+        SQLPP_DDL("/ddl/sqlpp"),
+        QUERY_STATUS("/query/status"),
+        QUERY_RESULT("/query/result"),
+        QUERY_SERVICE("/query/service"),
+        CONNECTOR("/connector"),
+        SHUTDOWN("/admin/shutdown"),
+        VERSION("/admin/version"),
+        CLUSTER_STATE("/admin/cluster/*"),
+        CLUSTER_STATE_NODE_DETAIL("/admin/cluster/node/*"),
+        CLUSTER_STATE_CC_DETAIL("/admin/cluster/cc/*"),
+        DIAGNOSTICS("/admin/diagnostics");
+
+        private final String path;
+
+        Lets(String path) {
+            this.path = path;
+        }
+
+        public String getPath() {
+            return path;
+        }
+
+    }
+
+    private LetUtil() {
+        throw new AssertionError("No objects of this class should be created.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java
deleted file mode 100644
index f0b124d..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java
+++ /dev/null
@@ -1,58 +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.common.utils;
-
-public class ServletUtil {
-
-    public enum Servlets {
-        AQL("/aql"),
-        AQL_QUERY("/query"),
-        AQL_UPDATE("/update"),
-        AQL_DDL("/ddl"),
-        SQLPP("/sqlpp"),
-        SQLPP_QUERY("/query/sqlpp"),
-        SQLPP_UPDATE("/update/sqlpp"),
-        SQLPP_DDL("/ddl/sqlpp"),
-        QUERY_STATUS("/query/status"),
-        QUERY_RESULT("/query/result"),
-        QUERY_SERVICE("/query/service"),
-        CONNECTOR("/connector"),
-        SHUTDOWN("/admin/shutdown"),
-        VERSION("/admin/version"),
-        CLUSTER_STATE("/admin/cluster/*"),
-        CLUSTER_STATE_NODE_DETAIL("/admin/cluster/node/*"),
-        CLUSTER_STATE_CC_DETAIL("/admin/cluster/cc/*"),
-        DIAGNOSTICS("/admin/diagnostics");
-
-        private final String path;
-
-        Servlets(String path) {
-            this.path = path;
-        }
-
-        public String getPath() {
-            return path;
-        }
-
-    }
-
-    private ServletUtil() {
-        throw new AssertionError("No objects of this class should be created.");
-    }
-}


[4/5] asterixdb git commit: Replace Servlets with Netty Based HTTP Servers

Posted by ti...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
new file mode 100644
index 0000000..b302bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -0,0 +1,557 @@
+/*
+ * 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.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResultUtil;
+import org.apache.asterix.app.translator.QueryTranslator;
+import org.apache.asterix.common.api.IClusterManagementWork;
+import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.utils.JSONUtil;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.aql.parser.TokenMgrError;
+import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutor.Stats;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.asterix.translator.SessionConfig;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.dataset.IHyracksDataset;
+import org.apache.hyracks.client.dataset.HyracksDataset;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class QueryServiceServlet extends AbstractServlet {
+    private static final Logger LOGGER = Logger.getLogger(QueryServiceServlet.class.getName());
+    private final ILangCompilationProvider compilationProvider;
+    private final IStatementExecutorFactory statementExecutorFactory;
+
+    public QueryServiceServlet(ConcurrentMap<String, Object> ctx, String[] paths,
+            ILangCompilationProvider compilationProvider,
+            IStatementExecutorFactory statementExecutorFactory) {
+        super(ctx, paths);
+        this.compilationProvider = compilationProvider;
+        this.statementExecutorFactory = statementExecutorFactory;
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        if (request.getHttpRequest().method() != HttpMethod.POST) {
+            response.setStatus(HttpResponseStatus.METHOD_NOT_ALLOWED);
+            return;
+        }
+        try {
+            handleRequest(getRequestParameters(request), response);
+        } catch (IOException e) {
+            // Servlet methods should not throw exceptions
+            // http://cwe.mitre.org/data/definitions/600.html
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
+        }
+    }
+
+    public enum Parameter {
+        STATEMENT("statement"),
+        FORMAT("format"),
+        CLIENT_ID("client_context_id"),
+        PRETTY("pretty"),
+        MODE("mode");
+
+        private final String str;
+
+        Parameter(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
+    private enum Attribute {
+        HEADER("header"),
+        LOSSLESS("lossless");
+
+        private final String str;
+
+        Attribute(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
+    public enum ResultFields {
+        REQUEST_ID("requestID"),
+        CLIENT_ID("clientContextID"),
+        SIGNATURE("signature"),
+        TYPE("type"),
+        STATUS("status"),
+        RESULTS("results"),
+        HANDLE("handle"),
+        ERRORS("errors"),
+        METRICS("metrics");
+
+        private final String str;
+
+        ResultFields(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
+    public enum ResultStatus {
+        SUCCESS("success"),
+        TIMEOUT("timeout"),
+        ERRORS("errors"),
+        FATAL("fatal");
+
+        private final String str;
+
+        ResultStatus(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
+    private enum ErrorField {
+        CODE("code"),
+        MSG("msg"),
+        STACK("stack");
+
+        private final String str;
+
+        ErrorField(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
+    private enum Metrics {
+        ELAPSED_TIME("elapsedTime"),
+        EXECUTION_TIME("executionTime"),
+        RESULT_COUNT("resultCount"),
+        RESULT_SIZE("resultSize");
+
+        private final String str;
+
+        Metrics(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
+    public enum TimeUnit {
+        SEC("s", 9),
+        MILLI("ms", 6),
+        MICRO("�s", 3),
+        NANO("ns", 0);
+
+        String unit;
+        int nanoDigits;
+
+        TimeUnit(String unit, int nanoDigits) {
+            this.unit = unit;
+            this.nanoDigits = nanoDigits;
+        }
+
+        public static String formatNanos(long nanoTime) {
+            final String strTime = String.valueOf(nanoTime);
+            final int len = strTime.length();
+            for (TimeUnit tu : TimeUnit.values()) {
+                if (len > tu.nanoDigits) {
+                    final String integer = strTime.substring(0, len - tu.nanoDigits);
+                    final String fractional = strTime.substring(len - tu.nanoDigits);
+                    return integer + (fractional.length() > 0 ? "." + fractional : "") + tu.unit;
+                }
+            }
+            return "illegal string value: " + strTime;
+        }
+    }
+
+    static class RequestParameters {
+        String statement;
+        String format;
+        boolean pretty;
+        String clientContextID;
+        String mode;
+
+        @Override
+        public String toString() {
+            return append(new StringBuilder()).toString();
+        }
+
+        public StringBuilder append(final StringBuilder sb) {
+            sb.append("{ ");
+            sb.append("\"statement\": \"");
+            JSONUtil.escape(sb, statement);
+            sb.append("\", ");
+            sb.append("\"format\": \"").append(format).append("\", ");
+            sb.append("\"pretty\": ").append(pretty).append(", ");
+            sb.append("\"mode\": ").append(mode).append(", ");
+            sb.append("\"clientContextID\": \"").append(clientContextID).append("\" ");
+            sb.append('}');
+            return sb;
+        }
+    }
+
+    private static String getParameterValue(String content, String attribute) {
+        if (content == null || attribute == null) {
+            return null;
+        }
+        int sc = content.indexOf(';');
+        if (sc < 0) {
+            return null;
+        }
+        int eq = content.indexOf('=', sc + 1);
+        if (eq < 0) {
+            return null;
+        }
+        if (content.substring(sc + 1, eq).trim().equalsIgnoreCase(attribute)) {
+            return content.substring(eq + 1).trim().toLowerCase();
+        }
+        return null;
+    }
+
+    private static String toLower(String s) {
+        return s != null ? s.toLowerCase() : s;
+    }
+
+    private static SessionConfig.OutputFormat getFormat(String format) {
+        if (format != null) {
+            if (format.startsWith(IServlet.ContentType.CSV)) {
+                return SessionConfig.OutputFormat.CSV;
+            }
+            if (format.equals(IServlet.ContentType.APPLICATION_ADM)) {
+                return SessionConfig.OutputFormat.ADM;
+            }
+            if (format.startsWith(IServlet.ContentType.APPLICATION_JSON)) {
+                return Boolean.parseBoolean(getParameterValue(format, Attribute.LOSSLESS.str()))
+                        ? SessionConfig.OutputFormat.LOSSLESS_JSON : SessionConfig.OutputFormat.CLEAN_JSON;
+            }
+        }
+        return SessionConfig.OutputFormat.CLEAN_JSON;
+    }
+
+    private static SessionConfig createSessionConfig(RequestParameters param, PrintWriter resultWriter) {
+        SessionConfig.ResultDecorator resultPrefix = new SessionConfig.ResultDecorator() {
+            int resultNo = -1;
+
+            @Override
+            public AlgebricksAppendable append(AlgebricksAppendable app) throws AlgebricksException {
+                app.append("\t\"");
+                app.append(ResultFields.RESULTS.str());
+                if (resultNo >= 0) {
+                    app.append('-').append(String.valueOf(resultNo));
+                }
+                ++resultNo;
+                app.append("\": ");
+                return app;
+            }
+        };
+
+        SessionConfig.ResultDecorator resultPostfix = (AlgebricksAppendable app) -> app.append("\t,\n");
+
+        SessionConfig.ResultDecorator handlePrefix = new SessionConfig.ResultDecorator() {
+            @Override
+            public AlgebricksAppendable append(AlgebricksAppendable app) throws AlgebricksException {
+                app.append("\t\"");
+                app.append(ResultFields.HANDLE.str());
+                app.append("\": ");
+                return app;
+            }
+        };
+
+        SessionConfig.ResultDecorator handlePostfix = (AlgebricksAppendable app) -> app.append(",\n");
+
+        SessionConfig.OutputFormat format = getFormat(param.format);
+        SessionConfig sessionConfig = new SessionConfig(resultWriter, format, resultPrefix, resultPostfix,
+                handlePrefix, handlePostfix);
+        sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, true);
+        sessionConfig.set(SessionConfig.FORMAT_INDENT_JSON, param.pretty);
+        sessionConfig.set(SessionConfig.FORMAT_QUOTE_RECORD,
+                format != SessionConfig.OutputFormat.CLEAN_JSON && format != SessionConfig.OutputFormat.LOSSLESS_JSON);
+        sessionConfig.set(SessionConfig.FORMAT_CSV_HEADER, format == SessionConfig.OutputFormat.CSV
+                && "present".equals(getParameterValue(param.format, Attribute.HEADER.str())));
+        return sessionConfig;
+    }
+
+    private static void printField(PrintWriter pw, String name, String value) {
+        printField(pw, name, value, true);
+    }
+
+    private static void printField(PrintWriter pw, String name, String value, boolean comma) {
+        pw.print("\t\"");
+        pw.print(name);
+        pw.print("\": \"");
+        pw.print(value);
+        pw.print('"');
+        if (comma) {
+            pw.print(',');
+        }
+        pw.print('\n');
+    }
+
+    private static UUID printRequestId(PrintWriter pw) {
+        UUID requestId = UUID.randomUUID();
+        printField(pw, ResultFields.REQUEST_ID.str(), requestId.toString());
+        return requestId;
+    }
+
+    private static void printClientContextID(PrintWriter pw, RequestParameters params) {
+        if (params.clientContextID != null && !params.clientContextID.isEmpty()) {
+            printField(pw, ResultFields.CLIENT_ID.str(), params.clientContextID);
+        }
+    }
+
+    private static void printSignature(PrintWriter pw) {
+        printField(pw, ResultFields.SIGNATURE.str(), "*");
+    }
+
+    private static void printType(PrintWriter pw, SessionConfig sessionConfig) {
+        switch (sessionConfig.fmt()) {
+            case ADM:
+                printField(pw, ResultFields.TYPE.str(), IServlet.ContentType.APPLICATION_ADM);
+                break;
+            case CSV:
+                String contentType = IServlet.ContentType.CSV + "; header="
+                        + (sessionConfig.is(SessionConfig.FORMAT_CSV_HEADER) ? "present" : "absent");
+                printField(pw, ResultFields.TYPE.str(), contentType);
+                break;
+            default:
+                break;
+        }
+    }
+
+    private static void printStatus(PrintWriter pw, ResultStatus rs) {
+        printField(pw, ResultFields.STATUS.str(), rs.str());
+    }
+
+    private static void printError(PrintWriter pw, Throwable e) throws JsonProcessingException {
+        Throwable rootCause = ResultUtil.getRootCause(e);
+        if (rootCause == null) {
+            rootCause = e;
+        }
+        final boolean addStack = false;
+        pw.print("\t\"");
+        pw.print(ResultFields.ERRORS.str());
+        pw.print("\": [{ \n");
+        printField(pw, ErrorField.CODE.str(), "1");
+        final String msg = rootCause.getMessage();
+        printField(pw, ErrorField.MSG.str(), JSONUtil.escape(msg != null ? msg : rootCause.getClass().getSimpleName()),
+                addStack);
+        pw.print("\t}],\n");
+    }
+
+    private static void printMetrics(PrintWriter pw, long elapsedTime, long executionTime, long resultCount,
+            long resultSize) {
+        pw.print("\t\"");
+        pw.print(ResultFields.METRICS.str());
+        pw.print("\": {\n");
+        pw.print("\t");
+        printField(pw, Metrics.ELAPSED_TIME.str(), TimeUnit.formatNanos(elapsedTime));
+        pw.print("\t");
+        printField(pw, Metrics.EXECUTION_TIME.str(), TimeUnit.formatNanos(executionTime));
+        pw.print("\t");
+        printField(pw, Metrics.RESULT_COUNT.str(), String.valueOf(resultCount));
+        pw.print("\t");
+        printField(pw, Metrics.RESULT_SIZE.str(), String.valueOf(resultSize), false);
+        pw.print("\t}\n");
+    }
+
+    private String getOptText(JsonNode node, String fieldName) {
+        final JsonNode value = node.get(fieldName);
+        return value != null ? value.asText() : null;
+    }
+
+    private boolean getOptBoolean(JsonNode node, String fieldName, boolean defaultValue) {
+        final JsonNode value = node.get(fieldName);
+        return value != null ? value.asBoolean() : defaultValue;
+    }
+
+    private RequestParameters getRequestParameters(IServletRequest request) throws IOException {
+        final String contentTypeParam = request.getHttpRequest().headers().get(HttpHeaderNames.CONTENT_TYPE);
+        int sep = contentTypeParam.indexOf(';');
+        final String contentType = sep < 0 ? contentTypeParam.trim() : contentTypeParam.substring(0, sep).trim();
+        RequestParameters param = new RequestParameters();
+        if (IServlet.ContentType.APPLICATION_JSON.equals(contentType)) {
+            try {
+                JsonNode jsonRequest = new ObjectMapper().readTree(getRequestBody(request));
+                param.statement = jsonRequest.get(Parameter.STATEMENT.str()).asText();
+                param.format = toLower(getOptText(jsonRequest, Parameter.FORMAT.str()));
+                param.pretty = getOptBoolean(jsonRequest, Parameter.PRETTY.str(), false);
+                param.mode = toLower(getOptText(jsonRequest, Parameter.MODE.str()));
+                param.clientContextID = getOptText(jsonRequest, Parameter.CLIENT_ID.str());
+            } catch (JsonParseException | JsonMappingException e) {
+                // if the JSON parsing fails, the statement is empty and we get an empty statement error
+                GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
+            }
+        } else {
+            param.statement = request.getParameter(Parameter.STATEMENT.str());
+            if (param.statement == null) {
+                param.statement = getRequestBody(request);
+            }
+            param.format = toLower(request.getParameter(Parameter.FORMAT.str()));
+            param.pretty = Boolean.parseBoolean(request.getParameter(Parameter.PRETTY.str()));
+            param.mode = toLower(request.getParameter(Parameter.MODE.str()));
+            param.clientContextID = request.getParameter(Parameter.CLIENT_ID.str());
+        }
+        return param;
+    }
+
+    private static String getRequestBody(IServletRequest request) throws IOException {
+        return request.getHttpRequest().content().toString(StandardCharsets.UTF_8);
+    }
+
+    private static QueryTranslator.ResultDelivery parseResultDelivery(String mode) {
+        if ("async".equals(mode)) {
+            return QueryTranslator.ResultDelivery.ASYNC;
+        } else if ("deferred".equals(mode)) {
+            return QueryTranslator.ResultDelivery.DEFERRED;
+        } else {
+            return QueryTranslator.ResultDelivery.IMMEDIATE;
+        }
+    }
+
+    private void handleRequest(RequestParameters param, IServletResponse response) throws IOException {
+        LOGGER.info(param.toString());
+        long elapsedStart = System.nanoTime();
+        final StringWriter stringWriter = new StringWriter();
+        final PrintWriter resultWriter = new PrintWriter(stringWriter);
+
+        QueryTranslator.ResultDelivery delivery = parseResultDelivery(param.mode);
+
+        SessionConfig sessionConfig = createSessionConfig(param, resultWriter);
+        IServletResponse.setContentType(response, IServlet.ContentType.APPLICATION_JSON, IServlet.Encoding.UTF8);
+
+        HttpResponseStatus status = HttpResponseStatus.OK;
+        Stats stats = new Stats();
+        long execStart = -1;
+        long execEnd = -1;
+
+        resultWriter.print("{\n");
+        printRequestId(resultWriter);
+        printClientContextID(resultWriter, param);
+        printSignature(resultWriter);
+        printType(resultWriter, sessionConfig);
+        try {
+            final IClusterManagementWork.ClusterState clusterState = ClusterStateManager.INSTANCE.getState();
+            if (clusterState != IClusterManagementWork.ClusterState.ACTIVE) {
+                // using a plain IllegalStateException here to get into the right catch clause for a 500
+                throw new IllegalStateException("Cannot execute request, cluster is " + clusterState);
+            }
+            if (param.statement == null || param.statement.isEmpty()) {
+                throw new AsterixException("Empty request, no statement provided");
+            }
+            IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
+            IHyracksDataset hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
+            if (hds == null) {
+                synchronized (ctx) {
+                    hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
+                    if (hds == null) {
+                        hds = new HyracksDataset(hcc, ResultReader.FRAME_SIZE, ResultReader.NUM_READERS);
+                        ctx.put(HYRACKS_DATASET_ATTR, hds);
+                    }
+                }
+            }
+            IParser parser = compilationProvider.getParserFactory().createParser(param.statement);
+            List<Statement> statements = parser.parse();
+            MetadataManager.INSTANCE.init();
+            IStatementExecutor translator = statementExecutorFactory.create(statements, sessionConfig,
+                    compilationProvider);
+            execStart = System.nanoTime();
+            translator.compileAndExecute(hcc, hds, delivery, stats);
+            execEnd = System.nanoTime();
+            printStatus(resultWriter, ResultStatus.SUCCESS);
+        } catch (AsterixException | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, pe.getMessage(), pe);
+            printError(resultWriter, pe);
+            printStatus(resultWriter, ResultStatus.FATAL);
+            status = HttpResponseStatus.BAD_REQUEST;
+        } catch (Exception e) {
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
+            printError(resultWriter, e);
+            printStatus(resultWriter, ResultStatus.FATAL);
+            status = HttpResponseStatus.INTERNAL_SERVER_ERROR;
+        } finally {
+            if (execStart == -1) {
+                execEnd = -1;
+            } else if (execEnd == -1) {
+                execEnd = System.nanoTime();
+            }
+        }
+        printMetrics(resultWriter, System.nanoTime() - elapsedStart, execEnd - execStart, stats.getCount(),
+                stats.getSize());
+        resultWriter.print("}\n");
+        resultWriter.flush();
+        String result = stringWriter.toString();
+
+        GlobalConfig.ASTERIX_LOGGER.log(Level.FINE, result);
+
+        response.setStatus(status);
+        response.writer().print(result);
+        if (response.writer().checkError()) {
+            LOGGER.warning("Error flushing output writer");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
new file mode 100644
index 0000000..197a39c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
@@ -0,0 +1,118 @@
+/*
+ * 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.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.dataset.IHyracksDataset;
+import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.client.dataset.HyracksDataset;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class QueryStatusApiServlet extends AbstractServlet {
+    private static final Logger LOGGER = Logger.getLogger(QueryStatusApiServlet.class.getName());
+
+    public QueryStatusApiServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        if (request.getHttpRequest().method() != HttpMethod.GET) {
+            response.setStatus(HttpResponseStatus.METHOD_NOT_ALLOWED);
+            return;
+        }
+        response.setStatus(HttpResponseStatus.OK);
+        try {
+            IServletResponse.setContentType(response, IServlet.ContentType.TEXT_HTML, IServlet.Encoding.UTF8);
+        } catch (IOException e) {
+            LOGGER.log(Level.WARNING, "Failure setting content type", e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        }
+        String strHandle = request.getParameter("handle");
+        PrintWriter out = response.writer();
+        try {
+            IHyracksDataset hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
+            if (hds == null) {
+                synchronized (ctx) {
+                    hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
+                    if (hds == null) {
+                        hds = new HyracksDataset((IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR),
+                                ResultReader.FRAME_SIZE, ResultReader.NUM_READERS);
+                        ctx.put(HYRACKS_DATASET_ATTR, hds);
+                    }
+                }
+            }
+            ObjectMapper om = new ObjectMapper();
+            JsonNode handleObj = om.readTree(strHandle);
+            JsonNode handle = handleObj.get("handle");
+            JobId jobId = new JobId(handle.get(0).asLong());
+            ResultSetId rsId = new ResultSetId(handle.get(1).asLong());
+
+            /* TODO(madhusudancs): We need to find a way to LOSSLESS_JSON serialize default format obtained from
+             * metadataProvider in the AQLTranslator and store it as part of the result handle.
+             */
+            ResultReader resultReader = new ResultReader(hds);
+            resultReader.open(jobId, rsId);
+
+            ObjectNode jsonResponse = om.createObjectNode();
+            String status;
+            switch (resultReader.getStatus()) {
+                case RUNNING:
+                    status = "RUNNING";
+                    break;
+                case SUCCESS:
+                    status = "SUCCESS";
+                    break;
+                default:
+                    status = "ERROR";
+                    break;
+            }
+            jsonResponse.put("status", status);
+            out.write(jsonResponse.toString());
+
+        } catch (Exception e) {
+            LOGGER.log(Level.SEVERE, "Failure handling a request", e);
+            out.println(e.getMessage());
+            e.printStackTrace(out);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryWebInterfaceServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryWebInterfaceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryWebInterfaceServlet.java
new file mode 100644
index 0000000..d924cf1
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryWebInterfaceServlet.java
@@ -0,0 +1,133 @@
+/*
+ * 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.api.http.server;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.config.ExternalProperties;
+import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.commons.io.IOUtils;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class QueryWebInterfaceServlet extends AbstractServlet {
+    private static final Logger LOGGER = Logger.getLogger(QueryWebInterfaceServlet.class.getName());
+
+    public QueryWebInterfaceServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        try {
+            if (request.getHttpRequest().method() == HttpMethod.GET) {
+                doGet(request, response);
+            } else if (request.getHttpRequest().method() == HttpMethod.POST) {
+                doPost(response);
+            } else {
+                response.setStatus(HttpResponseStatus.METHOD_NOT_ALLOWED);
+            }
+        } catch (IOException e) {
+            LOGGER.log(Level.WARNING, "Failure setting content type", e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        }
+    }
+
+    private void doGet(IServletRequest request, IServletResponse response) throws IOException {
+        String resourcePath = null;
+        String requestURI = request.getHttpRequest().uri();
+        response.setStatus(HttpResponseStatus.OK);
+
+        if ("/".equals(requestURI)) {
+            IServletResponse.setContentType(response, IServlet.ContentType.TEXT_HTML);
+            resourcePath = "/queryui/queryui.html";
+        } else {
+            resourcePath = requestURI;
+        }
+
+        try (InputStream is = QueryWebInterfaceServlet.class.getResourceAsStream(resourcePath)) {
+            if (is == null) {
+                response.setStatus(HttpResponseStatus.NOT_FOUND);
+                return;
+            }
+            int i = resourcePath.lastIndexOf('.');
+            if (i >= 0) {
+                String extension = resourcePath.substring(i);
+                String mime = IServlet.ContentType.mime(extension);
+                if (mime != null) {
+                    OutputStream out = response.outputStream();
+                    IServletResponse.setContentType(response, mime);
+                    try {
+                        IOUtils.copy(is, out);
+                    } catch (Exception e) {
+                        LOGGER.log(Level.WARNING, "Failure copying response", e);
+                    } finally {
+                        if (out != null) {
+                            IOUtils.closeQuietly(out);
+                        }
+                        IOUtils.closeQuietly(is);
+                    }
+                    return;
+                }
+            }
+            response.setStatus(HttpResponseStatus.BAD_REQUEST);
+        }
+    }
+
+    private void doPost(IServletResponse response) throws IOException {
+        IServletResponse.setContentType(response, IServlet.ContentType.APPLICATION_JSON, IServlet.Encoding.UTF8);
+        ExternalProperties externalProperties = AppContextInfo.INSTANCE.getExternalProperties();
+        response.setStatus(HttpResponseStatus.OK);
+        ObjectMapper om = new ObjectMapper();
+        ObjectNode obj = om.createObjectNode();
+        try {
+            PrintWriter out = response.writer();
+            obj.put("api_port", String.valueOf(externalProperties.getAPIServerPort()));
+            out.println(obj.toString());
+            return;
+        } catch (Exception e) {
+            LOGGER.log(Level.SEVERE, "Failure writing response", e);
+        }
+        try {
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+        } catch (Exception e) {
+            LOGGER.log(Level.SEVERE, "Failure setting response status", e);
+        }
+    }
+
+    public static String extension(String path) {
+        int i = path.lastIndexOf('.');
+        return i < 1 ? "" : path.substring(i);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
new file mode 100644
index 0000000..c2d1d33
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
@@ -0,0 +1,248 @@
+/*
+ * 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.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResultUtil;
+import org.apache.asterix.app.translator.QueryTranslator;
+import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.aql.parser.TokenMgrError;
+import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.IParserFactory;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.asterix.translator.SessionConfig;
+import org.apache.asterix.translator.SessionConfig.OutputFormat;
+import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.dataset.IHyracksDataset;
+import org.apache.hyracks.client.dataset.HyracksDataset;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public abstract class RestApiServlet extends AbstractServlet {
+    private static final Logger LOGGER = Logger.getLogger(RestApiServlet.class.getName());
+    private final ILangCompilationProvider compilationProvider;
+    private final IParserFactory parserFactory;
+    private final IStatementExecutorFactory statementExecutorFactory;
+
+    public RestApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
+        super(ctx, paths);
+        this.compilationProvider = compilationProvider;
+        this.parserFactory = compilationProvider.getParserFactory();
+        this.statementExecutorFactory = statementExecutorFactory;
+    }
+
+    /**
+     * Initialize the Content-Type of the response, and construct a
+     * SessionConfig with the appropriate output writer and output-format
+     * based on the Accept: header and other servlet parameters.
+     */
+    static SessionConfig initResponse(IServletRequest request, IServletResponse response) throws IOException {
+        IServletResponse.setContentType(response, IServlet.ContentType.TEXT_PLAIN, IServlet.Encoding.UTF8);
+
+        // CLEAN_JSON output is the default; most generally useful for a
+        // programmatic HTTP API
+        OutputFormat format = OutputFormat.CLEAN_JSON;
+        // First check the "output" servlet parameter.
+        String output = request.getParameter("output");
+        String accept = request.getHeader("Accept");
+        if (accept == null) {
+            accept = "";
+        }
+        if (output != null) {
+            if ("CSV".equals(output)) {
+                format = OutputFormat.CSV;
+            } else if ("ADM".equals(output)) {
+                format = OutputFormat.ADM;
+            }
+        } else {
+            // Second check the Accept: HTTP header.
+            if (accept.contains("application/x-adm")) {
+                format = OutputFormat.ADM;
+            } else if (accept.contains("text/csv")) {
+                format = OutputFormat.CSV;
+            }
+        }
+
+        // If it's JSON, check for the "lossless" flag
+
+        if (format == OutputFormat.CLEAN_JSON
+                && ("true".equals(request.getParameter("lossless")) || accept.contains("lossless=true"))) {
+            format = OutputFormat.LOSSLESS_JSON;
+        }
+
+        SessionConfig.ResultDecorator handlePrefix =
+                (AlgebricksAppendable app) -> app.append("{ \"").append("handle").append("\": ");
+        SessionConfig.ResultDecorator handlePostfix = (AlgebricksAppendable app) -> app.append(" }");
+
+        SessionConfig sessionConfig =
+                new SessionConfig(response.writer(), format, null, null, handlePrefix, handlePostfix);
+
+        // If it's JSON or ADM, check for the "wrapper-array" flag. Default is
+        // "true" for JSON and "false" for ADM. (Not applicable for CSV.)
+        boolean wrapperArray;
+        switch (format) {
+            case CLEAN_JSON:
+            case LOSSLESS_JSON:
+                wrapperArray = true;
+                break;
+            default:
+                wrapperArray = false;
+                break;
+        }
+        String wrapperParam = request.getParameter("wrapper-array");
+        if (wrapperParam != null) {
+            wrapperArray = Boolean.valueOf(wrapperParam);
+        } else if (accept.contains("wrap-array=true")) {
+            wrapperArray = true;
+        } else if (accept.contains("wrap-array=false")) {
+            wrapperArray = false;
+        }
+        sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, wrapperArray);
+        // Now that format is set, output the content-type
+        switch (format) {
+            case ADM:
+                IServletResponse.setContentType(response, "application/x-adm");
+                break;
+            case CLEAN_JSON:
+                // No need to reflect "clean-ness" in output type; fall through
+            case LOSSLESS_JSON:
+                IServletResponse.setContentType(response, "application/json");
+                break;
+            case CSV:
+                // Check for header parameter or in Accept:.
+                if ("present".equals(request.getParameter("header")) || accept.contains("header=present")) {
+                    IServletResponse.setContentType(response, "text/csv; header=present");
+                    sessionConfig.set(SessionConfig.FORMAT_CSV_HEADER, true);
+                } else {
+                    IServletResponse.setContentType(response, "text/csv; header=absent");
+                }
+                break;
+        }
+        return sessionConfig;
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        try {
+            String query = query(request);
+            // enable cross-origin resource sharing
+            response.setHeader("Access-Control-Allow-Origin", "*");
+            response.setHeader("Access-Control-Allow-Headers", "Origin, X-Requested-With, Content-Type, Accept");
+
+            SessionConfig sessionConfig = initResponse(request, response);
+            QueryTranslator.ResultDelivery resultDelivery = whichResultDelivery(request);
+            try {
+                response.setStatus(HttpResponseStatus.OK);
+                IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
+                IHyracksDataset hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
+                if (hds == null) {
+                    synchronized (ctx) {
+                        hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
+                        if (hds == null) {
+                            hds = new HyracksDataset(hcc, ResultReader.FRAME_SIZE, ResultReader.NUM_READERS);
+                            ctx.put(HYRACKS_DATASET_ATTR, hds);
+                        }
+                    }
+                }
+                IParser parser = parserFactory.createParser(query);
+                List<Statement> aqlStatements = parser.parse();
+                validate(aqlStatements);
+                MetadataManager.INSTANCE.init();
+                IStatementExecutor translator =
+                        statementExecutorFactory.create(aqlStatements, sessionConfig, compilationProvider);
+                translator.compileAndExecute(hcc, hds, resultDelivery);
+            } catch (AsterixException | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
+                GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, pe.getMessage(), pe);
+                String errorMessage = ResultUtil.buildParseExceptionMessage(pe, query);
+                ObjectNode errorResp =
+                        ResultUtil.getErrorResponse(2, errorMessage, "", ResultUtil.extractFullStackTrace(pe));
+                sessionConfig.out().write(new ObjectMapper().writeValueAsString(errorResp));
+                response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            } catch (Exception e) {
+                GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
+                ResultUtil.apiErrorHandler(sessionConfig.out(), e);
+                response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            }
+        } catch (Exception e) {
+            LOGGER.log(Level.WARNING, "Failure handling request", e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        }
+    }
+
+    //TODO: Both Get and Post of this API must use the same parameter names
+    private String query(IServletRequest request) {
+        if (request.getHttpRequest().method() == HttpMethod.POST) {
+            return request.getHttpRequest().content().toString(StandardCharsets.UTF_8);
+        } else {
+            return getQueryParameter(request);
+        }
+    }
+
+    private void validate(List<Statement> aqlStatements) throws AsterixException {
+        for (Statement st : aqlStatements) {
+            if ((st.getCategory() & getAllowedCategories()) == 0) {
+                throw new AsterixException(String.format(getErrorMessage(), st.getKind()));
+            }
+        }
+    }
+
+    protected QueryTranslator.ResultDelivery whichResultDelivery(IServletRequest request) {
+        String mode = request.getParameter("mode");
+        if (mode != null) {
+            if ("asynchronous".equals(mode) || "async".equals(mode)) {
+                return QueryTranslator.ResultDelivery.ASYNC;
+            } else if ("asynchronous-deferred".equals(mode) || "deferred".equals(mode)) {
+                return QueryTranslator.ResultDelivery.DEFERRED;
+            }
+        }
+        return QueryTranslator.ResultDelivery.IMMEDIATE;
+    }
+
+    protected abstract String getQueryParameter(IServletRequest request);
+
+    protected abstract byte getAllowedCategories();
+
+    protected abstract String getErrorMessage();
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ShutdownApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ShutdownApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ShutdownApiServlet.java
new file mode 100644
index 0000000..25be651
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ShutdownApiServlet.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Date;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class ShutdownApiServlet extends AbstractServlet {
+    private static final Logger LOGGER = Logger.getLogger(ShutdownApiServlet.class.getName());
+    public static final String NODE_ID_KEY = "node_id";
+    public static final String NCSERVICE_PID = "ncservice_pid";
+    public static final String INI = "ini";
+    public static final String PID = "pid";
+
+    public ShutdownApiServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        if (request.getHttpRequest().method() != HttpMethod.POST) {
+            response.setStatus(HttpResponseStatus.METHOD_NOT_ALLOWED);
+            return;
+        }
+        IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
+        boolean terminateNCServices = "true".equalsIgnoreCase(request.getParameter("all"));
+        Thread t = new Thread(() -> {
+            try {
+                hcc.stopCluster(terminateNCServices);
+            } catch (Exception e) {
+                GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, "Exception stopping cluster", e);
+            }
+        }, "Shutdown Servlet Worker");
+
+        try {
+            IServletResponse.setContentType(response, IServlet.ContentType.APPLICATION_JSON,
+                    IServlet.Encoding.UTF8);
+        } catch (IOException e) {
+            LOGGER.log(Level.WARNING, "Failure handling request", e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        }
+        response.setStatus(HttpResponseStatus.ACCEPTED);
+        ObjectMapper om = new ObjectMapper();
+        ObjectNode jsonObject = om.createObjectNode();
+        try {
+            jsonObject.put("status", "SHUTTING_DOWN");
+            jsonObject.putPOJO("date", new Date());
+            ObjectNode clusterState = ClusterStateManager.INSTANCE.getClusterStateDescription();
+            ArrayNode ncs = (ArrayNode) clusterState.get("ncs");
+            for (int i = 0; i < ncs.size(); i++) {
+                ObjectNode nc = (ObjectNode) ncs.get(i);
+                String node = nc.get(NODE_ID_KEY).asText();
+                ObjectNode details = (ObjectNode) om.readTree(hcc.getNodeDetailsJSON(node, false, true));
+                nc.set(PID, details.get(PID));
+                if (details.has(INI) && details.get(INI).has(NCSERVICE_PID)) {
+                    nc.put(NCSERVICE_PID, details.get(INI).get(NCSERVICE_PID).asInt());
+                }
+            }
+            jsonObject.set("cluster", clusterState);
+            final PrintWriter writer = response.writer();
+            writer.print(om.writeValueAsString(jsonObject));
+            writer.close();
+        } catch (Exception e) {
+            GlobalConfig.ASTERIX_LOGGER.log(Level.INFO, "Exception writing response", e);
+        }
+        t.start();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
new file mode 100644
index 0000000..1c173d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
@@ -0,0 +1,50 @@
+/*
+ * 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.api.http.server;
+
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.hyracks.http.server.IServletRequest;
+
+public class UpdateApiServlet extends RestApiServlet {
+    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE;
+
+    public UpdateApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
+        super(ctx, paths, compilationProvider, statementExecutorFactory);
+    }
+
+    @Override
+    protected String getQueryParameter(IServletRequest request) {
+        return request.getParameter("statements");
+    }
+
+    @Override
+    protected byte getAllowedCategories() {
+        return ALLOWED_CATEGORIES;
+    }
+
+    @Override
+    protected String getErrorMessage() {
+        return "Invalid statement: Non-Update statement %s to the Update API.";
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/VersionApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/VersionApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/VersionApiServlet.java
new file mode 100644
index 0000000..e5ceccc
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/VersionApiServlet.java
@@ -0,0 +1,75 @@
+/*
+ * 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.api.http.server;
+
+import static org.apache.asterix.api.http.servlet.ServletConstants.ASTERIX_BUILD_PROP_ATTR;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.hyracks.http.server.IServlet;
+import org.apache.hyracks.http.server.IServletRequest;
+import org.apache.hyracks.http.server.IServletResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+public class VersionApiServlet extends AbstractServlet {
+    private static final Logger LOGGER = Logger.getLogger(VersionApiServlet.class.getName());
+
+    public VersionApiServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
+        super(ctx, paths);
+    }
+
+    @Override
+    public void handle(IServletRequest request, IServletResponse response) {
+        if (request.getHttpRequest().method() != HttpMethod.GET) {
+            response.setStatus(HttpResponseStatus.METHOD_NOT_ALLOWED);
+            return;
+        }
+        response.setStatus(HttpResponseStatus.OK);
+        AppContextInfo props = (AppContextInfo) ctx.get(ASTERIX_BUILD_PROP_ATTR);
+        Map<String, String> buildProperties = props.getBuildProperties().getAllProps();
+        ObjectMapper om = new ObjectMapper();
+        ObjectNode responseObject = om.createObjectNode();
+        for (Map.Entry<String, String> e : buildProperties.entrySet()) {
+            responseObject.put(e.getKey(), e.getValue());
+        }
+        try {
+            IServletResponse.setContentType(response, IServlet.ContentType.TEXT_PLAIN, IServlet.Encoding.UTF8);
+        } catch (IOException e) {
+            LOGGER.log(Level.WARNING, "Failure handling request", e);
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        }
+        PrintWriter responseWriter = response.writer();
+        responseWriter.write(responseObject.toString());
+        responseWriter.flush();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/APIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/APIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/APIServlet.java
deleted file mode 100644
index b693407..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/APIServlet.java
+++ /dev/null
@@ -1,205 +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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_DATASET_ATTR;
-
-import java.awt.image.BufferedImage;
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.PrintWriter;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import javax.imageio.ImageIO;
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import org.apache.asterix.app.result.ResultReader;
-import org.apache.asterix.app.result.ResultUtil;
-import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.aql.parser.TokenMgrError;
-import org.apache.asterix.lang.common.base.IParser;
-import org.apache.asterix.lang.common.base.IParserFactory;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.translator.IStatementExecutor;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.asterix.translator.SessionConfig;
-import org.apache.asterix.translator.SessionConfig.OutputFormat;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.client.dataset.HyracksDataset;
-
-public class APIServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(APIServlet.class.getName());
-    public static final String HTML_STATEMENT_SEPARATOR = "<!-- BEGIN -->";
-
-    private final ILangCompilationProvider aqlCompilationProvider;
-    private final ILangCompilationProvider sqlppCompilationProvider;
-    private final transient IStatementExecutorFactory statementExectorFactory;
-
-    public APIServlet(ILangCompilationProvider aqlCompilationProvider,
-            ILangCompilationProvider sqlppCompilationProvider, IStatementExecutorFactory statementExecutorFactory) {
-        this.aqlCompilationProvider = aqlCompilationProvider;
-        this.sqlppCompilationProvider = sqlppCompilationProvider;
-        this.statementExectorFactory = statementExecutorFactory;
-    }
-
-    @Override
-    public void doPost(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        // Query language
-        ILangCompilationProvider compilationProvider = "AQL".equals(request.getParameter("query-language"))
-                ? aqlCompilationProvider : sqlppCompilationProvider;
-        IParserFactory parserFactory = compilationProvider.getParserFactory();
-
-        // Output format.
-        OutputFormat format;
-        boolean csv_and_header = false;
-        String output = request.getParameter("output-format");
-        try {
-            format = OutputFormat.valueOf(output);
-        } catch (IllegalArgumentException e) {
-            LOGGER.info(output + ": unsupported output-format, using " + OutputFormat.CLEAN_JSON + " instead");
-            // Default output format
-            format = OutputFormat.CLEAN_JSON;
-        }
-
-        String query = request.getParameter("query");
-        String wrapperArray = request.getParameter("wrapper-array");
-        String printExprParam = request.getParameter("print-expr-tree");
-        String printRewrittenExprParam = request.getParameter("print-rewritten-expr-tree");
-        String printLogicalPlanParam = request.getParameter("print-logical-plan");
-        String printOptimizedLogicalPlanParam = request.getParameter("print-optimized-logical-plan");
-        String printJob = request.getParameter("print-job");
-        String executeQuery = request.getParameter("execute-query");
-        response.setCharacterEncoding("utf-8");
-        response.setContentType("text/html");
-        PrintWriter out = response.getWriter();
-        ServletContext context = getServletContext();
-        IHyracksClientConnection hcc;
-        IHyracksDataset hds;
-
-        try {
-            synchronized (context) {
-                hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
-
-                hds = (IHyracksDataset) context.getAttribute(HYRACKS_DATASET_ATTR);
-                if (hds == null) {
-                    hds = new HyracksDataset(hcc, ResultReader.FRAME_SIZE, ResultReader.NUM_READERS);
-                    context.setAttribute(HYRACKS_DATASET_ATTR, hds);
-                }
-            }
-            IParser parser = parserFactory.createParser(query);
-            List<Statement> aqlStatements = parser.parse();
-            SessionConfig sessionConfig = new SessionConfig(out, format, true, isSet(executeQuery), true);
-            sessionConfig.set(SessionConfig.FORMAT_HTML, true);
-            sessionConfig.set(SessionConfig.FORMAT_CSV_HEADER, csv_and_header);
-            sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, isSet(wrapperArray));
-            sessionConfig.setOOBData(isSet(printExprParam), isSet(printRewrittenExprParam),
-                    isSet(printLogicalPlanParam), isSet(printOptimizedLogicalPlanParam), isSet(printJob));
-            MetadataManager.INSTANCE.init();
-            IStatementExecutor translator =
-                    statementExectorFactory.create(aqlStatements, sessionConfig, compilationProvider);
-            double duration = 0;
-            long startTime = System.currentTimeMillis();
-            translator.compileAndExecute(hcc, hds, IStatementExecutor.ResultDelivery.IMMEDIATE);
-            long endTime = System.currentTimeMillis();
-            duration = (endTime - startTime) / 1000.00;
-            out.println(HTML_STATEMENT_SEPARATOR);
-            out.println("<PRE>Duration of all jobs: " + duration + " sec</PRE>");
-        } catch (AsterixException | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
-            GlobalConfig.ASTERIX_LOGGER.log(Level.INFO, pe.toString(), pe);
-            ResultUtil.webUIParseExceptionHandler(out, pe, query);
-        } catch (Exception e) {
-            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
-            ResultUtil.webUIErrorHandler(out, e);
-        }
-    }
-
-    @Override
-    public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        String resourcePath = null;
-        String requestURI = request.getRequestURI();
-
-        if (requestURI.equals("/")) {
-            response.setContentType("text/html");
-            resourcePath = "/webui/querytemplate.html";
-        } else {
-            resourcePath = requestURI;
-        }
-
-        try (InputStream is = APIServlet.class.getResourceAsStream(resourcePath)) {
-            if (is == null) {
-                response.sendError(HttpServletResponse.SC_NOT_FOUND);
-                return;
-            }
-
-            // Special handler for font files and .png resources
-            if (resourcePath.endsWith(".png")) {
-
-                BufferedImage img = ImageIO.read(is);
-                OutputStream outputStream = response.getOutputStream();
-                String formatName = "png";
-                response.setContentType("image/png");
-                ImageIO.write(img, formatName, outputStream);
-                outputStream.close();
-                return;
-            }
-
-            response.setCharacterEncoding("utf-8");
-            InputStreamReader isr = new InputStreamReader(is);
-            StringBuilder sb = new StringBuilder();
-            BufferedReader br = new BufferedReader(isr);
-            String line;
-            try {
-                line = br.readLine();
-            } catch (NullPointerException e) {
-                LOGGER.log(Level.WARNING, "NPE reading resource " + resourcePath
-                        + ", assuming JDK-8080094; returning 404", e);
-                // workaround lame JDK bug where a broken InputStream is returned in case the resourcePath is a
-                // directory; see https://bugs.openjdk.java.net/browse/JDK-8080094
-                response.sendError(HttpServletResponse.SC_NOT_FOUND);
-                return;
-            }
-
-            while (line != null) {
-                sb.append(line);
-                line = br.readLine();
-            }
-
-            PrintWriter out = response.getWriter();
-            out.println(sb.toString());
-        }
-    }
-
-    private static boolean isSet(String requestParameter) {
-        return (requestParameter != null && requestParameter.equals("true"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
deleted file mode 100644
index 988b5bb..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
+++ /dev/null
@@ -1,178 +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.api.http.servlet;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-import java.util.regex.Pattern;
-
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.SerializationFeature;
-import org.apache.asterix.common.config.AsterixProperties;
-import org.apache.asterix.common.config.ExternalProperties;
-import org.apache.asterix.common.utils.JSONUtil;
-import org.apache.asterix.common.config.AbstractProperties;
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.asterix.runtime.util.ClusterStateManager;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-public class ClusterAPIServlet extends HttpServlet {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(ClusterAPIServlet.class.getName());
-
-    protected static final String NODE_ID_KEY = "node_id";
-    protected static final String CONFIG_URI_KEY = "configUri";
-    protected static final String STATS_URI_KEY = "statsUri";
-    protected static final String THREAD_DUMP_URI_KEY = "threadDumpUri";
-    protected static final String SHUTDOWN_URI_KEY = "shutdownUri";
-    protected static final String FULL_SHUTDOWN_URI_KEY = "fullShutdownUri";
-    protected static final String VERSION_URI_KEY = "versionUri";
-    protected static final String DIAGNOSTICS_URI_KEY = "diagnosticsUri";
-    protected static final String REPLICATION_URI_KEY = "replicationUri";
-    private static final Pattern PARENT_DIR = Pattern.compile("/[^./]+/\\.\\./");
-    private static final Pattern REPLICATION_PROPERTY = Pattern.compile("^replication\\.");
-    private final ObjectMapper om = new ObjectMapper();
-
-    @Override
-    public final void doGet(HttpServletRequest request, HttpServletResponse response) {
-        try {
-            getUnsafe(request, response);
-        } catch (IOException e) {
-            LOGGER.log(Level.WARNING, "Unhandled IOException thrown from " + getClass().getName() + " get impl", e);
-        }
-    }
-
-    protected void getUnsafe(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        response.setContentType("application/json");
-        response.setCharacterEncoding("utf-8");
-        PrintWriter responseWriter = response.getWriter();
-        try {
-            ObjectNode json;
-            switch (request.getPathInfo() == null ? "" : request.getPathInfo()) {
-                case "":
-                    json = getClusterStateJSON(request, "");
-                    break;
-                case "/replication":
-                    json = getReplicationJSON();
-                    break;
-                case "/summary":
-                    json = getClusterStateSummaryJSON();
-                    break;
-                default:
-                    throw new IllegalArgumentException();
-            }
-            response.setStatus(HttpServletResponse.SC_OK);
-            responseWriter.write(JSONUtil.convertNode(json));
-        } catch (IllegalArgumentException e) { // NOSONAR - exception not logged or rethrown
-            response.sendError(HttpServletResponse.SC_NOT_FOUND);
-        } catch (Exception e) {
-            LOGGER.log(Level.INFO, "exception thrown for " + request, e);
-            response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, e.toString());
-        }
-        responseWriter.flush();
-    }
-
-    protected ObjectNode getClusterStateSummaryJSON() {
-        return ClusterStateManager.INSTANCE.getClusterStateSummary();
-    }
-
-    protected ObjectNode getReplicationJSON() {
-        for (AbstractProperties props : getPropertiesInstances()) {
-            if (props instanceof ReplicationProperties) {
-                ObjectNode json = om.createObjectNode();
-                json.putPOJO("config", props.getProperties(key -> REPLICATION_PROPERTY.matcher(key).replaceFirst("")));
-                return json;
-            }
-        }
-        throw new IllegalStateException("ERROR: replication properties not found");
-    }
-
-    protected Map<String, Object> getAllClusterProperties() {
-        Map<String, Object> allProperties = new HashMap<>();
-        for (AbstractProperties properties : getPropertiesInstances()) {
-            if (!(properties instanceof ReplicationProperties)) {
-                allProperties.putAll(properties.getProperties());
-            }
-        }
-        return allProperties;
-    }
-
-    protected List<AbstractProperties> getPropertiesInstances() {
-        return AbstractProperties.getImplementations();
-    }
-
-    protected ObjectNode getClusterStateJSON(HttpServletRequest request, String pathToNode) {
-        ObjectNode json = ClusterStateManager.INSTANCE.getClusterStateDescription();
-        Map<String, Object> allProperties = getAllClusterProperties();
-        json.putPOJO("config", allProperties);
-
-        ArrayNode ncs = (ArrayNode) json.get("ncs");
-        final StringBuilder requestURL = new StringBuilder(request.getRequestURL());
-        if (requestURL.charAt(requestURL.length() - 1) != '/') {
-            requestURL.append('/');
-        }
-        requestURL.append(pathToNode);
-        String clusterURL = canonicalize(requestURL);
-        String adminURL = canonicalize(clusterURL + "../");
-        String nodeURL = clusterURL + "node/";
-        for (int i = 0; i < ncs.size(); i++) {
-            ObjectNode nc = (ObjectNode) ncs.get(i);
-            nc.put(CONFIG_URI_KEY, nodeURL + nc.get(NODE_ID_KEY).asText() + "/config");
-            nc.put(STATS_URI_KEY, nodeURL + nc.get(NODE_ID_KEY).asText() + "/stats");
-            nc.put(THREAD_DUMP_URI_KEY, nodeURL + nc.get(NODE_ID_KEY).asText() + "/threaddump");
-        }
-        ObjectNode cc;
-        if (json.has("cc")) {
-            cc = (ObjectNode) json.get("cc");
-        } else {
-            cc = om.createObjectNode();
-            json.set("cc", cc);
-        }
-        cc.put(CONFIG_URI_KEY, clusterURL + "cc/config");
-        cc.put(STATS_URI_KEY, clusterURL + "cc/stats");
-        cc.put(THREAD_DUMP_URI_KEY, clusterURL + "cc/threaddump");
-        json.put(REPLICATION_URI_KEY, clusterURL + "replication");
-        json.put(SHUTDOWN_URI_KEY, adminURL + "shutdown");
-        json.put(FULL_SHUTDOWN_URI_KEY, adminURL + "shutdown?all=true");
-        json.put(VERSION_URI_KEY, adminURL + "version");
-        json.put(DIAGNOSTICS_URI_KEY, adminURL + "diagnostics");
-        return json;
-    }
-
-    private String canonicalize(CharSequence requestURL) {
-        String clusterURL = "";
-        String newClusterURL = requestURL.toString();
-        while (!clusterURL.equals(newClusterURL)) {
-            clusterURL = newClusterURL;
-            newClusterURL = PARENT_DIR.matcher(clusterURL).replaceAll("/");
-        }
-        return clusterURL;
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/60e7f12b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterCCDetailsAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterCCDetailsAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterCCDetailsAPIServlet.java
deleted file mode 100644
index 569d5a0..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterCCDetailsAPIServlet.java
+++ /dev/null
@@ -1,101 +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.api.http.servlet;
-
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-public class ClusterCCDetailsAPIServlet extends ClusterAPIServlet {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(ClusterCCDetailsAPIServlet.class.getName());
-    private final ObjectMapper om = new ObjectMapper();
-
-    @Override
-    protected void getUnsafe(HttpServletRequest request, HttpServletResponse response) throws IOException {
-        PrintWriter responseWriter = response.getWriter();
-        ServletContext context = getServletContext();
-        IHyracksClientConnection hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
-        try {
-            ObjectNode json;
-            if (request.getPathInfo() == null) {
-                json = (ObjectNode) getClusterStateJSON(request, "../").get("cc");
-            } else {
-                json = processNode(request, hcc);
-            }
-            response.setStatus(HttpServletResponse.SC_OK);
-            response.setContentType("application/json");
-            response.setCharacterEncoding("utf-8");
-            responseWriter.write(new ObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(json));
-        } catch (IllegalArgumentException e) { // NOSONAR - exception not logged or rethrown
-            response.sendError(HttpServletResponse.SC_NOT_FOUND);
-        } catch (Exception e) {
-            LOGGER.log(Level.INFO, "exception thrown for " + request, e);
-            response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, e.toString());
-        }
-        responseWriter.flush();
-    }
-
-    private ObjectNode processNode(HttpServletRequest request, IHyracksClientConnection hcc) throws Exception {
-        String pathInfo = request.getPathInfo();
-        if (pathInfo.endsWith("/")) {
-            throw new IllegalArgumentException();
-        }
-        String[] parts = pathInfo.substring(1).split("/");
-
-        if (request.getPathInfo() == null) {
-            return (ObjectNode) getClusterStateJSON(request, "../../").get("cc");
-        } else if (parts.length == 1) {
-            switch (parts[0]) {
-                case "config":
-                    return om.readValue(hcc.getNodeDetailsJSON(null, false, true), ObjectNode.class);
-                case "stats":
-                    return om.readValue(hcc.getNodeDetailsJSON(null, true, false), ObjectNode.class);
-                case "threaddump":
-                    return processCCThreadDump(hcc);
-
-                default:
-                    throw new IllegalArgumentException();
-            }
-
-        } else {
-            throw new IllegalArgumentException();
-        }
-    }
-
-    private ObjectNode processCCThreadDump(IHyracksClientConnection hcc) throws Exception {
-        String dump = hcc.getThreadDump(null);
-        if (dump == null) {
-            throw new IllegalArgumentException();
-        }
-        return (ObjectNode) om.readTree(dump);
-    }
-
-}