You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pr...@apache.org on 2015/08/10 01:59:04 UTC

hive git commit: HIVE-11496: Better tests for evaluating ORC predicate pushdown (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Repository: hive
Updated Branches:
  refs/heads/master 3e68cdc99 -> 7536edec1


HIVE-11496: Better tests for evaluating ORC predicate pushdown (Prasanth Jayachandran reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7536edec
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7536edec
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7536edec

Branch: refs/heads/master
Commit: 7536edec1dc39028fca7d53f1a09aa56f9531682
Parents: 3e68cdc
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Sun Aug 9 16:58:52 2015 -0700
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Sun Aug 9 16:58:52 2015 -0700

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../ql/hooks/PostExecTezSummaryPrinter.java     |  72 ++
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |  18 +-
 .../hive/ql/io/sarg/ConvertAstToSearchArg.java  |  14 +-
 .../test/queries/clientpositive/orc_ppd_basic.q | 177 +++++
 .../clientpositive/tez/orc_ppd_basic.q.out      | 701 +++++++++++++++++++
 6 files changed, 975 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7536edec/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index c710b0b..bed621d 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -331,6 +331,7 @@ minitez.query.files=bucket_map_join_tez1.q,\
   mapjoin_decimal.q,\
   lvj_mapjoin.q, \
   mrr.q,\
+  orc_ppd_basic.q,\
   tez_bmj_schema_evolution.q,\
   tez_dml.q,\
   tez_fsstat.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/7536edec/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecTezSummaryPrinter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecTezSummaryPrinter.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecTezSummaryPrinter.java
new file mode 100644
index 0000000..60c587f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecTezSummaryPrinter.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hooks;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.tez.TezTask;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.tez.common.counters.CounterGroup;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.common.counters.TezCounters;
+
+/**
+ * Post execution hook to print hive tez counters to console error stream.
+ */
+public class PostExecTezSummaryPrinter implements ExecuteWithHookContext {
+  private static final Log LOG = LogFactory.getLog(PostExecTezSummaryPrinter.class.getName());
+
+  @Override
+  public void run(HookContext hookContext) throws Exception {
+    assert (hookContext.getHookType() == HookContext.HookType.POST_EXEC_HOOK);
+    HiveConf conf = hookContext.getConf();
+    if (!"tez".equals(HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE))) {
+      return;
+    }
+
+    LOG.info("Executing post execution hook to print tez summary..");
+    SessionState ss = SessionState.get();
+    SessionState.LogHelper console = ss.getConsole();
+    QueryPlan plan = hookContext.getQueryPlan();
+    if (plan == null) {
+      return;
+    }
+
+    List<TezTask> rootTasks = Utilities.getTezTasks(plan.getRootTasks());
+    for (TezTask tezTask : rootTasks) {
+      LOG.info("Printing summary for tez task: " + tezTask.getName());
+      TezCounters counters = tezTask.getTezCounters();
+      if (counters != null) {
+        for (CounterGroup group : counters) {
+          if ("HIVE".equals(group.getDisplayName())) {
+            console.printError(tezTask.getId() + " HIVE COUNTERS:");
+            for (TezCounter counter : group) {
+              console.printError("   " + counter.getDisplayName() + ": " + counter.getValue());
+            }
+          }
+        }
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7536edec/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
index f85420d..0d765b1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
@@ -376,7 +376,7 @@ class RecordReaderImpl implements RecordReader {
       Object predObj = getBaseObjectForComparison(predicate.getType(), baseObj);
 
       result = evaluatePredicateMinMax(predicate, predObj, minValue, maxValue, hasNull);
-      if (bloomFilter != null && result != TruthValue.NO_NULL && result != TruthValue.NO) {
+      if (shouldEvaluateBloomFilter(predicate, result, bloomFilter)) {
         result = evaluatePredicateBloomFilter(predicate, predObj, bloomFilter, hasNull);
       }
       // in case failed conversion, return the default YES_NO_NULL truth value
@@ -394,6 +394,22 @@ class RecordReaderImpl implements RecordReader {
     return result;
   }
 
+  private static boolean shouldEvaluateBloomFilter(PredicateLeaf predicate,
+      TruthValue result, BloomFilterIO bloomFilter) {
+    // evaluate bloom filter only when
+    // 1) Bloom filter is available
+    // 2) Min/Max evaluation yield YES or MAYBE
+    // 3) Predicate is EQUALS or IN list
+    if (bloomFilter != null
+        && result != TruthValue.NO_NULL && result != TruthValue.NO
+        && (predicate.getOperator().equals(PredicateLeaf.Operator.EQUALS)
+            || predicate.getOperator().equals(PredicateLeaf.Operator.NULL_SAFE_EQUALS)
+            || predicate.getOperator().equals(PredicateLeaf.Operator.IN))) {
+      return true;
+    }
+    return false;
+  }
+
   private static TruthValue evaluatePredicateMinMax(PredicateLeaf predicate, Object predObj,
       Object minValue,
       Object maxValue,

http://git-wip-us.apache.org/repos/asf/hive/blob/7536edec/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
index eb8c03f..5c4b7ea 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
@@ -18,14 +18,15 @@
 
 package org.apache.hadoop.hive.ql.io.sarg;
 
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Input;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
@@ -51,9 +52,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.List;
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
 
 public class ConvertAstToSearchArg {
   private static final Log LOG = LogFactory.getLog(ConvertAstToSearchArg.class);
@@ -145,7 +145,7 @@ public class ConvertAstToSearchArg {
         return ((Number) lit).longValue();
       case STRING:
         if (lit instanceof HiveChar) {
-          lit = ((HiveChar) lit).getPaddedValue();
+          return ((HiveChar) lit).getPaddedValue();
         } else if (lit instanceof String) {
           return lit;
         } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/7536edec/ql/src/test/queries/clientpositive/orc_ppd_basic.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/orc_ppd_basic.q b/ql/src/test/queries/clientpositive/orc_ppd_basic.q
new file mode 100644
index 0000000..f9dafef
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/orc_ppd_basic.q
@@ -0,0 +1,177 @@
+SET hive.fetch.task.conversion=none;
+SET hive.optimize.index.filter=true;
+SET hive.cbo.enable=false;
+
+CREATE TABLE staging(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE staging;
+LOAD DATA LOCAL INPATH '../../data/files/over1k' INTO TABLE staging;
+
+CREATE TABLE orc_ppd_staging(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           c char(50),
+           v varchar(50),
+           da date,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+STORED AS ORC tblproperties("orc.row.index.stride" = "1000", "orc.bloom.filter.columns"="*");
+
+insert overwrite table orc_ppd_staging select t, si, i, b, f, d, bo, s, cast(s as char(50)), cast(s as varchar(50)), cast(ts as date), ts, dec, bin from staging order by t, s;
+
+-- just to introduce a gap in min/max range for bloom filters. The dataset has contiguous values
+-- which makes it hard to test bloom filters
+insert into orc_ppd_staging select -10,-321,-65680,-4294967430,-97.94,-13.07,true,"aaa","aaa","aaa","1990-03-11","1990-03-11 10:11:58.703308",-71.54,"aaa" from staging limit 1;
+insert into orc_ppd_staging select 127,331,65690,4294967440,107.94,23.07,true,"zzz","zzz","zzz","2023-03-11","2023-03-11 10:11:58.703308",71.54,"zzz" from staging limit 1;
+
+CREATE TABLE orc_ppd(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           c char(50),
+           v varchar(50),
+           da date,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+STORED AS ORC tblproperties("orc.row.index.stride" = "1000", "orc.bloom.filter.columns"="*");
+
+insert overwrite table orc_ppd select t, si, i, b, f, d, bo, s, cast(s as char(50)), cast(s as varchar(50)), cast(ts as date), ts, dec, bin from orc_ppd_staging order by t, s;
+
+SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecTezSummaryPrinter;
+
+-- Row group statistics for column t:
+-- Entry 0: count: 994 hasNull: true min: -10 max: 54 sum: 26014 positions: 0,0,0,0,0,0,0
+-- Entry 1: count: 1000 hasNull: false min: 54 max: 118 sum: 86812 positions: 0,2,124,0,0,116,11
+-- Entry 2: count: 100 hasNull: false min: 118 max: 127 sum: 12151 positions: 0,4,119,0,0,244,19
+
+-- INPUT_RECORDS: 2100 (all row groups)
+select count(*) from orc_ppd;
+
+-- INPUT_RECORDS: 0 (no row groups)
+select count(*) from orc_ppd where t > 127;
+
+-- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t = 55;
+select count(*) from orc_ppd where t <=> 50;
+select count(*) from orc_ppd where t <=> 100;
+
+-- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where t = "54";
+
+-- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t = -10.0;
+
+-- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t = cast(53 as float);
+select count(*) from orc_ppd where t = cast(53 as double);
+
+-- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where t < 100;
+
+-- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t < 100 and t > 98;
+
+-- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where t <= 100;
+
+-- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t is null;
+
+-- INPUT_RECORDS: 1100 (2 row groups)
+select count(*) from orc_ppd where t in (5, 120);
+
+-- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t between 60 and 80;
+
+-- bloom filter tests
+-- INPUT_RECORDS: 0
+select count(*) from orc_ppd where t = -100;
+select count(*) from orc_ppd where t <=> -100;
+select count(*) from orc_ppd where t = 125;
+select count(*) from orc_ppd where t IN (-100, 125, 200);
+
+-- Row group statistics for column s:
+-- Entry 0: count: 1000 hasNull: false min:  max: zach young sum: 12907 positions: 0,0,0
+-- Entry 1: count: 1000 hasNull: false min: alice allen max: zach zipper sum: 12704 positions: 0,1611,191
+-- Entry 2: count: 100 hasNull: false min: bob davidson max: zzz sum: 1281 positions: 0,3246,373
+
+-- INPUT_RECORDS: 0 (no row groups)
+select count(*) from orc_ppd where s > "zzz";
+
+-- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where s = "zach young";
+select count(*) from orc_ppd where s <=> "zach zipper";
+select count(*) from orc_ppd where s <=> "";
+
+-- INPUT_RECORDS: 0
+select count(*) from orc_ppd where s is null;
+
+-- INPUT_RECORDS: 2100
+select count(*) from orc_ppd where s is not null;
+
+-- INPUT_RECORDS: 0
+select count(*) from orc_ppd where s = cast("zach young" as char(50));
+
+-- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where s = cast("zach young" as char(10));
+select count(*) from orc_ppd where s = cast("zach young" as varchar(10));
+select count(*) from orc_ppd where s = cast("zach young" as varchar(50));
+
+-- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where s < "b";
+
+-- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where s > "alice" and s < "bob";
+
+-- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where s in ("alice allen", "");
+
+-- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where s between "" and "alice allen";
+
+-- INPUT_RECORDS: 100 (1 row group)
+select count(*) from orc_ppd where s between "zz" and "zzz";
+
+-- INPUT_RECORDS: 1100 (2 row groups)
+select count(*) from orc_ppd where s between "zach zipper" and "zzz";
+
+-- bloom filter tests
+-- INPUT_RECORDS: 0
+select count(*) from orc_ppd where s = "hello world";
+select count(*) from orc_ppd where s <=> "apache hive";
+select count(*) from orc_ppd where s IN ("a", "z");
+
+-- INPUT_RECORDS: 100
+select count(*) from orc_ppd where s = "sarah ovid";
+
+-- INPUT_RECORDS: 1100
+select count(*) from orc_ppd where s = "wendy king";
+
+-- INPUT_RECORDS: 1000
+select count(*) from orc_ppd where s = "wendy king" and t < 0;
+
+-- INPUT_RECORDS: 100
+select count(*) from orc_ppd where s = "wendy king" and t > 100;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7536edec/ql/src/test/results/clientpositive/tez/orc_ppd_basic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/orc_ppd_basic.q.out b/ql/src/test/results/clientpositive/tez/orc_ppd_basic.q.out
new file mode 100644
index 0000000..2d0984b
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/orc_ppd_basic.q.out
@@ -0,0 +1,701 @@
+PREHOOK: query: CREATE TABLE staging(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@staging
+POSTHOOK: query: CREATE TABLE staging(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@staging
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE staging
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@staging
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE staging
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@staging
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/over1k' INTO TABLE staging
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@staging
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/over1k' INTO TABLE staging
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@staging
+PREHOOK: query: CREATE TABLE orc_ppd_staging(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           c char(50),
+           v varchar(50),
+           da date,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+STORED AS ORC tblproperties("orc.row.index.stride" = "1000", "orc.bloom.filter.columns"="*")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@orc_ppd_staging
+POSTHOOK: query: CREATE TABLE orc_ppd_staging(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           c char(50),
+           v varchar(50),
+           da date,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+STORED AS ORC tblproperties("orc.row.index.stride" = "1000", "orc.bloom.filter.columns"="*")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@orc_ppd_staging
+PREHOOK: query: insert overwrite table orc_ppd_staging select t, si, i, b, f, d, bo, s, cast(s as char(50)), cast(s as varchar(50)), cast(ts as date), ts, dec, bin from staging order by t, s
+PREHOOK: type: QUERY
+PREHOOK: Input: default@staging
+PREHOOK: Output: default@orc_ppd_staging
+POSTHOOK: query: insert overwrite table orc_ppd_staging select t, si, i, b, f, d, bo, s, cast(s as char(50)), cast(s as varchar(50)), cast(ts as date), ts, dec, bin from staging order by t, s
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@staging
+POSTHOOK: Output: default@orc_ppd_staging
+POSTHOOK: Lineage: orc_ppd_staging.b SIMPLE [(staging)staging.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.bin SIMPLE [(staging)staging.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.bo SIMPLE [(staging)staging.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.c EXPRESSION [(staging)staging.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.d SIMPLE [(staging)staging.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.da EXPRESSION [(staging)staging.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.dec SIMPLE [(staging)staging.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.f SIMPLE [(staging)staging.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.i SIMPLE [(staging)staging.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.s SIMPLE [(staging)staging.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.si SIMPLE [(staging)staging.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.t SIMPLE [(staging)staging.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.ts SIMPLE [(staging)staging.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: orc_ppd_staging.v EXPRESSION [(staging)staging.FieldSchema(name:s, type:string, comment:null), ]
+PREHOOK: query: -- just to introduce a gap in min/max range for bloom filters. The dataset has contiguous values
+-- which makes it hard to test bloom filters
+insert into orc_ppd_staging select -10,-321,-65680,-4294967430,-97.94,-13.07,true,"aaa","aaa","aaa","1990-03-11","1990-03-11 10:11:58.703308",-71.54,"aaa" from staging limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@staging
+PREHOOK: Output: default@orc_ppd_staging
+POSTHOOK: query: -- just to introduce a gap in min/max range for bloom filters. The dataset has contiguous values
+-- which makes it hard to test bloom filters
+insert into orc_ppd_staging select -10,-321,-65680,-4294967430,-97.94,-13.07,true,"aaa","aaa","aaa","1990-03-11","1990-03-11 10:11:58.703308",-71.54,"aaa" from staging limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@staging
+POSTHOOK: Output: default@orc_ppd_staging
+POSTHOOK: Lineage: orc_ppd_staging.b EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.bin EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.bo SIMPLE []
+POSTHOOK: Lineage: orc_ppd_staging.c EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.d EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.da EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.dec EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.f EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.i EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.s SIMPLE []
+POSTHOOK: Lineage: orc_ppd_staging.si EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.t EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.ts EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.v EXPRESSION []
+PREHOOK: query: insert into orc_ppd_staging select 127,331,65690,4294967440,107.94,23.07,true,"zzz","zzz","zzz","2023-03-11","2023-03-11 10:11:58.703308",71.54,"zzz" from staging limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@staging
+PREHOOK: Output: default@orc_ppd_staging
+POSTHOOK: query: insert into orc_ppd_staging select 127,331,65690,4294967440,107.94,23.07,true,"zzz","zzz","zzz","2023-03-11","2023-03-11 10:11:58.703308",71.54,"zzz" from staging limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@staging
+POSTHOOK: Output: default@orc_ppd_staging
+POSTHOOK: Lineage: orc_ppd_staging.b SIMPLE []
+POSTHOOK: Lineage: orc_ppd_staging.bin EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.bo SIMPLE []
+POSTHOOK: Lineage: orc_ppd_staging.c EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.d SIMPLE []
+POSTHOOK: Lineage: orc_ppd_staging.da EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.dec EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.f EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.i SIMPLE []
+POSTHOOK: Lineage: orc_ppd_staging.s SIMPLE []
+POSTHOOK: Lineage: orc_ppd_staging.si EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.t EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.ts EXPRESSION []
+POSTHOOK: Lineage: orc_ppd_staging.v EXPRESSION []
+PREHOOK: query: CREATE TABLE orc_ppd(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           c char(50),
+           v varchar(50),
+           da date,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+STORED AS ORC tblproperties("orc.row.index.stride" = "1000", "orc.bloom.filter.columns"="*")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@orc_ppd
+POSTHOOK: query: CREATE TABLE orc_ppd(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           c char(50),
+           v varchar(50),
+           da date,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+STORED AS ORC tblproperties("orc.row.index.stride" = "1000", "orc.bloom.filter.columns"="*")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@orc_ppd
+PREHOOK: query: insert overwrite table orc_ppd select t, si, i, b, f, d, bo, s, cast(s as char(50)), cast(s as varchar(50)), cast(ts as date), ts, dec, bin from orc_ppd_staging order by t, s
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd_staging
+PREHOOK: Output: default@orc_ppd
+POSTHOOK: query: insert overwrite table orc_ppd select t, si, i, b, f, d, bo, s, cast(s as char(50)), cast(s as varchar(50)), cast(ts as date), ts, dec, bin from orc_ppd_staging order by t, s
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_ppd_staging
+POSTHOOK: Output: default@orc_ppd
+POSTHOOK: Lineage: orc_ppd.b SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.bin SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.bo SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.c EXPRESSION [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.d SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.da EXPRESSION [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.dec SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: orc_ppd.f SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.i SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.s SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.si SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.t SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.ts SIMPLE [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: orc_ppd.v EXPRESSION [(orc_ppd_staging)orc_ppd_staging.FieldSchema(name:s, type:string, comment:null), ]
+PREHOOK: query: -- Row group statistics for column t:
+-- Entry 0: count: 994 hasNull: true min: -10 max: 54 sum: 26014 positions: 0,0,0,0,0,0,0
+-- Entry 1: count: 1000 hasNull: false min: 54 max: 118 sum: 86812 positions: 0,2,124,0,0,116,11
+-- Entry 2: count: 100 hasNull: false min: 118 max: 127 sum: 12151 positions: 0,4,119,0,0,244,19
+
+-- INPUT_RECORDS: 2100 (all row groups)
+select count(*) from orc_ppd
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 2100
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+2100
+PREHOOK: query: -- INPUT_RECORDS: 0 (no row groups)
+select count(*) from orc_ppd where t > 127
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   RECORDS_OUT_0: 1
+0
+PREHOOK: query: -- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t = 55
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+8
+PREHOOK: query: select count(*) from orc_ppd where t <=> 50
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+22
+PREHOOK: query: select count(*) from orc_ppd where t <=> 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+16
+PREHOOK: query: -- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where t = "54"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 2000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+18
+PREHOOK: query: -- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t = -10.0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+1
+PREHOOK: query: -- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t = cast(53 as float)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+32
+PREHOOK: query: select count(*) from orc_ppd where t = cast(53 as double)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+32
+PREHOOK: query: -- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where t < 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 2000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+1697
+PREHOOK: query: -- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t < 100 and t > 98
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+12
+PREHOOK: query: -- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where t <= 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 2000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+1713
+PREHOOK: query: -- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+6
+PREHOOK: query: -- INPUT_RECORDS: 1100 (2 row groups)
+select count(*) from orc_ppd where t in (5, 120)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1100
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+50
+PREHOOK: query: -- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where t between 60 and 80
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+318
+PREHOOK: query: -- bloom filter tests
+-- INPUT_RECORDS: 0
+select count(*) from orc_ppd where t = -100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   RECORDS_OUT_0: 1
+0
+PREHOOK: query: select count(*) from orc_ppd where t <=> -100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   RECORDS_OUT_0: 1
+0
+PREHOOK: query: select count(*) from orc_ppd where t = 125
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 0
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+0
+PREHOOK: query: select count(*) from orc_ppd where t IN (-100, 125, 200)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 0
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+0
+PREHOOK: query: -- Row group statistics for column s:
+-- Entry 0: count: 1000 hasNull: false min:  max: zach young sum: 12907 positions: 0,0,0
+-- Entry 1: count: 1000 hasNull: false min: alice allen max: zach zipper sum: 12704 positions: 0,1611,191
+-- Entry 2: count: 100 hasNull: false min: bob davidson max: zzz sum: 1281 positions: 0,3246,373
+
+-- INPUT_RECORDS: 0 (no row groups)
+select count(*) from orc_ppd where s > "zzz"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   RECORDS_OUT_0: 1
+0
+PREHOOK: query: -- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where s = "zach young"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+2
+PREHOOK: query: select count(*) from orc_ppd where s <=> "zach zipper"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+6
+PREHOOK: query: select count(*) from orc_ppd where s <=> ""
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+6
+PREHOOK: query: -- INPUT_RECORDS: 0
+select count(*) from orc_ppd where s is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   RECORDS_OUT_0: 1
+0
+PREHOOK: query: -- INPUT_RECORDS: 2100
+select count(*) from orc_ppd where s is not null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 2100
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+2100
+PREHOOK: query: -- INPUT_RECORDS: 0
+select count(*) from orc_ppd where s = cast("zach young" as char(50))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 0
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+0
+PREHOOK: query: -- INPUT_RECORDS: 1000 (1 row group)
+select count(*) from orc_ppd where s = cast("zach young" as char(10))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+2
+PREHOOK: query: select count(*) from orc_ppd where s = cast("zach young" as varchar(10))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+2
+PREHOOK: query: select count(*) from orc_ppd where s = cast("zach young" as varchar(50))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+2
+PREHOOK: query: -- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where s < "b"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 2000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+81
+PREHOOK: query: -- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where s > "alice" and s < "bob"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 2000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+74
+PREHOOK: query: -- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where s in ("alice allen", "")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 2000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+12
+PREHOOK: query: -- INPUT_RECORDS: 2000 (2 row groups)
+select count(*) from orc_ppd where s between "" and "alice allen"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 2000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+13
+PREHOOK: query: -- INPUT_RECORDS: 100 (1 row group)
+select count(*) from orc_ppd where s between "zz" and "zzz"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 100
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+1
+PREHOOK: query: -- INPUT_RECORDS: 1100 (2 row groups)
+select count(*) from orc_ppd where s between "zach zipper" and "zzz"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1100
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+7
+PREHOOK: query: -- bloom filter tests
+-- INPUT_RECORDS: 0
+select count(*) from orc_ppd where s = "hello world"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 0
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+0
+PREHOOK: query: select count(*) from orc_ppd where s <=> "apache hive"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 0
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+0
+PREHOOK: query: select count(*) from orc_ppd where s IN ("a", "z")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 0
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+0
+PREHOOK: query: -- INPUT_RECORDS: 100
+select count(*) from orc_ppd where s = "sarah ovid"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 100
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+2
+PREHOOK: query: -- INPUT_RECORDS: 1100
+select count(*) from orc_ppd where s = "wendy king"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1100
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+6
+PREHOOK: query: -- INPUT_RECORDS: 1000
+select count(*) from orc_ppd where s = "wendy king" and t < 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 1000
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+2
+PREHOOK: query: -- INPUT_RECORDS: 100
+select count(*) from orc_ppd where s = "wendy king" and t > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_ppd
+#### A masked pattern was here ####
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 100
+   RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Map_1: 1
+2