You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by dz...@apache.org on 2022/10/21 11:49:18 UTC

[drill] 05/09: DRILL-8281: Info schema LIKE with ESCAPE push down bug. (#2627)

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

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

commit 86f7eab40bb75c0ab485f83261210cd6562dac82
Author: James Turton <91...@users.noreply.github.com>
AuthorDate: Mon Aug 22 05:53:09 2022 +0200

    DRILL-8281: Info schema LIKE with ESCAPE push down bug. (#2627)
---
 .../drill/exec/store/ischema/InfoSchemaFilter.java |  26 +-
 .../org/apache/drill/exec/sql/TestInfoSchema.java  | 270 ++++++++++++---------
 2 files changed, 174 insertions(+), 122 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilter.java
index bc71c98c0b..ebf6c19e47 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaFilter.java
@@ -147,7 +147,7 @@ public class InfoSchemaFilter {
    * Evaluate the filter for given <COLUMN NAME, VALUE> pairs.
    *
    * @param recordValues map of field names and their values
-   * @param prefixMatchesInconclusive whether a prefix match between a column value and a filter value
+   * @param prefixMatchesInconclusive whether a prefix match between a schema path and a filter value
    *                                  results in Result.INCONCLUSIVE.  Used for pruning the schema search
    *                                  tree, e.g. "dfs" need not be recursed to find a schema of "cp.default"
    * @return evaluation result
@@ -194,15 +194,23 @@ public class InfoSchemaFilter {
           : sqlToRegexLike(pattern.value, escape.value);
 
         if (Pattern.matches(spi.getJavaPatternString(), fieldValue)) {
+          // E.g. pattern = 'dfs.%', schema path = 'dfs.tmp'.
+          // The entire schema path matches.
           return Result.TRUE;
         }
         if (!prefixMatchesInconclusive) {
+          // E.g. pattern = 'dfs.%', schema path = 'dfs'.
+          // There may be prefix match but prefixMatchesInconclusive is false.
           return Result.FALSE;
         }
         if ((spi.getPatternType() == SqlPatternType.STARTS_WITH || spi.getPatternType() == SqlPatternType.CONSTANT) &&
-          !pattern.value.startsWith(fieldValue)) {
+          !spi.getSimplePatternString().startsWith(fieldValue)) {
+            // E.g. pattern = 'dfs.%', schema path = 'cp'.
+            // No match, not even to a prefix.
             return Result.FALSE;
           }
+        // E.g. pattern = 'dfs.%', schema path = 'dfs'.
+        // A prefix matches
         return Result.INCONCLUSIVE;
       }
       case FunctionNames.EQ:
@@ -220,24 +228,18 @@ public class InfoSchemaFilter {
         boolean exactMatch = prefixMatch && arg.value.equals(value);
 
         if (exprNode.function.equals(FunctionNames.EQ)) {
+          // Equality case
           if (exactMatch) {
             return Result.TRUE;
           } else {
-            if (prefixMatchesInconclusive && prefixMatch) {
-              return Result.INCONCLUSIVE;
-            } else {
-              return Result.FALSE;
-            }
+            return prefixMatchesInconclusive && prefixMatch ? Result.INCONCLUSIVE: Result.FALSE;
           }
         } else {
+          // Inequality case
           if (exactMatch) {
             return Result.FALSE;
           } else {
-            if (prefixMatchesInconclusive && prefixMatch) {
-              return Result.INCONCLUSIVE;
-            } else {
-              return Result.TRUE;
-            }
+            return prefixMatchesInconclusive && prefixMatch ? Result.INCONCLUSIVE : Result.TRUE;
           }
         }
       }
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 fa616e6623..469462e601 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
@@ -18,16 +18,23 @@
 package org.apache.drill.exec.sql;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.drill.PlanTestBase;
 import org.apache.drill.categories.SqlTest;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.common.exceptions.UserRemoteException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.DirectRowSet;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.physical.rowSet.RowSetReader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
-import org.apache.drill.exec.vector.NullableVarCharVector;
-import org.apache.drill.test.BaseTestQuery;
+import org.apache.drill.test.ClusterFixtureBuilder;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
 import org.apache.drill.test.TestBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -41,8 +48,10 @@ import static com.fasterxml.jackson.databind.SerializationFeature.INDENT_OUTPUT;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_CONNECT;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_DESCRIPTION;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_NAME;
+import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -52,25 +61,27 @@ import static org.junit.Assert.assertTrue;
  * -- USE schema
  */
 @Category(SqlTest.class)
-public class TestInfoSchema extends BaseTestQuery {
+public class TestInfoSchema extends ClusterTest {
   private static final String TEST_SUB_DIR = "testSubDir";
   private static final ObjectMapper mapper = new ObjectMapper().enable(INDENT_OUTPUT);
 
   @BeforeClass
-  public static void setupFiles() {
+  public static void setupFiles() throws Exception {
+    ClusterTest.startCluster(new ClusterFixtureBuilder(dirTestWatcher));
     dirTestWatcher.copyFileToRoot(Paths.get("sample-data"));
     dirTestWatcher.makeRootSubDir(Paths.get(TEST_SUB_DIR));
+
   }
 
   @Test
-  public void selectFromAllTables() throws Exception{
-    test("select * from INFORMATION_SCHEMA.SCHEMATA");
-    test("select * from INFORMATION_SCHEMA.CATALOGS");
-    test("select * from INFORMATION_SCHEMA.VIEWS");
-    test("select * from INFORMATION_SCHEMA.`TABLES`");
-    test("select * from INFORMATION_SCHEMA.COLUMNS");
-    test("select * from INFORMATION_SCHEMA.`FILES`");
-    test("select * from INFORMATION_SCHEMA.`PARTITIONS`");
+  public void selectFromAllTables() throws Exception {
+    runAndLog("select * from INFORMATION_SCHEMA.SCHEMATA");
+    runAndLog("select * from INFORMATION_SCHEMA.CATALOGS");
+    runAndLog("select * from INFORMATION_SCHEMA.VIEWS");
+    runAndLog("select * from INFORMATION_SCHEMA.`TABLES`");
+    runAndLog("select * from INFORMATION_SCHEMA.COLUMNS");
+    runAndLog("select * from INFORMATION_SCHEMA.`FILES`");
+    runAndLog("select * from INFORMATION_SCHEMA.`PARTITIONS`");
   }
 
   @Test
@@ -84,7 +95,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void showTablesFromDb() throws Exception{
+  public void showTablesFromDb() throws Exception {
     List<String[]> expected = Arrays.asList(
         new String[]{"information_schema", "VIEWS"},
         new String[]{"information_schema", "COLUMNS"},
@@ -114,7 +125,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void showTablesFromDbWhere() throws Exception{
+  public void showTablesFromDbWhere() throws Exception {
     testBuilder()
         .sqlQuery("SHOW TABLES FROM INFORMATION_SCHEMA WHERE TABLE_NAME='VIEWS'")
         .unOrdered()
@@ -124,7 +135,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void showTablesLike() throws Exception{
+  public void showTablesLike() throws Exception {
     testBuilder()
         .sqlQuery("SHOW TABLES LIKE '%CH%'")
         .unOrdered()
@@ -135,26 +146,37 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void showDatabases() throws Exception{
-    List<String> expected = Arrays.asList("dfs.default", "dfs.root", "dfs.tmp", "cp.default", "sys", "information_schema");
-
-    TestBuilder t1 = testBuilder()
-        .sqlQuery("SHOW DATABASES")
-        .unOrdered()
-        .baselineColumns("SCHEMA_NAME");
-    expected.forEach(t1::baselineValues);
-    t1.go();
+  public void showDatabases() throws Exception {
+    QueryBuilder builder = client.queryBuilder().sql("SHOW DATABASES");
+    DirectRowSet sets = builder.rowSet();
 
-    TestBuilder t2 = testBuilder()
-        .sqlQuery("SHOW SCHEMAS")
-        .unOrdered()
-        .baselineColumns("SCHEMA_NAME");
-    expected.forEach(t2::baselineValues);
-    t2.go();
+    try {
+      TupleMetadata schema = new SchemaBuilder()
+        .addNullable("SCHEMA_NAME", TypeProtos.MinorType.VARCHAR)
+        .build();
+
+      RowSet expected = new RowSetBuilder(client.allocator(), schema)
+        .addRow("dfs.default")
+        .addRow("dfs.root")
+        .addRow("dfs.tmp")
+        .addRow("cp.default")
+        .addRow("sys")
+        .addRow("information_schema")
+        .addRow("mock")
+        .build();
+
+      new RowSetComparison(expected).unorderedVerifyAndClearAll(sets);
+
+      builder = client.queryBuilder().sql("SHOW SCHEMAS");
+      sets = builder.rowSet();
+      new RowSetComparison(expected).unorderedVerifyAndClearAll(sets);
+    } finally {
+      sets.clear();
+    }
   }
 
   @Test
-  public void showDatabasesWhere() throws Exception{
+  public void showDatabasesWhere() throws Exception {
     testBuilder()
         .sqlQuery("SHOW DATABASES WHERE SCHEMA_NAME='dfs.tmp'")
         .unOrdered()
@@ -175,17 +197,41 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void showDatabasesLike() throws Exception{
+  public void showDatabasesLike() throws Exception {
     testBuilder()
         .sqlQuery("SHOW DATABASES LIKE '%y%'")
         .unOrdered()
         .baselineColumns("SCHEMA_NAME")
         .baselineValues("sys")
         .go();
+
+
+  }
+
+  @Test // DRILL-8281
+  public void likePatternWithEscapeChar() throws Exception {
+    StoragePluginRegistry pluginRegistry = cluster.drillbit().getContext().getStorage();
+    pluginRegistry.validatedPut(
+      "dfs_with_underscore",
+      pluginRegistry.getDefinedConfig("dfs")
+      );
+
+    try {
+      testBuilder()
+        .sqlQuery("SHOW DATABASES WHERE schema_name LIKE 'dfs^_with^_underscore.%' escape '^'")
+        .unOrdered()
+        .baselineColumns("SCHEMA_NAME")
+        .baselineValues("dfs_with_underscore.default")
+        .baselineValues("dfs_with_underscore.tmp")
+        .baselineValues("dfs_with_underscore.root")
+        .go();
+    } finally {
+      pluginRegistry.remove("dfs_with_underscore");
+    }
   }
 
   @Test
-  public void describeTable() throws Exception{
+  public void describeTable() throws Exception {
     testBuilder()
         .sqlQuery("DESCRIBE CATALOGS")
         .unOrdered()
@@ -199,7 +245,7 @@ public class TestInfoSchema extends BaseTestQuery {
 
   @Test
   public void describeTableWithTableKeyword() throws Exception {
-    test("USE INFORMATION_SCHEMA");
+    runAndLog("USE INFORMATION_SCHEMA");
     testBuilder()
         .sqlQuery("DESCRIBE TABLE CATALOGS")
         .unOrdered()
@@ -208,7 +254,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void describeTableWithSchema() throws Exception{
+  public void describeTableWithSchema() throws Exception {
     testBuilder()
         .sqlQuery("DESCRIBE INFORMATION_SCHEMA.`TABLES`")
         .unOrdered()
@@ -234,10 +280,10 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void describeWhenSameTableNameExistsInMultipleSchemas() throws Exception{
+  public void describeWhenSameTableNameExistsInMultipleSchemas() throws Exception {
     try {
-      test("USE dfs.tmp");
-      test("CREATE OR REPLACE VIEW `TABLES` AS SELECT full_name FROM cp.`employee.json`");
+      runAndLog("USE dfs.tmp");
+      runAndLog("CREATE OR REPLACE VIEW `TABLES` AS SELECT full_name FROM cp.`employee.json`");
 
       testBuilder()
           .sqlQuery("DESCRIBE `TABLES`")
@@ -261,15 +307,15 @@ public class TestInfoSchema extends BaseTestQuery {
           .baselineValues("LAST_MODIFIED_TIME", "TIMESTAMP", "NO")
           .go();
     } finally {
-      test("DROP VIEW IF EXISTS dfs.tmp.`TABLES`");
+      runAndLog("DROP VIEW IF EXISTS dfs.tmp.`TABLES`");
     }
   }
 
   @Test
   public void describeWhenSameTableNameExistsInMultipleSchemasWithTableKeyword() throws Exception {
     try {
-      test("USE dfs.tmp");
-      test("CREATE OR REPLACE VIEW `TABLES` AS SELECT full_name FROM cp.`employee.json`");
+      runAndLog("USE dfs.tmp");
+      runAndLog("CREATE OR REPLACE VIEW `TABLES` AS SELECT full_name FROM cp.`employee.json`");
 
       testBuilder()
           .sqlQuery("DESCRIBE TABLE `TABLES`")
@@ -283,12 +329,12 @@ public class TestInfoSchema extends BaseTestQuery {
           .sqlBaselineQuery("DESCRIBE INFORMATION_SCHEMA.`TABLES`")
           .go();
     } finally {
-      test("DROP VIEW IF EXISTS dfs.tmp.`TABLES`");
+      runAndLog("DROP VIEW IF EXISTS dfs.tmp.`TABLES`");
     }
   }
 
   @Test
-  public void describeTableWithColumnName() throws Exception{
+  public void describeTableWithColumnName() throws Exception {
     testBuilder()
         .sqlQuery("DESCRIBE `TABLES` TABLE_CATALOG")
         .unOrdered()
@@ -300,7 +346,7 @@ public class TestInfoSchema extends BaseTestQuery {
 
   @Test
   public void describeTableWithColumnNameAndTableKeyword() throws Exception {
-    test("USE INFORMATION_SCHEMA");
+    runAndLog("USE INFORMATION_SCHEMA");
     testBuilder()
         .sqlQuery("DESCRIBE TABLE `TABLES` TABLE_CATALOG")
         .unOrdered()
@@ -309,7 +355,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void describeTableWithSchemaAndColumnName() throws Exception{
+  public void describeTableWithSchemaAndColumnName() throws Exception {
     testBuilder()
         .sqlQuery("DESCRIBE INFORMATION_SCHEMA.`TABLES` TABLE_CATALOG")
         .unOrdered()
@@ -328,7 +374,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void describeTableWithColQualifier() throws Exception{
+  public void describeTableWithColQualifier() throws Exception {
     testBuilder()
         .sqlQuery("DESCRIBE COLUMNS 'TABLE%'")
         .unOrdered()
@@ -341,7 +387,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void describeTableWithSchemaAndColQualifier() throws Exception{
+  public void describeTableWithSchemaAndColQualifier() throws Exception {
     testBuilder()
         .sqlQuery("DESCRIBE INFORMATION_SCHEMA.SCHEMATA 'SCHEMA%'")
         .unOrdered()
@@ -352,7 +398,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void defaultSchemaDfs() throws Exception{
+  public void defaultSchemaDfs() throws Exception {
     testBuilder()
         .sqlQuery("SELECT R_REGIONKEY FROM `sample-data/region.parquet` LIMIT 1")
         .unOrdered()
@@ -363,7 +409,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void defaultSchemaClasspath() throws Exception{
+  public void defaultSchemaClasspath() throws Exception {
     testBuilder()
         .sqlQuery("SELECT full_name FROM `employee.json` LIMIT 1")
         .unOrdered()
@@ -375,7 +421,7 @@ public class TestInfoSchema extends BaseTestQuery {
 
 
   @Test
-  public void queryFromNonDefaultSchema() throws Exception{
+  public void queryFromNonDefaultSchema() throws Exception {
     testBuilder()
         .sqlQuery("SELECT full_name FROM cp.`employee.json` LIMIT 1")
         .unOrdered()
@@ -386,7 +432,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void useSchema() throws Exception{
+  public void useSchema() throws Exception {
     testBuilder()
         .sqlQuery("USE dfs.`default`")
         .unOrdered()
@@ -396,7 +442,7 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void useSubSchemaWithinSchema() throws Exception{
+  public void useSubSchemaWithinSchema() throws Exception {
     testBuilder()
         .sqlQuery("USE dfs")
         .unOrdered()
@@ -420,89 +466,93 @@ public class TestInfoSchema extends BaseTestQuery {
   }
 
   @Test
-  public void useSchemaNegative() throws Exception{
-    errorMsgTestHelper("USE invalid.schema",
-        "Schema [invalid.schema] is not valid with respect to either root schema or current default schema.");
+  public void useSchemaNegative() throws Exception {
+    try {
+      queryBuilder().sql("USE invalid.schema").run();
+    } catch (UserRemoteException e) {
+      assertThat(
+        e.getMessage(),
+        containsString("Schema [invalid.schema] is not valid with respect to either root schema or current default schema.")
+      );
+    }
   }
 
   // Tests using backticks around the complete schema path
   // select * from `dfs.tmp`.`/tmp/nation.parquet`;
   @Test
   public void completeSchemaRef1() throws Exception {
-    test("SELECT * FROM `cp.default`.`employee.json` limit 2");
+    runAndLog("SELECT * FROM `cp.default`.`employee.json` limit 2");
   }
 
   @Test
   public void describeSchemaSyntax() throws Exception {
-    test("describe schema dfs");
-    test("describe schema dfs.`default`");
-    test("describe database dfs.`default`");
+    runAndLog("describe schema dfs");
+    runAndLog("describe schema dfs.`default`");
+    runAndLog("describe database dfs.`default`");
   }
 
   @Test
   public void describePartialSchema() throws Exception {
-    test("use dfs");
-    test("describe schema tmp");
+    runAndLog("use dfs");
+    runAndLog("describe schema tmp");
   }
 
   @Test
   public void describeSchemaOutput() throws Exception {
-    List<QueryDataBatch> result = testSqlWithResults("describe schema dfs.tmp");
-    assertEquals(1, result.size());
-    QueryDataBatch batch = result.get(0);
-    RecordBatchLoader loader = new RecordBatchLoader(getDrillbitContext().getAllocator());
-    loader.load(batch.getHeader().getDef(), batch.getData());
-
-    // check schema column value
-    VectorWrapper<?> schemaValueVector = loader.getValueAccessorById(
-        NullableVarCharVector.class,
-        loader.getValueVectorId(SchemaPath.getCompoundPath("schema")).getFieldIds());
-    String schema = schemaValueVector.getValueVector().getAccessor().getObject(0).toString();
-    assertEquals("dfs.tmp", schema);
-
-    // check properties column value
-    VectorWrapper<?> propertiesValueVector = loader.getValueAccessorById(
-        NullableVarCharVector.class,
-        loader.getValueVectorId(SchemaPath.getCompoundPath("properties")).getFieldIds());
-    String properties = propertiesValueVector.getValueVector().getAccessor().getObject(0).toString();
-    Map<?, ?> configMap = mapper.readValue(properties, Map.class);
-
-    // check some stable properties existence
-    assertTrue(configMap.containsKey("connection"));
-    assertTrue(configMap.containsKey("formats"));
-    assertFalse(configMap.containsKey("workspaces"));
-
-    // check some stable properties values
-    assertEquals("file", configMap.get("type"));
-
-    FileSystemConfig testConfig = (FileSystemConfig) bits[0].getContext().getStorage().getPlugin("dfs").getConfig();
-    String tmpSchemaLocation = testConfig.getWorkspaces().get("tmp").getLocation();
-    assertEquals(tmpSchemaLocation, configMap.get("location"));
-
-    batch.release();
-    loader.clear();
+    QueryBuilder builder = queryBuilder().sql("describe schema dfs.tmp");
+    DirectRowSet rows = builder.rowSet();
+    try {
+      assertEquals(1, rows.rowCount());
+
+      RowSetReader reader = rows.reader();
+      assertTrue(reader.next());
+      String schema = (String) reader.column(0).reader().getObject();
+      assertEquals("dfs.tmp", schema);
+
+      String properties = (String) reader.column(1).reader().getObject();
+      Map<?, ?> configMap = mapper.readValue(properties, Map.class);
+      // check some stable properties existence
+      assertTrue(configMap.containsKey("connection"));
+      assertTrue(configMap.containsKey("formats"));
+      assertFalse(configMap.containsKey("workspaces"));
+
+      // check some stable properties values
+      assertEquals("file", configMap.get("type"));
+
+      DrillbitContext ctx = cluster.drillbit().getContext();
+      FileSystemConfig testConfig = (FileSystemConfig) ctx.getStorage().getPlugin("dfs").getConfig();
+      String tmpSchemaLocation = testConfig.getWorkspaces().get("tmp").getLocation();
+      assertEquals(tmpSchemaLocation, configMap.get("location"));
+    } finally {
+      rows.clear();
+    }
   }
 
   @Test
   public void describeSchemaInvalid() throws Exception {
-    errorMsgTestHelper("describe schema invalid.schema", "Invalid schema name [invalid.schema]");
+    try {
+      queryBuilder().sql("describe schema invalid.schema").run();
+    } catch (UserRemoteException e) {
+      assertThat(
+        e.getMessage(),
+        containsString("Invalid schema name [invalid.schema]")
+      );
+    }
   }
 
   @Test
   public void testDescribeAlias() throws Exception {
-    test("desc schema dfs.tmp");
-    test("desc information_schema.`catalogs`");
-    test("desc table information_schema.`catalogs`");
+    runAndLog("desc schema dfs.tmp");
+    runAndLog("desc information_schema.`catalogs`");
+    runAndLog("desc table information_schema.`catalogs`");
   }
 
   @Test
   public void testSerDe() throws Exception {
-    String sql = "select * from information_schema.`tables` where table_name = 'schemata' order by 1";
+    String sql = "select count() from information_schema.`tables` where table_name = 'SCHEMATA'";
+    String plan = queryBuilder().sql(sql).explainJson();
+    long cnt = queryBuilder().physical(plan).singletonLong();
 
-    testBuilder()
-      .sqlQuery(sql)
-      .unOrdered()
-      .physicalPlanBaseline(PlanTestBase.getPhysicalJsonPlan(sql))
-      .go();
+    assertEquals("Counts should match", 1, cnt);
   }
 }