You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2020/05/06 19:54:03 UTC

[hive] branch master updated: HIVE-19064: Add mode to support delimited identifiers enclosed within double quotation (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c3007d4  HIVE-19064: Add mode to support delimited identifiers enclosed within double quotation (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)
c3007d4 is described below

commit c3007d497ca58c595da8bd78455538a1ce0e3485
Author: Krisztian Kasa <kk...@cloudera.com>
AuthorDate: Wed May 6 12:53:14 2020 -0700

    HIVE-19064: Add mode to support delimited identifiers enclosed within double quotation (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |    18 +-
 .../test/resources/testconfiguration.properties    |     3 +
 .../apache/hadoop/hive/cli/control/CliConfigs.java |     1 -
 parser/pom.xml                                     |     1 +
 .../hive/ql/parse/ANTLRNoCaseStringStream.java     |    55 +
 .../hadoop/hive/ql/parse/GenericHiveLexer.java     |   106 +
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g    |   528 +-
 .../ql/parse/{HiveLexer.g => HiveLexerParent.g}    |    47 +-
 .../hadoop/hive/ql/parse/HiveLexerStandard.g       |    40 +
 .../org/apache/hadoop/hive/ql/parse/HiveParser.g   |    52 +-
 .../org/apache/hadoop/hive/ql/parse/Quotation.java |    71 +
 .../add/AlterViewAddPartitionAnalyzer.java         |     6 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java    |     7 +-
 .../hadoop/hive/ql/lockmgr/HiveLockObject.java     |     5 +-
 .../apache/hadoop/hive/ql/metadata/HiveUtils.java  |    16 +-
 .../ql/parse/ColumnStatsAutoGatherContext.java     |     8 +-
 .../hive/ql/parse/ColumnStatsSemanticAnalyzer.java |    69 +-
 .../apache/hadoop/hive/ql/parse/ParseDriver.java   |   132 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java     |    14 +-
 .../hadoop/hive/ql/stats/BasicStatsTask.java       |     6 +-
 .../hive/ql/udf/generic/GenericUDFCastFormat.java  |    12 +-
 .../hadoop/hive/ql/metadata/TestHiveUtils.java     |    97 +
 .../ql/parse/TestReplicationSemanticAnalyzer.java  |    43 +-
 .../ql/udf/generic/TestGenericUDFCastFormat.java   |    12 +-
 .../clientnegative/database_create_invalid_name.q  |     2 +-
 .../test/queries/clientpositive/quotedid_basic.q   |    12 +-
 .../clientpositive/quotedid_basic_standard.q       |    46 +
 .../special_character_in_tabnames_1.q              |    19 +-
 .../special_character_in_tabnames_quotes_1.q       |  1091 ++
 .../special_character_in_tabnames_quotes_2.q       |    24 +
 .../database_create_invalid_name.q.out             |     6 +-
 .../llap/special_character_in_tabnames_1.q.out     | 19456 +++++++++++++++++++
 .../special_character_in_tabnames_2.q.out          |    36 +-
 .../special_character_in_tabnames_quotes_1.q.out   | 19456 +++++++++++++++++++
 .../special_character_in_tabnames_quotes_2.q.out   |   153 +
 .../results/clientpositive/quotedid_basic.q.out    |    52 +-
 ...d_basic.q.out => quotedid_basic_standard.q.out} |   223 +-
 .../hadoop/hive/metastore/conf/MetastoreConf.java  |     4 +-
 .../hive/metastore/utils/MetaStoreUtils.java       |    18 +-
 .../hadoop/hive/metastore/HiveMetaStore.java       |     2 +-
 .../hadoop/hive/metastore/TestHiveMetaStore.java   |     2 +-
 .../hive/metastore/client/TestDatabases.java       |     2 +-
 .../client/TestTablesCreateDropAlterTruncate.java  |     4 +-
 43 files changed, 41002 insertions(+), 955 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 61db90c4..32b0d91 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -4057,10 +4057,16 @@ public class HiveConf extends Configuration {
         "The name of counter group for internal Hive variables (CREATED_FILE, FATAL_ERROR, etc.)"),
 
     HIVE_QUOTEDID_SUPPORT("hive.support.quoted.identifiers", "column",
-        new StringSet("none", "column"),
-        "Whether to use quoted identifier. 'none' or 'column' can be used. \n" +
-        "  none: default(past) behavior. Implies only alphaNumeric and underscore are valid characters in identifiers.\n" +
-        "  column: implies column names can contain any character."
+        new StringSet("none", "column", "standard"),
+        "Whether to use quoted identifier. 'none', 'column', and 'standard' can be used. \n" +
+        "  none: Quotation of identifiers and special characters in identifiers are not allowed but regular " +
+        "expressions in backticks are supported for column names.\n" +
+        "  column: Use the backtick character to quote identifiers having special characters. `col1` " +
+        "Use single quotes to quote string literals. 'value' " +
+        "Double quotes are also accepted but not recommended." +
+        "  standard: SQL standard way to quote identifiers. " +
+        "Use double quotes to quote identifiers having special characters \"col1\" " +
+        "and single quotes for string literals. 'value'"
     ),
     /**
      * @deprecated Use MetastoreConf.SUPPORT_SPECIAL_CHARACTERS_IN_TABLE_NAMES
@@ -4069,8 +4075,8 @@ public class HiveConf extends Configuration {
     HIVE_SUPPORT_SPECICAL_CHARACTERS_IN_TABLE_NAMES("hive.support.special.characters.tablename", true,
         "This flag should be set to true to enable support for special characters in table names.\n"
         + "When it is set to false, only [a-zA-Z_0-9]+ are supported.\n"
-        + "The only supported special character right now is '/'. This flag applies only to quoted table names.\n"
-        + "The default value is true."),
+        + "The supported special characters are %&'()*+,-./:;<=>?[]_|{}$^!~#@ and space. This flag applies only to"
+        + " quoted table names.\nThe default value is true."),
     HIVE_CREATE_TABLES_AS_INSERT_ONLY("hive.create.as.insert.only", false,
         "Whether the eligible tables should be created as ACID insert-only by default. Does \n" +
         "not apply to external tables, the ones using storage handlers, etc."),
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index e3a25e7..2036f29 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -799,6 +799,9 @@ minillaplocal.query.files=\
   smb_cache.q,\
   sort_acid.q,\
   special_character_in_tabnames_1.q,\
+  special_character_in_tabnames_2.q,\
+  special_character_in_tabnames_quotes_1.q,\
+  special_character_in_tabnames_quotes_2.q,\
   sqlmerge.q,\
   sqlmerge_stats.q,\
   stats_based_fetch_decision.q,\
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
index 1c0c62f..1ecd0d1 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
@@ -250,7 +250,6 @@ public class CliConfigs {
         includesFrom(testConfigProps, "minillaplocal.query.files");
         includesFrom(testConfigProps, "minillaplocal.shared.query.files");
         excludeQuery("bucket_map_join_tez1.q"); // Disabled in HIVE-19509
-        excludeQuery("special_character_in_tabnames_1.q"); // Disabled in HIVE-19509
         excludeQuery("tez_smb_1.q"); // Disabled in HIVE-19509
         excludeQuery("union_fast_stats.q"); // Disabled in HIVE-19509
         excludeQuery("schema_evol_orc_acidvec_part.q"); // Disabled in HIVE-19509
diff --git a/parser/pom.xml b/parser/pom.xml
index 18e0ad8..0edae27 100644
--- a/parser/pom.xml
+++ b/parser/pom.xml
@@ -77,6 +77,7 @@
           <sourceDirectory>${basedir}/src/java</sourceDirectory>
           <includes>
             <include>**/HiveLexer.g</include>
+            <include>**/HiveLexerStandard.g</include>
             <include>**/HiveParser.g</include>
             <include>**/HintParser.g</include>
           </includes>
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/ANTLRNoCaseStringStream.java b/parser/src/java/org/apache/hadoop/hive/ql/parse/ANTLRNoCaseStringStream.java
new file mode 100644
index 0000000..29d2010
--- /dev/null
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/ANTLRNoCaseStringStream.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse;
+
+import org.antlr.runtime.ANTLRStringStream;
+import org.antlr.runtime.CharStream;
+
+/**
+ * ANTLRNoCaseStringStream.
+ * This class provides and implementation for a case insensitive token checker
+ * for the lexical analysis part of antlr. By converting the token stream into
+ * upper case at the time when lexical rules are checked, this class ensures that the
+ * lexical rules need to just match the token with upper case letters as opposed to
+ * combination of upper case and lower case characteres. This is purely used for matching lexical
+ * rules. The actual token text is stored in the same way as the user input without
+ * actually converting it into an upper case. The token values are generated by the consume()
+ * function of the super class ANTLRStringStream. The LA() function is the lookahead funtion
+ * and is purely used for matching lexical rules. This also means that the grammar will only
+ * accept capitalized tokens in case it is run from other tools like antlrworks which
+ * do not have the ANTLRNoCaseStringStream implementation.
+ */
+public class ANTLRNoCaseStringStream extends ANTLRStringStream {
+
+  public ANTLRNoCaseStringStream(String input) {
+    super(input);
+  }
+
+  @Override
+  public int LA(int i) {
+
+    int returnChar = super.LA(i);
+    if (returnChar == CharStream.EOF) {
+      return returnChar;
+    } else if (returnChar == 0) {
+      return returnChar;
+    }
+
+    return Character.toUpperCase((char) returnChar);
+  }
+}
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/GenericHiveLexer.java b/parser/src/java/org/apache/hadoop/hive/ql/parse/GenericHiveLexer.java
new file mode 100644
index 0000000..b7551e0
--- /dev/null
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/GenericHiveLexer.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse;
+
+import java.util.ArrayList;
+import org.antlr.runtime.CharStream;
+import org.antlr.runtime.Lexer;
+import org.antlr.runtime.NoViableAltException;
+import org.antlr.runtime.RecognitionException;
+import org.antlr.runtime.RecognizerSharedState;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Common class of Legacy and SQL Standard Hive Lexer.
+ */
+public abstract class GenericHiveLexer extends Lexer {
+
+  public static GenericHiveLexer of(String statement, Configuration configuration) {
+    GenericHiveLexer lexer;
+    if (Quotation.from(configuration) == Quotation.STANDARD) {
+      lexer = new HiveLexerStandard(new ANTLRNoCaseStringStream(statement));
+    } else {
+      lexer = new HiveLexer(new ANTLRNoCaseStringStream(statement));
+    }
+
+    lexer.setHiveConf(configuration);
+    for (GenericHiveLexer wrappedLexers : lexer.getDelegates()) {
+      wrappedLexers.setHiveConf(configuration);
+    }
+
+    return lexer;
+  }
+
+  private final ArrayList<ParseError> errors;
+  private Configuration hiveConf;
+  private Quotation quotation;
+
+  public GenericHiveLexer() {
+    errors = new ArrayList<>();
+  }
+
+  public GenericHiveLexer(CharStream input) {
+    super(input);
+    errors = new ArrayList<>();
+  }
+
+  public GenericHiveLexer(CharStream input, RecognizerSharedState state) {
+    super(input, state);
+    errors = new ArrayList<>();
+  }
+
+  public void setHiveConf(Configuration hiveConf) {
+    this.hiveConf = hiveConf;
+  }
+
+  public abstract GenericHiveLexer[] getDelegates();
+
+  protected Quotation allowQuotedId() {
+    if (quotation == null) {
+      quotation = Quotation.from(hiveConf);
+    }
+    return quotation;
+  }
+
+  @Override
+  public void displayRecognitionError(String[] tokenNames, RecognitionException e) {
+    errors.add(new ParseError(this, e, tokenNames));
+  }
+
+  @Override
+  public String getErrorMessage(RecognitionException e, String[] tokenNames) {
+    String msg;
+
+    if (e instanceof NoViableAltException) {
+      //NoViableAltException nvae = (NoViableAltException) e;
+      // for development, can add
+      // "decision=<<"+nvae.grammarDecisionDescription+">>"
+      // and "(decision="+nvae.decisionNumber+") and
+      // "state "+nvae.stateNumber
+      msg = "character " + getCharErrorDisplay(e.c) + " not supported here";
+    } else {
+      msg = super.getErrorMessage(e, tokenNames);
+    }
+
+    return msg;
+  }
+
+  public ArrayList<ParseError> getErrors() {
+    return errors;
+  }
+}
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index 23f74ba..fff0fda 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -16,534 +16,24 @@
 */
 lexer grammar HiveLexer;
 
+options {
+superClass = GenericHiveLexer;
+}
+import HiveLexerParent;
+
 @lexer::header {
 package org.apache.hadoop.hive.ql.parse;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.commons.lang3.StringUtils;
 }
 
-@lexer::members {
-  private Configuration hiveConf;
-  
-  public void setHiveConf(Configuration hiveConf) {
-    this.hiveConf = hiveConf;
-  }
-  
-  protected boolean allowQuotedId() {
-    if(hiveConf == null){
-      return false;
-    }
-    String supportedQIds = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT);
-    return !"none".equals(supportedQIds);
-  }
-}
-
-// Keywords
-
-KW_TRUE : 'TRUE';
-KW_FALSE : 'FALSE';
-KW_UNKNOWN : 'UNKNOWN';
-KW_ALL : 'ALL';
-KW_SOME : 'SOME';
-KW_NONE: 'NONE';
-KW_AND : 'AND';
-KW_OR : 'OR';
-KW_NOT : 'NOT' | '!';
-KW_LIKE : 'LIKE';
-KW_ANY : 'ANY';
-
-KW_IF : 'IF';
-KW_EXISTS : 'EXISTS';
-
-KW_ASC : 'ASC';
-KW_DESC : 'DESC';
-KW_NULLS : 'NULLS';
-KW_LAST : 'LAST';
-KW_ORDER : 'ORDER';
-KW_GROUP : 'GROUP';
-KW_BY : 'BY';
-KW_HAVING : 'HAVING';
-KW_WHERE : 'WHERE';
-KW_FROM : 'FROM';
-KW_AS : 'AS';
-KW_SELECT : 'SELECT';
-KW_DISTINCT : 'DISTINCT';
-KW_INSERT : 'INSERT';
-KW_OVERWRITE : 'OVERWRITE';
-KW_OUTER : 'OUTER';
-KW_UNIQUEJOIN : 'UNIQUEJOIN';
-KW_PRESERVE : 'PRESERVE';
-KW_JOIN : 'JOIN';
-KW_LEFT : 'LEFT';
-KW_RIGHT : 'RIGHT';
-KW_FULL : 'FULL';
-KW_ON : 'ON';
-KW_PARTITION : 'PARTITION';
-KW_PARTITIONS : 'PARTITIONS';
-KW_TABLE: 'TABLE';
-KW_TABLES: 'TABLES';
-KW_COLUMNS: 'COLUMNS';
-KW_INDEX: 'INDEX';
-KW_INDEXES: 'INDEXES';
-KW_REBUILD: 'REBUILD';
-KW_FUNCTIONS: 'FUNCTIONS';
-KW_SHOW: 'SHOW';
-KW_MSCK: 'MSCK';
-KW_REPAIR: 'REPAIR';
-KW_DIRECTORY: 'DIRECTORY';
-KW_LOCAL: 'LOCAL';
-KW_TRANSFORM : 'TRANSFORM';
-KW_USING: 'USING';
-KW_CLUSTER: 'CLUSTER';
-KW_DISTRIBUTE: 'DISTRIBUTE';
-KW_SORT: 'SORT';
-KW_UNION: 'UNION';
-KW_EXCEPT: 'EXCEPT';
-KW_LOAD: 'LOAD';
-KW_EXPORT: 'EXPORT';
-KW_IMPORT: 'IMPORT';
-KW_REPLICATION: 'REPLICATION';
-KW_METADATA: 'METADATA';
-KW_DATA: 'DATA';
-KW_INPATH: 'INPATH';
-KW_IS: 'IS';
-KW_NULL: 'NULL';
-KW_CREATE: 'CREATE';
-KW_EXTERNAL: 'EXTERNAL';
-KW_ALTER: 'ALTER';
-KW_CHANGE: 'CHANGE';
-KW_COLUMN: 'COLUMN';
-KW_FIRST: 'FIRST';
-KW_AFTER: 'AFTER';
-KW_DESCRIBE: 'DESCRIBE';
-KW_DROP: 'DROP';
-KW_RENAME: 'RENAME';
-KW_TO: 'TO';
-KW_COMMENT: 'COMMENT';
-KW_BOOLEAN: 'BOOLEAN';
-KW_TINYINT: 'TINYINT';
-KW_SMALLINT: 'SMALLINT';
-KW_INT: 'INT' | 'INTEGER';
-KW_BIGINT: 'BIGINT';
-KW_FLOAT: 'FLOAT';
-KW_REAL: 'REAL';
-KW_DOUBLE: 'DOUBLE';
-KW_PRECISION: 'PRECISION';
-KW_DATE: 'DATE';
-KW_DATETIME: 'DATETIME';
-KW_TIMESTAMP: 'TIMESTAMP';
-KW_TIMESTAMPLOCALTZ: 'TIMESTAMPLOCALTZ';
-KW_TIME: 'TIME';
-KW_ZONE: 'ZONE';
-KW_INTERVAL: 'INTERVAL';
-KW_DECIMAL: 'DECIMAL' | 'DEC' | 'NUMERIC';
-KW_STRING: 'STRING';
-KW_CHAR: 'CHAR';
-KW_VARCHAR: 'VARCHAR';
-KW_ARRAY: 'ARRAY';
-KW_STRUCT: 'STRUCT';
-KW_MAP: 'MAP';
-KW_UNIONTYPE: 'UNIONTYPE';
-KW_REDUCE: 'REDUCE';
-KW_PARTITIONED: 'PARTITIONED';
-KW_CLUSTERED: 'CLUSTERED';
-KW_DISTRIBUTED: 'DISTRIBUTED';
-KW_SORTED: 'SORTED';
-KW_INTO: 'INTO';
-KW_BUCKETS: 'BUCKETS';
-KW_ROW: 'ROW';
-KW_ROWS: 'ROWS';
-KW_FORMAT: 'FORMAT';
-KW_DELIMITED: 'DELIMITED';
-KW_FIELDS: 'FIELDS';
-KW_TERMINATED: 'TERMINATED';
-KW_ESCAPED: 'ESCAPED';
-KW_COLLECTION: 'COLLECTION';
-KW_ITEMS: 'ITEMS';
-KW_KEYS: 'KEYS';
-KW_KEY_TYPE: '$KEY$';
-KW_KILL: 'KILL';
-KW_LINES: 'LINES';
-KW_STORED: 'STORED';
-KW_FILEFORMAT: 'FILEFORMAT';
-KW_INPUTFORMAT: 'INPUTFORMAT';
-KW_OUTPUTFORMAT: 'OUTPUTFORMAT';
-KW_INPUTDRIVER: 'INPUTDRIVER';
-KW_OUTPUTDRIVER: 'OUTPUTDRIVER';
-KW_ENABLE: 'ENABLE' | 'ENABLED';
-KW_DISABLE: 'DISABLE' | 'DISABLED';
-KW_EXECUTED: 'EXECUTED';
-KW_EXECUTE: 'EXECUTE';
-KW_LOCATION: 'LOCATION';
-KW_MANAGEDLOCATION: 'MANAGEDLOCATION';
-KW_TABLESAMPLE: 'TABLESAMPLE';
-KW_BUCKET: 'BUCKET';
-KW_OUT: 'OUT';
-KW_OF: 'OF';
-KW_PERCENT: 'PERCENT';
-KW_CAST: 'CAST';
-KW_ADD: 'ADD';
-KW_REPLACE: 'REPLACE';
-KW_RLIKE: 'RLIKE';
-KW_REGEXP: 'REGEXP';
-KW_TEMPORARY: 'TEMPORARY';
-KW_FUNCTION: 'FUNCTION';
-KW_MACRO: 'MACRO';
-KW_FILE: 'FILE';
-KW_JAR: 'JAR';
-KW_EXPLAIN: 'EXPLAIN';
-KW_EXTENDED: 'EXTENDED';
-KW_DEBUG: 'DEBUG';
-KW_FORMATTED: 'FORMATTED';
-KW_DEPENDENCY: 'DEPENDENCY';
-KW_LOGICAL: 'LOGICAL';
-KW_CBO: 'CBO';
-KW_SERDE: 'SERDE';
-KW_WITH: 'WITH';
-KW_DEFERRED: 'DEFERRED';
-KW_SERDEPROPERTIES: 'SERDEPROPERTIES';
-KW_DBPROPERTIES: 'DBPROPERTIES';
-KW_LIMIT: 'LIMIT';
-KW_OFFSET: 'OFFSET';
-KW_SET: 'SET';
-KW_UNSET: 'UNSET';
-KW_TBLPROPERTIES: 'TBLPROPERTIES';
-KW_IDXPROPERTIES: 'IDXPROPERTIES';
-KW_VALUE_TYPE: '$VALUE$';
-KW_ELEM_TYPE: '$ELEM$';
-KW_DEFINED: 'DEFINED';
-KW_CASE: 'CASE';
-KW_WHEN: 'WHEN';
-KW_THEN: 'THEN';
-KW_ELSE: 'ELSE';
-KW_END: 'END';
-KW_MAPJOIN: 'MAPJOIN';
-KW_STREAMTABLE: 'STREAMTABLE';
-KW_CLUSTERSTATUS: 'CLUSTERSTATUS';
-KW_UTC: 'UTC';
-KW_UTCTIMESTAMP: 'UTC_TMESTAMP';
-KW_LONG: 'LONG';
-KW_DELETE: 'DELETE';
-KW_PLUS: 'PLUS';
-KW_MINUS: 'MINUS';
-KW_FETCH: 'FETCH';
-KW_INTERSECT: 'INTERSECT';
-KW_VIEW: 'VIEW';
-KW_VIEWS: 'VIEWS';
-KW_IN: 'IN';
-KW_DATABASE: 'DATABASE';
-KW_DATABASES: 'DATABASES';
-KW_MATERIALIZED: 'MATERIALIZED';
-KW_SCHEMA: 'SCHEMA';
-KW_SCHEMAS: 'SCHEMAS';
-KW_GRANT: 'GRANT';
-KW_REVOKE: 'REVOKE';
-KW_SSL: 'SSL';
-KW_UNDO: 'UNDO';
-KW_LOCK: 'LOCK';
-KW_LOCKS: 'LOCKS';
-KW_UNLOCK: 'UNLOCK';
-KW_SHARED: 'SHARED';
-KW_EXCLUSIVE: 'EXCLUSIVE';
-KW_PROCEDURE: 'PROCEDURE';
-KW_UNSIGNED: 'UNSIGNED';
-KW_WHILE: 'WHILE';
-KW_READ: 'READ';
-KW_READS: 'READS';
-KW_PURGE: 'PURGE';
-KW_RANGE: 'RANGE';
-KW_ANALYZE: 'ANALYZE';
-KW_BEFORE: 'BEFORE';
-KW_BETWEEN: 'BETWEEN';
-KW_BOTH: 'BOTH';
-KW_BINARY: 'BINARY';
-KW_CROSS: 'CROSS';
-KW_CONTINUE: 'CONTINUE';
-KW_CURSOR: 'CURSOR';
-KW_TRIGGER: 'TRIGGER';
-KW_RECORDREADER: 'RECORDREADER';
-KW_RECORDWRITER: 'RECORDWRITER';
-KW_SEMI: 'SEMI';
-KW_LATERAL: 'LATERAL';
-KW_TOUCH: 'TOUCH';
-KW_ARCHIVE: 'ARCHIVE';
-KW_UNARCHIVE: 'UNARCHIVE';
-KW_COMPUTE: 'COMPUTE';
-KW_STATISTICS: 'STATISTICS';
-KW_USE: 'USE';
-KW_OPTION: 'OPTION';
-KW_CONCATENATE: 'CONCATENATE';
-KW_SHOW_DATABASE: 'SHOW_DATABASE';
-KW_UPDATE: 'UPDATE';
-KW_RESTRICT: 'RESTRICT';
-KW_CASCADE: 'CASCADE';
-KW_SKEWED: 'SKEWED';
-KW_ROLLUP: 'ROLLUP';
-KW_CUBE: 'CUBE';
-KW_DIRECTORIES: 'DIRECTORIES';
-KW_FOR: 'FOR';
-KW_WINDOW: 'WINDOW';
-KW_UNBOUNDED: 'UNBOUNDED';
-KW_PRECEDING: 'PRECEDING';
-KW_FOLLOWING: 'FOLLOWING';
-KW_CURRENT: 'CURRENT';
-KW_CURRENT_DATE: 'CURRENT_DATE';
-KW_CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP';
-KW_LESS: 'LESS';
-KW_MORE: 'MORE';
-KW_OVER: 'OVER';
-KW_GROUPING: 'GROUPING';
-KW_SETS: 'SETS';
-KW_TRUNCATE: 'TRUNCATE';
-KW_NOSCAN: 'NOSCAN';
-KW_USER: 'USER';
-KW_ROLE: 'ROLE';
-KW_ROLES: 'ROLES';
-KW_INNER: 'INNER';
-KW_EXCHANGE: 'EXCHANGE';
-KW_URI: 'URI';
-KW_SERVER : 'SERVER';
-KW_ADMIN: 'ADMIN';
-KW_OWNER: 'OWNER';
-KW_PRINCIPALS: 'PRINCIPALS';
-KW_COMPACT: 'COMPACT';
-KW_COMPACTIONS: 'COMPACTIONS';
-KW_TRANSACTIONS: 'TRANSACTIONS';
-KW_TRANSACTIONAL: 'TRANSACTIONAL';
-KW_REWRITE : 'REWRITE';
-KW_AUTHORIZATION: 'AUTHORIZATION';
-KW_REOPTIMIZATION: 'REOPTIMIZATION';
-KW_CONF: 'CONF';
-KW_VALUES: 'VALUES';
-KW_RELOAD: 'RELOAD';
-KW_YEAR: 'YEAR' | 'YEARS';
-KW_QUERY: 'QUERY';
-KW_QUARTER: 'QUARTER';
-KW_MONTH: 'MONTH' | 'MONTHS';
-KW_WEEK: 'WEEK' | 'WEEKS';
-KW_DAY: 'DAY' | 'DAYS';
-KW_DOW: 'DAYOFWEEK';
-KW_HOUR: 'HOUR' | 'HOURS';
-KW_MINUTE: 'MINUTE' | 'MINUTES';
-KW_SECOND: 'SECOND' | 'SECONDS';
-KW_START: 'START';
-KW_TRANSACTION: 'TRANSACTION';
-KW_COMMIT: 'COMMIT';
-KW_ROLLBACK: 'ROLLBACK';
-KW_WORK: 'WORK';
-KW_ONLY: 'ONLY';
-KW_WRITE: 'WRITE';
-KW_ISOLATION: 'ISOLATION';
-KW_LEVEL: 'LEVEL';
-KW_SNAPSHOT: 'SNAPSHOT';
-KW_AUTOCOMMIT: 'AUTOCOMMIT';
-KW_CACHE: 'CACHE';
-KW_PRIMARY: 'PRIMARY';
-KW_FOREIGN: 'FOREIGN';
-KW_REFERENCES: 'REFERENCES';
-KW_CONSTRAINT: 'CONSTRAINT';
-KW_FORCE: 'FORCE';
-KW_ENFORCED: 'ENFORCED';
-KW_VALIDATE: 'VALIDATE';
-KW_NOVALIDATE: 'NOVALIDATE';
-KW_RELY: 'RELY';
-KW_NORELY: 'NORELY';
-KW_UNIQUE: 'UNIQUE';
-KW_KEY: 'KEY';
-KW_ABORT: 'ABORT';
-KW_EXTRACT: 'EXTRACT';
-KW_FLOOR: 'FLOOR';
-KW_MERGE: 'MERGE';
-KW_MATCHED: 'MATCHED';
-KW_REPL: 'REPL';
-KW_DUMP: 'DUMP';
-KW_STATUS: 'STATUS';
-KW_VECTORIZATION: 'VECTORIZATION';
-KW_SUMMARY: 'SUMMARY';
-KW_OPERATOR: 'OPERATOR';
-KW_EXPRESSION: 'EXPRESSION';
-KW_DETAIL: 'DETAIL';
-KW_WAIT: 'WAIT';
-KW_RESOURCE: 'RESOURCE';
-KW_PLAN: 'PLAN';
-KW_QUERY_PARALLELISM: 'QUERY_PARALLELISM';
-KW_PLANS: 'PLANS';
-KW_ACTIVATE: 'ACTIVATE';
-KW_DEFAULT: 'DEFAULT';
-KW_CHECK: 'CHECK';
-KW_POOL: 'POOL';
-KW_MOVE: 'MOVE';
-KW_DO: 'DO';
-KW_ALLOC_FRACTION: 'ALLOC_FRACTION';
-KW_SCHEDULING_POLICY: 'SCHEDULING_POLICY';
-KW_SCHEDULED: 'SCHEDULED';
-KW_EVERY: 'EVERY';
-KW_AT: 'AT';
-KW_CRON: 'CRON';
-KW_PATH: 'PATH';
-KW_MAPPING: 'MAPPING';
-KW_WORKLOAD: 'WORKLOAD';
-KW_MANAGEMENT: 'MANAGEMENT';
-KW_ACTIVE: 'ACTIVE';
-KW_UNMANAGED: 'UNMANAGED';
-KW_APPLICATION: 'APPLICATION';
-KW_SYNC: 'SYNC';
-KW_AST: 'AST';
-KW_COST: 'COST';
-KW_JOINCOST: 'JOINCOST';
-KW_WITHIN: 'WITHIN';
-
-// Operators
-// NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.
-
-DOT : '.'; // generated as a part of Number rule
-COLON : ':' ;
-COMMA : ',' ;
-SEMICOLON : ';' ;
-
-LPAREN : '(' ;
-RPAREN : ')' ;
-LSQUARE : '[' ;
-RSQUARE : ']' ;
-LCURLY : '{';
-RCURLY : '}';
-
-EQUAL : '=' | '==';
-EQUAL_NS : '<=>';
-NOTEQUAL : '<>' | '!=';
-LESSTHANOREQUALTO : '<=';
-LESSTHAN : '<';
-GREATERTHANOREQUALTO : '>=';
-GREATERTHAN : '>';
-
-DIVIDE : '/';
-PLUS : '+';
-MINUS : '-';
-STAR : '*';
-MOD : '%';
-DIV : 'DIV';
-
-AMPERSAND : '&';
-TILDE : '~';
-BITWISEOR : '|';
-CONCATENATE : '||';
-BITWISEXOR : '^';
-QUESTION : '?';
-DOLLAR : '$';
-
-// LITERALS
-fragment
-Letter
-    : 'a'..'z' | 'A'..'Z'
-    ;
-
-fragment
-HexDigit
-    : 'a'..'f' | 'A'..'F'
-    ;
-
-fragment
-Digit
-    :
-    '0'..'9'
-    ;
-
-fragment
-Exponent
-    :
-    ('e' | 'E') ( PLUS|MINUS )? (Digit)+
-    ;
-
-fragment
-RegexComponent
-    : 'a'..'z' | 'A'..'Z' | '0'..'9' | '_'
-    | PLUS | STAR | QUESTION | MINUS | DOT
-    | LPAREN | RPAREN | LSQUARE | RSQUARE | LCURLY | RCURLY
-    | BITWISEXOR | BITWISEOR | DOLLAR | '!'
-    ;
-
 StringLiteral
     :
-    ( '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
-    | '\"' ( ~('\"'|'\\') | ('\\' .) )* '\"'
-    )+
-    ;
-
-CharSetLiteral
-    :
-    StringLiteral
-    | '0' 'X' (HexDigit|Digit)+
+    ( '\"' ( ~('\"'|'\\') | ('\\' .) )* '\"' | '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' )+
     ;
 
-IntegralLiteral
-    :
-    (Digit)+ ('L' | 'S' | 'Y')
-    ;
-
-NumberLiteral
-    :
-    Number ('D' | 'B' 'D')
-    ;
-
-ByteLengthLiteral
-    :
-    (Digit)+ ('b' | 'B' | 'k' | 'K' | 'm' | 'M' | 'g' | 'G')
-    ;
-
-Number
-    :
-    (Digit)+ ( DOT (Digit)* (Exponent)? | Exponent)?
-    ;
-
-/*
-An Identifier can be:
-- tableName
-- columnName
-- select expr alias
-- lateral view aliases
-- database name
-- view name
-- subquery alias
-- function name
-- ptf argument identifier
-- index name
-- property name for: db,tbl,partition...
-- fileFormat
-- role name
-- privilege name
-- principal name
-- macro name
-- hint name
-- window name
-*/    
-Identifier
-    :
-    (Letter | Digit) (Letter | Digit | '_')*
-    | {allowQuotedId()}? QuotedIdentifier  /* though at the language level we allow all Identifiers to be QuotedIdentifiers; 
-                                              at the API level only columns are allowed to be of this form */
-    | '`' RegexComponent+ '`'
-    ;
-
-fragment    
-QuotedIdentifier 
-    :
-    '`'  ( '``' | ~('`') )* '`' { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "``", "`")); }
-    ;
-
-CharSetName
+fragment
+QuotedIdentifier
     :
-    '_' (Letter | Digit | '_' | '-' | '.' | ':' )+
-    ;
-
-WS  :  (' '|'\r'|'\t'|'\n') {$channel=HIDDEN;}
-    ;
-
-LINE_COMMENT
-    : '--' (~('\n'|'\r'))* { $channel=HIDDEN; }
-    ;
-
-QUERY_HINT
-    : '/*' (options { greedy=false; } : QUERY_HINT|.)* '*/' { if(getText().charAt(2) != '+') { $channel=HIDDEN; } else { setText(getText().substring(3, getText().length() - 2)); } }
+    ('`'  ( '``' | ~('`') )* '`') { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "``", "`")); }
     ;
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g
similarity index 90%
copy from parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
copy to parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g
index 23f74ba..5bd831d 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g
@@ -1,9 +1,9 @@
 /**
-   Licensed to the Apache Software Foundation (ASF) under one or more 
-   contributor license agreements.  See the NOTICE file distributed with 
+   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 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
@@ -14,31 +14,7 @@
    See the License for the specific language governing permissions and
    limitations under the License.
 */
-lexer grammar HiveLexer;
-
-@lexer::header {
-package org.apache.hadoop.hive.ql.parse;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.commons.lang3.StringUtils;
-}
-
-@lexer::members {
-  private Configuration hiveConf;
-  
-  public void setHiveConf(Configuration hiveConf) {
-    this.hiveConf = hiveConf;
-  }
-  
-  protected boolean allowQuotedId() {
-    if(hiveConf == null){
-      return false;
-    }
-    String supportedQIds = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT);
-    return !"none".equals(supportedQIds);
-  }
-}
+lexer grammar HiveLexerParent;
 
 // Keywords
 
@@ -466,9 +442,7 @@ RegexComponent
 
 StringLiteral
     :
-    ( '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
-    | '\"' ( ~('\"'|'\\') | ('\\' .) )* '\"'
-    )+
+    // This is implemented by subclass.
     ;
 
 CharSetLiteral
@@ -517,19 +491,18 @@ An Identifier can be:
 - macro name
 - hint name
 - window name
-*/    
+*/
 Identifier
     :
     (Letter | Digit) (Letter | Digit | '_')*
-    | {allowQuotedId()}? QuotedIdentifier  /* though at the language level we allow all Identifiers to be QuotedIdentifiers; 
-                                              at the API level only columns are allowed to be of this form */
+    | {allowQuotedId() != Quotation.NONE}? QuotedIdentifier
     | '`' RegexComponent+ '`'
     ;
 
-fragment    
-QuotedIdentifier 
+fragment
+QuotedIdentifier
     :
-    '`'  ( '``' | ~('`') )* '`' { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "``", "`")); }
+    // This is implemented by subclass.
     ;
 
 CharSetName
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerStandard.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerStandard.g
new file mode 100644
index 0000000..466b200
--- /dev/null
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerStandard.g
@@ -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.
+*/
+lexer grammar HiveLexerStandard;
+
+options {
+superClass = GenericHiveLexer;
+}
+import HiveLexerParent;
+
+@lexer::header {
+package org.apache.hadoop.hive.ql.parse;
+
+import org.apache.commons.lang3.StringUtils;
+}
+
+StringLiteral
+    :
+    ( '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' )+
+    ;
+
+fragment
+QuotedIdentifier
+    :
+    ('"'  ( '""' | ~('"') )* '"') { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "\"\"", "\"")); }
+    | ('`'  ( '``' | ~('`') )* '`') { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "``", "`")); }
+    ;
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index b03b098..29986da 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -1,9 +1,9 @@
 /**
-   Licensed to the Apache Software Foundation (ASF) under one or more 
-   contributor license agreements.  See the NOTICE file distributed with 
+   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 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
@@ -18,7 +18,7 @@ parser grammar HiveParser;
 
 options
 {
-tokenVocab=HiveLexer;
+tokenVocab=HiveLexerParent;
 output=AST;
 ASTLabelType=ASTNode;
 backtrack=false;
@@ -724,7 +724,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 
     return header;
   }
-  
+
   @Override
   public String getErrorMessage(RecognitionException e, String[] tokenNames) {
     String msg = null;
@@ -761,7 +761,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
     }
     return msg;
   }
-  
+
   public void pushMsg(String msg, RecognizerSharedState state) {
     // ANTLR generated code does not wrap the @init code wit this backtracking check,
     //  even if the matching @after has it. If we have parser rules with that are doing
@@ -1491,7 +1491,7 @@ alterTblPartitionStatementSuffixSkewedLocation
   : KW_SET KW_SKEWED KW_LOCATION skewedLocations
   -> ^(TOK_ALTERTABLE_SKEWED_LOCATION skewedLocations)
   ;
-  
+
 skewedLocations
 @init { pushMsg("skewed locations", state); }
 @after { popMsg(state); }
@@ -1521,7 +1521,7 @@ alterStatementSuffixLocation[boolean partition]
   ->              ^(TOK_ALTERTABLE_LOCATION $newLoc)
   ;
 
-	
+
 alterStatementSuffixSkewedby
 @init {pushMsg("alter skewed by statement", state);}
 @after{popMsg(state);}
@@ -1613,10 +1613,10 @@ tabTypeExpr
    (identifier (DOT^
    (
    (KW_ELEM_TYPE) => KW_ELEM_TYPE
-   | 
+   |
    (KW_KEY_TYPE) => KW_KEY_TYPE
-   | 
-   (KW_VALUE_TYPE) => KW_VALUE_TYPE 
+   |
+   (KW_VALUE_TYPE) => KW_VALUE_TYPE
    | identifier
    ))*
    )?
@@ -1674,7 +1674,7 @@ showStatement
     | KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tableName ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)?
     -> ^(TOK_SHOWCOLUMNS tableName (TOK_FROM $db_name)? showStmtIdentifier?)
     | KW_SHOW KW_FUNCTIONS (KW_LIKE showFunctionIdentifier)?  -> ^(TOK_SHOWFUNCTIONS KW_LIKE? showFunctionIdentifier?)
-    | KW_SHOW KW_PARTITIONS tabName=tableName partitionSpec? -> ^(TOK_SHOWPARTITIONS $tabName partitionSpec?) 
+    | KW_SHOW KW_PARTITIONS tabName=tableName partitionSpec? -> ^(TOK_SHOWPARTITIONS $tabName partitionSpec?)
     | KW_SHOW KW_CREATE (
         (KW_DATABASE|KW_SCHEMA) => (KW_DATABASE|KW_SCHEMA) db_name=identifier -> ^(TOK_SHOW_CREATEDATABASE $db_name)
         |
@@ -1683,7 +1683,7 @@ showStatement
     | KW_SHOW KW_TABLE KW_EXTENDED ((KW_FROM|KW_IN) db_name=identifier)? KW_LIKE showStmtIdentifier partitionSpec?
     -> ^(TOK_SHOW_TABLESTATUS showStmtIdentifier $db_name? partitionSpec?)
     | KW_SHOW KW_TBLPROPERTIES tableName (LPAREN prptyName=StringLiteral RPAREN)? -> ^(TOK_SHOW_TBLPROPERTIES tableName $prptyName?)
-    | KW_SHOW KW_LOCKS 
+    | KW_SHOW KW_LOCKS
       (
       (KW_DATABASE|KW_SCHEMA) => (KW_DATABASE|KW_SCHEMA) (dbName=identifier) (isExtended=KW_EXTENDED)? -> ^(TOK_SHOWDBLOCKS $dbName $isExtended?)
       |
@@ -1808,7 +1808,7 @@ showCurrentRole
 setRole
 @init {pushMsg("set role", state);}
 @after {popMsg(state);}
-    : KW_SET KW_ROLE 
+    : KW_SET KW_ROLE
     (
     (KW_ALL) => (all=KW_ALL) -> ^(TOK_SET_ROLE Identifier[$all.text])
     |
@@ -2116,7 +2116,7 @@ createScheduledQueryStatement
             definedAsSpec
         )
     ;
-    
+
 dropScheduledQueryStatement
 @init { pushMsg("drop scheduled query statement", state); }
 @after { popMsg(state); }
@@ -2126,7 +2126,7 @@ dropScheduledQueryStatement
         )
     ;
 
-    
+
 alterScheduledQueryStatement
 @init { pushMsg("alter scheduled query statement", state); }
 @after { popMsg(state); }
@@ -2137,7 +2137,7 @@ alterScheduledQueryStatement
             $mod
         )
     ;
-    
+
 alterScheduledQueryChange
 @init { pushMsg("alter scheduled query change", state); }
 @after { popMsg(state); }
@@ -2153,7 +2153,7 @@ scheduleSpec
 @after { popMsg(state); }
         : KW_CRON cronString=StringLiteral -> ^(TOK_CRON $cronString)
         | KW_EVERY value=Number? qualifier=intervalQualifiers
-        ((KW_AT|KW_OFFSET KW_BY) offsetTs=StringLiteral)? -> ^(TOK_SCHEDULE ^(TOK_EVERY $value?) $qualifier $offsetTs?) 
+        ((KW_AT|KW_OFFSET KW_BY) offsetTs=StringLiteral)? -> ^(TOK_SCHEDULE ^(TOK_EVERY $value?) $qualifier $offsetTs?)
         ;
 
 executedAsSpec
@@ -2509,7 +2509,7 @@ alterForeignKeyWithName
 skewedValueElement
 @init { pushMsg("skewed value element", state); }
 @after { popMsg(state); }
-    : 
+    :
       skewedColumnValues
      | skewedColumnValuePairList
     ;
@@ -2523,8 +2523,8 @@ skewedColumnValuePairList
 skewedColumnValuePair
 @init { pushMsg("column value pair", state); }
 @after { popMsg(state); }
-    : 
-      LPAREN colValues=skewedColumnValues RPAREN 
+    :
+      LPAREN colValues=skewedColumnValues RPAREN
       -> ^(TOK_TABCOLVALUES $colValues)
     ;
 
@@ -2544,7 +2544,7 @@ skewedColumnValue
 skewedValueLocationElement
 @init { pushMsg("skewed value location element", state); }
 @after { popMsg(state); }
-    : 
+    :
       skewedColumnValue
      | skewedColumnValuePair
     ;
@@ -2865,7 +2865,7 @@ fromStatement
 	            {adaptor.create(Identifier, generateUnionAlias())}
 	           )
 	        )
-	       ^(TOK_INSERT 
+            ^(TOK_INSERT
 	          ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
 	          ^(TOK_SELECT ^(TOK_SELEXPR TOK_SETCOLREF))
 	        )
@@ -3095,8 +3095,8 @@ setColumnsClause
    KW_SET columnAssignmentClause (COMMA columnAssignmentClause)* -> ^(TOK_SET_COLUMNS_CLAUSE columnAssignmentClause* )
    ;
 
-/* 
-  UPDATE <table> 
+/*
+  UPDATE <table>
   SET col1 = val1, col2 = val2... WHERE ...
 */
 updateStatement
@@ -3229,4 +3229,4 @@ killQueryStatement
 @after { popMsg(state); }
   :
   KW_KILL KW_QUERY ( StringLiteral )+ -> ^(TOK_KILL_QUERY ( StringLiteral )+)
-  ;
+  ;
\ No newline at end of file
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/Quotation.java b/parser/src/java/org/apache/hadoop/hive/ql/parse/Quotation.java
new file mode 100644
index 0000000..da94ed9
--- /dev/null
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/Quotation.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * Identifier quotation.
+ * @see HiveConf.ConfVars#HIVE_QUOTEDID_SUPPORT
+ */
+public enum Quotation {
+  /**
+   * Quotation of identifiers and special characters in identifiers are not allowed.
+   * But regular expressions in backticks are supported for column names.
+   */
+  NONE("none"),
+  /**
+   * Use the backtick character to quote identifiers having special characters.
+   * Use single quotes to quote string literals. Double quotes are also accepted but not recommended.
+   */
+  BACKTICKS("column"),
+  /**
+   * SQL standard way to quote identifiers.
+   * Use double quotes to quote identifiers having special characters and single quotes for string literals.
+   */
+  STANDARD("standard");
+
+  Quotation(String stringValue) {
+    this.stringValue = stringValue;
+  }
+
+  private final String stringValue;
+
+  public String stringValue() {
+    return stringValue;
+  }
+
+  public static Quotation from(Configuration configuration) {
+    String supportedQIds;
+    if (configuration == null) {
+      supportedQIds = HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT.defaultStrVal;
+    } else {
+      supportedQIds = HiveConf.getVar(configuration, HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT);
+    }
+
+    for (Quotation quotation : values()) {
+      if (quotation.stringValue.equalsIgnoreCase(supportedQIds)) {
+        return quotation;
+      }
+    }
+
+    throw new EnumConstantNotPresentException(Quotation.class,
+        "Option not recognized for " + HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT.varname + "value: " + supportedQIds);
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/add/AlterViewAddPartitionAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/add/AlterViewAddPartitionAnalyzer.java
index c1d2887..a412fa5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/add/AlterViewAddPartitionAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/add/AlterViewAddPartitionAnalyzer.java
@@ -60,8 +60,8 @@ public class AlterViewAddPartitionAnalyzer extends AbstractAddPartitionAnalyzer
   protected void postProcess(TableName tableName, Table table, AlterTableAddPartitionDesc desc, Task<DDLWork> ddlTask)
       throws SemanticException {
     // Compile internal query to capture underlying table partition dependencies
-    String dbTable = HiveUtils.unparseIdentifier(tableName.getDb()) + "." +
-        HiveUtils.unparseIdentifier(tableName.getTable());
+    String dbTable = HiveUtils.unparseIdentifier(tableName.getDb(), conf) + "." +
+        HiveUtils.unparseIdentifier(tableName.getTable(), conf);
 
     StringBuilder where = new StringBuilder();
     boolean firstOr = true;
@@ -79,7 +79,7 @@ public class AlterViewAddPartitionAnalyzer extends AbstractAddPartitionAnalyzer
         } else {
           where.append(" AND ");
         }
-        where.append(HiveUtils.unparseIdentifier(entry.getKey()));
+        where.append(HiveUtils.unparseIdentifier(entry.getKey(), conf));
         where.append(" = '");
         where.append(HiveUtils.escapeString(entry.getValue()));
         where.append("'");
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
index 7820013..0383881 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.lockmgr;
 
+import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
 import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
@@ -405,9 +406,9 @@ class DummyTxnManager extends HiveTxnManagerImpl {
         try {
           locks.add(new HiveLockObj(
                       new HiveLockObject(new DummyPartition(p.getTable(), p.getTable().getDbName()
-                                                            + "/" + org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.encodeTableName(p.getTable().getTableName())
-                                                            + "/" + partialName,
-                                                              partialSpec), lockData), mode));
+                          + "/" + FileUtils.escapePathName(p.getTable().getTableName()).toLowerCase()
+                          + "/" + partialName,
+                          partialSpec), lockData), mode));
           partialName += "/";
         } catch (HiveException e) {
           throw new LockException(e.getMessage());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
index 08aeeb2..55ecbb4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
@@ -22,6 +22,7 @@ import java.util.Map;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StringInternUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.metadata.DummyPartition;
@@ -197,12 +198,12 @@ public class HiveLockObject {
   }
 
   public HiveLockObject(Table tbl, HiveLockObjectData lockData) {
-    this(new String[] {tbl.getDbName(), org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.encodeTableName(tbl.getTableName())}, lockData);
+    this(new String[] {tbl.getDbName(), FileUtils.escapePathName(tbl.getTableName()).toLowerCase()}, lockData);
   }
 
   public HiveLockObject(Partition par, HiveLockObjectData lockData) {
     this(new String[] {par.getTable().getDbName(),
-                       org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.encodeTableName(par.getTable().getTableName()), par.getName()}, lockData);
+        FileUtils.escapePathName(par.getTable().getTableName()).toLowerCase(), par.getName()}, lockData);
   }
 
   public HiveLockObject(DummyPartition par, HiveLockObjectData lockData) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
index 26c7a60..cf756bc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.parse.Quotation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -274,19 +275,12 @@ public final class HiveUtils {
   /**
    * Regenerate an identifier as part of unparsing it back to SQL text.
    */
-  public static String unparseIdentifier(String identifier) {
-    return unparseIdentifier(identifier, null);
-  }
-
   public static String unparseIdentifier(String identifier, Configuration conf) {
-    // In the future, if we support arbitrary characters in
-    // identifiers, then we'll need to escape any backticks
+    // We support arbitrary characters in
+    // identifiers, then we need to escape any backticks
     // in identifier by doubling them up.
-
-    // the time has come
-    String qIdSupport = conf == null ? null :
-      HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT);
-    if ( qIdSupport != null && !"none".equals(qIdSupport) ) {
+    Quotation quotation = Quotation.from(conf);
+    if (quotation != Quotation.NONE) {
       identifier = identifier.replaceAll("`", "``");
     }
     return "`" + identifier + "`";
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
index 9bcc472..997132f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import static org.apache.hadoop.hive.ql.metadata.HiveUtils.unparseIdentifier;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -103,9 +105,9 @@ public class ColumnStatsAutoGatherContext {
    * However, we do not need to specify the partition-spec because (1) the data is going to be inserted to that specific partition
    * (2) we can compose the static/dynamic partition using a select operator in replaceSelectOperatorProcess.
    */
-  public void insertAnalyzePipeline() throws SemanticException{
-    String analyzeCommand = "analyze table `" + tbl.getDbName() + "`.`" + tbl.getTableName() + "`"
-        + " compute statistics for columns ";
+  public void insertAnalyzePipeline() throws SemanticException {
+    String analyzeCommand = "analyze table " + unparseIdentifier(tbl.getDbName(), conf)
+        + "." + unparseIdentifier(tbl.getTableName(), conf) + " compute statistics for columns ";
     insertAnalyzePipeline(analyzeCommand, false);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
index 35f7ec6..b8231d2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import static org.apache.hadoop.hive.ql.metadata.HiveUtils.unparseIdentifier;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -34,6 +36,7 @@ import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.QueryState;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -73,6 +76,17 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     super(queryState);
   }
 
+  public static String getQuote(HiveConf conf) {
+    String qIdSupport = conf.getVar(ConfVars.HIVE_QUOTEDID_SUPPORT);
+    if ("column".equals(qIdSupport)) {
+      return "`";
+    } else if ("standard".equals(qIdSupport)) {
+      return "\"";
+    } else {
+      return "";
+    }
+  }
+
   private boolean shouldRewrite(ASTNode tree) {
     boolean rwt = false;
     if (tree.getChildCount() > 1) {
@@ -148,23 +162,23 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     }
   }
 
-  private static StringBuilder genPartitionClause(Table tbl, Map<String, String> partSpec)
+  private static StringBuilder genPartitionClause(Table tbl, Map<String, String> partSpec, HiveConf conf)
       throws SemanticException {
     StringBuilder whereClause = new StringBuilder(" where ");
     boolean predPresent = false;
     StringBuilder groupByClause = new StringBuilder(" group by ");
     boolean aggPresent = false;
 
-    for (String partKey : partSpec.keySet()) {
-      String value;
-      if ((value = partSpec.get(partKey)) != null) {
+    for (Map.Entry<String, String> part : partSpec.entrySet()) {
+      String value = part.getValue();
+      if (value != null) {
         if (!predPresent) {
           predPresent = true;
         } else {
           whereClause.append(" and ");
         }
-        whereClause.append("`").append(partKey).append("` = ")
-            .append(genPartValueString(getColTypeOf(tbl, partKey), value));
+        whereClause.append(unparseIdentifier(part.getKey(), conf)).append(" = ")
+            .append(genPartValueString(getColTypeOf(tbl, part.getKey()), value));
       }
     }
 
@@ -174,7 +188,7 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
       } else {
         groupByClause.append(',');
       }
-      groupByClause.append("`" + fs.getName() + "`");
+      groupByClause.append(unparseIdentifier(fs.getName(), conf));
     }
 
     // attach the predicate and group by to the return clause
@@ -215,15 +229,11 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     return colTypes;
   }
 
-  private static String escapeBackTicks(String colName) {
-    return colName.replaceAll("`", "``");
-  }
-
   private String genRewrittenQuery(List<String> colNames, HiveConf conf, Map<String, String> partSpec,
-      boolean isPartitionStats, boolean useTableValues) throws SemanticException {
-    String rewrittenQuery = genRewrittenQuery(tbl, colNames, conf, partSpec, isPartitionStats, useTableValues);
+      boolean isPartitionStats) throws SemanticException {
+    String rewritten = genRewrittenQuery(tbl, colNames, conf, partSpec, isPartitionStats, false);
     isRewritten = true;
-    return rewrittenQuery;
+    return rewritten;
   }
 
   public static String genRewrittenQuery(Table tbl, List<String> colNames, HiveConf conf, Map<String, String> partSpec,
@@ -239,10 +249,10 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
         columnDummyValuesBuilder.append(" , ");
       }
       String func = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_STATS_NDV_ALGO).toLowerCase();
-      rewrittenQueryBuilder.append("compute_stats(`");
-      final String columnName = escapeBackTicks(colNames.get(i));
+      rewrittenQueryBuilder.append("compute_stats(");
+      final String columnName = unparseIdentifier(colNames.get(i), conf);
       rewrittenQueryBuilder.append(columnName);
-      rewrittenQueryBuilder.append("`, '" + func + "'");
+      rewrittenQueryBuilder.append(", '" + func + "'");
       if ("fm".equals(func)) {
         int numBitVectors = 0;
         try {
@@ -254,7 +264,7 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
       }
       rewrittenQueryBuilder.append(')');
 
-      columnNamesBuilder.append("`" + columnName + "`");
+      columnNamesBuilder.append(unparseIdentifier(columnName, conf));
 
       columnDummyValuesBuilder.append(
           "cast(null as " + TypeInfoUtils.getTypeInfoFromTypeString(tbl.getCols().get(i).getType()).toString() + ")");
@@ -262,13 +272,12 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
 
     if (isPartitionStats) {
       for (FieldSchema fs : tbl.getPartCols()) {
-        final String partColumnName = " , `" + fs.getName() + "`";
-        rewrittenQueryBuilder.append(partColumnName);
-
-        columnNamesBuilder.append(partColumnName);
+        String identifier = unparseIdentifier(fs.getName(), conf);
+        rewrittenQueryBuilder.append(" , ").append(identifier);
+        columnNamesBuilder.append(" , ").append(identifier);
 
-        columnDummyValuesBuilder.append(
-            " , cast(null as " + TypeInfoUtils.getTypeInfoFromTypeString(fs.getType()).toString() + ")");
+        columnDummyValuesBuilder.append(" , cast(null as ")
+            .append(TypeInfoUtils.getTypeInfoFromTypeString(fs.getType()).toString()).append(")");
       }
     }
 
@@ -279,19 +288,21 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
       // Values
       rewrittenQueryBuilder.append(columnDummyValuesBuilder.toString());
       rewrittenQueryBuilder.append(")) as ");
-      rewrittenQueryBuilder.append("`" + tbl.getTableName() + "`");
+      rewrittenQueryBuilder.append(unparseIdentifier(tbl.getTableName() ,conf));
       rewrittenQueryBuilder.append("(");
       // Columns
       rewrittenQueryBuilder.append(columnNamesBuilder.toString());
       rewrittenQueryBuilder.append(")");
     } else {
-      rewrittenQueryBuilder.append("`" + tbl.getDbName() + "`.`" + tbl.getTableName() + "`");
+      rewrittenQueryBuilder.append(unparseIdentifier(tbl.getDbName(), conf));
+      rewrittenQueryBuilder.append(".");
+      rewrittenQueryBuilder.append(unparseIdentifier(tbl.getTableName(), conf));
     }
 
     // If partition level statistics is requested, add predicate and group by as needed to rewritten
     // query
     if (isPartitionStats) {
-      rewrittenQueryBuilder.append(genPartitionClause(tbl, partSpec));
+      rewrittenQueryBuilder.append(genPartitionClause(tbl, partSpec, conf));
     }
 
     String rewrittenQuery = rewrittenQueryBuilder.toString();
@@ -389,7 +400,7 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
         isTableLevel = true;
       }
       colType = getColumnTypes(tbl, colNames);
-      rewrittenQuery = genRewrittenQuery(colNames, conf, partSpec, isPartitionStats, false);
+      rewrittenQuery = genRewrittenQuery(colNames, conf, partSpec, isPartitionStats);
       rewrittenTree = genRewrittenTree(rewrittenQuery);
     } else {
       // Not an analyze table column compute statistics statement - don't do any rewrites
@@ -458,7 +469,7 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
 
     isTableLevel = !isPartitionStats;
 
-    rewrittenQuery = genRewrittenQuery(colNames, conf, partSpec, isPartitionStats, false);
+    rewrittenQuery = genRewrittenQuery(colNames, conf, partSpec, isPartitionStats);
     rewrittenTree = genRewrittenTree(rewrittenQuery);
 
     return rewrittenTree;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java
index 48c0a4a..46f1ec0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java
@@ -18,11 +18,9 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import java.util.ArrayList;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CharStream;
+
 import org.antlr.runtime.CommonToken;
-import org.antlr.runtime.NoViableAltException;
+import org.antlr.runtime.ParserRuleReturnScope;
 import org.antlr.runtime.RecognitionException;
 import org.antlr.runtime.Token;
 import org.antlr.runtime.TokenRewriteStream;
@@ -30,6 +28,7 @@ import org.antlr.runtime.TokenStream;
 import org.antlr.runtime.tree.CommonTree;
 import org.antlr.runtime.tree.CommonTreeAdaptor;
 import org.antlr.runtime.tree.TreeAdaptor;
+import org.apache.hadoop.conf.Configuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,91 +43,6 @@ public class ParseDriver {
   private static final Logger LOG = LoggerFactory.getLogger(ParseDriver.class);
 
   /**
-   * ANTLRNoCaseStringStream.
-   *
-   */
-  //This class provides and implementation for a case insensitive token checker
-  //for the lexical analysis part of antlr. By converting the token stream into
-  //upper case at the time when lexical rules are checked, this class ensures that the
-  //lexical rules need to just match the token with upper case letters as opposed to
-  //combination of upper case and lower case characteres. This is purely used for matching lexical
-  //rules. The actual token text is stored in the same way as the user input without
-  //actually converting it into an upper case. The token values are generated by the consume()
-  //function of the super class ANTLRStringStream. The LA() function is the lookahead funtion
-  //and is purely used for matching lexical rules. This also means that the grammar will only
-  //accept capitalized tokens in case it is run from other tools like antlrworks which
-  //do not have the ANTLRNoCaseStringStream implementation.
-  public class ANTLRNoCaseStringStream extends ANTLRStringStream {
-
-    public ANTLRNoCaseStringStream(String input) {
-      super(input);
-    }
-
-    @Override
-    public int LA(int i) {
-
-      int returnChar = super.LA(i);
-      if (returnChar == CharStream.EOF) {
-        return returnChar;
-      } else if (returnChar == 0) {
-        return returnChar;
-      }
-
-      return Character.toUpperCase((char) returnChar);
-    }
-  }
-
-  /**
-   * HiveLexerX.
-   *
-   */
-  public class HiveLexerX extends HiveLexer {
-
-    private final ArrayList<ParseError> errors;
-
-    public HiveLexerX() {
-      super();
-      errors = new ArrayList<ParseError>();
-    }
-
-    public HiveLexerX(CharStream input) {
-      super(input);
-      errors = new ArrayList<ParseError>();
-    }
-
-    @Override
-    public void displayRecognitionError(String[] tokenNames,
-        RecognitionException e) {
-
-      errors.add(new ParseError(this, e, tokenNames));
-    }
-
-    @Override
-    public String getErrorMessage(RecognitionException e, String[] tokenNames) {
-      String msg = null;
-
-      if (e instanceof NoViableAltException) {
-        @SuppressWarnings("unused")
-        NoViableAltException nvae = (NoViableAltException) e;
-        // for development, can add
-        // "decision=<<"+nvae.grammarDecisionDescription+">>"
-        // and "(decision="+nvae.decisionNumber+") and
-        // "state "+nvae.stateNumber
-        msg = "character " + getCharErrorDisplay(e.c) + " not supported here";
-      } else {
-        msg = super.getErrorMessage(e, tokenNames);
-      }
-
-      return msg;
-    }
-
-    public ArrayList<ParseError> getErrors() {
-      return errors;
-    }
-
-  }
-
-  /**
    * Tree adaptor for making antlr return ASTNodes instead of CommonTree nodes
    * so that the graph walking algorithms and the rules framework defined in
    * ql.lib can be used with the AST Nodes.
@@ -208,7 +122,8 @@ public class ParseDriver {
       LOG.debug("Parsing command: " + command);
     }
 
-    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
+    Configuration configuration = ctx == null ? null : ctx.getConf();
+    GenericHiveLexer lexer = GenericHiveLexer.of(command, configuration);
     TokenRewriteStream tokens = new TokenRewriteStream(lexer);
     if (ctx != null) {
       if (viewFullyQualifiedName == null) {
@@ -218,14 +133,11 @@ public class ParseDriver {
         // It is a view
         ctx.addViewTokenRewriteStream(viewFullyQualifiedName, tokens);
       }
-      lexer.setHiveConf(ctx.getConf());
     }
     HiveParser parser = new HiveParser(tokens);
-    if (ctx != null) {
-      parser.setHiveConf(ctx.getConf());
-    }
     parser.setTreeAdaptor(adaptor);
-    HiveParser.statement_return r = null;
+    parser.setHiveConf(configuration);
+    ParserRuleReturnScope r;
     try {
       r = parser.statement();
     } catch (RecognitionException e) {
@@ -251,7 +163,7 @@ public class ParseDriver {
   public ASTNode parseHint(String command) throws ParseException {
     LOG.debug("Parsing hint: {}", command);
 
-    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
+    GenericHiveLexer lexer = GenericHiveLexer.of(command, null);
     TokenRewriteStream tokens = new TokenRewriteStream(lexer);
     HintParser parser = new HintParser(tokens);
     parser.setTreeAdaptor(adaptor);
@@ -286,14 +198,16 @@ public class ParseDriver {
   public ASTNode parseSelect(String command, Context ctx) throws ParseException {
     LOG.debug("Parsing command: {}", command);
 
-    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
+    Configuration configuration = ctx == null ? null : ctx.getConf();
+    GenericHiveLexer lexer = GenericHiveLexer.of(command, configuration);
     TokenRewriteStream tokens = new TokenRewriteStream(lexer);
     if (ctx != null) {
       ctx.setTokenRewriteStream(tokens);
     }
     HiveParser parser = new HiveParser(tokens);
     parser.setTreeAdaptor(adaptor);
-    HiveParser_SelectClauseParser.selectClause_return r = null;
+    parser.setHiveConf(configuration);
+    ParserRuleReturnScope r;
     try {
       r = parser.selectClause();
     } catch (RecognitionException e) {
@@ -308,16 +222,16 @@ public class ParseDriver {
       throw new ParseException(parser.errors);
     }
 
-    return r.getTree();
+    return (ASTNode) r.getTree();
   }
   public ASTNode parseExpression(String command) throws ParseException {
     LOG.debug("Parsing expression: {}", command);
 
-    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
+    GenericHiveLexer lexer = GenericHiveLexer.of(command, null);
     TokenRewriteStream tokens = new TokenRewriteStream(lexer);
     HiveParser parser = new HiveParser(tokens);
     parser.setTreeAdaptor(adaptor);
-    HiveParser_IdentifiersParser.expression_return r = null;
+    ParserRuleReturnScope r;
     try {
       r = parser.expression();
     } catch (RecognitionException e) {
@@ -336,13 +250,13 @@ public class ParseDriver {
   }
 
   public ASTNode parseTriggerExpression(String command) throws ParseException {
-    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
+    GenericHiveLexer lexer = GenericHiveLexer.of(command, null);
     TokenRewriteStream tokens = new TokenRewriteStream(lexer);
     HiveParser parser = new HiveParser(tokens);
     parser.setTreeAdaptor(adaptor);
-    HiveParser_ResourcePlanParser.triggerExpressionStandalone_return r = null;
+    ParserRuleReturnScope r;
     try {
-      r = parser.gResourcePlanParser.triggerExpressionStandalone();
+      r = parser.triggerExpressionStandalone();
     } catch (RecognitionException e) {
       throw new ParseException(parser.errors);
     }
@@ -352,17 +266,17 @@ public class ParseDriver {
       throw new ParseException(parser.errors);
     }
 
-    return r.getTree();
+    return (ASTNode) r.getTree();
   }
 
   public ASTNode parseTriggerActionExpression(String command) throws ParseException {
-    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
+    GenericHiveLexer lexer = GenericHiveLexer.of(command, null);
     TokenRewriteStream tokens = new TokenRewriteStream(lexer);
     HiveParser parser = new HiveParser(tokens);
     parser.setTreeAdaptor(adaptor);
-    HiveParser_ResourcePlanParser.triggerActionExpressionStandalone_return r = null;
+    ParserRuleReturnScope r;
     try {
-      r = parser.gResourcePlanParser.triggerActionExpressionStandalone();
+      r = parser.triggerActionExpressionStandalone();
     } catch (RecognitionException e) {
       throw new ParseException(parser.errors);
     }
@@ -372,6 +286,6 @@ public class ParseDriver {
       throw new ParseException(parser.errors);
     }
 
-    return r.getTree();
+    return (ASTNode) r.getTree();
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 00fb059..aa8d84e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -4398,7 +4398,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
    */
   boolean isRegex(String pattern, HiveConf conf) {
     String qIdSupport = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT);
-    if ( "column".equals(qIdSupport)) {
+    if (!"none".equals(qIdSupport)) {
       return false;
     }
     for (int i = 0; i < pattern.length(); i++) {
@@ -11508,8 +11508,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     // db_name.table_name + partitionSec
     // as the prefix for easy of read during explain and debugging.
     // Currently, partition spec can only be static partition.
-    String k = org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.encodeTableName(tblName) + Path.SEPARATOR;
-    tsDesc.setStatsAggPrefix(tab.getDbName()+"."+k);
+    String k = FileUtils.escapePathName(tblName).toLowerCase() + Path.SEPARATOR;
+    tsDesc.setStatsAggPrefix(FileUtils.escapePathName(tab.getDbName()).toLowerCase() + "." + k);
 
     // set up WriteEntity for replication and txn stats
     WriteEntity we = new WriteEntity(tab, WriteEntity.WriteType.DDL_SHARED);
@@ -14956,7 +14956,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     int endIdx = ast.getTokenStopIndex();
 
     boolean queryNeedsQuotes = true;
-    if (conf.getVar(ConfVars.HIVE_QUOTEDID_SUPPORT).equals("none")) {
+    Quotation quotation = Quotation.from(conf);
+    if (quotation == Quotation.NONE) {
       queryNeedsQuotes = false;
     }
 
@@ -14967,10 +14968,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       } else if (queryNeedsQuotes && curTok.getType() == HiveLexer.Identifier) {
         // The Tokens have no distinction between Identifiers and QuotedIdentifiers.
         // Ugly solution is just to surround all identifiers with quotes.
-        sb.append('`');
-        // Re-escape any backtick (`) characters in the identifier.
-        sb.append(curTok.getText().replaceAll("`", "``"));
-        sb.append('`');
+        sb.append(HiveUtils.unparseIdentifier(curTok.getText(), conf));
       } else {
         sb.append(curTok.getText());
       }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java
index 6eb1ca2..5b72430 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java
@@ -32,6 +32,7 @@ import java.util.concurrent.Future;
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -39,7 +40,6 @@ import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -193,10 +193,10 @@ public class BasicStatsTask implements Serializable, IStatsProcessor {
     private String getAggregationPrefix0(Table table, Partition partition) throws MetaException {
 
       // prefix is of the form dbName.tblName
-      String prefix = table.getDbName() + "." + MetaStoreUtils.encodeTableName(table.getTableName());
+      String prefix = FileUtils.escapePathName(table.getDbName()).toLowerCase() + "." +
+          FileUtils.escapePathName(table.getTableName()).toLowerCase();
       // FIXME: this is a secret contract; reusein getAggrKey() creates a more closer relation to the StatsGatherer
       // prefix = work.getAggKey();
-      prefix = prefix.toLowerCase();
       if (partition != null) {
         return Utilities.join(prefix, Warehouse.makePartPath(partition.getSpec()));
       }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCastFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCastFormat.java
index 81540ba..6e24f9c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCastFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCastFormat.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.parse.HiveParser_IdentifiersParser;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
@@ -75,11 +75,11 @@ public class GenericUDFCastFormat extends GenericUDF implements Serializable {
 
   @VisibleForTesting
   static final Map<Integer, String> OUTPUT_TYPES = ImmutableMap.<Integer, String>builder()
-      .put(HiveParser_IdentifiersParser.TOK_STRING, serdeConstants.STRING_TYPE_NAME)
-      .put(HiveParser_IdentifiersParser.TOK_VARCHAR, serdeConstants.VARCHAR_TYPE_NAME)
-      .put(HiveParser_IdentifiersParser.TOK_CHAR, serdeConstants.CHAR_TYPE_NAME)
-      .put(HiveParser_IdentifiersParser.TOK_TIMESTAMP, serdeConstants.TIMESTAMP_TYPE_NAME)
-      .put(HiveParser_IdentifiersParser.TOK_DATE, serdeConstants.DATE_TYPE_NAME).build();
+      .put(HiveParser.TOK_STRING, serdeConstants.STRING_TYPE_NAME)
+      .put(HiveParser.TOK_VARCHAR, serdeConstants.VARCHAR_TYPE_NAME)
+      .put(HiveParser.TOK_CHAR, serdeConstants.CHAR_TYPE_NAME)
+      .put(HiveParser.TOK_TIMESTAMP, serdeConstants.TIMESTAMP_TYPE_NAME)
+      .put(HiveParser.TOK_DATE, serdeConstants.DATE_TYPE_NAME).build();
 
   private transient HiveSqlDateTimeFormatter formatter;
   private transient PrimitiveObjectInspector outputOI;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveUtils.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveUtils.java
new file mode 100644
index 0000000..b528b20
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveUtils.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.metadata;
+
+import static org.apache.hadoop.hive.ql.metadata.HiveUtils.unparseIdentifier;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.parse.Quotation;
+import org.junit.Test;
+
+/**
+ * Test class for testing methods in {@link HiveUtils}.
+ */
+public class TestHiveUtils {
+  @Test
+  public void testUnparseIdentifierWithBackTicksWhenQuotationIsNone() {
+    HiveConf conf = createConf(Quotation.NONE);
+    String id = "any``id";
+
+    String unparsed = unparseIdentifier(id, conf);
+
+    assertThat(unparsed, is("`any``id`"));
+  }
+
+  @Test
+  public void testUnparseIdentifierWithBackTicksWhenQuotationIsBackTicks() {
+    HiveConf conf = createConf(Quotation.BACKTICKS);
+    String id = "any``id";
+
+    String unparsed = unparseIdentifier(id, conf);
+
+    assertThat(unparsed, is("`any````id`"));
+  }
+
+  @Test
+  public void testUnparseIdentifierWithBackTicksWhenQuotationIsStandard() {
+    HiveConf conf = createConf(Quotation.STANDARD);
+    String id = "any``id";
+
+    String unparsed = unparseIdentifier(id, conf);
+
+    assertThat(unparsed, is("`any````id`"));
+  }
+
+  @Test
+  public void testUnparseIdentifierWithDoubleQuotesWhenQuotationIsNone() {
+    HiveConf conf = createConf(Quotation.NONE);
+    String id = "any\"\"id";
+
+    String unparsed = unparseIdentifier(id, conf);
+
+    assertThat(unparsed, is("`any\"\"id`"));
+  }
+
+  @Test
+  public void testUnparseIdentifierWithDoubleQuotesWhenQuotationIsBackTicks() {
+    HiveConf conf = createConf(Quotation.BACKTICKS);
+    String id = "any\"\"id";
+
+    String unparsed = unparseIdentifier(id, conf);
+
+    assertThat(unparsed, is("`any\"\"id`"));
+  }
+
+  @Test
+  public void testUnparseIdentifierWithDoubleQuotesWhenQuotationIsStandard() {
+    HiveConf conf = createConf(Quotation.STANDARD);
+    String id = "any\"\"id";
+
+    String unparsed = unparseIdentifier(id, conf);
+
+    assertThat(unparsed, is("`any\"\"id`"));
+  }
+
+  private HiveConf createConf(Quotation quotation) {
+    HiveConf conf = new HiveConf();
+    conf.setVar(HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT, quotation.stringValue());
+    return conf;
+  }
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestReplicationSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestReplicationSemanticAnalyzer.java
index 81ab01d..e1c6eed 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestReplicationSemanticAnalyzer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestReplicationSemanticAnalyzer.java
@@ -17,18 +17,35 @@
  */
 package org.apache.hadoop.hive.ql.parse;
 
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.runners.Enclosed;
 import org.junit.runner.RunWith;
 
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT;
 import static org.junit.Assert.assertEquals;
 
 @RunWith(Enclosed.class)
 public class TestReplicationSemanticAnalyzer {
   private static ParseDriver driver = new ParseDriver();
+  private static HiveConf hiveConf;
 
-  private static ASTNode parse(String command) throws ParseException {
-    return (ASTNode) driver.parse(command).getChild(0);
+  private TestReplicationSemanticAnalyzer() {
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    hiveConf = new HiveConf();
+    hiveConf.setVar(HIVE_QUOTEDID_SUPPORT, Quotation.NONE.stringValue());
+  }
+
+  private static ASTNode parse(String command) throws Exception {
+    SessionState.start(hiveConf);
+    Context context = new Context(hiveConf);
+    return (ASTNode) driver.parse(command, context).getChild(0);
   }
 
   private static void assertWithClause(ASTNode root, int replConfigIndex) {
@@ -91,7 +108,7 @@ public class TestReplicationSemanticAnalyzer {
   public static class ReplDump {
 
     @Test
-    public void parseDbPattern() throws ParseException {
+    public void parseDbPattern() throws Exception {
       ASTNode root = parse("repl dump `*`");
       assertEquals("TOK_REPL_DUMP", root.getText());
       assertEquals(1, root.getChildCount());
@@ -101,13 +118,13 @@ public class TestReplicationSemanticAnalyzer {
     }
 
     @Test
-    public void parseDb() throws ParseException {
+    public void parseDb() throws Exception {
       ASTNode root = parse("repl dump testDb");
       assertDatabase(1, root);
     }
 
     @Test
-    public void parseTableName() throws ParseException {
+    public void parseTableName() throws Exception {
       ASTNode root = parse("repl dump testDb.'test_table'");
       assertDatabase(2, root);
       assertTableName(root);
@@ -117,14 +134,14 @@ public class TestReplicationSemanticAnalyzer {
   public static class ReplDumpWithClause {
 
     @Test
-    public void parseDb() throws ParseException {
+    public void parseDb() throws Exception {
       ASTNode root = parse("repl dump testDb with ('key.1'='value.1','key.2'='value.2')");
       assertDatabase(2, root);
       assertWithClause(root, 1);
     }
 
     @Test
-    public void parseTableName() throws ParseException {
+    public void parseTableName() throws Exception {
       ASTNode root =
           parse("repl dump testDb.'test_table' with ('key.1'='value.1','key.2'='value.2')");
       assertDatabase(3, root);
@@ -136,20 +153,20 @@ public class TestReplicationSemanticAnalyzer {
   public static class ReplLoad {
 
     @Test
-    public void parseFromLocation() throws ParseException {
+    public void parseFromLocation() throws Exception {
       ASTNode root = parse("repl load testDbName");
       assertFromLocation(1, root);
     }
 
     @Test
-    public void parseTargetDbName() throws ParseException {
+    public void parseTargetDbName() throws Exception {
       ASTNode root = parse("repl load testDbName into targetTestDbName");
       assertFromLocation(2, root);
       assertTargetDatabaseName(root);
     }
 
     @Test
-    public void parseWithClause() throws ParseException {
+    public void parseWithClause() throws Exception {
       ASTNode root = parse("repl load testDbName into targetTestDbName"
           + " with ('mapred.job.queue.name'='repl','hive.repl.approx.max.load.tasks'='100')");
       assertFromLocation(3, root);
@@ -194,13 +211,13 @@ public class TestReplicationSemanticAnalyzer {
   public static class ReplStatus {
 
     @Test
-    public void parseTargetDbName() throws ParseException {
+    public void parseTargetDbName() throws Exception {
       ASTNode root = parse("repl status targetTestDbName");
       assertTargetDatabaseName(root);
     }
 
     @Test
-    public void parseWithClause() throws ParseException {
+    public void parseWithClause() throws Exception {
       ASTNode root = parse("repl status targetTestDbName with"
           + "('hive.metastore.uris'='thrift://localhost:12341')");
       assertTargetDatabaseName(root);
@@ -223,4 +240,4 @@ public class TestReplicationSemanticAnalyzer {
       assertEquals(0, child.getChildCount());
     }
   }
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFCastFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFCastFormat.java
index 9afd5af..2233b41 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFCastFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFCastFormat.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.parse.HiveParser_IdentifiersParser;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
@@ -44,11 +44,11 @@ import static org.junit.Assert.fail;
 public class TestGenericUDFCastFormat {
 
   //type codes
-  public static final int CHAR = HiveParser_IdentifiersParser.TOK_CHAR;
-  public static final int VARCHAR = HiveParser_IdentifiersParser.TOK_VARCHAR;
-  public static final int STRING = HiveParser_IdentifiersParser.TOK_STRING;
-  public static final int DATE = HiveParser_IdentifiersParser.TOK_DATE;
-  public static final int TIMESTAMP = HiveParser_IdentifiersParser.TOK_TIMESTAMP;
+  public static final int CHAR = HiveParser.TOK_CHAR;
+  public static final int VARCHAR = HiveParser.TOK_VARCHAR;
+  public static final int STRING = HiveParser.TOK_STRING;
+  public static final int DATE = HiveParser.TOK_DATE;
+  public static final int TIMESTAMP = HiveParser.TOK_TIMESTAMP;
 
   @Test
   public void testDateToStringWithFormat() throws HiveException {
diff --git a/ql/src/test/queries/clientnegative/database_create_invalid_name.q b/ql/src/test/queries/clientnegative/database_create_invalid_name.q
index 5d67495..1fd14c1 100644
--- a/ql/src/test/queries/clientnegative/database_create_invalid_name.q
+++ b/ql/src/test/queries/clientnegative/database_create_invalid_name.q
@@ -1,4 +1,4 @@
 SHOW DATABASES;
 
 -- Try to create a database with an invalid name
-CREATE DATABASE `test.db`;
+CREATE DATABASE `test§db`;
diff --git a/ql/src/test/queries/clientpositive/quotedid_basic.q b/ql/src/test/queries/clientpositive/quotedid_basic.q
index cb718f0..a316c43 100644
--- a/ql/src/test/queries/clientpositive/quotedid_basic.q
+++ b/ql/src/test/queries/clientpositive/quotedid_basic.q
@@ -19,18 +19,18 @@ from t1_n7 where `!@#$%^&*()_q` = '1' group by `x+1`, `y&Y`, `!@#$%^&*()_q` havi
 
 
 -- escaped back ticks
-create table t4(`x+1``` string, `y&y` string);
-describe t4;
-insert into table t4 select * from src;
+create table `t4```(`x+1``` string, `y&y` string);
+describe `t4```;
+insert into table `t4``` select * from src;
 select `x+1```, `y&y`, rank() over(partition by `x+1``` order by  `y&y`)  
-from t4 where `x+1``` = '10' group by `x+1```, `y&y` having `x+1``` = '10';
+from `t4``` where `x+1``` = '10' group by `x+1```, `y&y` having `x+1``` = '10';
 
 -- view
 create view v1 as 
 select `x+1```, `y&y`
-from t4 where `x+1``` < '200';
+from `t4``` where `x+1``` < '200';
 
 select `x+1```, `y&y`, rank() over(partition by `x+1``` order by  `y&y`)
 from v1
 group by `x+1```, `y&y`
-;
\ No newline at end of file
+;
diff --git a/ql/src/test/queries/clientpositive/quotedid_basic_standard.q b/ql/src/test/queries/clientpositive/quotedid_basic_standard.q
new file mode 100644
index 0000000..df33c99
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/quotedid_basic_standard.q
@@ -0,0 +1,46 @@
+--! qt:dataset:src
+
+set hive.mapred.mode=nonstrict;
+
+set hive.support.quoted.identifiers=standard;
+
+select 3 as "a", 10 as "~!@#$%^&*()_q<>";
+
+-- basic
+create table t1("x+1" string, "y&y" string, "~!@#$%^&*()_q<>" string);
+describe t1;
+select "x+1", "y&y", "~!@#$%^&*()_q<>" from t1;
+select "x+1", `y&y`, `~!@#$%^&*()_q<>` from t1;
+explain select "x+1", "y&y", "~!@#$%^&*()_q<>" from t1;
+explain select "x+1", "y&y", "~!@#$%^&*()_q<>" from t1 where "~!@#$%^&*()_q<>" = '1';
+explain select "x+1", "y&y", "~!@#$%^&*()_q<>" from t1 where "~!@#$%^&*()_q<>" = '1' group by "x+1", "y&y", "~!@#$%^&*()_q<>" having "~!@#$%^&*()_q<>" = '1';
+explain select "x+1", "y&y", "~!@#$%^&*()_q<>", rank() over(partition by "~!@#$%^&*()_q<>" order by  "y&y")
+from t1 where "~!@#$%^&*()_q<>" = '1' group by "x+1", "y&y", "~!@#$%^&*()_q<>" having "~!@#$%^&*()_q<>" = '1';
+
+-- case insensitive
+explain select "X+1", "Y&y", "~!@#$%^&*()_q<>", rank() over(partition by "~!@#$%^&*()_q<>" order by  "y&y")
+from t1 where "~!@#$%^&*()_q<>" = '1' group by "x+1", "y&Y", "~!@#$%^&*()_q<>" having "~!@#$%^&*()_q<>" = '1';
+
+
+-- escaped back ticks
+create table `t4```("x+1""" string, "y&y" string);
+describe `t4```;
+insert into table `t4``` select * from src;
+select "x+1""", "y&y", rank() over(partition by "x+1""" order by  "y&y")
+from `t4``` where "x+1""" = '10' group by "x+1""", "y&y" having "x+1""" = '10';
+
+-- view
+create view v1 as
+select "x+1""", "y&y"
+from `t4``` where "x+1""" < '200';
+
+select "x+1""", "y&y", rank() over(partition by "x+1""" order by  "y&y")
+from v1
+group by "x+1""", "y&y"
+;
+
+create table lv_table(c1 string) partitioned by(c2 string);
+create view "lv~!@#$%^&*()_q<>" partitioned on (c2) as select c1, c2 from lv_table;
+alter view "lv~!@#$%^&*()_q<>" add partition (c2='a');
+
+set hive.support.quoted.identifiers=column;
diff --git a/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q b/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
index 08df0d8..2159052 100644
--- a/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
+++ b/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
@@ -12,6 +12,9 @@ set hive.strict.checks.cartesian.product=false;
 
 -- SORT_QUERY_RESULTS
 
+create database `db~!@#$%^&*(),<>`;
+use `db~!@#$%^&*(),<>`;
+
 create table `c/b/o_t1`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE;
 create table `//cbo_t2`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE;
 create table `cbo_/t3////`(key string, value string, c_int int, c_float float, c_boolean boolean)  row format delimited fields terminated by ',' STORED AS TEXTFILE;
@@ -55,7 +58,7 @@ FIELDS TERMINATED BY '|';
 
 LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE `line/item`;
 
-create table `src/_/cbo` as select * from src;
+create table `src/_/cbo` as select * from default.src;
 
 analyze table `c/b/o_t1` partition (dt) compute statistics;
 
@@ -115,11 +118,11 @@ set hive.auto.convert.join=false;
 
 -- 21. Test groupby is empty and there is no other cols in aggr
 
-select unionsrc.key FROM (select 'tst1' as key, count(1) as value from src) unionsrc;
+select unionsrc.key FROM (select 'tst1' as key, count(1) as value from default.src) unionsrc;
 
 
 
-select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src) unionsrc;
+select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from default.src) unionsrc;
 
 
 
@@ -1067,15 +1070,15 @@ select * from (select max(c_int) over (partition by key order by value Rows UNBO
 
 select i, a, h, b, c, d, e, f, g, a as x, a +1 as y from (select max(c_int) over (partition by key order by value range UNBOUNDED PRECEDING) a, min(c_int) over (partition by key order by value range current row) b, count(c_int) over(partition by key order by value range 1 PRECEDING) c, avg(value) over (partition by key order by value range between unbounded preceding and unbounded following) d, sum(value) over (partition by key order by value range between unbounded preceding and current [...]
 
-select *, rank() over(partition by key order by value) as rr from src1;
+select *, rank() over(partition by key order by value) as rr from default.src1;
 
-select *, rank() over(partition by key order by value) from src1;
+select *, rank() over(partition by key order by value) from default.src1;
 
-insert into table `src/_/cbo` select * from src;
+insert into table `src/_/cbo` select * from default.src;
 
 select * from `src/_/cbo` limit 1;
 
-insert overwrite table `src/_/cbo` select * from src;
+insert overwrite table `src/_/cbo` select * from default.src;
 
 select * from `src/_/cbo` limit 1;
 
@@ -1085,3 +1088,5 @@ insert into `t//` values(1);
 insert into `t//` values(null);
 analyze table `t//` compute statistics;
 explain select * from `t//`;
+
+drop database `db~!@#$%^&*(),<>` cascade;
diff --git a/ql/src/test/queries/clientpositive/special_character_in_tabnames_quotes_1.q b/ql/src/test/queries/clientpositive/special_character_in_tabnames_quotes_1.q
new file mode 100644
index 0000000..64c9922
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/special_character_in_tabnames_quotes_1.q
@@ -0,0 +1,1091 @@
+--! qt:dataset:src1
+--! qt:dataset:src
+set hive.cbo.enable=true;
+set hive.exec.check.crossproducts=false;
+set hive.stats.fetch.column.stats=true;
+set hive.auto.convert.join=false;
+set hive.strict.checks.cartesian.product=false;
+set hive.support.quoted.identifiers=standard;
+
+-- SORT_QUERY_RESULTS
+
+create database "db~!@#$%^&*(),<>";
+use "db~!@#$%^&*(),<>";
+
+create table "c/b/o_t1"(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE;
+create table "//cbo_t2"(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE;
+create table "cbo_/t3////"(key string, value string, c_int int, c_float float, c_boolean boolean)  row format delimited fields terminated by ',' STORED AS TEXTFILE;
+
+load data local inpath '../../data/files/cbo_t1.txt' into table "c/b/o_t1" partition (dt='2014');
+load data local inpath '../../data/files/cbo_t2.txt' into table "//cbo_t2" partition (dt='2014');
+load data local inpath '../../data/files/cbo_t3.txt' into table "cbo_/t3////";
+
+CREATE TABLE "p/a/r/t"(
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table "p/a/r/t";
+
+CREATE TABLE "line/item" (L_ORDERKEY      INT,
+                                L_PARTKEY       INT,
+                                L_SUPPKEY       INT,
+                                L_LINENUMBER    INT,
+                                L_QUANTITY      DOUBLE,
+                                L_EXTENDEDPRICE DOUBLE,
+                                L_DISCOUNT      DOUBLE,
+                                L_TAX           DOUBLE,
+                                L_RETURNFLAG    STRING,
+                                L_LINESTATUS    STRING,
+                                l_shipdate      STRING,
+                                L_COMMITDATE    STRING,
+                                L_RECEIPTDATE   STRING,
+                                L_SHIPINSTRUCT  STRING,
+                                L_SHIPMODE      STRING,
+                                L_COMMENT       STRING)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|';
+
+LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE "line/item";
+
+create table "src/_/cbo" as select * from default.src;
+
+analyze table "c/b/o_t1" partition (dt) compute statistics;
+
+analyze table "c/b/o_t1" compute statistics for columns key, value, c_int, c_float, c_boolean;
+
+analyze table "//cbo_t2" partition (dt) compute statistics;
+
+analyze table "//cbo_t2" compute statistics for columns key, value, c_int, c_float, c_boolean;
+
+analyze table "cbo_/t3////" compute statistics;
+
+analyze table "cbo_/t3////" compute statistics for columns key, value, c_int, c_float, c_boolean;
+
+analyze table "src/_/cbo" compute statistics;
+
+analyze table "src/_/cbo" compute statistics for columns;
+
+analyze table "p/a/r/t" compute statistics;
+
+analyze table "p/a/r/t" compute statistics for columns;
+
+analyze table "line/item" compute statistics;
+
+analyze table "line/item" compute statistics for columns;
+
+select key, (c_int+1)+2 as x, sum(c_int) from "c/b/o_t1" group by c_float, "c/b/o_t1".c_int, key;
+
+select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from "c/b/o_t1" group by c_float, "c/b/o_t1".c_int, key) R group by y, x;
+
+select "cbo_/t3////".c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from "c/b/o_t1" where ("c/b/o_t1".c_int + 1 >= 0) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0) group by c_float, "c/b/o_t1".c_int, key order by a) "c/b/o_t1" join (select key as p, c_int+1 as q, sum(c_int) as r from "//cbo_t2" where ("//cbo_t2".c_int + 1 >= 0) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)  group by c_float, "//cbo_t2".c_int, key order by q/10 desc, r asc) "//cbo_ [...]
+
+select "cbo_/t3////".c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from "c/b/o_t1" where ("c/b/o_t1".c_int + 1 >= 0) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)  group by c_float, "c/b/o_t1".c_int, key having "c/b/o_t1".c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) "c/b/o_t1" left outer join (select key as p, c_int+1 as q, sum(c_int) as r from "//cbo_t2" where ("//cbo_t2".c_int + 1 >= 0) and ("//cbo_ [...]
+
+select "cbo_/t3////".c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from "c/b/o_t1" where ("c/b/o_t1".c_int + 1 >= 0) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)  group by c_float, "c/b/o_t1".c_int, key having "c/b/o_t1".c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) "c/b/o_t1" right outer join (select key as p, c_int+1 as q, sum(c_int) as r from "//cbo_t2" where ("//cbo_t2".c_int + 1 >= 0) and ("//cbo_t2".c [...]
+
+
+
+select "cbo_/t3////".c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from "c/b/o_t1" where ("c/b/o_t1".c_int + 1 >= 0) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)  group by c_float, "c/b/o_t1".c_int, key having "c/b/o_t1".c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) "c/b/o_t1" full outer join (select key as p, c_int+1 as q, sum(c_int) as r from "//cbo_t2" where ("//cbo_t2".c_int + 1 >= 0) and ("//cbo_t2".c_int [...]
+
+
+
+select "cbo_/t3////".c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from "c/b/o_t1" where ("c/b/o_t1".c_int + 1 >= 0) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)  group by c_float, "c/b/o_t1".c_int, key having "c/b/o_t1".c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) "c/b/o_t1" join (select key as p, c_int+1 as q, sum(c_int) as r from "//cbo_t2" where ("//cbo_t2".c_int + 1 >= 0) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= [...]
+
+
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 21. Test groupby is empty and there is no other cols in aggr
+
+select unionsrc.key FROM (select 'tst1' as key, count(1) as value from default.src) unionsrc;
+
+
+
+select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from default.src) unionsrc;
+
+
+
+select unionsrc.key FROM (select 'max' as key, max(c_int) as value from "cbo_/t3////" s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from "cbo_/t3////" s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from "cbo_/t3////" s3) unionsrc order by unionsrc.key;
+
+
+
+select unionsrc.key, unionsrc.value FROM (select 'max' as key, max(c_int) as value from "cbo_/t3////" s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from "cbo_/t3////" s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from "cbo_/t3////" s3) unionsrc order by unionsrc.key;
+
+
+
+select unionsrc.key, count(1) FROM (select 'max' as key, max(c_int) as value from "cbo_/t3////" s1
+
+    UNION  ALL
+
+        select 'min' as key,  min(c_int) as value from "cbo_/t3////" s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from "cbo_/t3////" s3) unionsrc group by unionsrc.key order by unionsrc.key;
+
+
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- SORT_QUERY_RESULTS
+
+-- 4. Test Select + Join + TS
+
+select "c/b/o_t1".c_int, "//cbo_t2".c_int from "c/b/o_t1" join             "//cbo_t2" on "c/b/o_t1".key="//cbo_t2".key;
+
+select "c/b/o_t1".key from "c/b/o_t1" join "cbo_/t3////";
+
+select "c/b/o_t1".key from "c/b/o_t1" join "cbo_/t3////" where "c/b/o_t1".key="cbo_/t3////".key and "c/b/o_t1".key >= 1;
+
+select "c/b/o_t1".c_int, "//cbo_t2".c_int from "c/b/o_t1" left outer join  "//cbo_t2" on "c/b/o_t1".key="//cbo_t2".key;
+
+select "c/b/o_t1".c_int, "//cbo_t2".c_int from "c/b/o_t1" right outer join "//cbo_t2" on "c/b/o_t1".key="//cbo_t2".key;
+
+select "c/b/o_t1".c_int, "//cbo_t2".c_int from "c/b/o_t1" full outer join  "//cbo_t2" on "c/b/o_t1".key="//cbo_t2".key;
+
+
+
+select b, "c/b/o_t1".c, "//cbo_t2".p, q, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1") "c/b/o_t1" join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2") "//cbo_t2" on "c/b/o_t1".a=p join "cbo_/t3////" on "c/b/o_t1".a=key;
+
+select key, "c/b/o_t1".c_int, "//cbo_t2".p, q from "c/b/o_t1" join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2") "//cbo_t2" on "c/b/o_t1".key=p join (select key as a, c_int as b, "cbo_/t3////".c_float as c from "cbo_/t3////")"cbo_/t3////" on "c/b/o_t1".key=a;
+
+select a, "c/b/o_t1".b, key, "//cbo_t2".c_int, "cbo_/t3////".p from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1") "c/b/o_t1" join "//cbo_t2"  on "c/b/o_t1".a=key join (select key as p, c_int as q, "cbo_/t3////".c_float as r from "cbo_/t3////")"cbo_/t3////" on "c/b/o_t1".a="cbo_/t3////".p;
+
+select b, "c/b/o_t1".c, "//cbo_t2".c_int, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1") "c/b/o_t1" join "//cbo_t2" on "c/b/o_t1".a="//cbo_t2".key join "cbo_/t3////" on "c/b/o_t1".a="cbo_/t3////".key;
+
+select "cbo_/t3////".c_int, b, "//cbo_t2".c_int, "c/b/o_t1".c from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1") "c/b/o_t1" join "//cbo_t2" on "c/b/o_t1".a="//cbo_t2".key join "cbo_/t3////" on "c/b/o_t1".a="cbo_/t3////".key;
+
+
+
+select b, "c/b/o_t1".c, "//cbo_t2".p, q, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1") "c/b/o_t1" left outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2") "//cbo_t2" on "c/b/o_t1".a=p join "cbo_/t3////" on "c/b/o_t1".a=key;
+
+select key, "c/b/o_t1".c_int, "//cbo_t2".p, q from "c/b/o_t1" join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2") "//cbo_t2" on "c/b/o_t1".key=p left outer join (select key as a, c_int as b, "cbo_/t3////".c_float as c from "cbo_/t3////")"cbo_/t3////" on "c/b/o_t1".key=a;
+
+
+
+select b, "c/b/o_t1".c, "//cbo_t2".p, q, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1") "c/b/o_t1" right outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2") "//cbo_t2" on "c/b/o_t1".a=p join "cbo_/t3////" on "c/b/o_t1".a=key;
+
+select key, "c/b/o_t1".c_int, "//cbo_t2".p, q from "c/b/o_t1" join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2") "//cbo_t2" on "c/b/o_t1".key=p right outer join (select key as a, c_int as b, "cbo_/t3////".c_float as c from "cbo_/t3////")"cbo_/t3////" on "c/b/o_t1".key=a;
+
+
+
+select b, "c/b/o_t1".c, "//cbo_t2".p, q, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1") "c/b/o_t1" full outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2") "//cbo_t2" on "c/b/o_t1".a=p join "cbo_/t3////" on "c/b/o_t1".a=key;
+
+select key, "c/b/o_t1".c_int, "//cbo_t2".p, q from "c/b/o_t1" join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2") "//cbo_t2" on "c/b/o_t1".key=p full outer join (select key as a, c_int as b, "cbo_/t3////".c_float as c from "cbo_/t3////")"cbo_/t3////" on "c/b/o_t1".key=a;
+
+
+
+-- 5. Test Select + Join + FIL + TS
+
+select "c/b/o_t1".c_int, "//cbo_t2".c_int from "c/b/o_t1" join "//cbo_t2" on "c/b/o_t1".key="//cbo_t2".key where ("c/b/o_t1".c_int + "//cbo_t2".c_int == 2) and ("c/b/o_t1".c_int > 0 or "//cbo_t2".c_float >= 0);
+
+select "c/b/o_t1".c_int, "//cbo_t2".c_int from "c/b/o_t1" left outer join  "//cbo_t2" on "c/b/o_t1".key="//cbo_t2".key where ("c/b/o_t1".c_int + "//cbo_t2".c_int == 2) and ("c/b/o_t1".c_int > 0 or "//cbo_t2".c_float >= 0);
+
+select "c/b/o_t1".c_int, "//cbo_t2".c_int from "c/b/o_t1" right outer join "//cbo_t2" on "c/b/o_t1".key="//cbo_t2".key where ("c/b/o_t1".c_int + "//cbo_t2".c_int == 2) and ("c/b/o_t1".c_int > 0 or "//cbo_t2".c_float >= 0);
+
+select "c/b/o_t1".c_int, "//cbo_t2".c_int from "c/b/o_t1" full outer join  "//cbo_t2" on "c/b/o_t1".key="//cbo_t2".key where ("c/b/o_t1".c_int + "//cbo_t2".c_int == 2) and ("c/b/o_t1".c_int > 0 or "//cbo_t2".c_float >= 0);
+
+
+
+select b, "c/b/o_t1".c, "//cbo_t2".p, q, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p join "cbo_/t3////" on "c/b/o_t1".a=key where (b +  [...]
+
+
+
+select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" left outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p join "cbo_/t3////" on "c/b/o_t1".a=key  [...]
+
+
+
+select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" right outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p join "cbo_/t3////" on "c/b/o_t1".a=key [...]
+
+
+
+select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" full outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p join "cbo_/t3////" on "c/b/o_t1".a=key  [...]
+
+
+
+select * from (select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" full outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p join "cbo_/t3////" on "c [...]
+
+
+
+select * from (select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" left outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p left outer join "cbo_/t3 [...]
+
+
+
+select * from (select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" left outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p right outer join "cbo_/t [...]
+
+
+
+select * from (select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" left outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p full outer join "cbo_/t3 [...]
+
+
+
+select * from (select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" right outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p right outer join "cbo_/ [...]
+
+
+
+select * from (select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" right outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p left outer join "cbo_/t [...]
+
+
+
+select * from (select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" right outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p full outer join "cbo_/t [...]
+
+
+
+select * from (select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" full outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p full outer join "cbo_/t3 [...]
+
+
+
+select * from (select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" full outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p left outer join "cbo_/t3 [...]
+
+
+
+select * from (select q, b, "//cbo_t2".p, "c/b/o_t1".c, "cbo_/t3////".c_int from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" full outer join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p right outer join "cbo_/t [...]
+
+
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 7. Test Select + TS + Join + Fil + GB + GB Having + Limit
+
+select key, (c_int+1)+2 as x, sum(c_int) from "c/b/o_t1" group by c_float, "c/b/o_t1".c_int, key order by x limit 1;
+
+select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from "c/b/o_t1" group by c_float, "c/b/o_t1".c_int, key) R group by y, x order by x,y limit 1;
+
+select key from(select key from (select key from "c/b/o_t1" order by key limit 5)"//cbo_t2" order by key limit 5)"cbo_/t3////" order by key limit 5;
+
+select key, c_int from(select key, c_int from (select key, c_int from "c/b/o_t1" order by c_int limit 5)"c/b/o_t1"  order by c_int limit 5)"//cbo_t2"  order by c_int limit 5;
+
+
+
+select "cbo_/t3////".c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from "c/b/o_t1" where ("c/b/o_t1".c_int + 1 >= 0) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0) group by c_float, "c/b/o_t1".c_int, key order by a limit 5) "c/b/o_t1" join (select key as p, c_int+1 as q, sum(c_int) as r from "//cbo_t2" where ("//cbo_t2".c_int + 1 >= 0) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)  group by c_float, "//cbo_t2".c_int, key order by q/10 desc, r asc  [...]
+
+
+
+select "cbo_/t3////".c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from "c/b/o_t1" where ("c/b/o_t1".c_int + 1 >= 0) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)  group by c_float, "c/b/o_t1".c_int, key having "c/b/o_t1".c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) "c/b/o_t1" left outer join (select key as p, c_int+1 as q, sum(c_int) as r from "//cbo_t2" where ("//cbo_t2".c_int + 1 >= 0) and  [...]
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 12. SemiJoin
+
+select "c/b/o_t1".c_int           from "c/b/o_t1" left semi join   "//cbo_t2" on "c/b/o_t1".key="//cbo_t2".key;
+
+select "c/b/o_t1".c_int           from "c/b/o_t1" left semi join   "//cbo_t2" on "c/b/o_t1".key="//cbo_t2".key where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0);
+
+select * from (select c, b, a from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" left semi join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p left semi join "cbo_/t3////" on "c/b/o_t1".a=key where (b + 1 == 2) and [...]
+
+select * from (select "cbo_/t3////".c_int, "c/b/o_t1".c, b from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 = 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" left semi join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p left outer join "cbo_/t3////" on "c/b/o_t1" [...]
+
+select * from (select c_int, b, "c/b/o_t1".c from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" left semi join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p right outer join "cbo_/t3////" on "c/b/o_t1".a=key where [...]
+
+select * from (select c_int, b, "c/b/o_t1".c from (select key as a, c_int as b, "c/b/o_t1".c_float as c from "c/b/o_t1"  where ("c/b/o_t1".c_int + 1 == 2) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)) "c/b/o_t1" left semi join (select "//cbo_t2".key as p, "//cbo_t2".c_int as q, c_float as r from "//cbo_t2"  where ("//cbo_t2".c_int + 1 == 2) and ("//cbo_t2".c_int > 0 or "//cbo_t2".c_float >= 0)) "//cbo_t2" on "c/b/o_t1".a=p full outer join "cbo_/t3////" on "c/b/o_t1".a=key where  [...]
+
+select a, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from "c/b/o_t1" where ("c/b/o_t1".c_int + 1 >= 0) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)  group by c_float, "c/b/o_t1".c_int, key having "c/b/o_t1".c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc) "c/b/o_t1" left semi join (select key as p, c_int+1 as q, sum(c_int) as r from "//cbo_t2" where ("//cbo_t2".c_int + 1 >= 0) and ("//cbo_t2".c_int > 0 or "//c [...]
+
+select a, c, count(*)  from (select key as a, c_int+1 as b, sum(c_int) as c from "c/b/o_t1" where ("c/b/o_t1".c_int + 1 >= 0) and ("c/b/o_t1".c_int > 0 or "c/b/o_t1".c_float >= 0)  group by c_float, "c/b/o_t1".c_int, key having "c/b/o_t1".c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by a+b desc, c asc limit 5) "c/b/o_t1" left semi join (select key as p, c_int+1 as q, sum(c_int) as r from "//cbo_t2" where ("//cbo_t2".c_int + 1 >= 0) and ("//cbo_t2".c_int >  [...]
+
+
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 1. Test Select + TS
+
+select * from "c/b/o_t1";
+
+select * from "c/b/o_t1" as "c/b/o_t1";
+
+select * from "c/b/o_t1" as "//cbo_t2";
+
+
+
+select "c/b/o_t1".key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from "c/b/o_t1";
+
+select * from "c/b/o_t1" where (((key=1) and (c_float=10)) and (c_int=20));
+
+
+
+-- 2. Test Select + TS + FIL
+
+select * from "c/b/o_t1" where "c/b/o_t1".c_int >= 0;
+
+select * from "c/b/o_t1" as "c/b/o_t1"  where "c/b/o_t1".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+select * from "c/b/o_t1" as "//cbo_t2" where "//cbo_t2".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+
+
+select "//cbo_t2".key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from "c/b/o_t1" as "//cbo_t2"  where "//cbo_t2".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+
+
+-- 3 Test Select + Select + TS + FIL
+
+select * from (select * from "c/b/o_t1" where "c/b/o_t1".c_int >= 0) as "c/b/o_t1";
+
+select * from (select * from "c/b/o_t1" as "c/b/o_t1"  where "c/b/o_t1".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as "c/b/o_t1";
+
+select * from (select * from "c/b/o_t1" as "//cbo_t2" where "//cbo_t2".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as "c/b/o_t1";
+
+select * from (select "//cbo_t2".key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from "c/b/o_t1" as "//cbo_t2"  where "//cbo_t2".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as "c/b/o_t1";
+
+
+
+select * from (select * from "c/b/o_t1" where "c/b/o_t1".c_int >= 0) as "c/b/o_t1" where "c/b/o_t1".c_int >= 0;
+
+select * from (select * from "c/b/o_t1" as "c/b/o_t1"  where "c/b/o_t1".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as "c/b/o_t1"  where "c/b/o_t1".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+select * from (select * from "c/b/o_t1" as "//cbo_t2" where "//cbo_t2".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as "//cbo_t2" where "//cbo_t2".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+select * from (select "//cbo_t2".key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from "c/b/o_t1" as "//cbo_t2"  where "//cbo_t2".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as "c/b/o_t1" where "c/b/o_t1".c_int >= 0 and y+c_int >= 0 or x <= 100;
+
+
+
+select "c/b/o_t1".c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from "c/b/o_t1" where "c/b/o_t1".c_int >= 0) as "c/b/o_t1" where "c/b/o_t1".c_int >= 0;
+
+select "//cbo_t2".c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from "c/b/o_t1" where "c/b/o_t1".c_int >= 0) as "//cbo_t2" where "//cbo_t2".c_int >= 0;
+
+
+
+
+
+
+
+select * from (select * from "c/b/o_t1" where "c/b/o_t1".c_int >= 0) as "c/b/o_t1" where "c/b/o_t1".c_int >= 0;
+
+select * from (select * from "c/b/o_t1" as "c/b/o_t1"  where "c/b/o_t1".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as "c/b/o_t1"  where "c/b/o_t1".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+select * from (select * from "c/b/o_t1" as "//cbo_t2" where "//cbo_t2".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as "//cbo_t2" where "//cbo_t2".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100;
+
+select * from (select "//cbo_t2".key as x, c_int as c_int, (((c_int+c_float)*10)+5) as y from "c/b/o_t1" as "//cbo_t2"  where "//cbo_t2".c_int >= 0 and c_float+c_int >= 0 or c_float <= 100) as "c/b/o_t1" where "c/b/o_t1".c_int >= 0 and y+c_int >= 0 or x <= 100;
+
+
+
+select "c/b/o_t1".c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from "c/b/o_t1" where "c/b/o_t1".c_int >= 0) as "c/b/o_t1" where "c/b/o_t1".c_int >= 0;
+
+select "//cbo_t2".c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from "c/b/o_t1" where "c/b/o_t1".c_int >= 0) as "//cbo_t2" where "//cbo_t2".c_int >= 0;
+
+
+
+
+
+
+
+-- 13. null expr in select list
+
+select null from "cbo_/t3////";
+
+
+
+-- 14. unary operator
+
+select key from "c/b/o_t1" where c_int = -6  or c_int = +6;
+
+
+
+-- 15. query referencing only partition columns
+
+select count("c/b/o_t1".dt) from "c/b/o_t1" join "//cbo_t2" on "c/b/o_t1".dt  = "//cbo_t2".dt  where "c/b/o_t1".dt = '2014' ;
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 20. Test get stats with empty partition list
+
+select "c/b/o_t1".value from "c/b/o_t1" join "//cbo_t2" on "c/b/o_t1".key = "//cbo_t2".key where "c/b/o_t1".dt = '10' and "c/b/o_t1".c_boolean = true;
+
+
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 18. SubQueries Not Exists
+
+-- distinct, corr
+
+select *
+
+from "src/_/cbo" b
+
+where not exists
+
+  (select distinct a.key
+
+  from "src/_/cbo" a
+
+  where b.value = a.value and a.value > 'val_2'
+
+  )
+
+;
+
+
+
+-- no agg, corr, having
+
+select *
+
+from "src/_/cbo" b
+
+group by key, value
+
+having not exists
+
+  (select a.key
+
+  from "src/_/cbo" a
+
+  where b.value = a.value  and a.key = b.key and a.value > 'val_12'
+
+  )
+
+;
+
+
+
+-- 19. SubQueries Exists
+
+-- view test
+
+create view cv1 as
+
+select *
+
+from "src/_/cbo" b
+
+where exists
+
+  (select a.key
+
+  from "src/_/cbo" a
+
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+
+;
+
+
+
+select * from cv1
+
+;
+
+
+
+-- sq in from
+
+select *
+
+from (select *
+
+      from "src/_/cbo" b
+
+      where exists
+
+          (select a.key
+
+          from "src/_/cbo" a
+
+          where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+
+     ) a
+
+;
+
+
+
+-- sq in from, having
+
+select *
+
+from (select b.key, count(*)
+
+  from "src/_/cbo" b
+
+  group by b.key
+
+  having exists
+
+    (select a.key
+
+    from "src/_/cbo" a
+
+    where a.key = b.key and a.value > 'val_9'
+
+    )
+
+) a
+
+;
+
+
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 17. SubQueries In
+
+-- non agg, non corr
+
+select *
+
+from "src/_/cbo"
+
+where "src/_/cbo".key in (select key from "src/_/cbo" s1 where s1.key > '9') order by key
+
+;
+
+
+
+-- agg, corr
+
+-- add back once rank issue fixed for cbo
+
+
+
+-- distinct, corr
+
+select *
+
+from "src/_/cbo" b
+
+where b.key in
+
+        (select distinct a.key
+
+         from "src/_/cbo" a
+
+         where b.value = a.value and a.key > '9'
+
+        ) order by b.key
+
+;
+
+
+
+-- non agg, corr, with join in Parent Query
+
+select p.p_partkey, li.l_suppkey
+
+from (select distinct l_partkey as p_partkey from "line/item") p join "line/item" li on p.p_partkey = li.l_partkey
+
+where li.l_linenumber = 1 and
+
+ li.l_orderkey in (select l_orderkey from "line/item" where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber)
+
+ order by p.p_partkey
+
+;
+
+
+
+-- where and having
+
+-- Plan is:
+
+-- Stage 1: b semijoin sq1:"src/_/cbo" (subquery in where)
+
+-- Stage 2: group by Stage 1 o/p
+
+-- Stage 5: group by on sq2:"src/_/cbo" (subquery in having)
+
+-- Stage 6: Stage 2 o/p semijoin Stage 5
+
+select key, value, count(*)
+
+from "src/_/cbo" b
+
+where b.key in (select key from "src/_/cbo" where "src/_/cbo".key > '8')
+
+group by key, value
+
+having count(*) in (select count(*) from "src/_/cbo" s1 where s1.key > '9' group by s1.key ) order by key
+
+;
+
+
+
+-- non agg, non corr, windowing
+
+select p_mfgr, p_name, avg(p_size)
+
+from "p/a/r/t"
+
+group by p_mfgr, p_name
+
+having p_name in
+
+  (select first_value(p_name) over(partition by p_mfgr order by p_size) from "p/a/r/t") order by p_mfgr
+
+;
+
+
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 16. SubQueries Not In
+
+-- non agg, non corr
+
+select *
+
+from "src/_/cbo"
+
+where "src/_/cbo".key not in
+
+  ( select key  from "src/_/cbo" s1
+
+    where s1.key > '2'
+
+  ) order by key
+
+;
+
+
+
+-- non agg, corr
+
+select p_mfgr, b.p_name, p_size
+
+from "p/a/r/t" b
+
+where b.p_name not in
+
+  (select p_name
+
+  from (select p_mfgr, p_name, p_size as r from "p/a/r/t") a
+
+  where r < 10 and b.p_mfgr = a.p_mfgr
+
+  ) order by p_mfgr,p_size
+
+;
+
+
+
+-- agg, non corr
+
+select p_name, p_size
+
+from
+
+"p/a/r/t" where "p/a/r/t".p_size not in
+
+  (select avg(p_size)
+
+  from (select p_size from "p/a/r/t") a
+
+  where p_size < 10
+
+  ) order by p_name
+
+;
+
+
+
+-- agg, corr
+
+select p_mfgr, p_name, p_size
+
+from "p/a/r/t" b where b.p_size not in
+
+  (select min(p_size)
+
+  from (select p_mfgr, p_size from "p/a/r/t") a
+
+  where p_size < 10 and b.p_mfgr = a.p_mfgr
+
+  ) order by  p_name
+
+;
+
+
+
+-- non agg, non corr, Group By in Parent Query
+
+select li.l_partkey, count(*)
+
+from "line/item" li
+
+where li.l_linenumber = 1 and
+
+  li.l_orderkey not in (select l_orderkey from "line/item" where l_shipmode = 'AIR')
+
+group by li.l_partkey order by li.l_partkey
+
+;
+
+
+
+-- add null check test from sq_notin.q once HIVE-7721 resolved.
+
+
+
+-- non agg, corr, having
+
+select b.p_mfgr, min(p_retailprice)
+
+from "p/a/r/t" b
+
+group by b.p_mfgr
+
+having b.p_mfgr not in
+
+  (select p_mfgr
+
+  from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from "p/a/r/t" group by p_mfgr) a
+
+  where min(p_retailprice) = l and r - l > 600
+
+  )
+
+  order by b.p_mfgr
+
+;
+
+
+
+-- agg, non corr, having
+
+select b.p_mfgr, min(p_retailprice)
+
+from "p/a/r/t" b
+
+group by b.p_mfgr
+
+having b.p_mfgr not in
+
+  (select p_mfgr
+
+  from "p/a/r/t" a
+
+  group by p_mfgr
+
+  having max(p_retailprice) - min(p_retailprice) > 600
+
+  )
+
+  order by b.p_mfgr
+
+;
+
+
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- SORT_QUERY_RESULTS
+
+
+
+-- 8. Test UDF/UDAF
+
+select count(*), count(c_int), sum(c_int), avg(c_int), max(c_int), min(c_int) from "c/b/o_t1";
+
+select count(*), count(c_int) as a, sum(c_int), avg(c_int), max(c_int), min(c_int), case c_int when 0  then 1 when 1 then 2 else 3 end, sum(case c_int when 0  then 1 when 1 then 2 else 3 end) from "c/b/o_t1" group by c_int order by a;
+
+select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f from "c/b/o_t1") "c/b/o_t1";
+
+select * from (select count(*) as a, count(distinct c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f, case c_int when 0  then 1 when 1 then 2 else 3 end as g, sum(case c_int when 0  then 1 when 1 then 2 else 3 end) as h from "c/b/o_t1" group by c_int) "c/b/o_t1" order by a;
+
+select f,a,e,b from (select count(*) as a, count(c_int) as b, sum(c_int) as c, avg(c_int) as d, max(c_int) as e, min(c_int) as f from "c/b/o_t1") "c/b/o_t1";
+
+select f,a,e,b from (select count(*) as a, count(distinct c_int) as b, sum(distinct c_int) as c, avg(distinct c_int) as d, max(distinct c_int) as e, min(distinct c_int) as f from "c/b/o_t1") "c/b/o_t1";
+
+select key,count(c_int) as a, avg(c_float) from "c/b/o_t1" group by key order by a;
+
+select count(distinct c_int) as a, avg(c_float) from "c/b/o_t1" group by c_float order by a;
+
+select count(distinct c_int) as a, avg(c_float) from "c/b/o_t1" group by c_int order by a;
+
+select count(distinct c_int) as a, avg(c_float) from "c/b/o_t1" group by c_float, c_int order by a;
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- SORT_QUERY_RESULTS
+
+
+
+-- 11. Union All
+
+select * from (select * from "c/b/o_t1" order by key, c_boolean, value, dt)a union all select * from (select * from "//cbo_t2" order by key, c_boolean, value, dt)b;
+
+select key from (select key, c_int from (select * from "c/b/o_t1" union all select * from "//cbo_t2" where "//cbo_t2".key >=0)r1 union all select key, c_int from "cbo_/t3////")r2 where key >=0 order by key;
+
+select r2.key from (select key, c_int from (select key, c_int from "c/b/o_t1" union all select key, c_int from "cbo_/t3////" )r1 union all select key, c_int from "cbo_/t3////")r2 join   (select key, c_int from (select * from "c/b/o_t1" union all select * from "//cbo_t2" where "//cbo_t2".key >=0)r1 union all select key, c_int from "cbo_/t3////")r3 on r2.key=r3.key where r3.key >=0 order by r2.key;
+
+
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 10. Test views
+
+create view v1 as select c_int, value, c_boolean, dt from "c/b/o_t1";
+
+create view v2 as select c_int, value from "//cbo_t2";
+
+
+
+select value from v1 where c_boolean=false;
+
+select max(c_int) from v1 group by (c_boolean);
+
+
+
+select count(v1.c_int)  from v1 join "//cbo_t2" on v1.c_int = "//cbo_t2".c_int;
+
+select count(v1.c_int)  from v1 join v2 on v1.c_int = v2.c_int;
+
+
+
+select count(*) from v1 a join v1 b on a.value = b.value;
+
+
+
+create view v3 as select v1.value val from v1 join "c/b/o_t1" on v1.c_boolean = "c/b/o_t1".c_boolean;
+
+
+
+select count(val) from v3 where val != '1';
+
+with q1 as ( select key from "c/b/o_t1" where key = '1')
+
+select count(*) from q1;
+
+
+
+with q1 as ( select value from v1 where c_boolean = false)
+
+select count(value) from q1 ;
+
+
+
+create view v4 as
+
+with q1 as ( select key,c_int from "c/b/o_t1"  where key = '1')
+
+select * from q1
+
+;
+
+
+
+with q1 as ( select c_int from q2 where c_boolean = false),
+
+q2 as ( select c_int,c_boolean from v1  where value = '1')
+
+select sum(c_int) from (select c_int from q1) a;
+
+
+
+with q1 as ( select "c/b/o_t1".c_int c_int from q2 join "c/b/o_t1" where q2.c_int = "c/b/o_t1".c_int  and "c/b/o_t1".dt='2014'),
+
+q2 as ( select c_int,c_boolean from v1  where value = '1' or dt = '14')
+
+select count(*) from q1 join q2 join v4 on q1.c_int = q2.c_int and v4.c_int = q2.c_int;
+
+
+
+
+
+drop view v1;
+
+drop view v2;
+
+drop view v3;
+
+drop view v4;
+
+set hive.cbo.enable=false;
+
+set hive.exec.check.crossproducts=false;
+
+
+
+set hive.stats.fetch.column.stats=true;
+
+set hive.auto.convert.join=false;
+
+
+
+-- 9. Test Windowing Functions
+
+-- SORT_QUERY_RESULTS
+
+
+
+select count(c_int) over() from "c/b/o_t1";
+
+select count(c_int) over(partition by c_float order by key), sum(c_float) over(partition by c_float order by key), max(c_int) over(partition by c_float order by key), min(c_int) over(partition by c_float order by key), row_number() over(partition by c_float order by key) as rn, rank() over(partition by c_float order by key), dense_rank() over(partition by c_float order by key), round(percent_rank() over(partition by c_float order by key), 2), lead(c_int, 2, c_int) over(partition by c_flo [...]
+
+select * from (select count(c_int) over(partition by c_float order by key), sum(c_float) over(partition by c_float order by key), max(c_int) over(partition by c_float order by key), min(c_int) over(partition by c_float order by key), row_number() over(partition by c_float order by key) as rn, rank() over(partition by c_float order by key), dense_rank() over(partition by c_float order by key), round(percent_rank() over(partition by c_float order by key),2), lead(c_int, 2, c_int) over(part [...]
+
+select x from (select count(c_int) over() as x, sum(c_float) over() from "c/b/o_t1") "c/b/o_t1";
+
+select 1+sum(c_int) over() from "c/b/o_t1";
+
+select sum(c_int)+sum(sum(c_int)) over() from "c/b/o_t1";
+
+select * from (select max(c_int) over (partition by key order by value Rows UNBOUNDED PRECEDING), min(c_int) over (partition by key order by value rows current row), count(c_int) over(partition by key order by value ROWS 1 PRECEDING), avg(value) over (partition by key order by value Rows between unbounded preceding and unbounded following), sum(value) over (partition by key order by value rows between unbounded preceding and current row), avg(c_float) over (partition by key order by valu [...]
+
+select i, a, h, b, c, d, e, f, g, a as x, a +1 as y from (select max(c_int) over (partition by key order by value range UNBOUNDED PRECEDING) a, min(c_int) over (partition by key order by value range current row) b, count(c_int) over(partition by key order by value range 1 PRECEDING) c, avg(value) over (partition by key order by value range between unbounded preceding and unbounded following) d, sum(value) over (partition by key order by value range between unbounded preceding and current [...]
+
+select *, rank() over(partition by key order by value) as rr from default.src1;
+
+select *, rank() over(partition by key order by value) from default.src1;
+
+insert into table "src/_/cbo" select * from default.src;
+
+select * from "src/_/cbo" order by key limit 1;
+
+insert overwrite table "src/_/cbo" select * from default.src;
+
+select * from "src/_/cbo" order by key limit 1;
+
+drop table "t//";
+create table "t//" (col string);
+insert into "t//" values(1);
+insert into "t//" values(null);
+analyze table "t//" compute statistics;
+explain select * from "t//";
+
+drop database "db~!@#$%^&*(),<>" cascade;
+
+set hive.support.quoted.identifiers=column;
diff --git a/ql/src/test/queries/clientpositive/special_character_in_tabnames_quotes_2.q b/ql/src/test/queries/clientpositive/special_character_in_tabnames_quotes_2.q
new file mode 100644
index 0000000..3aec316
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/special_character_in_tabnames_quotes_2.q
@@ -0,0 +1,24 @@
+set hive.cbo.enable=true;
+set hive.support.quoted.identifiers=standard;
+
+-- try the query without indexing, with manual indexing, and with automatic indexing
+-- SORT_QUERY_RESULTS
+
+DROP TABLE IF EXISTS "s/c";
+
+CREATE TABLE "s/c" (key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE "s/c";
+
+ANALYZE TABLE "s/c" COMPUTE STATISTICS;
+
+ANALYZE TABLE "s/c" COMPUTE STATISTICS FOR COLUMNS key,value;
+
+describe formatted "s/c";
+
+SELECT key, value FROM "s/c" WHERE key > 80 AND key < 100;
+
+EXPLAIN SELECT key, value FROM "s/c" WHERE key > 80 AND key < 100;
+SELECT key, value FROM "s/c" WHERE key > 80 AND key < 100;
+
+set hive.support.quoted.identifiers=column;
diff --git a/ql/src/test/results/clientnegative/database_create_invalid_name.q.out b/ql/src/test/results/clientnegative/database_create_invalid_name.q.out
index 4b2cd1e..9dba025 100644
--- a/ql/src/test/results/clientnegative/database_create_invalid_name.q.out
+++ b/ql/src/test/results/clientnegative/database_create_invalid_name.q.out
@@ -3,7 +3,7 @@ PREHOOK: type: SHOWDATABASES
 POSTHOOK: query: SHOW DATABASES
 POSTHOOK: type: SHOWDATABASES
 default
-PREHOOK: query: CREATE DATABASE `test.db`
+PREHOOK: query: CREATE DATABASE `test§db`
 PREHOOK: type: CREATEDATABASE
-PREHOOK: Output: database:test.db
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.ddl.DDLTask. InvalidObjectException(message:test.db is not a valid database name)
+PREHOOK: Output: database:test§db
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.ddl.DDLTask. InvalidObjectException(message:test§db is not a valid database name)
diff --git a/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out b/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out
new file mode 100644
index 0000000..14c8fad
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out
@@ -0,0 +1,19456 @@
+PREHOOK: query: create database `db~!@#$%^&*(),<>`
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:db~!@#$%^&*(),<>
+POSTHOOK: query: create database `db~!@#$%^&*(),<>`
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:db~!@#$%^&*(),<>
+PREHOOK: query: use `db~!@#$%^&*(),<>`
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:db~!@#$%^&*(),<>
+POSTHOOK: query: use `db~!@#$%^&*(),<>`
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:db~!@#$%^&*(),<>
+PREHOOK: query: create table `c/b/o_t1`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:db~!@#$%^&*(),<>
+PREHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: query: create table `c/b/o_t1`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:db~!@#$%^&*(),<>
+POSTHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: query: create table `//cbo_t2`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:db~!@#$%^&*(),<>
+PREHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: query: create table `//cbo_t2`(key string, value string, c_int int, c_float float, c_boolean boolean)  partitioned by (dt string) row format delimited fields terminated by ',' STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:db~!@#$%^&*(),<>
+POSTHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: query: create table `cbo_/t3////`(key string, value string, c_int int, c_float float, c_boolean boolean)  row format delimited fields terminated by ',' STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:db~!@#$%^&*(),<>
+PREHOOK: Output: db~!@#$%^&*(),<>@cbo_/t3////
+POSTHOOK: query: create table `cbo_/t3////`(key string, value string, c_int int, c_float float, c_boolean boolean)  row format delimited fields terminated by ',' STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:db~!@#$%^&*(),<>
+POSTHOOK: Output: db~!@#$%^&*(),<>@cbo_/t3////
+PREHOOK: query: load data local inpath '../../data/files/cbo_t1.txt' into table `c/b/o_t1` partition (dt='2014')
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: query: load data local inpath '../../data/files/cbo_t1.txt' into table `c/b/o_t1` partition (dt='2014')
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: query: load data local inpath '../../data/files/cbo_t2.txt' into table `//cbo_t2` partition (dt='2014')
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: query: load data local inpath '../../data/files/cbo_t2.txt' into table `//cbo_t2` partition (dt='2014')
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: query: load data local inpath '../../data/files/cbo_t3.txt' into table `cbo_/t3////`
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: db~!@#$%^&*(),<>@cbo_/t3////
+POSTHOOK: query: load data local inpath '../../data/files/cbo_t3.txt' into table `cbo_/t3////`
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: db~!@#$%^&*(),<>@cbo_/t3////
+PREHOOK: query: CREATE TABLE `p/a/r/t`(
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:db~!@#$%^&*(),<>
+PREHOOK: Output: db~!@#$%^&*(),<>@p/a/r/t
+POSTHOOK: query: CREATE TABLE `p/a/r/t`(
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:db~!@#$%^&*(),<>
+POSTHOOK: Output: db~!@#$%^&*(),<>@p/a/r/t
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table `p/a/r/t`
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: db~!@#$%^&*(),<>@p/a/r/t
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table `p/a/r/t`
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: db~!@#$%^&*(),<>@p/a/r/t
+PREHOOK: query: CREATE TABLE `line/item` (L_ORDERKEY      INT,
+                                L_PARTKEY       INT,
+                                L_SUPPKEY       INT,
+                                L_LINENUMBER    INT,
+                                L_QUANTITY      DOUBLE,
+                                L_EXTENDEDPRICE DOUBLE,
+                                L_DISCOUNT      DOUBLE,
+                                L_TAX           DOUBLE,
+                                L_RETURNFLAG    STRING,
+                                L_LINESTATUS    STRING,
+                                l_shipdate      STRING,
+                                L_COMMITDATE    STRING,
+                                L_RECEIPTDATE   STRING,
+                                L_SHIPINSTRUCT  STRING,
+                                L_SHIPMODE      STRING,
+                                L_COMMENT       STRING)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:db~!@#$%^&*(),<>
+PREHOOK: Output: db~!@#$%^&*(),<>@line/item
+POSTHOOK: query: CREATE TABLE `line/item` (L_ORDERKEY      INT,
+                                L_PARTKEY       INT,
+                                L_SUPPKEY       INT,
+                                L_LINENUMBER    INT,
+                                L_QUANTITY      DOUBLE,
+                                L_EXTENDEDPRICE DOUBLE,
+                                L_DISCOUNT      DOUBLE,
+                                L_TAX           DOUBLE,
+                                L_RETURNFLAG    STRING,
+                                L_LINESTATUS    STRING,
+                                l_shipdate      STRING,
+                                L_COMMITDATE    STRING,
+                                L_RECEIPTDATE   STRING,
+                                L_SHIPINSTRUCT  STRING,
+                                L_SHIPMODE      STRING,
+                                L_COMMENT       STRING)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:db~!@#$%^&*(),<>
+POSTHOOK: Output: db~!@#$%^&*(),<>@line/item
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE `line/item`
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: db~!@#$%^&*(),<>@line/item
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE `line/item`
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: db~!@#$%^&*(),<>@line/item
+PREHOOK: query: create table `src/_/cbo` as select * from default.src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:db~!@#$%^&*(),<>
+PREHOOK: Output: db~!@#$%^&*(),<>@src/_/cbo
+POSTHOOK: query: create table `src/_/cbo` as select * from default.src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:db~!@#$%^&*(),<>
+POSTHOOK: Output: db~!@#$%^&*(),<>@src/_/cbo
+POSTHOOK: Lineage: src/_/cbo.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src/_/cbo.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: analyze table `c/b/o_t1` partition (dt) compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: query: analyze table `c/b/o_t1` partition (dt) compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: query: analyze table `c/b/o_t1` compute statistics for columns key, value, c_int, c_float, c_boolean
+PREHOOK: type: ANALYZE_TABLE
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `c/b/o_t1` compute statistics for columns key, value, c_int, c_float, c_boolean
+POSTHOOK: type: ANALYZE_TABLE
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Output: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+PREHOOK: query: analyze table `//cbo_t2` partition (dt) compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: query: analyze table `//cbo_t2` partition (dt) compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: query: analyze table `//cbo_t2` compute statistics for columns key, value, c_int, c_float, c_boolean
+PREHOOK: type: ANALYZE_TABLE
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `//cbo_t2` compute statistics for columns key, value, c_int, c_float, c_boolean
+POSTHOOK: type: ANALYZE_TABLE
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Output: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+#### A masked pattern was here ####
+PREHOOK: query: analyze table `cbo_/t3////` compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+PREHOOK: Output: db~!@#$%^&*(),<>@cbo_/t3////
+POSTHOOK: query: analyze table `cbo_/t3////` compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+POSTHOOK: Output: db~!@#$%^&*(),<>@cbo_/t3////
+PREHOOK: query: analyze table `cbo_/t3////` compute statistics for columns key, value, c_int, c_float, c_boolean
+PREHOOK: type: ANALYZE_TABLE
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+PREHOOK: Output: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `cbo_/t3////` compute statistics for columns key, value, c_int, c_float, c_boolean
+POSTHOOK: type: ANALYZE_TABLE
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+POSTHOOK: Output: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+PREHOOK: query: analyze table `src/_/cbo` compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@src/_/cbo
+PREHOOK: Output: db~!@#$%^&*(),<>@src/_/cbo
+POSTHOOK: query: analyze table `src/_/cbo` compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@src/_/cbo
+POSTHOOK: Output: db~!@#$%^&*(),<>@src/_/cbo
+PREHOOK: query: analyze table `src/_/cbo` compute statistics for columns
+PREHOOK: type: ANALYZE_TABLE
+PREHOOK: Input: db~!@#$%^&*(),<>@src/_/cbo
+PREHOOK: Output: db~!@#$%^&*(),<>@src/_/cbo
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `src/_/cbo` compute statistics for columns
+POSTHOOK: type: ANALYZE_TABLE
+POSTHOOK: Input: db~!@#$%^&*(),<>@src/_/cbo
+POSTHOOK: Output: db~!@#$%^&*(),<>@src/_/cbo
+#### A masked pattern was here ####
+PREHOOK: query: analyze table `p/a/r/t` compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@p/a/r/t
+PREHOOK: Output: db~!@#$%^&*(),<>@p/a/r/t
+POSTHOOK: query: analyze table `p/a/r/t` compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@p/a/r/t
+POSTHOOK: Output: db~!@#$%^&*(),<>@p/a/r/t
+PREHOOK: query: analyze table `p/a/r/t` compute statistics for columns
+PREHOOK: type: ANALYZE_TABLE
+PREHOOK: Input: db~!@#$%^&*(),<>@p/a/r/t
+PREHOOK: Output: db~!@#$%^&*(),<>@p/a/r/t
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `p/a/r/t` compute statistics for columns
+POSTHOOK: type: ANALYZE_TABLE
+POSTHOOK: Input: db~!@#$%^&*(),<>@p/a/r/t
+POSTHOOK: Output: db~!@#$%^&*(),<>@p/a/r/t
+#### A masked pattern was here ####
+PREHOOK: query: analyze table `line/item` compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@line/item
+PREHOOK: Output: db~!@#$%^&*(),<>@line/item
+POSTHOOK: query: analyze table `line/item` compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@line/item
+POSTHOOK: Output: db~!@#$%^&*(),<>@line/item
+PREHOOK: query: analyze table `line/item` compute statistics for columns
+PREHOOK: type: ANALYZE_TABLE
+PREHOOK: Input: db~!@#$%^&*(),<>@line/item
+PREHOOK: Output: db~!@#$%^&*(),<>@line/item
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table `line/item` compute statistics for columns
+POSTHOOK: type: ANALYZE_TABLE
+POSTHOOK: Input: db~!@#$%^&*(),<>@line/item
+POSTHOOK: Output: db~!@#$%^&*(),<>@line/item
+#### A masked pattern was here ####
+PREHOOK: query: select key, (c_int+1)+2 as x, sum(c_int) from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select key, (c_int+1)+2 as x, sum(c_int) from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+ 1	4	2
+ 1 	4	2
+1	4	12
+1 	4	2
+NULL	NULL	NULL
+PREHOOK: query: select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key) R group by y, x
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select x, y, count(*) from (select key, (c_int+c_float+1+2) as x, sum(c_int) as y from `c/b/o_t1` group by c_float, `c/b/o_t1`.c_int, key) R group by y, x
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+5.0	12	1
+5.0	2	3
+NULL	NULL	1
+PREHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0) group by c_float, `c/b/o_t1`.c_int, key order by a) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key order by q/10 desc [...]
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0) group by c_float, `c/b/o_t1`.c_int, key order by a) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cbo_t2`.c_float >= 0)  group by c_float, `//cbo_t2`.c_int, key order by q/10 des [...]
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	12	6
+1	2	6
+PREHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) `c/b/o_t1` left outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= [...]
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc) `c/b/o_t1` left outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 > [...]
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	12	6
+1	2	6
+PREHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) `c/b/o_t1` right outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) a [...]
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) `c/b/o_t1` right outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0)  [...]
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	12	6
+1	2	6
+PREHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) `c/b/o_t1` full outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and ( [...]
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by c+a desc) `c/b/o_t1` full outer join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and  [...]
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	12	6
+1	2	6
+PREHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//cb [...]
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from `c/b/o_t1` where (`c/b/o_t1`.c_int + 1 >= 0) and (`c/b/o_t1`.c_int > 0 or `c/b/o_t1`.c_float >= 0)  group by c_float, `c/b/o_t1`.c_int, key having `c/b/o_t1`.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) `c/b/o_t1` join (select key as p, c_int+1 as q, sum(c_int) as r from `//cbo_t2` where (`//cbo_t2`.c_int + 1 >= 0) and (`//cbo_t2`.c_int > 0 or `//c [...]
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	12	6
+1	2	6
+PREHOOK: query: select unionsrc.key FROM (select 'tst1' as key, count(1) as value from default.src) unionsrc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select unionsrc.key FROM (select 'tst1' as key, count(1) as value from default.src) unionsrc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+tst1
+PREHOOK: query: select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from default.src) unionsrc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from default.src) unionsrc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+tst1	500
+PREHOOK: query: select unionsrc.key FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc order by unionsrc.key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select unionsrc.key FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc order by unionsrc.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+avg
+max
+min
+PREHOOK: query: select unionsrc.key, unionsrc.value FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc order by unionsrc.key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select unionsrc.key, unionsrc.value FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+UNION  ALL
+
+    select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc order by unionsrc.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+avg	1.5
+max	3.0
+min	1.0
+PREHOOK: query: select unionsrc.key, count(1) FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+    UNION  ALL
+
+        select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc group by unionsrc.key order by unionsrc.key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select unionsrc.key, count(1) FROM (select 'max' as key, max(c_int) as value from `cbo_/t3////` s1
+
+    UNION  ALL
+
+        select 'min' as key,  min(c_int) as value from `cbo_/t3////` s2
+
+    UNION ALL
+
+        select 'avg' as key,  avg(c_int) as value from `cbo_/t3////` s3) unionsrc group by unionsrc.key order by unionsrc.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+avg	1
+max	1
+min	1
+PREHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` join             `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` join             `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+PREHOOK: query: select `c/b/o_t1`.key from `c/b/o_t1` join `cbo_/t3////`
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.key from `c/b/o_t1` join `cbo_/t3////`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+ 1 
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+1 
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+PREHOOK: query: select `c/b/o_t1`.key from `c/b/o_t1` join `cbo_/t3////` where `c/b/o_t1`.key=`cbo_/t3////`.key and `c/b/o_t1`.key >= 1
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.key from `c/b/o_t1` join `cbo_/t3////` where `c/b/o_t1`.key=`cbo_/t3////`.key and `c/b/o_t1`.key >= 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+ 1
+ 1
+ 1
+ 1
+ 1 
+ 1 
+ 1 
+ 1 
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1 
+1 
+1 
+1 
+PREHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` left outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` left outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+NULL	NULL
+NULL	NULL
+PREHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` right outer join `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` right outer join `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+NULL	2
+NULL	2
+NULL	2
+NULL	2
+NULL	2
+NULL	NULL
+NULL	NULL
+PREHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` full outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select `c/b/o_t1`.c_int, `//cbo_t2`.c_int from `c/b/o_t1` full outer join  `//cbo_t2` on `c/b/o_t1`.key=`//cbo_t2`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+#### A masked pattern was here ####
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+NULL	2
+NULL	2
+NULL	2
+NULL	2
+NULL	2
+NULL	NULL
+NULL	NULL
+NULL	NULL
+NULL	NULL
+PREHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+PREHOOK: query: select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+PREHOOK: query: select a, `c/b/o_t1`.b, key, `//cbo_t2`.c_int, `cbo_/t3////`.p from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2`  on `c/b/o_t1`.a=key join (select key as p, c_int as q, `cbo_/t3////`.c_float as r from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.p
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select a, `c/b/o_t1`.b, key, `//cbo_t2`.c_int, `cbo_/t3////`.p from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2`  on `c/b/o_t1`.a=key join (select key as p, c_int as q, `cbo_/t3////`.c_float as r from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.p
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1	1	 1	1	 1
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+ 1 	1	 1 	1	 1 
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1	1	1	1	1
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+1 	1	1 	1	1 
+PREHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.c_int, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.a=`//cbo_t2`.key join `cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.c_int, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.a=`//cbo_t2`.key join `cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+1	1.0	1	1
+PREHOOK: query: select `cbo_/t3////`.c_int, b, `//cbo_t2`.c_int, `c/b/o_t1`.c from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.a=`//cbo_t2`.key join `cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select `cbo_/t3////`.c_int, b, `//cbo_t2`.c_int, `c/b/o_t1`.c from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` join `//cbo_t2` on `c/b/o_t1`.a=`//cbo_t2`.key join `cbo_/t3////` on `c/b/o_t1`.a=`cbo_/t3////`.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+1	1	1	1.0
+PREHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` left outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` left outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+PREHOOK: query: select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p left outer join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p left outer join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+PREHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` right outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` right outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+1	1.0	1 	1	1
+PREHOOK: query: select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p right outer join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select key, `c/b/o_t1`.c_int, `//cbo_t2`.p, q from `c/b/o_t1` join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.key=p right outer join (select key as a, c_int as b, `cbo_/t3////`.c_float as c from `cbo_/t3////`)`cbo_/t3////` on `c/b/o_t1`.key=a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1	1	 1	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+ 1 	1	 1 	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1	1	1	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+1 	1	1 	1
+NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL
+NULL	NULL	NULL	NULL
+PREHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` full outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+PREHOOK: type: QUERY
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+PREHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+PREHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+PREHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+POSTHOOK: query: select b, `c/b/o_t1`.c, `//cbo_t2`.p, q, `cbo_/t3////`.c_int from (select key as a, c_int as b, `c/b/o_t1`.c_float as c from `c/b/o_t1`) `c/b/o_t1` full outer join (select `//cbo_t2`.key as p, `//cbo_t2`.c_int as q, c_float as r from `//cbo_t2`) `//cbo_t2` on `c/b/o_t1`.a=p join `cbo_/t3////` on `c/b/o_t1`.a=key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2
+POSTHOOK: Input: db~!@#$%^&*(),<>@//cbo_t2@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1
+POSTHOOK: Input: db~!@#$%^&*(),<>@c/b/o_t1@dt=2014
+POSTHOOK: Input: db~!@#$%^&*(),<>@cbo_/t3////
+#### A masked pattern was here ####
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	 1 	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
+1	1.0	1	1	1
... 33215 lines suppressed ...