You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2015/02/09 21:01:03 UTC

drill git commit: DRILL-2172: make web UI bound check before accessing a vector, returning null for overflowing indices; fix a non-string type rendering problem;

Repository: drill
Updated Branches:
  refs/heads/master 0d867b0da -> 3d863b5eb


DRILL-2172: make web UI bound check before accessing a vector, returning null for overflowing indices; fix a non-string type rendering problem;


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

Branch: refs/heads/master
Commit: 3d863b5ebff53ba5b8fcde922303643137f4b19b
Parents: 0d867b0
Author: Hanifi Gunes <hg...@maprtech.com>
Authored: Thu Feb 5 14:16:52 2015 -0800
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Mon Feb 9 12:00:45 2015 -0800

----------------------------------------------------------------------
 .../drill/exec/server/rest/QueryResources.java  |  70 +++++------
 .../drill/exec/server/rest/QueryWrapper.java    | 126 +++++++++----------
 .../src/main/resources/rest/query/result.ftl    |   8 +-
 3 files changed, 88 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/3d863b5e/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
index f8998ed..9ef6676 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.server.rest;
 
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
@@ -30,6 +30,7 @@ import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.coord.ClusterCoordinator;
@@ -55,7 +56,7 @@ public class QueryResources {
   @Path("/query.json")
   @Consumes(MediaType.APPLICATION_JSON)
   @Produces(MediaType.APPLICATION_JSON)
-  public List<Map<String, Object>> submitQueryJSON(QueryWrapper query) throws Exception {
+  public QueryWrapper.QueryResult submitQueryJSON(QueryWrapper query) throws Exception {
     final DrillConfig config = work.getContext().getConfig();
     final ClusterCoordinator coordinator = work.getContext().getClusterCoordinator();
     final BufferAllocator allocator = work.getContext().getAllocator();
@@ -67,58 +68,43 @@ public class QueryResources {
   @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
   @Produces(MediaType.TEXT_HTML)
   public Viewable submitQuery(@FormParam("query") String query, @FormParam("queryType") String queryType) throws Exception {
-    List<Map<String, Object>> result = submitQueryJSON(new QueryWrapper(query, queryType));
-
-    List<String> columnNames;
-    if (result.isEmpty()) {
-      columnNames = Lists.newArrayList();
-    } else {
-      columnNames = Lists.newArrayList(result.get(0).keySet());
-    }
-    List<List<Object>> records = new ArrayList<>();
-
-    if(!isEmptyResult(result)) {
-      for (Map m : result) {
-        records.add(new ArrayList<Object>(m.values()));
-      }
-    }
-
-    Table table = new Table(columnNames, records);
-
-    return new Viewable("/rest/query/result.ftl", table);
+    final QueryWrapper.QueryResult result = submitQueryJSON(new QueryWrapper(query, queryType));
+    return new Viewable("/rest/query/result.ftl", new TabularResult(result));
   }
 
-  private boolean isEmptyResult(List<Map<String, Object>> result) {
-    if (result.size() > 1) {
-      return false;
-    } else {
-      for(Object col : result.get(0).values()) {
-        if(col != null) {
-          return false;
+  public static class TabularResult {
+    private final List<String> columns;
+    private final List<List<String>> rows;
+
+    public TabularResult(QueryWrapper.QueryResult result) {
+      final List<List<String>> rows = Lists.newArrayList();
+      for (Map<String, String> rowMap:result.rows) {
+        final List<String> row = Lists.newArrayList();
+        for (String col:result.columns) {
+          row.add(rowMap.get(col));
         }
+        rows.add(row);
       }
 
-      return true;
+      this.columns = ImmutableList.copyOf(result.columns);
+      this.rows = rows;
     }
-  }
 
-  public class Table {
-    private List<String> columnNames;
-    private List<List<Object>> records;
-
-    public Table(List<String> columnNames, List<List<Object>> records) {
-      this.columnNames = columnNames;
-      this.records = records;
+    public TabularResult(List<String> columns, List<List<String>> rows) {
+      this.columns = columns;
+      this.rows = rows;
     }
 
-    public boolean isEmpty() { return getColumnNames().isEmpty(); }
+    public boolean isEmpty() {
+      return columns.isEmpty();
+    }
 
-    public List<String> getColumnNames() {
-      return columnNames;
+    public List<String> getColumns() {
+      return columns;
     }
 
-    public List<List<Object>> getRecords() {
-      return records;
+    public List<List<String>> getRows() {
+      return rows;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/3d863b5e/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
index 922533a..8996a69 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
@@ -18,24 +18,23 @@
 
 package org.apache.drill.exec.server.rest;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.xml.bind.annotation.XmlRootElement;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.physical.impl.flatten.FlattenRecordBatch;
 import org.apache.drill.exec.proto.UserBitShared;
-import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.RpcException;
@@ -43,10 +42,10 @@ 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.proto.UserBitShared.SerializedField;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import parquet.Preconditions;
 
 @XmlRootElement
 public class QueryWrapper {
@@ -80,24 +79,36 @@ public class QueryWrapper {
     return type;
   }
 
-  public List<Map<String, Object>> run(DrillConfig config, ClusterCoordinator coordinator, BufferAllocator allocator)
+  public QueryResult run(DrillConfig config, ClusterCoordinator coordinator, BufferAllocator allocator)
     throws Exception {
     try(DrillClient client = new DrillClient(config, coordinator, allocator)){
-      Listener listener = new Listener(new RecordBatchLoader(allocator));
+      Listener listener = new Listener(allocator);
 
       client.connect();
       client.runQuery(getType(), query, listener);
 
-      List<Map<String, Object>> result = listener.waitForCompletion();
-      if (result.isEmpty()) {
-        Map<String, Object> dumbRecord = new HashMap<>();
-        for (String columnName : listener.getColumnNames()) {
-          dumbRecord.put(columnName, null);
+      listener.waitForCompletion();
+      if (listener.results.isEmpty()) {
+        listener.results.add(Maps.<String, String>newHashMap());
+      }
+      final Map<String, String> first = listener.results.get(0);
+      for (String columnName : listener.columns) {
+        if (!first.containsKey(columnName)) {
+          first.put(columnName, null);
         }
-        result.add(dumbRecord);
       }
 
-      return result;
+      return new QueryResult(listener.columns, listener.results);
+    }
+  }
+
+  public static class QueryResult {
+    public final Collection<String> columns;
+    public final List<Map<String, String>> rows;
+
+    public QueryResult(Collection<String> columns, List<Map<String, String>> rows) {
+      this.columns = columns;
+      this.rows = rows;
     }
   }
 
@@ -109,15 +120,13 @@ public class QueryWrapper {
 
   private static class Listener implements UserResultsListener {
     private volatile Exception exception;
-    private AtomicInteger count = new AtomicInteger();
-    private CountDownLatch latch = new CountDownLatch(1);
-    private List<Map<String, Object>> output = new LinkedList<>();
-    private ArrayList<String> columnNames;
-    private RecordBatchLoader loader;
-    private boolean schemaAdded = false;
-
-    Listener(RecordBatchLoader loader) {
-      this.loader = loader;
+    private final CountDownLatch latch = new CountDownLatch(1);
+    private final BufferAllocator allocator;
+    public final List<Map<String, String>> results = Lists.newArrayList();
+    public final Set<String> columns = Sets.newLinkedHashSet();
+
+    Listener(BufferAllocator allocator) {
+      this.allocator = Preconditions.checkNotNull(allocator, "allocator cannot be null");
     }
 
     @Override
@@ -129,68 +138,45 @@ public class QueryWrapper {
 
     @Override
     public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
-      int rows = result.getHeader().getRowCount();
-      if (result.hasData()) {
-        count.addAndGet(rows);
-        try {
+      try {
+        final int rows = result.getHeader().getRowCount();
+        if (result.hasData()) {
+          final RecordBatchLoader loader = new RecordBatchLoader(allocator);
           loader.load(result.getHeader().getDef(), result.getData());
-          if (!schemaAdded || output.isEmpty()) {
-                columnNames = new ArrayList<>();
-            for (int i = 0; i < loader.getSchema().getFieldCount(); ++i) {
-              columnNames.add(loader.getSchema().getColumn(i).getPath().getAsUnescapedPath());
-            }
-            schemaAdded = true;
+          for (int i = 0; i < loader.getSchema().getFieldCount(); ++i) {
+            columns.add(loader.getSchema().getColumn(i).getPath().getAsUnescapedPath());
           }
-        } catch (SchemaChangeException e) {
-          throw new RuntimeException(e);
-        }
-        for (int i = 0; i < rows; ++i) {
-          Map<String, Object> record = new HashMap<>();
-          int j = 0;
-          for (VectorWrapper<?> vw : loader) {
-            ValueVector.Accessor accessor = vw.getValueVector().getAccessor();
-            Object object = accessor.getObject(i);
-            if (object != null && (! object.getClass().getName().startsWith("java.lang"))) {
-              object = object.toString();
-            }
-            if (object != null) {
-              record.put(columnNames.get(j), object);
-            } else {
-              record.put(columnNames.get(j), null);
+          for (int i = 0; i < rows; ++i) {
+            final Map<String, String> record = Maps.newHashMap();
+            for (VectorWrapper<?> vw : loader) {
+              final String field = vw.getValueVector().getMetadata().getNamePart().getName();
+              final ValueVector.Accessor accessor = vw.getValueVector().getAccessor();
+              final Object value = i < accessor.getValueCount() ? accessor.getObject(i) : null;
+              final String display = value == null ? null : value.toString();
+              record.put(field, display);
             }
-            ++j;
+            results.add(record);
           }
-          output.add(record);
         }
-      } else if (!schemaAdded) {
-        columnNames = new ArrayList<>();
-        schemaAdded = true;
-        for (SerializedField fmd : result.getHeader().getDef().getFieldList()) {
-          MaterializedField fieldDef = MaterializedField.create(fmd);
-          columnNames.add(fieldDef.getPath().getAsUnescapedPath());
+      } catch (SchemaChangeException e) {
+        throw new RuntimeException(e);
+      } finally {
+        result.release();
+        if (result.getHeader().getIsLastChunk()) {
+          latch.countDown();
         }
       }
-
-      result.release();
-      if (result.getHeader().getIsLastChunk()) {
-        latch.countDown();
-      }
     }
 
     @Override
     public void queryIdArrived(UserBitShared.QueryId queryId) {
     }
 
-    public List<Map<String, Object>> waitForCompletion() throws Exception {
+    public void waitForCompletion() throws Exception {
       latch.await();
       if (exception != null) {
         throw exception;
       }
-      return output;
-    }
-
-    public List<String> getColumnNames() {
-      return new ArrayList<String> (columnNames);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/3d863b5e/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
index b69bd6d..7fe52a4 100644
--- a/exec/java-exec/src/main/resources/rest/query/result.ftl
+++ b/exec/java-exec/src/main/resources/rest/query/result.ftl
@@ -33,16 +33,16 @@
     <table id="result" class="table table-striped table-bordered table-condensed" style="table-layout: auto; width=100%;">
       <thead>
         <tr>
-          <#list model.getColumnNames() as value>
-          <th>${value}</th>
+          <#list model.getColumns() as value>
+          <th>${value?html}</th>
           </#list>
         </tr>
       </thead>
       <tbody>
-      <#list model.getRecords() as record>
+      <#list model.getRows() as record>
         <tr>
           <#list record as value>
-          <td><#if value??>${value}<#else>null</#if></td>
+          <td>${value!"null"?html}</td>
           </#list>
         </tr>
       </#list>