You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ti...@apache.org on 2018/09/24 23:21:06 UTC

[drill] branch master updated (87ed03f -> 3bc3b66)

This is an automated email from the ASF dual-hosted git repository.

timothyfarkas pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git.


    from 87ed03f  DRILL-6747:
     new c1bd6f6  DRILL-6753: Fix show files command to return result the same way as before
     new 2a8d5bb  DRILL-6752: Surround Drill quotes with double quotes
     new e82f5ef  DRILL-6746: Query can hang when PartitionSender task thread sees a connection failure while sending data batches to remote fragment
     new 3bc3b66  DRILL-5782: Web UI: do not attempt to build visualized plan when plan is absent

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../resources/drill-sqlline-override-example.conf  |   2 +-
 .../planner/sql/handlers/ShowFilesHandler.java     |  98 ++++++++++----------
 .../exec/store/ischema/InfoSchemaConstants.java    |   1 +
 .../drill/exec/store/ischema/InfoSchemaTable.java  |  10 +-
 .../apache/drill/exec/store/ischema/Records.java   |  25 ++++-
 .../drill/exec/work/batch/BaseRawBatchBuffer.java  |  39 ++++++--
 .../exec/work/batch/SpoolingRawBatchBuffer.java    |  17 ++--
 .../exec/work/batch/UnlimitedRawBatchBuffer.java   |  10 ++
 .../src/main/resources/drill-sqlline.conf          |  58 ++++++------
 .../src/main/resources/rest/static/js/graph.js     |  19 ++--
 .../exec/client/DrillSqlLineApplicationTest.java   |   8 +-
 .../impersonation/TestImpersonationMetadata.java   |  47 ++++------
 .../org/apache/drill/exec/sql/TestInfoSchema.java  |   1 -
 .../drill/exec/store/ischema/TestFilesTable.java   | 103 +++++++++++----------
 .../exec/work/metadata/TestMetadataProvider.java   |   2 +-
 .../resources/drill-sqlline-test-override.conf     |   5 +-
 .../src/test/resources/drill-sqlline-test.conf     |  40 --------
 17 files changed, 257 insertions(+), 228 deletions(-)
 delete mode 100644 exec/java-exec/src/test/resources/drill-sqlline-test.conf


[drill] 02/04: DRILL-6752: Surround Drill quotes with double quotes

Posted by ti...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

timothyfarkas pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 2a8d5bb546a629aab60b95edd496ecedd81fd23f
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Fri Sep 21 15:44:14 2018 +0300

    DRILL-6752: Surround Drill quotes with double quotes
    
    1. Surround Drill quotes with double quotes.
    2. Remove drill-sqlline-test.conf, use drill-sqlline.conf for tests instead.
    
    closes #1475
---
 .../resources/drill-sqlline-override-example.conf  |  2 +-
 .../src/main/resources/drill-sqlline.conf          | 58 +++++++++++-----------
 .../exec/client/DrillSqlLineApplicationTest.java   |  8 +--
 .../resources/drill-sqlline-test-override.conf     |  5 +-
 .../src/test/resources/drill-sqlline-test.conf     | 40 ---------------
 5 files changed, 38 insertions(+), 75 deletions(-)

diff --git a/distribution/src/resources/drill-sqlline-override-example.conf b/distribution/src/resources/drill-sqlline-override-example.conf
index 34decdc..6507079 100644
--- a/distribution/src/resources/drill-sqlline-override-example.conf
+++ b/distribution/src/resources/drill-sqlline-override-example.conf
@@ -20,7 +20,7 @@ drill.sqlline {
   # ${?drill.sqlline.quotes} is used to append new quotes to the existing ones
   # remove it if you want to override existing quotes completely
   quotes: ${?drill.sqlline.quotes} [
-    "east or west, drill is the best"
+    "East or West, Drill is the best."
   ],
   # overrides default SqlLine properties
   # for the full list of properties use !set command in SqlLine
diff --git a/exec/java-exec/src/main/resources/drill-sqlline.conf b/exec/java-exec/src/main/resources/drill-sqlline.conf
index b977d9f..e081794 100644
--- a/exec/java-exec/src/main/resources/drill-sqlline.conf
+++ b/exec/java-exec/src/main/resources/drill-sqlline.conf
@@ -17,36 +17,36 @@
 # This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
 
 drill.sqlline {
-  info_message_template: "apache drill %s\n%s",
+  info_message_template: "Apache Drill %s%n\"%s\"",
   quotes: [
-    "start your sql engine",
-    "this isn't your grandfather's sql",
-    "a little sql for your nosql",
-    "json ain't no thang",
-    "drill baby drill",
-    "just drill it",
-    "say hello to my little drill",
-    "what ever the mind of man can conceive and believe, drill can query",
-    "the only truly happy people are children, the creative minority and drill users",
-    "a drill is a terrible thing to waste",
-    "got drill?",
-    "a drill in the hand is better than two in the bush",
-    "drill never goes out of style",
-    "everything is easier with drill",
-    "you told me to drill, sergeant",
-    "you're gonna need a bigger drill",
-    "keep your data close, but your drillbits closer",
-    "let's drill something more solid than concrete",
-    "drill must go on",
-    "in drill we trust",
-    "two things are infinite: the universe and drill; and i'm not sure about the universe",
-    "a query result is never late, nor is it early, it arrives precisely when it means to",
-    "think different, think drill",
-    "there are two types of analysts in the world: those who use drill and those who don't",
-    "good friends, good books and drill cluster: this is the ideal life",
-    "data is the new oil: ready to drill some?",
-    "you won't meet santa, but drill supports clauses",
-    "your drill is the drill that will pierce the heavens"
+    "Start your SQL engine.",
+    "This isn't your grandfather's SQL.",
+    "A little SQL for your NoSQL.",
+    "JSON ain't no thang.",
+    "Drill, baby, Drill.",
+    "Just Drill It.",
+    "Say hello to my little Drill.",
+    "What ever the mind of man can conceive and believe, Drill can query.",
+    "The only truly happy people are children, the creative minority and Drill users.",
+    "A Drill is a terrible thing to waste.",
+    "Got Drill?",
+    "A Drill in the hand is better than two in the bush.",
+    "Drill never goes out of style.",
+    "Everything is easier with Drill.",
+    "You told me to, Drill Sergeant!",
+    "You're gonna need a bigger Drill.",
+    "Keep your data close, but your Drillbits closer.",
+    "Let's Drill something more solid than concrete.",
+    "Drill must go on.",
+    "In Drill We Trust.",
+    "Two things are infinite: the universe and Drill; and I'm not sure about the universe.",
+    "A query result is never late, nor is it early, it arrives precisely when it means to.",
+    "Think different, think Drill.",
+    "There are two types of analysts in the world: those who use Drill and those who don't.",
+    "Good friends, good books and Drill cluster: this is the ideal life.",
+    "Data is the new oil. Ready to Drill some?",
+    "You won't meet Santa, but Drill supports Clauses.",
+    "Your Drill is the Drill that will pierce the heavens."
   ],
   drivers: [
     org.apache.drill.jdbc.Driver
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillSqlLineApplicationTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillSqlLineApplicationTest.java
index 4996ce2..8d3c290 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillSqlLineApplicationTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillSqlLineApplicationTest.java
@@ -43,14 +43,14 @@ public class DrillSqlLineApplicationTest {
   @BeforeClass
   public static void init() {
     application = new DrillSqlLineApplication(
-        "drill-sqlline-test.conf",
+        "drill-sqlline.conf",
         "drill-sqlline-test-override.conf");
   }
 
   @Test
   public void testInfoMessage() throws Exception {
     String infoMessage = application.getInfoMessage();
-    assertThat(infoMessage, containsString("all code is guilty until proven innocent"));
+    assertThat(infoMessage, containsString("\"All code is guilty until proven innocent.\""));
   }
 
   @Test
@@ -99,14 +99,14 @@ public class DrillSqlLineApplicationTest {
   @Test
   public void testEmptyConfig() {
     DrillSqlLineApplication application = new DrillSqlLineApplication(
-        "missing.conf", "missing_example.conf");
+        "missing.conf", "missing_override.conf");
     assertTrue(application.getConfig().isEmpty());
   }
 
   @Test
   public void testConfigWithoutOverride() {
     DrillSqlLineApplication application = new DrillSqlLineApplication(
-        "drill-sqlline-test.conf", "missing_example.conf");
+        "drill-sqlline.conf", "missing_override.conf");
     assertFalse(application.getConfig().isEmpty());
   }
 
diff --git a/exec/java-exec/src/test/resources/drill-sqlline-test-override.conf b/exec/java-exec/src/test/resources/drill-sqlline-test-override.conf
index 380785c..b7d949e 100644
--- a/exec/java-exec/src/test/resources/drill-sqlline-test-override.conf
+++ b/exec/java-exec/src/test/resources/drill-sqlline-test-override.conf
@@ -17,12 +17,15 @@
 # This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
 
 drill.sqlline {
+  # override all quotes for the test
   quotes: [
-    "all code is guilty until proven innocent"
+    "All code is guilty until proven innocent."
   ],
+  # override all url examples for the test
   connection_url_examples: [
     "jdbc:drill:zk=local"
   ],
+  # override maxWidth and add timeout for the test
   opts: {
     maxWidth: 20000,
     timeout: 200
diff --git a/exec/java-exec/src/test/resources/drill-sqlline-test.conf b/exec/java-exec/src/test/resources/drill-sqlline-test.conf
deleted file mode 100644
index d0d9a89..0000000
--- a/exec/java-exec/src/test/resources/drill-sqlline-test.conf
+++ /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.
-
-# This file customizes SqlLine application.
-# This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
-
-drill.sqlline {
-  info_message_template: "apache drill %s\n%s",
-  quotes: [
-    "start your sql engine"
-  ],
-  drivers: [
-    org.apache.drill.jdbc.Driver
-  ],
-  connection_url_examples: [
-    "jdbc:drill:drillbit=localhost",
-    "jdbc:drill:zk=local"
-  ],
-  commands.exclude: [
-    describe,
-    indexes
-  ],
-  opts: {
-    incremental: false,
-    isolation: TRANSACTION_NONE,
-    maxWidth: 10000
-  }
-}


[drill] 03/04: DRILL-6746: Query can hang when PartitionSender task thread sees a connection failure while sending data batches to remote fragment

Posted by ti...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

timothyfarkas pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit e82f5ef689f1051b6cffc17921462e8e5524629b
Author: Sorabh Hamirwasia <sh...@maprtech.com>
AuthorDate: Sun Sep 16 21:01:15 2018 -0700

    DRILL-6746: Query can hang when PartitionSender task thread sees a connection failure while sending data batches to remote fragment
    
    closes #1470
---
 .../drill/exec/work/batch/BaseRawBatchBuffer.java  | 39 ++++++++++++++++++----
 .../exec/work/batch/SpoolingRawBatchBuffer.java    | 17 ++++++----
 .../exec/work/batch/UnlimitedRawBatchBuffer.java   | 10 ++++++
 3 files changed, 54 insertions(+), 12 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java
index 43abd8e..6d77d63 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.work.batch;
 
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
@@ -36,8 +37,9 @@ public abstract class BaseRawBatchBuffer<T> implements RawBatchBuffer {
 
   protected interface BufferQueue<T> {
     void addOomBatch(RawFragmentBatch batch);
-    RawFragmentBatch poll() throws IOException;
+    RawFragmentBatch poll() throws IOException, InterruptedException;
     RawFragmentBatch take() throws IOException, InterruptedException;
+    RawFragmentBatch poll(long timeout, TimeUnit timeUnit) throws InterruptedException, IOException;
     boolean checkForOutOfMemory();
     int size();
     boolean isEmpty();
@@ -127,17 +129,24 @@ public abstract class BaseRawBatchBuffer<T> implements RawBatchBuffer {
    * responses pending
    */
   private void clearBufferWithBody() {
+    RawFragmentBatch batch;
     while (!bufferQueue.isEmpty()) {
-      final RawFragmentBatch batch;
+      batch = null;
       try {
         batch = bufferQueue.poll();
         assertAckSent(batch);
       } catch (IOException e) {
         context.getExecutorState().fail(e);
         continue;
-      }
-      if (batch.getBody() != null) {
-        batch.getBody().release();
+      } catch (InterruptedException e) {
+        context.getExecutorState().fail(e);
+        // keep the state that the thread is interrupted
+        Thread.currentThread().interrupt();
+        continue;
+      } finally {
+        if (batch != null && batch.getBody() != null) {
+          batch.getBody().release();
+        }
       }
     }
   }
@@ -167,7 +176,25 @@ public abstract class BaseRawBatchBuffer<T> implements RawBatchBuffer {
 
       // if we didn't get a batch, block on waiting for queue.
       if (b == null && (!isTerminated() || !bufferQueue.isEmpty())) {
-        b = bufferQueue.take();
+        // We shouldn't block infinitely here. There can be a condition such that due to a failure FragmentExecutor
+        // state is changed to FAILED and queue is empty. Because of this the minor fragment main thread will block
+        // here waiting for next batch to arrive. Meanwhile when next batch arrived and was enqueued it sees
+        // FragmentExecutor failure state and doesn't enqueue the batch and cleans up the buffer queue. Hence this
+        // thread will stuck forever. So we pool for 5 seconds until we get a batch or FragmentExecutor state is in
+        // error condition.
+        while (b == null) {
+          b = bufferQueue.poll(5, TimeUnit.SECONDS);
+          if (!context.getExecutorState().shouldContinue()) {
+            kill(context);
+            if (b != null) {
+              assertAckSent(b);
+              if (b.getBody() != null) {
+                b.getBody().release();
+              }
+              b = null;
+            }
+          } // else b will be assigned a valid batch
+        }
       }
     } catch (final InterruptedException e) {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
index 5d4b3a1..50f582d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
@@ -102,14 +102,10 @@ public class SpoolingRawBatchBuffer extends BaseRawBatchBuffer<SpoolingRawBatchB
     }
 
     @Override
-    public RawFragmentBatch poll() throws IOException {
+    public RawFragmentBatch poll() throws IOException, InterruptedException {
       RawFragmentBatchWrapper batchWrapper = buffer.poll();
       if (batchWrapper != null) {
-        try {
-          return batchWrapper.get();
-        } catch (InterruptedException e) {
-          return null;
-        }
+        return batchWrapper.get();
       }
       return null;
     }
@@ -120,6 +116,15 @@ public class SpoolingRawBatchBuffer extends BaseRawBatchBuffer<SpoolingRawBatchB
     }
 
     @Override
+    public RawFragmentBatch poll(long timeout, TimeUnit timeUnit) throws InterruptedException, IOException {
+      RawFragmentBatchWrapper batchWrapper = buffer.poll(timeout, timeUnit);
+      if (batchWrapper != null) {
+        return batchWrapper.get();
+      }
+      return null;
+    }
+
+    @Override
     public boolean checkForOutOfMemory() {
       return buffer.peek().isOutOfMemory();
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
index bf14a74..0d36d5d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.work.batch;
 
 import java.io.IOException;
 import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.RawFragmentBatch;
@@ -64,6 +65,15 @@ public class UnlimitedRawBatchBuffer extends BaseRawBatchBuffer<RawFragmentBatch
     }
 
     @Override
+    public RawFragmentBatch poll(long timeout, TimeUnit timeUnit) throws InterruptedException, IOException {
+      RawFragmentBatch batch = buffer.poll(timeout, timeUnit);
+      if (batch != null) {
+        batch.sendOk();
+      }
+      return batch;
+    }
+
+    @Override
     public boolean checkForOutOfMemory() {
       return context.getAllocator().isOverLimit();
     }


[drill] 01/04: DRILL-6753: Fix show files command to return result the same way as before

Posted by ti...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

timothyfarkas pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit c1bd6f60476c1fc2bceba7dd73652dfd5f063454
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Fri Sep 21 16:25:37 2018 +0300

    DRILL-6753: Fix show files command to return result the same way as before
    
    1. Add ACCESS_TIME column.
    2. Re-write show files command to return result using ShowFilesCommandResult to maintain backward compatibility.
    
    closes #1477
---
 .../planner/sql/handlers/ShowFilesHandler.java     |  98 ++++++++++----------
 .../exec/store/ischema/InfoSchemaConstants.java    |   1 +
 .../drill/exec/store/ischema/InfoSchemaTable.java  |  10 +-
 .../apache/drill/exec/store/ischema/Records.java   |  25 ++++-
 .../impersonation/TestImpersonationMetadata.java   |  47 ++++------
 .../org/apache/drill/exec/sql/TestInfoSchema.java  |   1 -
 .../drill/exec/store/ischema/TestFilesTable.java   | 103 +++++++++++----------
 .../exec/work/metadata/TestMetadataProvider.java   |   2 +-
 8 files changed, 154 insertions(+), 133 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java
index c9bac32..75abfdd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java
@@ -19,53 +19,39 @@ package org.apache.drill.exec.planner.sql.handlers;
 
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.SqlSelect;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.Util;
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.apache.drill.exec.planner.sql.SchemaUtilites;
-import org.apache.drill.exec.planner.sql.parser.DrillParserUtil;
 import org.apache.drill.exec.planner.sql.parser.SqlShowFiles;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory.WorkspaceSchema;
-import org.apache.drill.exec.store.ischema.InfoSchemaTableType;
+import org.apache.drill.exec.store.ischema.Records;
+import org.apache.drill.exec.util.FileSystemUtil;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
+import org.apache.hadoop.fs.Path;
 
-import java.util.Arrays;
-import java.util.Collections;
+import java.io.IOException;
+import java.sql.Timestamp;
 import java.util.List;
-
-import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.FILES_COL_RELATIVE_PATH;
-import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.FILES_COL_SCHEMA_NAME;
-import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.IS_SCHEMA_NAME;
-
+import java.util.stream.Collectors;
 
 public class ShowFilesHandler extends DefaultSqlHandler {
+
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SetOptionHandler.class);
 
   public ShowFilesHandler(SqlHandlerConfig config) {
     super(config);
   }
 
-  /** Rewrite the parse tree as SELECT ... FROM INFORMATION_SCHEMA.FILES ... */
   @Override
-  public SqlNode rewrite(SqlNode sqlNode) throws ForemanSetupException {
-
-    List<SqlNode> selectList = Collections.singletonList(SqlIdentifier.star(SqlParserPos.ZERO));
-
-    SqlNode fromClause = new SqlIdentifier(Arrays.asList(IS_SCHEMA_NAME, InfoSchemaTableType.FILES.name()), SqlParserPos.ZERO);
-
+  public PhysicalPlan getPlan(SqlNode sqlNode) throws ForemanSetupException, IOException {
     SchemaPlus defaultSchema = config.getConverter().getDefaultSchema();
     SchemaPlus drillSchema = defaultSchema;
-
     SqlShowFiles showFiles = unwrap(sqlNode, SqlShowFiles.class);
     SqlIdentifier from = showFiles.getDb();
-    boolean addRelativePathLikeClause = false;
+    String fromDir = "./";
 
     // Show files can be used without from clause, in which case we display the files in the default schema
     if (from != null) {
@@ -75,7 +61,8 @@ public class ShowFilesHandler extends DefaultSqlHandler {
       if (drillSchema == null) {
         // Entire from clause is not a schema, try to obtain the schema without the last part of the specified clause.
         drillSchema = SchemaUtilites.findSchema(defaultSchema, from.names.subList(0, from.names.size() - 1));
-        addRelativePathLikeClause = true;
+        // Listing for specific directory: show files in dfs.tmp.specific_directory
+        fromDir = fromDir + from.names.get((from.names.size() - 1));
       }
 
       if (drillSchema == null) {
@@ -85,11 +72,9 @@ public class ShowFilesHandler extends DefaultSqlHandler {
       }
     }
 
-    String fullSchemaName;
-
+    WorkspaceSchema wsSchema;
     try {
-      WorkspaceSchema wsSchema = (WorkspaceSchema) drillSchema.unwrap(AbstractSchema.class).getDefaultSchema();
-      fullSchemaName = wsSchema.getFullSchemaName();
+      wsSchema = (WorkspaceSchema) drillSchema.unwrap(AbstractSchema.class).getDefaultSchema();
     } catch (ClassCastException e) {
       throw UserException.validationError()
           .message("SHOW FILES is supported in workspace type schema only. Schema [%s] is not a workspace schema.",
@@ -97,28 +82,43 @@ public class ShowFilesHandler extends DefaultSqlHandler {
           .build(logger);
     }
 
-    SqlNode whereClause = DrillParserUtil.createCondition(new SqlIdentifier(FILES_COL_SCHEMA_NAME, SqlParserPos.ZERO),
-        SqlStdOperatorTable.EQUALS, SqlLiteral.createCharString(fullSchemaName, SqlParserPos.ZERO));
+    Path path = new Path(wsSchema.getDefaultLocation(), fromDir);
+    List<ShowFilesCommandResult> records = FileSystemUtil.listAll(wsSchema.getFS(), path, false).stream()
+        // use ShowFilesCommandResult for backward compatibility
+        .map(fileStatus -> new ShowFilesCommandResult(new Records.File(wsSchema.getFullSchemaName(), wsSchema, fileStatus)))
+        .collect(Collectors.toList());
 
-    // listing for specific directory: show files in dfs.tmp.specific_directory
-    if (addRelativePathLikeClause) {
-      if (!context.getOptions().getBoolean(ExecConstants.LIST_FILES_RECURSIVELY)) {
-        throw UserException.validationError()
-            .message("To SHOW FILES in specific directory, enable option %s", ExecConstants.LIST_FILES_RECURSIVELY)
-            .build(logger);
-      }
-
-      // like clause: relative_path like 'specific_directory/%'
-      String folderPath = from.names.get(from.names.size() - 1);
-      folderPath = folderPath.endsWith("/") ? folderPath : folderPath + "/";
-      SqlNode likeLiteral = SqlLiteral.createCharString(folderPath + "%", Util.getDefaultCharset().name(), SqlParserPos.ZERO);
-      SqlNode likeClause = DrillParserUtil.createCondition(new SqlIdentifier(FILES_COL_RELATIVE_PATH, SqlParserPos.ZERO),
-          SqlStdOperatorTable.LIKE, likeLiteral);
+    return DirectPlan.createDirectPlan(context.getCurrentEndpoint(), records, ShowFilesCommandResult.class);
+  }
 
-      whereClause = DrillParserUtil.createCondition(whereClause, SqlStdOperatorTable.AND, likeClause);
+  /**
+   * Original show files command result holder is used as wrapper over new {@link Records.File} holder
+   * to maintain backward compatibility with ODBC driver etc. in column names and types.
+   */
+  public static class ShowFilesCommandResult {
+
+    public final String name;
+    public final boolean isDirectory;
+    public final boolean isFile;
+    public final long length;
+    public final String owner;
+    public final String group;
+    public final String permissions;
+    public final Timestamp accessTime;
+    public final Timestamp modificationTime;
+
+    public ShowFilesCommandResult(Records.File fileRecord) {
+      this.name = fileRecord.FILE_NAME;
+      this.isDirectory = fileRecord.IS_DIRECTORY;
+      this.isFile = fileRecord.IS_FILE;
+      this.length = fileRecord.LENGTH;
+      this.owner = fileRecord.OWNER;
+      this.group = fileRecord.GROUP;
+      this.permissions = fileRecord.PERMISSION;
+      this.accessTime = fileRecord.ACCESS_TIME;
+      this.modificationTime = fileRecord.MODIFICATION_TIME;
     }
 
-    return new SqlSelect(SqlParserPos.ZERO, null, new SqlNodeList(selectList, SqlParserPos.ZERO), fromClause, whereClause,
-        null, null, null, null, null, null);
   }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java
index 63e19d3..4c07f30 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java
@@ -100,5 +100,6 @@ public interface InfoSchemaConstants {
    String FILES_COL_OWNER = "OWNER";
    String FILES_COL_GROUP = "GROUP";
    String FILES_COL_PERMISSION = "PERMISSION";
+   String FILES_COL_ACCESS_TIME = "ACCESS_TIME";
    String FILES_COL_MODIFICATION_TIME = "MODIFICATION_TIME";
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTable.java
index 1f99c0e..e09942e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaTable.java
@@ -33,6 +33,7 @@ import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_N
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_NUMERIC_PRECISION_RADIX;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_NUMERIC_SCALE;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_ORDINAL_POSITION;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.FILES_COL_ACCESS_TIME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.FILES_COL_FILE_NAME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.FILES_COL_GROUP;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.FILES_COL_IS_DIRECTORY;
@@ -90,6 +91,7 @@ public abstract class InfoSchemaTable<S> {
   public static final MajorType BIGINT = Types.required(MinorType.BIGINT);
   public static final MajorType VARCHAR = Types.required(MinorType.VARCHAR);
   public static final MajorType BIT = Types.required(MinorType.BIT);
+  public static final MajorType TIMESTAMP = Types.required(MinorType.TIMESTAMP);
 
   private final List<Field> fields;
 
@@ -121,8 +123,11 @@ public abstract class InfoSchemaTable<S> {
         return typeFactory.createSqlType(SqlTypeName.VARCHAR, Integer.MAX_VALUE);
       case BIT:
         return typeFactory.createSqlType(SqlTypeName.BOOLEAN);
+      case TIMESTAMP:
+        return typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
       default:
-        throw new UnsupportedOperationException("Only INT, BIGINT, VARCHAR and BOOLEAN types are supported in " + InfoSchemaConstants.IS_SCHEMA_NAME);
+        throw new UnsupportedOperationException("Only INT, BIGINT, VARCHAR, BOOLEAN and TIMESTAMP types are supported in " +
+            InfoSchemaConstants.IS_SCHEMA_NAME);
     }
   }
 
@@ -275,7 +280,8 @@ public abstract class InfoSchemaTable<S> {
         Field.create(FILES_COL_OWNER, VARCHAR),
         Field.create(FILES_COL_GROUP, VARCHAR),
         Field.create(FILES_COL_PERMISSION, VARCHAR),
-        Field.create(FILES_COL_MODIFICATION_TIME, VARCHAR)
+        Field.create(FILES_COL_ACCESS_TIME, TIMESTAMP),
+        Field.create(FILES_COL_MODIFICATION_TIME, TIMESTAMP)
     );
 
     public Files() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
index 086aa0d..5608560 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
@@ -31,9 +31,10 @@ import org.slf4j.Logger;
 
 import org.apache.drill.shaded.guava.com.google.common.base.MoreObjects;
 
+import java.sql.Timestamp;
 import java.time.Instant;
+import java.time.ZoneId;
 import java.time.ZoneOffset;
-import java.time.format.DateTimeFormatter;
 
 public class Records {
 
@@ -565,7 +566,8 @@ public class Records {
     public final String OWNER;
     public final String GROUP;
     public final String PERMISSION;
-    public final String MODIFICATION_TIME;
+    public final Timestamp ACCESS_TIME;
+    public final Timestamp MODIFICATION_TIME;
 
     public File(String schemaName, WorkspaceSchemaFactory.WorkspaceSchema wsSchema, FileStatus fileStatus) {
       this.SCHEMA_NAME = schemaName;
@@ -580,9 +582,22 @@ public class Records {
       this.OWNER = fileStatus.getOwner();
       this.GROUP = fileStatus.getGroup();
       this.PERMISSION = fileStatus.getPermission().toString();
-      this.MODIFICATION_TIME = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")
-          .withZone(ZoneOffset.UTC)
-          .format(Instant.ofEpochMilli(fileStatus.getModificationTime()));
+      this.ACCESS_TIME = getTimestampWithReplacedZone(fileStatus.getAccessTime());
+      this.MODIFICATION_TIME = getTimestampWithReplacedZone(fileStatus.getModificationTime());
+    }
+
+    /**
+     * Convert milliseconds into sql timestamp.
+     * Get the timestamp in UTC because Drill's internal TIMESTAMP stores time in UTC.
+     *
+     * @param ms milliseconds
+     * @return sql timestamp instance
+     */
+    private Timestamp getTimestampWithReplacedZone(long ms) {
+      return Timestamp.from(Instant.ofEpochMilli(ms)
+          .atZone(ZoneId.systemDefault())
+          .withZoneSameLocal(ZoneOffset.UTC)
+          .toInstant());
     }
   }
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java
index fd747bb..9acd015 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java
@@ -23,7 +23,6 @@ import org.apache.drill.categories.SecurityTest;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.exceptions.UserRemoteException;
-import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.store.dfs.WorkspaceConfig;
 import org.apache.drill.categories.SlowTest;
 import org.apache.hadoop.fs.FileSystem;
@@ -33,15 +32,14 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 
 import java.util.Map;
 
 import static org.hamcrest.core.StringContains.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -60,6 +58,9 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
     UserGroupInformation.createUserForTesting(user2, new String[]{ group1 });
   }
 
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
   @BeforeClass
   public static void setup() throws Exception {
     startMiniDfsCluster(TestImpersonationMetadata.class.getSimpleName());
@@ -176,14 +177,11 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
   public void testShowFilesInWSWithNoPermissionsForQueryUser() throws Exception {
     updateClient(user2);
 
-    try {
-      setSessionOption(ExecConstants.LIST_FILES_RECURSIVELY, true);
-      // Try show tables in schema "drill_test_grp_1_700" which is owned by "user1"
-      int count = testSql(String.format("SHOW FILES IN %s.drill_test_grp_1_700", MINI_DFS_STORAGE_PLUGIN_NAME));
-      assertEquals("Counts should match", 0, count);
-    } finally {
-      resetSessionOption(ExecConstants.LIST_FILES_RECURSIVELY);
-    }
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("Permission denied: user=drillTestUser2, " +
+        "access=READ_EXECUTE, inode=\"/drill_test_grp_1_700\":drillTestUser1:drill_test_grp_1:drwx------"));
+    // Try show tables in schema "drill_test_grp_1_700" which is owned by "user1"
+    test("SHOW FILES IN %s.drill_test_grp_1_700", MINI_DFS_STORAGE_PLUGIN_NAME);
   }
 
   @Test
@@ -345,25 +343,18 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
   @Test
   public void testCreateTableInWSWithNoPermissionsForQueryUser() throws Exception {
     // Workspace dir owned by "processUser", workspace group is "group0" and "user2" is not part of "group0"
-    final String tableWS = "drill_test_grp_0_755";
-    final String tableName = "table1";
-
-    UserRemoteException ex = null;
+    String tableWS = "drill_test_grp_0_755";
+    String tableName = "table1";
 
-    try {
-      updateClient(user2);
-
-      test("USE " + Joiner.on(".").join(MINI_DFS_STORAGE_PLUGIN_NAME, tableWS));
+    updateClient(user2);
+    test("use %s.`%s`", MINI_DFS_STORAGE_PLUGIN_NAME, tableWS);
 
-      test("CREATE TABLE " + tableName + " AS SELECT " +
-          "c_custkey, c_nationkey FROM cp.`tpch/customer.parquet` ORDER BY c_custkey;");
-    } catch(UserRemoteException e) {
-      ex = e;
-    }
+    thrown.expect(UserRemoteException.class);
+    thrown.expectMessage(containsString("Permission denied: user=drillTestUser2, " +
+        "access=WRITE, inode=\"/drill_test_grp_0_755/"));
 
-    assertNotNull("UserRemoteException is expected", ex);
-    assertThat(ex.getMessage(),
-        containsString("SYSTEM ERROR: RemoteException: Permission denied: user=drillTestUser2, access=WRITE, inode=\"/drill_test_grp_0_755/"));
+    test("CREATE TABLE %s AS SELECT c_custkey, c_nationkey " +
+        "FROM cp.`tpch/customer.parquet` ORDER BY c_custkey", tableName);
   }
 
   @Test
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
index e295eee..ee89c5c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
@@ -49,7 +49,6 @@ import static org.junit.Assert.assertTrue;
  * -- InformationSchema
  * -- Queries on InformationSchema such as SHOW TABLES, SHOW SCHEMAS or DESCRIBE table
  * -- USE schema
- * -- SHOW FILES
  */
 @Category(SqlTest.class)
 public class TestInfoSchema extends BaseTestQuery {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestFilesTable.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestFilesTable.java
index f8ea9a1..35b0cbc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestFilesTable.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestFilesTable.java
@@ -18,11 +18,11 @@
 package org.apache.drill.exec.store.ischema;
 
 import org.apache.drill.categories.SqlTest;
-import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterFixtureBuilder;
 import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
@@ -31,6 +31,7 @@ import org.junit.rules.ExpectedException;
 
 import java.io.File;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
@@ -45,11 +46,22 @@ public class TestFilesTable extends ClusterTest {
     // create one workspace named files
     File filesWorkspace = cluster.makeDataDir("files", null, null);
 
-    // add data to the workspace: one file and folder with one file
-    assertTrue(new File(filesWorkspace, "file1.txt").createNewFile());
-    File folder = new File(filesWorkspace, "folder");
-    assertTrue(folder.mkdir());
-    assertTrue(new File(folder, "file2.txt").createNewFile());
+    /*
+      Add data to the workspace:
+       ../files
+       ../files/file0.txt
+       ../files/folder1
+       ../files/folder1/file1.txt
+       ../files/folder1/folder2
+       ../files/folder1/folder2/file2.txt
+     */
+    assertTrue(new File(filesWorkspace, "file0.txt").createNewFile());
+    File folder1 = new File(filesWorkspace, "folder1");
+    assertTrue(folder1.mkdir());
+    assertTrue(new File(folder1, "file1.txt").createNewFile());
+    File folder2 = new File(folder1, "folder2");
+    assertTrue(folder2.mkdir());
+    assertTrue(new File(folder2, "file2.txt").createNewFile());
   }
 
   @Rule
@@ -58,11 +70,12 @@ public class TestFilesTable extends ClusterTest {
   @Test
   public void testSelectWithoutRecursion() throws Exception {
     client.testBuilder()
-        .sqlQuery("select schema_name, root_schema_name, workspace_name, file_name, relative_path, is_directory, is_file from INFORMATION_SCHEMA.`FILES`")
+        .sqlQuery("select schema_name, root_schema_name, workspace_name, file_name, relative_path, is_directory, is_file " +
+            "from INFORMATION_SCHEMA.`FILES`")
         .unOrdered()
         .baselineColumns("schema_name", "root_schema_name", "workspace_name", "file_name", "relative_path", "is_directory", "is_file")
-        .baselineValues("dfs.files", "dfs", "files", "file1.txt", "file1.txt", false, true)
-        .baselineValues("dfs.files", "dfs", "files", "folder", "folder", true, false)
+        .baselineValues("dfs.files", "dfs", "files", "file0.txt", "file0.txt", false, true)
+        .baselineValues("dfs.files", "dfs", "files", "folder1", "folder1", true, false)
         .go();
   }
 
@@ -71,12 +84,15 @@ public class TestFilesTable extends ClusterTest {
     try {
       client.alterSession(ExecConstants.LIST_FILES_RECURSIVELY, true);
       client.testBuilder()
-          .sqlQuery("select schema_name, root_schema_name, workspace_name, file_name, relative_path, is_directory, is_file from INFORMATION_SCHEMA.`FILES`")
+          .sqlQuery("select schema_name, root_schema_name, workspace_name, file_name, relative_path, is_directory, is_file " +
+              "from INFORMATION_SCHEMA.`FILES`")
           .unOrdered()
           .baselineColumns("schema_name", "root_schema_name", "workspace_name", "file_name", "relative_path", "is_directory", "is_file")
-          .baselineValues("dfs.files", "dfs", "files", "file1.txt", "file1.txt", false, true)
-          .baselineValues("dfs.files", "dfs", "files", "folder", "folder", true, false)
-          .baselineValues("dfs.files", "dfs", "files", "file2.txt", "folder/file2.txt", false, true)
+          .baselineValues("dfs.files", "dfs", "files", "file0.txt", "file0.txt", false, true)
+          .baselineValues("dfs.files", "dfs", "files", "folder1", "folder1", true, false)
+          .baselineValues("dfs.files", "dfs", "files", "file1.txt", "folder1/file1.txt", false, true)
+          .baselineValues("dfs.files", "dfs", "files", "folder2", "folder1/folder2", true, false)
+          .baselineValues("dfs.files", "dfs", "files", "file2.txt", "folder1/folder2/file2.txt", false, true)
           .go();
     } finally {
       client.resetSession(ExecConstants.LIST_FILES_RECURSIVELY);
@@ -86,62 +102,47 @@ public class TestFilesTable extends ClusterTest {
 
   @Test
   public void testShowFilesWithInCondition() throws Exception {
-    client.testBuilder()
-        .sqlQuery("show files in dfs.`files`")
-        .unOrdered()
-        .sqlBaselineQuery("select * from INFORMATION_SCHEMA.`FILES` where schema_name = 'dfs.files'")
-        .go();
+    checkCounts("show files in dfs.`files`",
+        "select * from INFORMATION_SCHEMA.`FILES` where schema_name = 'dfs.files'");
   }
 
   @Test
-  public void testShowFilesForSpecificFolderSuccess() throws Exception {
+  public void testShowFilesForSpecificDirectory() throws Exception {
     try {
+      client.alterSession(ExecConstants.LIST_FILES_RECURSIVELY, false);
+      QueryBuilder queryBuilder = client.queryBuilder().sql("show files in dfs.`files`.folder1");
+      QueryBuilder.QuerySummary querySummary = queryBuilder.run();
+      assertTrue(querySummary.succeeded());
+      assertEquals(2, querySummary.recordCount());
+      // option has no effect
       client.alterSession(ExecConstants.LIST_FILES_RECURSIVELY, true);
-      client.testBuilder()
-          .sqlQuery("show files in dfs.`files`.folder")
-          .unOrdered()
-          .sqlBaselineQuery("select * from INFORMATION_SCHEMA.`FILES` where schema_name = 'dfs.files' and relative_path like 'folder/%'")
-          .go();
+      querySummary = queryBuilder.run();
+      assertTrue(querySummary.succeeded());
+      assertEquals(2, querySummary.recordCount());
     } finally {
       client.resetSession(ExecConstants.LIST_FILES_RECURSIVELY);
     }
   }
 
   @Test
-  public void testShowFilesForSpecificFolderFailure() throws Exception {
-    thrown.expect(UserRemoteException.class);
-    thrown.expectMessage(String.format("To SHOW FILES in specific directory, enable option %s", ExecConstants.LIST_FILES_RECURSIVELY));
-    queryBuilder().sql("show files in dfs.`files`.folder").run();
-  }
-
-  @Test
   public void testShowFilesWithUseClause() throws Exception {
     queryBuilder().sql("use dfs.`files`").run();
-    client.testBuilder()
-        .sqlQuery("show files")
-        .unOrdered()
-        .sqlBaselineQuery("select * from INFORMATION_SCHEMA.`FILES` where schema_name = 'dfs.files'")
-        .go();
+    checkCounts("show files",
+        "select * from INFORMATION_SCHEMA.`FILES` where schema_name = 'dfs.files'");
   }
 
   @Test
   public void testShowFilesWithPartialUseClause() throws Exception {
     queryBuilder().sql("use dfs").run();
-    client.testBuilder()
-        .sqlQuery("show files in `files`")
-        .unOrdered()
-        .sqlBaselineQuery("select * from INFORMATION_SCHEMA.`FILES` where schema_name = 'dfs.files'")
-        .go();
+    checkCounts("show files in `files`",
+        "select * from INFORMATION_SCHEMA.`FILES` where schema_name = 'dfs.files'");
   }
 
   @Test
   public void testShowFilesForDefaultSchema() throws Exception {
-    queryBuilder().sql("use dfs").run();
-    client.testBuilder()
-        .sqlQuery("show files")
-        .unOrdered()
-        .sqlBaselineQuery("select * from INFORMATION_SCHEMA.`FILES` where schema_name = 'dfs.default'")
-        .go();
+    queryBuilder().sql("use dfs").run().succeeded();
+    checkCounts("show files",
+        "select * from INFORMATION_SCHEMA.`FILES` where schema_name = 'dfs.default'");
   }
 
   @Test
@@ -165,5 +166,13 @@ public class TestFilesTable extends ClusterTest {
     assertFalse(plan.contains("Filter(condition="));
   }
 
+  private void checkCounts(String testQuery, String baseQuery) throws Exception {
+    QueryBuilder.QuerySummary testQuerySummary = queryBuilder().sql(testQuery).run();
+    assertTrue(testQuerySummary.succeeded());
+    QueryBuilder.QuerySummary baseQuerySummary = queryBuilder().sql(baseQuery).run();
+    assertTrue(baseQuerySummary.succeeded());
+    assertEquals(testQuerySummary.recordCount(), baseQuerySummary.recordCount());
+  }
+
 }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java b/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java
index 76c472c..de55f19 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java
@@ -250,7 +250,7 @@ public class TestMetadataProvider extends BaseTestQuery {
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<ColumnMetadata> columns = resp.getColumnsList();
-    assertEquals(134, columns.size());
+    assertEquals(135, columns.size());
     // too many records to verify the output.
   }
 


[drill] 04/04: DRILL-5782: Web UI: do not attempt to build visualized plan when plan is absent

Posted by ti...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

timothyfarkas pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 3bc3b66919536dc94015dc23ce43e7254623de5d
Author: Anton Gozhiy <an...@gmail.com>
AuthorDate: Fri Sep 21 15:19:33 2018 +0300

    DRILL-5782: Web UI: do not attempt to build visualized plan when plan is absent
    
    Added checks for undefined values. Now there are no js errors and also a JSON profile is now displayed for incorrect queries.
    
    closes #1476
---
 .../src/main/resources/rest/static/js/graph.js        | 19 +++++++++++--------
 1 file changed, 11 insertions(+), 8 deletions(-)

diff --git a/exec/java-exec/src/main/resources/rest/static/js/graph.js b/exec/java-exec/src/main/resources/rest/static/js/graph.js
index 9ea117c..5801245 100644
--- a/exec/java-exec/src/main/resources/rest/static/js/graph.js
+++ b/exec/java-exec/src/main/resources/rest/static/js/graph.js
@@ -19,10 +19,12 @@ $(window).on('load',(function () {
         return $.map(table, function (record, index) {
             var ra = [];
             var nested = faccess(record);
-            for (var i = 0; i < nested.length; i++) {
-                var newrec = $.extend({}, record);
-                newrec[dest] = nested[i];
-                ra.push(newrec);
+            if (nested !== undefined) {
+                for (var i = 0; i < nested.length; i++) {
+                    var newrec = $.extend({}, record);
+                    newrec[dest] = nested[i];
+                    ra.push(newrec);
+                }
             }
             return ra;
         });
@@ -350,9 +352,10 @@ $(window).on('load',(function () {
 
     function setupglobalconfig (profile) {
         globalconfig.profile = profile;
-        globalconfig.majorcolorscale = d3.scale.category20()
-            .domain([0, d3.max(profile.fragmentProfile, accessor("majorFragmentId"))]);
-
+        if (profile.fragmentProfile !== undefined) {
+            globalconfig.majorcolorscale = d3.scale.category20()
+                .domain([0, d3.max(profile.fragmentProfile, accessor("majorFragmentId"))]);
+        }
     }
 
     String.prototype.endsWith = function(suffix) {
@@ -368,7 +371,7 @@ $(window).on('load',(function () {
 
         // trigger svg drawing when visible
         $('#query-tabs').on('shown.bs.tab', function (e) {
-            if (queryvisualdrawn || !e.target.href.endsWith("#query-visual")) return;
+            if (profile.plan === undefined || queryvisualdrawn || !e.target.href.endsWith("#query-visual")) return;
             buildplangraph(d3.select("#query-visual-canvas"), profile.plan);
             queryvisualdrawn = true;
         })