You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ja...@apache.org on 2014/06/11 05:52:39 UTC

[54/61] [abbrv] git commit: web cleanup, enable multiphase and increase broadcast threshold

web cleanup, enable multiphase and increase broadcast threshold


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/ab9b2fe9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/ab9b2fe9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/ab9b2fe9

Branch: refs/heads/master
Commit: ab9b2fe933ec5b05cabfab94c837f24db83e3ba9
Parents: c6eb0ba
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun Jun 8 21:38:01 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Tue Jun 10 18:59:40 2014 -0700

----------------------------------------------------------------------
 .../exec/planner/physical/PlannerSettings.java  |   4 +-
 .../drill/exec/server/rest/DrillRestServer.java |   2 +
 .../drill/exec/server/rest/DrillRoot.java       | 195 +------------------
 .../exec/server/rest/ProfileResources.java      | 104 ++++++++++
 .../drill/exec/server/rest/QueryResources.java  | 131 +++++++++++++
 .../java-exec/src/main/resources/rest/error.ftl |  21 ++
 .../src/main/resources/rest/generic.ftl         |   4 +-
 .../java-exec/src/main/resources/rest/index.ftl |  27 +++
 .../src/main/resources/rest/profile/list.ftl    |  72 +++++++
 .../src/main/resources/rest/profile/profile.ftl |  65 +++++++
 .../src/main/resources/rest/query/query.ftl     |  54 +++++
 .../src/main/resources/rest/query/result.ftl    |  40 ++++
 .../src/main/resources/rest/status.ftl          |  29 +++
 .../src/main/resources/rest/status/error.ftl    |  21 --
 .../src/main/resources/rest/status/index.ftl    |  27 ---
 .../src/main/resources/rest/status/list.ftl     |  72 -------
 .../src/main/resources/rest/status/profile.ftl  |  65 -------
 .../src/main/resources/rest/status/query.ftl    |  54 -----
 .../src/main/resources/rest/status/result.ftl   |  40 ----
 .../src/main/resources/rest/status/status.ftl   |  29 ---
 20 files changed, 552 insertions(+), 504 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
index 2325e2f..1b76e2b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
@@ -37,9 +37,9 @@ public class PlannerSettings implements FrameworkContext{
   public static final OptionValidator STREAMAGG = new BooleanValidator("planner.enable_streamagg", true);
   public static final OptionValidator HASHJOIN = new BooleanValidator("planner.enable_hashjoin", true);
   public static final OptionValidator MERGEJOIN = new BooleanValidator("planner.enable_mergejoin", true);
-  public static final OptionValidator MULTIPHASE = new BooleanValidator("planner.enable_multiphase_agg", false);
+  public static final OptionValidator MULTIPHASE = new BooleanValidator("planner.enable_multiphase_agg", true);
   public static final OptionValidator BROADCAST = new BooleanValidator("planner.enable_broadcast_join", true);
-  public static final OptionValidator BROADCAST_THRESHOLD = new PositiveLongValidator("planner.broadcast_threshold", MAX_BROADCAST_THRESHOLD, 10000);
+  public static final OptionValidator BROADCAST_THRESHOLD = new PositiveLongValidator("planner.broadcast_threshold", MAX_BROADCAST_THRESHOLD, 1000000);
 
   public OptionManager options = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
index 38354ca..2a94e1c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
@@ -40,6 +40,8 @@ public class DrillRestServer extends ResourceConfig {
   public DrillRestServer(final WorkManager workManager) {
     register(DrillRoot.class);
     register(StorageResources.class);
+    register(ProfileResources.class);
+    register(QueryResources.class);
     register(FreemarkerMvcFeature.class);
     property(ServerProperties.METAINF_SERVICES_LOOKUP_DISABLE, true);
     register(MultiPartFeature.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
index 78d196a..b1cb18b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
@@ -17,59 +17,22 @@
  */
 package org.apache.drill.exec.server.rest;
 
-import java.io.IOException;
-import java.text.SimpleDateFormat;
-import java.util.AbstractMap;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.inject.Inject;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.FormParam;
 import javax.ws.rs.GET;
-import javax.ws.rs.POST;
 import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 
-import org.apache.drill.exec.cache.DistributedMap;
-import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.proto.UserBitShared;
-import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
-import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
-import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.user.ConnectionThrottle;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
-import org.apache.drill.exec.rpc.user.UserResultsListener;
-import org.apache.drill.exec.store.sys.PStore;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.work.WorkManager;
-import org.apache.drill.exec.work.foreman.QueryStatus;
 import org.glassfish.jersey.server.mvc.Viewable;
 
-import com.google.common.collect.Lists;
-
 @Path("/")
 public class DrillRoot {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRoot.class);
 
-  @Inject WorkManager work;
-
   @GET
   @Produces(MediaType.TEXT_HTML)
   public Viewable getHello() {
     String status = "Running!";
-    return new Viewable("/rest/status/index.ftl", status);
+    return new Viewable("/rest/index.ftl", status);
   }
 
   @GET
@@ -77,162 +40,8 @@ public class DrillRoot {
   @Produces(MediaType.TEXT_HTML)
   public Viewable getStatus() {
     String status = "Running!";
-    return new Viewable("/rest/status/status.ftl", status);
-  }
-
-  @GET
-  @Path("/queries")
-  @Produces(MediaType.TEXT_HTML)
-  public Viewable getResults() throws IOException {
-    PStore<QueryProfile> profiles = work.getContext().getPersistentStoreProvider().getPStore(QueryStatus.QUERY_PROFILE);
-
-    List<Map.Entry<String, Long>> runningIds = Lists.newArrayList();
-    List<Map.Entry<String, Long>> finishedIds = Lists.newArrayList();
-    for(Map.Entry<String, QueryProfile> entry : profiles){
-      QueryProfile q = entry.getValue();
-      if (q.getState() == QueryState.RUNNING || q.getState() == QueryState.PENDING) {
-        runningIds.add(new AbstractMap.SimpleEntry<>(entry.getKey(), q.getStart()));
-      } else {
-        finishedIds.add(new AbstractMap.SimpleEntry<>(entry.getKey(), q.getStart()));
-      }
-    }
-
-    Comparator<Map.Entry<String,Long>> comparator = new Comparator<Map.Entry<String,Long>>() {
-      @Override
-      public int compare(Map.Entry<String, Long> o1, Map.Entry<String, Long> o2) {
-        return o2.getValue().compareTo(o1.getValue());
-      }
-    };
-
-    Collections.sort(runningIds, comparator);
-    Collections.sort(finishedIds, comparator);
-
-    List<Map.Entry<String, String>> runningQueries = Lists.newArrayList();
-    List<Map.Entry<String, String>> oldQueries = Lists.newArrayList();
-    for(Map.Entry<String, Long> entry : runningIds){
-      runningQueries.add(new AbstractMap.SimpleEntry<>(entry.getKey(), new SimpleDateFormat("MM/dd/yyyy HH:mm:ss").format(new Date(entry.getValue()))));
-    }
-
-    for(Map.Entry<String, Long> entry : finishedIds){
-      oldQueries.add(new AbstractMap.SimpleEntry<>(entry.getKey(), new SimpleDateFormat("MM/dd/yyyy HH:mm:ss").format(new Date(entry.getValue()))));
-    }
-    // add status (running, done)
-
-    Queries queries = new Queries();
-    queries.runningQueries = runningQueries;
-    queries.oldQueries = oldQueries;
-
-    return new Viewable("/rest/status/list.ftl", queries);
-  }
-
-  public static class Queries {
-    List<Map.Entry<String, String>> runningQueries;
-    List<Map.Entry<String, String>> oldQueries;
-
-    public List<Map.Entry<String, String>> getRunningQueries() {
-      return runningQueries;
-    }
-
-    public List<Map.Entry<String, String>> getOldQueries() {
-      return oldQueries;
-    }
+    return new Viewable("/rest/status.ftl", status);
   }
 
-  @GET
-  @Path("/query/{queryid}")
-  @Produces(MediaType.TEXT_HTML)
-  public Viewable getQuery(@PathParam("queryid") String queryId) throws IOException {
-    PStore<QueryProfile> profiles = work.getContext().getPersistentStoreProvider().getPStore(QueryStatus.QUERY_PROFILE);
-    QueryProfile profile = profiles.get(queryId);
-    if(profile == null) profile = QueryProfile.getDefaultInstance();
 
-    return new Viewable("/rest/status/profile.ftl", profile);
-
-  }
-
-  @GET
-  @Path("/query")
-  @Produces(MediaType.TEXT_HTML)
-  public Viewable getQuery() {
-    return new Viewable("/rest/status/query.ftl");
-  }
-
-  @POST
-  @Path("/query")
-  @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
-  @Produces(MediaType.TEXT_HTML)
-  public Viewable submitQuery(@FormParam("query") String query, @FormParam("queryType") String queryType) throws Exception {
-    DrillClient client = new DrillClient(work.getContext().getConfig(), work.getContext().getClusterCoordinator());
-    client.connect();
-
-    UserBitShared.QueryType type = UserBitShared.QueryType.SQL;
-    switch (queryType){
-      case "SQL" : type = UserBitShared.QueryType.SQL; break;
-      case "LOGICAL" : type = UserBitShared.QueryType.LOGICAL; break;
-      case "PHYSICAL" : type = UserBitShared.QueryType.PHYSICAL; break;
-    }
-
-    Listener listener = new Listener(new RecordBatchLoader(work.getContext().getAllocator()));
-    client.runQuery(type, query, listener);
-    List<LinkedList<String>> result = listener.waitForCompletion();
-
-    return new Viewable("/rest/status/result.ftl", result);
-  }
-
-  private static class Listener implements UserResultsListener {
-    private volatile Exception exception;
-    private AtomicInteger count = new AtomicInteger();
-    private CountDownLatch latch = new CountDownLatch(1);
-    private LinkedList<LinkedList<String>> output = new LinkedList<>();
-    private RecordBatchLoader loader;
-
-    Listener(RecordBatchLoader loader) {
-      this.loader = loader;
-    }
-
-    @Override
-    public void submissionFailed(RpcException ex) {
-      exception = ex;
-      System.out.println("Query failed: " + ex.getMessage());
-      latch.countDown();
-    }
-
-    @Override
-    public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
-      int rows = result.getHeader().getRowCount();
-      if (result.getData() != null) {
-        count.addAndGet(rows);
-        try {
-          loader.load(result.getHeader().getDef(), result.getData());
-          output.add(new LinkedList<String>());
-          for (int i = 0; i < loader.getSchema().getFieldCount(); ++i) {
-            output.getLast().add(loader.getSchema().getColumn(i).getPath().getAsUnescapedPath());
-          }
-        } catch (SchemaChangeException e) {
-          throw new RuntimeException(e);
-        }
-        for(int i = 0; i < rows; ++i) {
-          output.add(new LinkedList<String>());
-          for (VectorWrapper<?> vw : loader) {
-            ValueVector.Accessor accessor = vw.getValueVector().getAccessor();
-            output.getLast().add(accessor.getObject(i).toString());
-            }
-        }
-      }
-      result.release();
-      if (result.getHeader().getIsLastChunk()) {
-        latch.countDown();
-      }
-    }
-
-    @Override
-    public void queryIdArrived(UserBitShared.QueryId queryId) {}
-
-    public List<LinkedList<String>> waitForCompletion() throws Exception {
-      latch.await();
-      if(exception != null) throw exception;
-      System.out.println();
-      return output;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/ProfileResources.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/ProfileResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/ProfileResources.java
new file mode 100644
index 0000000..d66cdbc
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/ProfileResources.java
@@ -0,0 +1,104 @@
+package org.apache.drill.exec.server.rest;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.AbstractMap;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
+import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
+import org.apache.drill.exec.store.sys.PStore;
+import org.apache.drill.exec.work.WorkManager;
+import org.apache.drill.exec.work.foreman.QueryStatus;
+import org.glassfish.jersey.server.mvc.Viewable;
+
+import com.google.common.collect.Lists;
+
+@Path("/profiles")
+public class ProfileResources {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProfileResources.class);
+
+  @Inject WorkManager work;
+
+
+  @GET
+  @Path("/{queryid}")
+  @Produces(MediaType.TEXT_HTML)
+  public Viewable getQuery(@PathParam("queryid") String queryId) throws IOException {
+    PStore<QueryProfile> profiles = work.getContext().getPersistentStoreProvider().getPStore(QueryStatus.QUERY_PROFILE);
+    QueryProfile profile = profiles.get(queryId);
+    if(profile == null) profile = QueryProfile.getDefaultInstance();
+
+    return new Viewable("/rest/profile/profile.ftl", profile);
+
+  }
+
+  @GET
+  @Produces(MediaType.TEXT_HTML)
+  public Viewable getResults() throws IOException {
+    PStore<QueryProfile> profiles = work.getContext().getPersistentStoreProvider().getPStore(QueryStatus.QUERY_PROFILE);
+
+    List<Map.Entry<String, Long>> runningIds = Lists.newArrayList();
+    List<Map.Entry<String, Long>> finishedIds = Lists.newArrayList();
+    for(Map.Entry<String, QueryProfile> entry : profiles){
+      QueryProfile q = entry.getValue();
+      if (q.getState() == QueryState.RUNNING || q.getState() == QueryState.PENDING) {
+        runningIds.add(new AbstractMap.SimpleEntry<>(entry.getKey(), q.getStart()));
+      } else {
+        finishedIds.add(new AbstractMap.SimpleEntry<>(entry.getKey(), q.getStart()));
+      }
+    }
+
+    Comparator<Map.Entry<String,Long>> comparator = new Comparator<Map.Entry<String,Long>>() {
+      @Override
+      public int compare(Map.Entry<String, Long> o1, Map.Entry<String, Long> o2) {
+        return o2.getValue().compareTo(o1.getValue());
+      }
+    };
+
+    Collections.sort(runningIds, comparator);
+    Collections.sort(finishedIds, comparator);
+
+    List<Map.Entry<String, String>> runningQueries = Lists.newArrayList();
+    List<Map.Entry<String, String>> oldQueries = Lists.newArrayList();
+    for(Map.Entry<String, Long> entry : runningIds){
+      runningQueries.add(new AbstractMap.SimpleEntry<>(entry.getKey(), new SimpleDateFormat("MM/dd/yyyy HH:mm:ss").format(new Date(entry.getValue()))));
+    }
+
+    for(Map.Entry<String, Long> entry : finishedIds){
+      oldQueries.add(new AbstractMap.SimpleEntry<>(entry.getKey(), new SimpleDateFormat("MM/dd/yyyy HH:mm:ss").format(new Date(entry.getValue()))));
+    }
+    // add status (running, done)
+
+    Queries queries = new Queries();
+    queries.runningQueries = runningQueries;
+    queries.oldQueries = oldQueries;
+
+    return new Viewable("/rest/profile/list.ftl", queries);
+  }
+
+  public static class Queries {
+    List<Map.Entry<String, String>> runningQueries;
+    List<Map.Entry<String, String>> oldQueries;
+
+    public List<Map.Entry<String, String>> getRunningQueries() {
+      return runningQueries;
+    }
+
+    public List<Map.Entry<String, String>> getOldQueries() {
+      return oldQueries;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryResources.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryResources.java
new file mode 100644
index 0000000..84fc581
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryResources.java
@@ -0,0 +1,131 @@
+package org.apache.drill.exec.server.rest;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.inject.Inject;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.FormParam;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.ConnectionThrottle;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.work.WorkManager;
+import org.glassfish.jersey.server.mvc.Viewable;
+
+@Path("/query")
+public class QueryResources {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryResources.class);
+
+  @Inject
+  WorkManager work;
+
+  @GET
+  @Produces(MediaType.TEXT_HTML)
+  public Viewable getQuery() {
+    return new Viewable("/rest/query/query.ftl");
+  }
+
+  @POST
+  @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
+  @Produces(MediaType.TEXT_HTML)
+  public Viewable submitQuery(@FormParam("query") String query, @FormParam("queryType") String queryType)
+      throws Exception {
+    try (DrillClient client = new DrillClient(work.getContext().getConfig(), work.getContext().getClusterCoordinator())) {
+
+      client.connect();
+
+      UserBitShared.QueryType type = UserBitShared.QueryType.SQL;
+      switch (queryType) {
+      case "SQL":
+        type = UserBitShared.QueryType.SQL;
+        break;
+      case "LOGICAL":
+        type = UserBitShared.QueryType.LOGICAL;
+        break;
+      case "PHYSICAL":
+        type = UserBitShared.QueryType.PHYSICAL;
+        break;
+      }
+
+      Listener listener = new Listener(new RecordBatchLoader(work.getContext().getAllocator()));
+      client.runQuery(type, query, listener);
+      List<LinkedList<String>> result = listener.waitForCompletion();
+
+      return new Viewable("/rest/query/result.ftl", result);
+    }
+  }
+
+  private static class Listener implements UserResultsListener {
+    private volatile Exception exception;
+    private AtomicInteger count = new AtomicInteger();
+    private CountDownLatch latch = new CountDownLatch(1);
+    private LinkedList<LinkedList<String>> output = new LinkedList<>();
+    private RecordBatchLoader loader;
+
+    Listener(RecordBatchLoader loader) {
+      this.loader = loader;
+    }
+
+    @Override
+    public void submissionFailed(RpcException ex) {
+      exception = ex;
+      System.out.println("Query failed: " + ex.getMessage());
+      latch.countDown();
+    }
+
+    @Override
+    public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
+      int rows = result.getHeader().getRowCount();
+      if (result.getData() != null) {
+        count.addAndGet(rows);
+        try {
+          loader.load(result.getHeader().getDef(), result.getData());
+          output.add(new LinkedList<String>());
+          for (int i = 0; i < loader.getSchema().getFieldCount(); ++i) {
+            output.getLast().add(loader.getSchema().getColumn(i).getPath().getAsUnescapedPath());
+          }
+        } catch (SchemaChangeException e) {
+          throw new RuntimeException(e);
+        }
+        for (int i = 0; i < rows; ++i) {
+          output.add(new LinkedList<String>());
+          for (VectorWrapper<?> vw : loader) {
+            ValueVector.Accessor accessor = vw.getValueVector().getAccessor();
+            output.getLast().add(accessor.getObject(i).toString());
+          }
+        }
+      }
+      result.release();
+      if (result.getHeader().getIsLastChunk()) {
+        latch.countDown();
+      }
+    }
+
+    @Override
+    public void queryIdArrived(UserBitShared.QueryId queryId) {
+    }
+
+    public List<LinkedList<String>> waitForCompletion() throws Exception {
+      latch.await();
+      if (exception != null)
+        throw exception;
+      System.out.println();
+      return output;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/error.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/error.ftl b/exec/java-exec/src/main/resources/rest/error.ftl
new file mode 100644
index 0000000..a07794a
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/error.ftl
@@ -0,0 +1,21 @@
+<#-- 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. -->
+
+<html>
+<a href="/queries">back</a><br />
+
+<pre>
+${model.printStackTrace()}
+</pre>
+
+
+
+<html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/generic.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/generic.ftl b/exec/java-exec/src/main/resources/rest/generic.ftl
index 8ff83a0..41700d0 100644
--- a/exec/java-exec/src/main/resources/rest/generic.ftl
+++ b/exec/java-exec/src/main/resources/rest/generic.ftl
@@ -55,8 +55,10 @@
             <ul class="nav navbar-nav">
               <li><a href="/"><font color="red" id="statusFontColor">Status</font></a></li>
               <li><a href="/query">Query</a></li>
-              <li><a href="/results">Results</a></li>
+              <li><a href="/profiles">Profiles</a></li>
               <li><a href="/storage">Storage</a></li>
+              <li><a href="/status/metrics">Metrics</a></li>
+              <li><a href="/status/threads">Threads</a></li>
             </ul>
           </div>
         </div>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/index.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/index.ftl b/exec/java-exec/src/main/resources/rest/index.ftl
new file mode 100644
index 0000000..a6b3075
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/index.ftl
@@ -0,0 +1,27 @@
+<#-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
+  license agreements. See the NOTICE file distributed with this work for additional
+  information regarding copyright ownership. The ASF licenses this file to
+  You under the Apache License, Version 2.0 (the "License"); you may not use
+  this file except in compliance with the License. You may obtain a copy of
+  the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
+  by applicable law or agreed to in writing, software distributed under the
+  License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
+  OF ANY KIND, either express or implied. See the License for the specific
+  language governing permissions and limitations under the License. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <a href="/queries">back</a><br/>
+  <div class="page-header">
+  </div>
+  <p class="lead"> Read about Apache Drill <a href="http://incubator.apache.org/drill/drill_overview.html" rel="nofollow">here</a>.</p>
+  <script>
+      var elem = document.getElementById("statusFontColor");
+      elem.style.color = "green";
+  </script>
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/profile/list.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/profile/list.ftl b/exec/java-exec/src/main/resources/rest/profile/list.ftl
new file mode 100644
index 0000000..613a0e9
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/profile/list.ftl
@@ -0,0 +1,72 @@
+<#-- 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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <a href="/queries">back</a><br/>
+  <div class="page-header">
+  </div>
+  <h3>Running Queries</h3>
+  <div class="table-responsive">
+    <table class="table table-hover">
+      <thead>
+         <td>Time</td>
+         <td>Query</td>
+      </thead>
+      <tbody>
+        <#list model.getRunningQueries() as query>
+        <tr>
+          <td>${query.getValue()}</td>
+          <td>
+            <a href="/query/${query.getKey()}">
+              <div style="height:100%;width:100%">
+                ${query.getKey()}
+              </div>
+            </a>
+          </td>
+        </tr>
+        </#list>
+      </tbody>
+    </table>
+  </div>
+  <h3>Completed Queries</h3>
+  <div class="table-responsive">
+    <table class="table table-hover">
+      <thead>
+         <td>Time</td>
+         <td>Query</td>
+      </thead>
+      <tbody>
+        <#list model.getOldQueries() as query>
+        <tr>
+          <td>${query.getValue()}</td>
+          <td>
+            <a href="/query/${query.getKey()}">
+              <div style="height:100%;width:100%">
+                ${query.getKey()}
+              </div>
+            </a>
+          </td>
+        </tr>
+        </#list>
+      </tbody>
+    </table>
+  </div>
+  <script>
+    var elem = document.getElementById("statusFontColor");
+    elem.style.color = "green";
+  </script>
+</#macro>
+
+<@page_html/>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/profile/profile.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/profile/profile.ftl b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
new file mode 100644
index 0000000..470f7ab
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
@@ -0,0 +1,65 @@
+<#-- 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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <a href="/queries">back</a><br/>
+  <div class="page-header">
+  </div>
+  <h2>Query</h2>
+  <form role="form" action="/query" method="POST">
+    <div class="form-group">
+      <textarea class="form-control" id="query" name="query">${model.query}</textarea>
+    </div>
+    <div class="form-group">
+      <div class="radio-inline">
+        <label>
+          <input type="radio" name="queryType" id="sql" value="SQL" checked>
+          SQL
+        </label>
+      </div>
+      <div class="radio-inline">
+        <label>
+          <input type="radio" name="queryType" id="physical" value="PHYSICAL">
+          PHYSICAL
+        </label>
+      </div>
+      <div class="radio-inline">
+        <label>
+          <input type="radio" name="queryType" id="logical" value="LOGICAL">
+          LOGICAL
+        </label>
+      </div>
+    </div>
+    <button type="submit" class="btn btn-default">Re-run query</button>
+  </form>
+  <div class="page-header">
+    <h2>Physical plan</h2>
+  </div>
+  <div class="well">
+    <p><font face="courier">${model.plan}</font></p>
+  </div>
+  <div class="page-header">
+    <h2>Complete Profile</h2>
+  </div>
+  <div class="well">
+    <p><font face="courier">${model.toString()}</font></p>
+  </div>
+  <script>
+      var elem = document.getElementById("statusFontColor");
+      elem.style.color = "green";
+  </script>
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/query/query.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/query/query.ftl b/exec/java-exec/src/main/resources/rest/query/query.ftl
new file mode 100644
index 0000000..a92696b
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/query/query.ftl
@@ -0,0 +1,54 @@
+<#-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
+  license agreements. See the NOTICE file distributed with this work for additional
+  information regarding copyright ownership. The ASF licenses this file to
+  You under the Apache License, Version 2.0 (the "License"); you may not use
+  this file except in compliance with the License. You may obtain a copy of
+  the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
+  by applicable law or agreed to in writing, software distributed under the
+  License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
+  OF ANY KIND, either express or implied. See the License for the specific
+  language governing permissions and limitations under the License. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <a href="/queries">back</a><br/>
+  <div class="page-header">
+  </div>
+  <form role="form" action="/query" method="POST">
+    <div class="form-group">
+      <label for="queryType">Query Type</label>
+      <div class="radio">
+        <label>
+          <input type="radio" name="queryType" id="sql" value="SQL" checked>
+          SQL
+        </label>
+      </div>
+      <div class="radio">
+        <label>
+          <input type="radio" name="queryType" id="physical" value="PHYSICAL">
+          PHYSICAL
+        </label>
+      </div>
+      <div class="radio">
+        <label>
+          <input type="radio" name="queryType" id="logical" value="LOGICAL">
+          LOGICAL
+        </label>
+      </div>
+    </div>
+    <div class="form-group">
+      <label for="query">Query</label>
+      <textarea class="form-control" id="query" rows="5" name="query"></textarea>
+    </div>
+    <button type="submit" class="btn btn-default">Submit</button>
+  </form>
+  <script>
+      var elem = document.getElementById("statusFontColor");
+      elem.style.color = "green";
+  </script>
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/query/result.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/query/result.ftl b/exec/java-exec/src/main/resources/rest/query/result.ftl
new file mode 100644
index 0000000..a44be9c
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/query/result.ftl
@@ -0,0 +1,40 @@
+<#-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
+  license agreements. See the NOTICE file distributed with this work for additional
+  information regarding copyright ownership. The ASF licenses this file to
+  You under the Apache License, Version 2.0 (the "License"); you may not use
+  this file except in compliance with the License. You may obtain a copy of
+  the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
+  by applicable law or agreed to in writing, software distributed under the
+  License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
+  OF ANY KIND, either express or implied. See the License for the specific
+  language governing permissions and limitations under the License. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <a href="/queries">back</a><br/>
+  <div class="page-header">
+      <h2>Result</h2>
+  </div>
+  <div class="table-responsive">
+    <table class="table">
+      <tbody>
+        <#list model as rows>
+        <tr>
+          <#list rows as row>
+          <td>${row}</td>
+          </#list>
+        </tr>
+        </#list>
+      </tbody>
+    </table>
+  </div>
+  <!-- <script>
+      var elem = document.getElementById("statusFontColor");
+      elem.style.color = "green";
+  </script> -->
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/status.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status.ftl b/exec/java-exec/src/main/resources/rest/status.ftl
new file mode 100644
index 0000000..1d4e3d3
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/status.ftl
@@ -0,0 +1,29 @@
+<#-- 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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <a href="/queries">back</a><br/>
+  <div class="page-header">
+    <div class="alert alert-success">
+      <strong>${model}</strong>
+    </div>
+  </div>
+  <script>
+      var elem = document.getElementById("statusFontColor");
+      elem.style.color = "green";
+  </script>
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/status/error.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/error.ftl b/exec/java-exec/src/main/resources/rest/status/error.ftl
deleted file mode 100644
index a07794a..0000000
--- a/exec/java-exec/src/main/resources/rest/status/error.ftl
+++ /dev/null
@@ -1,21 +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. -->
-
-<html>
-<a href="/queries">back</a><br />
-
-<pre>
-${model.printStackTrace()}
-</pre>
-
-
-
-<html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/status/index.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/index.ftl b/exec/java-exec/src/main/resources/rest/status/index.ftl
deleted file mode 100644
index a6b3075..0000000
--- a/exec/java-exec/src/main/resources/rest/status/index.ftl
+++ /dev/null
@@ -1,27 +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. -->
-
-<#include "*/generic.ftl">
-<#macro page_head>
-</#macro>
-
-<#macro page_body>
-  <a href="/queries">back</a><br/>
-  <div class="page-header">
-  </div>
-  <p class="lead"> Read about Apache Drill <a href="http://incubator.apache.org/drill/drill_overview.html" rel="nofollow">here</a>.</p>
-  <script>
-      var elem = document.getElementById("statusFontColor");
-      elem.style.color = "green";
-  </script>
-</#macro>
-
-<@page_html/>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/status/list.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/list.ftl b/exec/java-exec/src/main/resources/rest/status/list.ftl
deleted file mode 100644
index 613a0e9..0000000
--- a/exec/java-exec/src/main/resources/rest/status/list.ftl
+++ /dev/null
@@ -1,72 +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. -->
-
-<#include "*/generic.ftl">
-<#macro page_head>
-</#macro>
-
-<#macro page_body>
-  <a href="/queries">back</a><br/>
-  <div class="page-header">
-  </div>
-  <h3>Running Queries</h3>
-  <div class="table-responsive">
-    <table class="table table-hover">
-      <thead>
-         <td>Time</td>
-         <td>Query</td>
-      </thead>
-      <tbody>
-        <#list model.getRunningQueries() as query>
-        <tr>
-          <td>${query.getValue()}</td>
-          <td>
-            <a href="/query/${query.getKey()}">
-              <div style="height:100%;width:100%">
-                ${query.getKey()}
-              </div>
-            </a>
-          </td>
-        </tr>
-        </#list>
-      </tbody>
-    </table>
-  </div>
-  <h3>Completed Queries</h3>
-  <div class="table-responsive">
-    <table class="table table-hover">
-      <thead>
-         <td>Time</td>
-         <td>Query</td>
-      </thead>
-      <tbody>
-        <#list model.getOldQueries() as query>
-        <tr>
-          <td>${query.getValue()}</td>
-          <td>
-            <a href="/query/${query.getKey()}">
-              <div style="height:100%;width:100%">
-                ${query.getKey()}
-              </div>
-            </a>
-          </td>
-        </tr>
-        </#list>
-      </tbody>
-    </table>
-  </div>
-  <script>
-    var elem = document.getElementById("statusFontColor");
-    elem.style.color = "green";
-  </script>
-</#macro>
-
-<@page_html/>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/status/profile.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/profile.ftl b/exec/java-exec/src/main/resources/rest/status/profile.ftl
deleted file mode 100644
index 470f7ab..0000000
--- a/exec/java-exec/src/main/resources/rest/status/profile.ftl
+++ /dev/null
@@ -1,65 +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. -->
-
-<#include "*/generic.ftl">
-<#macro page_head>
-</#macro>
-
-<#macro page_body>
-  <a href="/queries">back</a><br/>
-  <div class="page-header">
-  </div>
-  <h2>Query</h2>
-  <form role="form" action="/query" method="POST">
-    <div class="form-group">
-      <textarea class="form-control" id="query" name="query">${model.query}</textarea>
-    </div>
-    <div class="form-group">
-      <div class="radio-inline">
-        <label>
-          <input type="radio" name="queryType" id="sql" value="SQL" checked>
-          SQL
-        </label>
-      </div>
-      <div class="radio-inline">
-        <label>
-          <input type="radio" name="queryType" id="physical" value="PHYSICAL">
-          PHYSICAL
-        </label>
-      </div>
-      <div class="radio-inline">
-        <label>
-          <input type="radio" name="queryType" id="logical" value="LOGICAL">
-          LOGICAL
-        </label>
-      </div>
-    </div>
-    <button type="submit" class="btn btn-default">Re-run query</button>
-  </form>
-  <div class="page-header">
-    <h2>Physical plan</h2>
-  </div>
-  <div class="well">
-    <p><font face="courier">${model.plan}</font></p>
-  </div>
-  <div class="page-header">
-    <h2>Complete Profile</h2>
-  </div>
-  <div class="well">
-    <p><font face="courier">${model.toString()}</font></p>
-  </div>
-  <script>
-      var elem = document.getElementById("statusFontColor");
-      elem.style.color = "green";
-  </script>
-</#macro>
-
-<@page_html/>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/status/query.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/query.ftl b/exec/java-exec/src/main/resources/rest/status/query.ftl
deleted file mode 100644
index a92696b..0000000
--- a/exec/java-exec/src/main/resources/rest/status/query.ftl
+++ /dev/null
@@ -1,54 +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. -->
-
-<#include "*/generic.ftl">
-<#macro page_head>
-</#macro>
-
-<#macro page_body>
-  <a href="/queries">back</a><br/>
-  <div class="page-header">
-  </div>
-  <form role="form" action="/query" method="POST">
-    <div class="form-group">
-      <label for="queryType">Query Type</label>
-      <div class="radio">
-        <label>
-          <input type="radio" name="queryType" id="sql" value="SQL" checked>
-          SQL
-        </label>
-      </div>
-      <div class="radio">
-        <label>
-          <input type="radio" name="queryType" id="physical" value="PHYSICAL">
-          PHYSICAL
-        </label>
-      </div>
-      <div class="radio">
-        <label>
-          <input type="radio" name="queryType" id="logical" value="LOGICAL">
-          LOGICAL
-        </label>
-      </div>
-    </div>
-    <div class="form-group">
-      <label for="query">Query</label>
-      <textarea class="form-control" id="query" rows="5" name="query"></textarea>
-    </div>
-    <button type="submit" class="btn btn-default">Submit</button>
-  </form>
-  <script>
-      var elem = document.getElementById("statusFontColor");
-      elem.style.color = "green";
-  </script>
-</#macro>
-
-<@page_html/>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/status/result.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/result.ftl b/exec/java-exec/src/main/resources/rest/status/result.ftl
deleted file mode 100644
index 4c054f4..0000000
--- a/exec/java-exec/src/main/resources/rest/status/result.ftl
+++ /dev/null
@@ -1,40 +0,0 @@
-<#-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
-  license agreements. See the NOTICE file distributed with this work for additional
-  information regarding copyright ownership. The ASF licenses this file to
-  You under the Apache License, Version 2.0 (the "License"); you may not use
-  this file except in compliance with the License. You may obtain a copy of
-  the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
-  by applicable law or agreed to in writing, software distributed under the
-  License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
-  OF ANY KIND, either express or implied. See the License for the specific
-  language governing permissions and limitations under the License. -->
-
-<#include "*/generic.ftl">
-<#macro page_head>
-</#macro>
-
-<#macro page_body>
-  <a href="/queries">back</a><br/>
-  <div class="page-header">
-      <h2>Result</h2>
-  </div>
-  <div class="table-responsive">
-    <table class="table">
-      <tbody>
-        <#list model as rows>
-        <tr>
-          <#list rows as row>
-          <td>${row}</td>
-          </#list>
-        </tr>
-        </#list>
-      </tbody>
-    </table>
-  </div>
-  <script>
-      var elem = document.getElementById("statusFontColor");
-      elem.style.color = "green";
-  </script>
-</#macro>
-
-<@page_html/>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ab9b2fe9/exec/java-exec/src/main/resources/rest/status/status.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/status.ftl b/exec/java-exec/src/main/resources/rest/status/status.ftl
deleted file mode 100644
index 1d4e3d3..0000000
--- a/exec/java-exec/src/main/resources/rest/status/status.ftl
+++ /dev/null
@@ -1,29 +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. -->
-
-<#include "*/generic.ftl">
-<#macro page_head>
-</#macro>
-
-<#macro page_body>
-  <a href="/queries">back</a><br/>
-  <div class="page-header">
-    <div class="alert alert-success">
-      <strong>${model}</strong>
-    </div>
-  </div>
-  <script>
-      var elem = document.getElementById("statusFontColor");
-      elem.style.color = "green";
-  </script>
-</#macro>
-
-<@page_html/>