DRILL-4264: Allow field names to include dots

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

Branch: refs/heads/master
Commit: d105950a7a9fb2ff3acd072ee65a51ef1fca120e
Parents: e57514a
Author: Volodymyr Vysotskyi <vvo...@gmail.com>
Authored: Wed Jul 5 19:08:59 2017 +0000
Committer: Aman Sinha <asi...@maprtech.com>
Committed: Mon Sep 4 19:23:49 2017 -0700

----------------------------------------------------------------------
 .../db/binary/CompareFunctionsProcessor.java    | 547 -------------------
 .../mapr/db/binary/MapRDBFilterBuilder.java     |  12 +-
 .../binary/MaprDBCompareFunctionsProcessor.java |  65 +++
 .../mapr/db/json/JsonConditionBuilder.java      |  28 +-
 .../store/hbase/CompareFunctionsProcessor.java  | 461 ++++++++--------
 .../exec/store/hbase/HBaseFilterBuilder.java    |   6 +-
 ...onvertHiveParquetScanToDrillParquetScan.java |   4 +-
 .../hive/HiveDrillNativeScanBatchCreator.java   |   6 +-
 .../drill/exec/store/kudu/KuduRecordReader.java |   6 +-
 .../exec/store/kudu/KuduRecordWriterImpl.java   |   4 +-
 .../exec/store/mongo/MongoFilterBuilder.java    |   4 +-
 .../templates/EventBasedRecordWriter.java       |   6 +-
 .../templates/StringOutputRecordWriter.java     |   4 +-
 .../org/apache/drill/exec/client/DumpCat.java   |  26 +-
 .../drill/exec/physical/impl/ScanBatch.java     |  12 +-
 .../impl/TopN/PriorityQueueTemplate.java        |   2 +-
 .../impl/aggregate/StreamingAggBatch.java       |  12 +-
 .../physical/impl/common/ChainedHashTable.java  |   6 +-
 .../impl/flatten/FlattenRecordBatch.java        |  11 +-
 .../exec/physical/impl/join/HashJoinBatch.java  |   2 +-
 .../exec/physical/impl/join/MergeJoinBatch.java |   4 +-
 .../physical/impl/join/NestedLoopJoinBatch.java |   4 +-
 .../OrderedPartitionRecordBatch.java            |   5 +-
 .../impl/producer/ProducerConsumerBatch.java    |   6 +-
 .../impl/project/ProjectRecordBatch.java        |  70 +--
 .../impl/union/UnionAllRecordBatch.java         |  95 ++--
 .../physical/impl/xsort/ExternalSortBatch.java  |   2 +-
 .../managed/PriorityQueueCopierWrapper.java     |   2 +-
 .../logical/partition/PruneScanRule.java        |   6 +-
 .../drill/exec/record/RecordBatchLoader.java    |   6 +-
 .../apache/drill/exec/record/SchemaUtil.java    |  21 +-
 .../drill/exec/record/VectorContainer.java      |  12 +-
 .../exec/server/rest/WebUserConnection.java     |   2 +-
 .../apache/drill/exec/store/ColumnExplorer.java |   4 +-
 .../store/ischema/InfoSchemaFilterBuilder.java  |   6 +-
 .../drill/exec/store/parquet/Metadata.java      |  44 +-
 .../exec/store/parquet/MetadataVersion.java     |  10 +-
 .../exec/store/parquet/ParquetRecordWriter.java |   6 +-
 .../columnreaders/ParquetColumnMetadata.java    |   6 +-
 .../parquet/columnreaders/ParquetSchema.java    |  10 +-
 .../exec/store/parquet2/DrillParquetReader.java |  19 +-
 .../apache/drill/exec/util/BatchPrinter.java    |  35 +-
 .../org/apache/drill/exec/util/VectorUtil.java  |  14 +-
 .../drill/exec/vector/complex/FieldIdUtil.java  |   4 +-
 .../drill/exec/vector/complex/MapUtility.java   |   4 +-
 .../java/org/apache/drill/BaseTestQuery.java    |   5 +
 .../java/org/apache/drill/DrillTestWrapper.java |  29 +-
 .../java/org/apache/drill/PlanTestBase.java     |   4 +-
 .../java/org/apache/drill/TestUnionAll.java     |  42 +-
 .../expr/TestSchemaPathMaterialization.java     |  21 +-
 .../exec/fn/impl/TestAggregateFunctions.java    |  33 +-
 .../exec/physical/impl/TestOptiqPlans.java      |  12 +-
 .../physical/impl/TestSimpleFragmentRun.java    |   8 +-
 .../impl/join/TestHashJoinAdvanced.java         |  43 +-
 .../exec/physical/impl/join/TestMergeJoin.java  |  10 +-
 .../impl/mergereceiver/TestMergingReceiver.java |  14 +-
 .../physical/unit/MiniPlanUnitTestBase.java     |   5 +-
 .../exec/record/TestMaterializedField.java      |   7 +-
 .../drill/exec/record/vector/TestLoad.java      |  16 +-
 .../drill/exec/store/TestOutputMutator.java     |   4 +-
 .../store/parquet/ParquetResultListener.java    |  20 +-
 .../exec/store/parquet/TestParquetComplex.java  |  10 +-
 .../store/parquet/TestParquetMetadataCache.java |  93 +++-
 .../store/parquet/TestParquetPhysicalPlan.java  |   8 +-
 .../vector/complex/writer/TestJsonReader.java   |  29 +
 .../apache/drill/test/rowSet/RowSetPrinter.java |   2 +-
 .../drill/test/rowSet/test/RowSetTest.java      | 290 +++++-----
 .../metadata_directories.requires_replace.txt   |   3 +
 .../V3/metadata_table.requires_replace.txt      | 108 ++++
 .../V3/metadata_table_t1.requires_replace.txt   |  76 +++
 .../V3/metadata_table_t2.requires_replace.txt   |  76 +++
 .../metadata_directories.requires_replace.txt   |   3 +
 .../v3_1/metadata_table.requires_replace.txt    | 108 ++++
 .../v3_1/metadata_table_t1.requires_replace.txt |  76 +++
 .../v3_1/metadata_table_t2.requires_replace.txt |  76 +++
 ...ies_with_absolute_paths.requires_replace.txt |   3 -
 ...ble_with_absolute_paths.requires_replace.txt | 108 ----
 ..._with_absolute_paths_t1.requires_replace.txt |  76 ---
 ..._with_absolute_paths_t2.requires_replace.txt |  76 ---
 .../store/parquet/complex/baseline8.json        |  25 +-
 .../jdbc/impl/DrillColumnMetaDataList.java      |   6 +-
 .../drill/jdbc/DrillColumnMetaDataListTest.java |   6 +-
 .../codegen/templates/FixedValueVectors.java    |   3 +-
 .../drill/exec/record/MaterializedField.java    |  41 +-
 .../drill/exec/vector/BaseValueVector.java      |   4 +-
 .../org/apache/drill/exec/vector/BitVector.java |   5 +-
 .../drill/exec/vector/VectorDescriptor.java     |   6 +-
 .../exec/vector/accessor/TupleAccessor.java     |  12 +
 .../drill/exec/vector/accessor/TupleReader.java |   7 +
 .../drill/exec/vector/accessor/TupleWriter.java |   7 +
 .../vector/complex/AbstractContainerVector.java |  14 +-
 .../exec/vector/complex/AbstractMapVector.java  |  10 +-
 .../drill/exec/vector/complex/ListVector.java   |   4 +-
 .../drill/exec/vector/complex/MapVector.java    |  12 +-
 .../exec/vector/complex/RepeatedMapVector.java  |  10 +-
 .../vector/complex/impl/PromotableWriter.java   |   4 +-
 .../drill/common/expression/FieldReference.java |  14 +-
 .../drill/common/expression/PathSegment.java    |  40 +-
 .../drill/common/expression/SchemaPath.java     |  87 ++-
 99 files changed, 1710 insertions(+), 1722 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/CompareFunctionsProcessor.java
----------------------------------------------------------------------
diff --git 
a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/CompareFunctionsProcessor.java
 
b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/CompareFunctionsProcessor.java
deleted file mode 100644
index a83abf3..0000000
--- 
a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/CompareFunctionsProcessor.java
+++ /dev/null
@@ -1,547 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mapr.db.binary;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-
-import org.apache.drill.common.expression.CastExpression;
-import org.apache.drill.common.expression.ConvertExpression;
-import org.apache.drill.common.expression.FunctionCall;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.ValueExpressions.BooleanExpression;
-import org.apache.drill.common.expression.ValueExpressions.DateExpression;
-import org.apache.drill.common.expression.ValueExpressions.DoubleExpression;
-import org.apache.drill.common.expression.ValueExpressions.FloatExpression;
-import org.apache.drill.common.expression.ValueExpressions.IntExpression;
-import org.apache.drill.common.expression.ValueExpressions.LongExpression;
-import org.apache.drill.common.expression.ValueExpressions.QuotedString;
-import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
-import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
-import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
-import org.apache.hadoop.hbase.util.Order;
-import org.apache.hadoop.hbase.util.PositionedByteRange;
-import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange;
-
-import org.apache.drill.exec.store.hbase.DrillHBaseConstants;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.PrefixFilter;
-
-import com.google.common.base.Charsets;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-
-class CompareFunctionsProcessor extends AbstractExprVisitor<Boolean, 
LogicalExpression, RuntimeException> {
-  private byte[] value;
-  private boolean success;
-  private boolean isEqualityFn;
-  private SchemaPath path;
-  private String functionName;
-  private boolean sortOrderAscending;
-
-  // Fields for row-key prefix comparison
-  // If the query is on row-key prefix, we cannot use a standard template to 
identify startRow, stopRow and filter
-  // Hence, we use these local variables(set depending upon the encoding type 
in user query)
-  private boolean isRowKeyPrefixComparison;
-  byte[] rowKeyPrefixStartRow;
-  byte[] rowKeyPrefixStopRow;
-  Filter rowKeyPrefixFilter;
-
-  public static boolean isCompareFunction(String functionName) {
-    return COMPARE_FUNCTIONS_TRANSPOSE_MAP.keySet().contains(functionName);
-  }
-
-  public static CompareFunctionsProcessor process(FunctionCall call, boolean 
nullComparatorSupported) {
-    String functionName = call.getName();
-    LogicalExpression nameArg = call.args.get(0);
-    LogicalExpression valueArg = call.args.size() >= 2 ? call.args.get(1) : 
null;
-    CompareFunctionsProcessor evaluator = new 
CompareFunctionsProcessor(functionName);
-
-    if (valueArg != null) { // binary function
-      if (VALUE_EXPRESSION_CLASSES.contains(nameArg.getClass())) {
-        LogicalExpression swapArg = valueArg;
-        valueArg = nameArg;
-        nameArg = swapArg;
-        evaluator.functionName = 
COMPARE_FUNCTIONS_TRANSPOSE_MAP.get(functionName);
-      }
-      evaluator.success = nameArg.accept(evaluator, valueArg);
-    } else if (nullComparatorSupported && call.args.get(0) instanceof 
SchemaPath) {
-      evaluator.success = true;
-      evaluator.path = (SchemaPath) nameArg;
-    }
-
-    return evaluator;
-  }
-
-  public CompareFunctionsProcessor(String functionName) {
-    this.success = false;
-    this.functionName = functionName;
-    this.isEqualityFn = 
COMPARE_FUNCTIONS_TRANSPOSE_MAP.containsKey(functionName)
-        && 
COMPARE_FUNCTIONS_TRANSPOSE_MAP.get(functionName).equals(functionName);
-    this.isRowKeyPrefixComparison = false;
-    this.sortOrderAscending = true;
-  }
-
-  public byte[] getValue() {
-    return value;
-  }
-
-  public boolean isSuccess() {
-    return success;
-  }
-
-  public SchemaPath getPath() {
-    return path;
-  }
-
-  public String getFunctionName() {
-    return functionName;
-  }
-
-  public boolean isRowKeyPrefixComparison() {
-       return isRowKeyPrefixComparison;
-  }
-
-  public byte[] getRowKeyPrefixStartRow() {
-    return rowKeyPrefixStartRow;
-  }
-
-  public byte[] getRowKeyPrefixStopRow() {
-  return rowKeyPrefixStopRow;
-  }
-
-  public Filter getRowKeyPrefixFilter() {
-  return rowKeyPrefixFilter;
-  }
-
-  public boolean isSortOrderAscending() {
-    return sortOrderAscending;
-  }
-
-  @Override
-  public Boolean visitCastExpression(CastExpression e, LogicalExpression 
valueArg) throws RuntimeException {
-    if (e.getInput() instanceof CastExpression || e.getInput() instanceof 
SchemaPath) {
-      return e.getInput().accept(this, valueArg);
-    }
-    return false;
-  }
-
-  @Override
-  public Boolean visitConvertExpression(ConvertExpression e, LogicalExpression 
valueArg) throws RuntimeException {
-    if (e.getConvertFunction() == ConvertExpression.CONVERT_FROM) {
-
-      String encodingType = e.getEncodingType();
-      int prefixLength    = 0;
-
-      // Handle scan pruning in the following scenario:
-      // The row-key is a composite key and the CONVERT_FROM() function has 
byte_substr() as input function which is
-      // querying for the first few bytes of the row-key(start-offset 1)
-      // Example WHERE clause:
-      // CONVERT_FROM(BYTE_SUBSTR(row_key, 1, 8), 'DATE_EPOCH_BE') < DATE 
'2015-06-17'
-      if (e.getInput() instanceof FunctionCall) {
-
-        // We can prune scan range only for big-endian encoded data
-        if (encodingType.endsWith("_BE") == false) {
-          return false;
-        }
-
-        FunctionCall call = (FunctionCall)e.getInput();
-        String functionName = call.getName();
-        if (!functionName.equalsIgnoreCase("byte_substr")) {
-          return false;
-        }
-
-        LogicalExpression nameArg = call.args.get(0);
-        LogicalExpression valueArg1 = call.args.size() >= 2 ? call.args.get(1) 
: null;
-        LogicalExpression valueArg2 = call.args.size() >= 3 ? call.args.get(2) 
: null;
-
-        if (((nameArg instanceof SchemaPath) == false) ||
-             (valueArg1 == null) || ((valueArg1 instanceof IntExpression) == 
false) ||
-             (valueArg2 == null) || ((valueArg2 instanceof IntExpression) == 
false)) {
-          return false;
-        }
-
-        boolean isRowKey = 
((SchemaPath)nameArg).getAsUnescapedPath().equals(DrillHBaseConstants.ROW_KEY);
-        int offset = ((IntExpression)valueArg1).getInt();
-
-        if (!isRowKey || (offset != 1)) {
-          return false;
-        }
-
-        this.path    = (SchemaPath)nameArg;
-        prefixLength = ((IntExpression)valueArg2).getInt();
-        this.isRowKeyPrefixComparison = true;
-        return visitRowKeyPrefixConvertExpression(e, prefixLength, valueArg);
-      }
-
-      if (e.getInput() instanceof SchemaPath) {
-        ByteBuf bb = null;
-
-        switch (encodingType) {
-        case "INT_BE":
-        case "INT":
-        case "UINT_BE":
-        case "UINT":
-        case "UINT4_BE":
-        case "UINT4":
-          if (valueArg instanceof IntExpression
-              && (isEqualityFn || encodingType.startsWith("U"))) {
-            bb = newByteBuf(4, encodingType.endsWith("_BE"));
-            bb.writeInt(((IntExpression)valueArg).getInt());
-          }
-          break;
-        case "BIGINT_BE":
-        case "BIGINT":
-        case "UINT8_BE":
-        case "UINT8":
-          if (valueArg instanceof LongExpression
-              && (isEqualityFn || encodingType.startsWith("U"))) {
-            bb = newByteBuf(8, encodingType.endsWith("_BE"));
-            bb.writeLong(((LongExpression)valueArg).getLong());
-          }
-          break;
-        case "FLOAT":
-          if (valueArg instanceof FloatExpression && isEqualityFn) {
-            bb = newByteBuf(4, true);
-            bb.writeFloat(((FloatExpression)valueArg).getFloat());
-          }
-          break;
-        case "DOUBLE":
-          if (valueArg instanceof DoubleExpression && isEqualityFn) {
-            bb = newByteBuf(8, true);
-            bb.writeDouble(((DoubleExpression)valueArg).getDouble());
-          }
-          break;
-        case "TIME_EPOCH":
-        case "TIME_EPOCH_BE":
-          if (valueArg instanceof TimeExpression) {
-            bb = newByteBuf(8, encodingType.endsWith("_BE"));
-            bb.writeLong(((TimeExpression)valueArg).getTime());
-          }
-          break;
-        case "DATE_EPOCH":
-        case "DATE_EPOCH_BE":
-          if (valueArg instanceof DateExpression) {
-            bb = newByteBuf(8, encodingType.endsWith("_BE"));
-            bb.writeLong(((DateExpression)valueArg).getDate());
-          }
-          break;
-        case "BOOLEAN_BYTE":
-          if (valueArg instanceof BooleanExpression) {
-            bb = newByteBuf(1, false /* does not matter */);
-            bb.writeByte(((BooleanExpression)valueArg).getBoolean() ? 1 : 0);
-          }
-          break;
-        case "DOUBLE_OB":
-        case "DOUBLE_OBD":
-          if (valueArg instanceof DoubleExpression) {
-            bb = newByteBuf(9, true);
-            PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 9);
-            if (encodingType.endsWith("_OBD")) {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat64(br,
-                  ((DoubleExpression)valueArg).getDouble(), Order.DESCENDING);
-              this.sortOrderAscending = false;
-            } else {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat64(br,
-                  ((DoubleExpression)valueArg).getDouble(), Order.ASCENDING);
-            }
-          }
-          break;
-        case "FLOAT_OB":
-        case "FLOAT_OBD":
-          if (valueArg instanceof FloatExpression) {
-            bb = newByteBuf(5, true);
-            PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 5);
-            if (encodingType.endsWith("_OBD")) {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat32(br,
-                  ((FloatExpression)valueArg).getFloat(), Order.DESCENDING);
-              this.sortOrderAscending = false;
-            } else {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat32(br,
-                        ((FloatExpression)valueArg).getFloat(), 
Order.ASCENDING);
-            }
-          }
-          break;
-        case "BIGINT_OB":
-        case "BIGINT_OBD":
-          if (valueArg instanceof LongExpression) {
-            bb = newByteBuf(9, true);
-            PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 9);
-            if (encodingType.endsWith("_OBD")) {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeInt64(br,
-                        ((LongExpression)valueArg).getLong(), 
Order.DESCENDING);
-              this.sortOrderAscending = false;
-            } else {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeInt64(br,
-                  ((LongExpression)valueArg).getLong(), Order.ASCENDING);
-            }
-          }
-          break;
-        case "INT_OB":
-        case "INT_OBD":
-          if (valueArg instanceof IntExpression) {
-            bb = newByteBuf(5, true);
-            PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 5);
-            if (encodingType.endsWith("_OBD")) {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeInt32(br,
-                  ((IntExpression)valueArg).getInt(), Order.DESCENDING);
-              this.sortOrderAscending = false;
-            } else {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeInt32(br,
-                        ((IntExpression)valueArg).getInt(), Order.ASCENDING);
-            }
-          }
-          break;
-        case "UTF8_OB":
-        case "UTF8_OBD":
-          if (valueArg instanceof QuotedString) {
-            int stringLen = ((QuotedString) 
valueArg).value.getBytes(Charsets.UTF_8).length;
-            bb = newByteBuf(stringLen + 2, true);
-            PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, stringLen + 2);
-            if (encodingType.endsWith("_OBD")) {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeString(br,
-                  ((QuotedString)valueArg).value, Order.DESCENDING);
-              this.sortOrderAscending = false;
-            } else {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeString(br,
-                        ((QuotedString)valueArg).value, Order.ASCENDING);
-            }
-          }
-          break;
-        case "UTF8":
-        // let visitSchemaPath() handle this.
-          return e.getInput().accept(this, valueArg);
-        }
-
-        if (bb != null) {
-          this.value = bb.array();
-          this.path = (SchemaPath)e.getInput();
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  private Boolean visitRowKeyPrefixConvertExpression(ConvertExpression e,
-    int prefixLength, LogicalExpression valueArg) {
-    String encodingType = e.getEncodingType();
-    rowKeyPrefixStartRow = HConstants.EMPTY_START_ROW;
-    rowKeyPrefixStopRow  = HConstants.EMPTY_START_ROW;
-    rowKeyPrefixFilter   = null;
-
-    if ((encodingType.compareTo("UINT4_BE") == 0) ||
-        (encodingType.compareTo("UINT_BE") == 0)) {
-      if (prefixLength != 4) {
-        throw new RuntimeException("Invalid length(" + prefixLength + ") of 
row-key prefix");
-      }
-
-      int val;
-      if ((valueArg instanceof IntExpression) == false) {
-        return false;
-      }
-
-      val = ((IntExpression)valueArg).getInt();
-
-      // For TIME_EPOCH_BE/BIGINT_BE encoding, the operators that we push-down 
are =, <>, <, <=, >, >=
-      switch (functionName) {
-      case "equal":
-        rowKeyPrefixFilter = new 
PrefixFilter(ByteBuffer.allocate(4).putInt(val).array());
-        rowKeyPrefixStartRow = ByteBuffer.allocate(4).putInt(val).array();
-        rowKeyPrefixStopRow = ByteBuffer.allocate(4).putInt(val + 1).array();
-        return true;
-      case "greater_than_or_equal_to":
-        rowKeyPrefixStartRow = ByteBuffer.allocate(4).putInt(val).array();
-        return true;
-      case "greater_than":
-        rowKeyPrefixStartRow = ByteBuffer.allocate(4).putInt(val + 1).array();
-        return true;
-      case "less_than_or_equal_to":
-        rowKeyPrefixStopRow = ByteBuffer.allocate(4).putInt(val + 1).array();
-        return true;
-      case "less_than":
-        rowKeyPrefixStopRow = ByteBuffer.allocate(4).putInt(val).array();
-        return true;
-      }
-
-      return false;
-    }
-
-    if ((encodingType.compareTo("TIMESTAMP_EPOCH_BE") == 0) ||
-        (encodingType.compareTo("TIME_EPOCH_BE") == 0) ||
-        (encodingType.compareTo("UINT8_BE") == 0)) {
-
-      if (prefixLength != 8) {
-        throw new RuntimeException("Invalid length(" + prefixLength + ") of 
row-key prefix");
-      }
-
-      long val;
-      if (encodingType.compareTo("TIME_EPOCH_BE") == 0) {
-        if ((valueArg instanceof TimeExpression) == false) {
-          return false;
-        }
-
-        val = ((TimeExpression)valueArg).getTime();
-      } else if (encodingType.compareTo("UINT8_BE") == 0){
-        if ((valueArg instanceof LongExpression) == false) {
-          return false;
-        }
-
-        val = ((LongExpression)valueArg).getLong();
-      } else if (encodingType.compareTo("TIMESTAMP_EPOCH_BE") == 0) {
-        if ((valueArg instanceof TimeStampExpression) == false) {
-          return false;
-        }
-
-        val = ((TimeStampExpression)valueArg).getTimeStamp();
-      } else {
-        // Should not reach here.
-        return false;
-      }
-
-      // For TIME_EPOCH_BE/BIGINT_BE encoding, the operators that we push-down 
are =, <>, <, <=, >, >=
-      switch (functionName) {
-      case "equal":
-        rowKeyPrefixFilter = new 
PrefixFilter(ByteBuffer.allocate(8).putLong(val).array());
-        rowKeyPrefixStartRow = ByteBuffer.allocate(8).putLong(val).array();
-        rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(val + 1).array();
-        return true;
-      case "greater_than_or_equal_to":
-        rowKeyPrefixStartRow = ByteBuffer.allocate(8).putLong(val).array();
-        return true;
-      case "greater_than":
-        rowKeyPrefixStartRow = ByteBuffer.allocate(8).putLong(val + 1).array();
-        return true;
-      case "less_than_or_equal_to":
-        rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(val + 1).array();
-        return true;
-      case "less_than":
-        rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(val).array();
-        return true;
-      }
-
-      return false;
-    }
-
-    if (encodingType.compareTo("DATE_EPOCH_BE") == 0) {
-      if ((valueArg instanceof DateExpression) == false) {
-        return false;
-      }
-
-      if (prefixLength != 8) {
-        throw new RuntimeException("Invalid length(" + prefixLength + ") of 
row-key prefix");
-      }
-
-      final long MILLISECONDS_IN_A_DAY  = (long)1000 * 60 * 60 * 24;
-      long dateToSet;
-      // For DATE encoding, the operators that we push-down are =, <>, <, <=, 
>, >=
-      switch (functionName) {
-      case "equal":
-        long startDate = ((DateExpression)valueArg).getDate();
-        rowKeyPrefixStartRow = 
ByteBuffer.allocate(8).putLong(startDate).array();
-        long stopDate  = ((DateExpression)valueArg).getDate() + 
MILLISECONDS_IN_A_DAY;
-        rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(stopDate).array();
-        return true;
-      case "greater_than_or_equal_to":
-        dateToSet = ((DateExpression)valueArg).getDate();
-        rowKeyPrefixStartRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
-        return true;
-      case "greater_than":
-        dateToSet = ((DateExpression)valueArg).getDate() + 
MILLISECONDS_IN_A_DAY;
-        rowKeyPrefixStartRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
-        return true;
-      case "less_than_or_equal_to":
-        dateToSet = ((DateExpression)valueArg).getDate() + 
MILLISECONDS_IN_A_DAY;
-        rowKeyPrefixStopRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
-        return true;
-      case "less_than":
-        dateToSet = ((DateExpression)valueArg).getDate();
-        rowKeyPrefixStopRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
-        return true;
-      }
-
-      return false;
-    }
-
-    return false;
-  }
-
-  @Override
-  public Boolean visitUnknown(LogicalExpression e, LogicalExpression valueArg) 
throws RuntimeException {
-    return false;
-  }
-
-  @Override
-  public Boolean visitSchemaPath(SchemaPath path, LogicalExpression valueArg) 
throws RuntimeException {
-    if (valueArg instanceof QuotedString) {
-      this.value = ((QuotedString) valueArg).value.getBytes(Charsets.UTF_8);
-      this.path = path;
-      return true;
-    }
-    return false;
-  }
-
-  private static ByteBuf newByteBuf(int size, boolean bigEndian) {
-    return Unpooled.wrappedBuffer(new byte[size])
-        .order(bigEndian ? ByteOrder.BIG_ENDIAN : ByteOrder.LITTLE_ENDIAN)
-        .writerIndex(0);
-  }
-
-  private static final ImmutableSet<Class<? extends LogicalExpression>> 
VALUE_EXPRESSION_CLASSES;
-  static {
-    ImmutableSet.Builder<Class<? extends LogicalExpression>> builder = 
ImmutableSet.builder();
-    VALUE_EXPRESSION_CLASSES = builder
-        .add(BooleanExpression.class)
-        .add(DateExpression.class)
-        .add(DoubleExpression.class)
-        .add(FloatExpression.class)
-        .add(IntExpression.class)
-        .add(LongExpression.class)
-        .add(QuotedString.class)
-        .add(TimeExpression.class)
-        .build();
-  }
-
-  private static final ImmutableMap<String, String> 
COMPARE_FUNCTIONS_TRANSPOSE_MAP;
-  static {
-    ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
-    COMPARE_FUNCTIONS_TRANSPOSE_MAP = builder
-        // unary functions
-        .put("isnotnull", "isnotnull")
-        .put("isNotNull", "isNotNull")
-        .put("is not null", "is not null")
-        .put("isnull", "isnull")
-        .put("isNull", "isNull")
-        .put("is null", "is null")
-        // binary functions
-        .put("like", "like")
-        .put("equal", "equal")
-        .put("not_equal", "not_equal")
-        .put("greater_than_or_equal_to", "less_than_or_equal_to")
-        .put("greater_than", "less_than")
-        .put("less_than_or_equal_to", "greater_than_or_equal_to")
-        .put("less_than", "greater_than")
-        .build();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MapRDBFilterBuilder.java
----------------------------------------------------------------------
diff --git 
a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MapRDBFilterBuilder.java
 
b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MapRDBFilterBuilder.java
index 3aba1e7..240d6b0 100644
--- 
a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MapRDBFilterBuilder.java
+++ 
b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MapRDBFilterBuilder.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -95,7 +95,7 @@ public class MapRDBFilterBuilder extends 
AbstractExprVisitor<HBaseScanSpec, Void
     String functionName = call.getName();
     ImmutableList<LogicalExpression> args = call.args;
 
-    if (CompareFunctionsProcessor.isCompareFunction(functionName)) {
+    if (MaprDBCompareFunctionsProcessor.isCompareFunction(functionName)) {
       /*
        * HBASE-10848: Bug in HBase versions (0.94.[0-18], 0.96.[0-2], 
0.98.[0-1])
        * causes a filter with NullComparator to fail. Enable only if specified 
in
@@ -105,7 +105,7 @@ public class MapRDBFilterBuilder extends 
AbstractExprVisitor<HBaseScanSpec, Void
         nullComparatorSupported = 
groupScan.getHBaseConf().getBoolean("drill.hbase.supports.null.comparator", 
false);
       }
 
-      CompareFunctionsProcessor processor = 
CompareFunctionsProcessor.process(call, nullComparatorSupported);
+      MaprDBCompareFunctionsProcessor processor = 
MaprDBCompareFunctionsProcessor.createFunctionsProcessorInstance(call, 
nullComparatorSupported);
       if (processor.isSuccess()) {
         nodeScanSpec = createHBaseScanSpec(call, processor);
       }
@@ -156,12 +156,12 @@ public class MapRDBFilterBuilder extends 
AbstractExprVisitor<HBaseScanSpec, Void
     return new HBaseScanSpec(groupScan.getTableName(), startRow, stopRow, 
newFilter);
   }
 
-  private HBaseScanSpec createHBaseScanSpec(FunctionCall call, 
CompareFunctionsProcessor processor) {
+  private HBaseScanSpec createHBaseScanSpec(FunctionCall call, 
MaprDBCompareFunctionsProcessor processor) {
     String functionName = processor.getFunctionName();
     SchemaPath field = processor.getPath();
     byte[] fieldValue = processor.getValue();
     boolean sortOrderAscending = processor.isSortOrderAscending();
-    boolean isRowKey = field.getAsUnescapedPath().equals(ROW_KEY);
+    boolean isRowKey = field.getRootSegmentPath().equals(ROW_KEY);
     if (!(isRowKey
         || (!field.getRootSegment().isLastPath()
             && field.getRootSegment().getChild().isLastPath()
@@ -338,7 +338,7 @@ public class MapRDBFilterBuilder extends 
AbstractExprVisitor<HBaseScanSpec, Void
   }
 
   private HBaseScanSpec createRowKeyPrefixScanSpec(FunctionCall call,
-      CompareFunctionsProcessor processor) {
+      MaprDBCompareFunctionsProcessor processor) {
     byte[] startRow = processor.getRowKeyPrefixStartRow();
     byte[] stopRow  = processor.getRowKeyPrefixStopRow();
     Filter filter   = processor.getRowKeyPrefixFilter();

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MaprDBCompareFunctionsProcessor.java
----------------------------------------------------------------------
diff --git 
a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MaprDBCompareFunctionsProcessor.java
 
b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MaprDBCompareFunctionsProcessor.java
new file mode 100644
index 0000000..5e17c27
--- /dev/null
+++ 
b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MaprDBCompareFunctionsProcessor.java
@@ -0,0 +1,65 @@
+/*
+ * 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.drill.exec.store.mapr.db.binary;
+
+import com.google.common.base.Charsets;
+import io.netty.buffer.ByteBuf;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.exec.store.hbase.CompareFunctionsProcessor;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange;
+
+class MaprDBCompareFunctionsProcessor extends CompareFunctionsProcessor {
+
+  public MaprDBCompareFunctionsProcessor(String functionName) {
+    super(functionName);
+  }
+
+  public static MaprDBCompareFunctionsProcessor 
createFunctionsProcessorInstance(FunctionCall call, boolean 
nullComparatorSupported) {
+    String functionName = call.getName();
+    MaprDBCompareFunctionsProcessor evaluator = new 
MaprDBCompareFunctionsProcessor(functionName);
+
+    return createFunctionsProcessorInstanceInternal(call, 
nullComparatorSupported, evaluator);
+  }
+
+  @Override
+  protected ByteBuf getByteBuf(LogicalExpression valueArg, String 
encodingType) {
+    switch (encodingType) {
+      case "UTF8_OB":
+      case "UTF8_OBD":
+        if (valueArg instanceof ValueExpressions.QuotedString) {
+          int stringLen = ((ValueExpressions.QuotedString) 
valueArg).value.getBytes(Charsets.UTF_8).length;
+          ByteBuf bb = newByteBuf(stringLen + 2, true);
+          PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, stringLen + 2);
+          if (encodingType.endsWith("_OBD")) {
+            org.apache.hadoop.hbase.util.OrderedBytes.encodeString(br, 
((ValueExpressions.QuotedString) valueArg).value,
+                                                                  
Order.DESCENDING);
+            setSortOrderAscending(false);
+          } else {
+            org.apache.hadoop.hbase.util.OrderedBytes.encodeString(br, 
((ValueExpressions.QuotedString) valueArg).value,
+                                                                  
Order.ASCENDING);
+          }
+          return bb;
+        }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
----------------------------------------------------------------------
diff --git 
a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
 
b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
index 16802ad..e8a8b6e 100644
--- 
a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
+++ 
b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -166,66 +166,66 @@ public class JsonConditionBuilder extends 
AbstractExprVisitor<JsonScanSpec, Void
     switch (functionName) {
     case "equal":
       cond = MapRDB.newCondition();
-      setIsCondition(cond, field.getAsUnescapedPath(), Op.EQUAL, fieldValue);
+      setIsCondition(cond, field.getRootSegmentPath(), Op.EQUAL, fieldValue);
       cond.build();
       break;
 
     case "not_equal":
       cond = MapRDB.newCondition();
-      setIsCondition(cond, field.getAsUnescapedPath(), Op.NOT_EQUAL, 
fieldValue);
+      setIsCondition(cond, field.getRootSegmentPath(), Op.NOT_EQUAL, 
fieldValue);
       cond.build();
       break;
 
     case "less_than":
       cond = MapRDB.newCondition();
-      setIsCondition(cond, field.getAsUnescapedPath(), Op.LESS, fieldValue);
+      setIsCondition(cond, field.getRootSegmentPath(), Op.LESS, fieldValue);
       cond.build();
       break;
 
     case "less_than_or_equal_to":
       cond = MapRDB.newCondition();
-      setIsCondition(cond, field.getAsUnescapedPath(), Op.LESS_OR_EQUAL, 
fieldValue);
+      setIsCondition(cond, field.getRootSegmentPath(), Op.LESS_OR_EQUAL, 
fieldValue);
       cond.build();
       break;
 
     case "greater_than":
       cond = MapRDB.newCondition();
-      setIsCondition(cond, field.getAsUnescapedPath(), Op.GREATER, fieldValue);
+      setIsCondition(cond, field.getRootSegmentPath(), Op.GREATER, fieldValue);
       cond.build();
       break;
 
     case "greater_than_or_equal_to":
       cond = MapRDB.newCondition();
-      setIsCondition(cond, field.getAsUnescapedPath(), Op.GREATER_OR_EQUAL, 
fieldValue);
+      setIsCondition(cond, field.getRootSegmentPath(), Op.GREATER_OR_EQUAL, 
fieldValue);
       cond.build();
       break;
 
     case "isnull":
-      cond = 
MapRDB.newCondition().notExists(field.getAsUnescapedPath()).build();
+      cond = 
MapRDB.newCondition().notExists(field.getRootSegmentPath()).build();
       break;
 
     case "isnotnull":
-      cond = MapRDB.newCondition().exists(field.getAsUnescapedPath()).build();
+      cond = MapRDB.newCondition().exists(field.getRootSegmentPath()).build();
       break;
 
     case "istrue":
-      cond = MapRDB.newCondition().is(field.getAsUnescapedPath(), Op.EQUAL, 
true).build();
+      cond = MapRDB.newCondition().is(field.getRootSegmentPath(), Op.EQUAL, 
true).build();
       break;
 
     case "isnotfalse":
-      cond = MapRDB.newCondition().is(field.getAsUnescapedPath(), 
Op.NOT_EQUAL, false).build();
+      cond = MapRDB.newCondition().is(field.getRootSegmentPath(), 
Op.NOT_EQUAL, false).build();
       break;
 
     case "isfalse":
-      cond = MapRDB.newCondition().is(field.getAsUnescapedPath(), Op.EQUAL, 
false).build();
+      cond = MapRDB.newCondition().is(field.getRootSegmentPath(), Op.EQUAL, 
false).build();
       break;
 
     case "isnottrue":
-      cond = MapRDB.newCondition().is(field.getAsUnescapedPath(), 
Op.NOT_EQUAL, true).build();
+      cond = MapRDB.newCondition().is(field.getRootSegmentPath(), 
Op.NOT_EQUAL, true).build();
       break;
 
     case "like":
-      cond = MapRDB.newCondition().like(field.getAsUnescapedPath(), 
fieldValue.getString()).build();
+      cond = MapRDB.newCondition().like(field.getRootSegmentPath(), 
fieldValue.getString()).build();
       break;
 
     default:

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/CompareFunctionsProcessor.java
----------------------------------------------------------------------
diff --git 
a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/CompareFunctionsProcessor.java
 
b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/CompareFunctionsProcessor.java
index 09e7be7..0672b53 100644
--- 
a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/CompareFunctionsProcessor.java
+++ 
b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/CompareFunctionsProcessor.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -50,7 +50,7 @@ import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 
-class CompareFunctionsProcessor extends AbstractExprVisitor<Boolean, 
LogicalExpression, RuntimeException> {
+public class CompareFunctionsProcessor extends AbstractExprVisitor<Boolean, 
LogicalExpression, RuntimeException> {
   private byte[] value;
   private boolean success;
   private boolean isEqualityFn;
@@ -62,31 +62,37 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
   // If the query is on row-key prefix, we cannot use a standard template to 
identify startRow, stopRow and filter
   // Hence, we use these local variables(set depending upon the encoding type 
in user query)
   private boolean isRowKeyPrefixComparison;
-  byte[] rowKeyPrefixStartRow;
-  byte[] rowKeyPrefixStopRow;
-  Filter rowKeyPrefixFilter;
+  private byte[] rowKeyPrefixStartRow;
+  private byte[] rowKeyPrefixStopRow;
+  private Filter rowKeyPrefixFilter;
 
   public static boolean isCompareFunction(String functionName) {
     return COMPARE_FUNCTIONS_TRANSPOSE_MAP.keySet().contains(functionName);
   }
 
-  public static CompareFunctionsProcessor process(FunctionCall call, boolean 
nullComparatorSupported) {
+  public static CompareFunctionsProcessor 
createFunctionsProcessorInstance(FunctionCall call, boolean 
nullComparatorSupported) {
     String functionName = call.getName();
-    LogicalExpression nameArg = call.args.get(0);
-    LogicalExpression valueArg = call.args.size() >= 2 ? call.args.get(1) : 
null;
     CompareFunctionsProcessor evaluator = new 
CompareFunctionsProcessor(functionName);
 
+    return createFunctionsProcessorInstanceInternal(call, 
nullComparatorSupported, evaluator);
+  }
+
+  protected static <T extends CompareFunctionsProcessor> T 
createFunctionsProcessorInstanceInternal(FunctionCall call,
+                                                                               
                     boolean nullComparatorSupported,
+                                                                               
                     T evaluator) {
+    LogicalExpression nameArg = call.args.get(0);
+    LogicalExpression valueArg = call.args.size() >= 2 ? call.args.get(1) : 
null;
     if (valueArg != null) { // binary function
       if (VALUE_EXPRESSION_CLASSES.contains(nameArg.getClass())) {
         LogicalExpression swapArg = valueArg;
         valueArg = nameArg;
         nameArg = swapArg;
-        evaluator.functionName = 
COMPARE_FUNCTIONS_TRANSPOSE_MAP.get(functionName);
+        
evaluator.setFunctionName(COMPARE_FUNCTIONS_TRANSPOSE_MAP.get(evaluator.getFunctionName()));
       }
-      evaluator.success = nameArg.accept(evaluator, valueArg);
+      evaluator.setSuccess(nameArg.accept(evaluator, valueArg));
     } else if (nullComparatorSupported && call.args.get(0) instanceof 
SchemaPath) {
-      evaluator.success = true;
-      evaluator.path = (SchemaPath) nameArg;
+      evaluator.setSuccess(true);
+      evaluator.setPath((SchemaPath) nameArg);
     }
 
     return evaluator;
@@ -109,14 +115,26 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
     return success;
   }
 
+  protected void setSuccess(boolean success) {
+    this.success = success;
+  }
+
   public SchemaPath getPath() {
     return path;
   }
 
+  protected void setPath(SchemaPath path) {
+    this.path = path;
+  }
+
   public String getFunctionName() {
     return functionName;
   }
 
+  protected void setFunctionName(String functionName) {
+    this.functionName = functionName;
+  }
+
   public boolean isRowKeyPrefixComparison() {
   return isRowKeyPrefixComparison;
   }
@@ -137,6 +155,10 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
     return sortOrderAscending;
   }
 
+  protected void setSortOrderAscending(boolean sortOrderAscending) {
+    this.sortOrderAscending = sortOrderAscending;
+  }
+
   @Override
   public Boolean visitCastExpression(CastExpression e, LogicalExpression 
valueArg) throws RuntimeException {
     if (e.getInput() instanceof CastExpression || e.getInput() instanceof 
SchemaPath) {
@@ -147,10 +169,10 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
 
   @Override
   public Boolean visitConvertExpression(ConvertExpression e, LogicalExpression 
valueArg) throws RuntimeException {
-    if (e.getConvertFunction() == ConvertExpression.CONVERT_FROM) {
+    if (ConvertExpression.CONVERT_FROM.equals(e.getConvertFunction())) {
 
       String encodingType = e.getEncodingType();
-      int prefixLength    = 0;
+      int prefixLength;
 
       // Handle scan pruning in the following scenario:
       // The row-key is a composite key and the CONVERT_FROM() function has 
byte_substr() as input function which is
@@ -160,7 +182,7 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
       if (e.getInput() instanceof FunctionCall) {
 
         // We can prune scan range only for big-endian encoded data
-        if (encodingType.endsWith("_BE") == false) {
+        if (!encodingType.endsWith("_BE")) {
           return false;
         }
 
@@ -174,21 +196,21 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
         LogicalExpression valueArg1 = call.args.size() >= 2 ? call.args.get(1) 
: null;
         LogicalExpression valueArg2 = call.args.size() >= 3 ? call.args.get(2) 
: null;
 
-        if (((nameArg instanceof SchemaPath) == false) ||
-             (valueArg1 == null) || ((valueArg1 instanceof IntExpression) == 
false) ||
-             (valueArg2 == null) || ((valueArg2 instanceof IntExpression) == 
false)) {
+        if (!(nameArg instanceof SchemaPath)
+            || (valueArg1 == null) || !(valueArg1 instanceof IntExpression)
+            || (valueArg2 == null) || !(valueArg2 instanceof IntExpression)) {
           return false;
         }
 
-        boolean isRowKey = 
((SchemaPath)nameArg).getAsUnescapedPath().equals(DrillHBaseConstants.ROW_KEY);
-        int offset = ((IntExpression)valueArg1).getInt();
+        boolean isRowKey = ((SchemaPath) 
nameArg).getRootSegmentPath().equals(DrillHBaseConstants.ROW_KEY);
+        int offset = ((IntExpression) valueArg1).getInt();
 
-        if (!isRowKey || (offset != 1)) {
+        if (!isRowKey || offset != 1) {
           return false;
         }
 
-        this.path    = (SchemaPath)nameArg;
-        prefixLength = ((IntExpression)valueArg2).getInt();
+        this.path = (SchemaPath) nameArg;
+        prefixLength = ((IntExpression) valueArg2).getInt();
         this.isRowKeyPrefixComparison = true;
         return visitRowKeyPrefixConvertExpression(e, prefixLength, valueArg);
       }
@@ -196,123 +218,125 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
       if (e.getInput() instanceof SchemaPath) {
         ByteBuf bb = null;
         switch (encodingType) {
-        case "INT_BE":
-        case "INT":
-        case "UINT_BE":
-        case "UINT":
-        case "UINT4_BE":
-        case "UINT4":
-          if (valueArg instanceof IntExpression
-              && (isEqualityFn || encodingType.startsWith("U"))) {
-            bb = newByteBuf(4, encodingType.endsWith("_BE"));
-            bb.writeInt(((IntExpression)valueArg).getInt());
-          }
-          break;
-        case "BIGINT_BE":
-        case "BIGINT":
-        case "UINT8_BE":
-        case "UINT8":
-          if (valueArg instanceof LongExpression
-              && (isEqualityFn || encodingType.startsWith("U"))) {
-            bb = newByteBuf(8, encodingType.endsWith("_BE"));
-            bb.writeLong(((LongExpression)valueArg).getLong());
-          }
-          break;
-        case "FLOAT":
-          if (valueArg instanceof FloatExpression && isEqualityFn) {
-          bb = newByteBuf(4, true);
-            bb.writeFloat(((FloatExpression)valueArg).getFloat());
-          }
-          break;
-        case "DOUBLE":
-          if (valueArg instanceof DoubleExpression && isEqualityFn) {
-            bb = newByteBuf(8, true);
-            bb.writeDouble(((DoubleExpression)valueArg).getDouble());
-          }
-          break;
-        case "TIME_EPOCH":
-        case "TIME_EPOCH_BE":
-          if (valueArg instanceof TimeExpression) {
-            bb = newByteBuf(8, encodingType.endsWith("_BE"));
-            bb.writeLong(((TimeExpression)valueArg).getTime());
-          }
-          break;
-        case "DATE_EPOCH":
-        case "DATE_EPOCH_BE":
-          if (valueArg instanceof DateExpression) {
-            bb = newByteBuf(8, encodingType.endsWith("_BE"));
-            bb.writeLong(((DateExpression)valueArg).getDate());
-          }
-          break;
-        case "BOOLEAN_BYTE":
-          if (valueArg instanceof BooleanExpression) {
-            bb = newByteBuf(1, false /* does not matter */);
-            bb.writeByte(((BooleanExpression)valueArg).getBoolean() ? 1 : 0);
-          }
-          break;
-        case "DOUBLE_OB":
-        case "DOUBLE_OBD":
-          if (valueArg instanceof DoubleExpression) {
-            bb = newByteBuf(9, true);
-            PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 9);
-            if (encodingType.endsWith("_OBD")) {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat64(br,
-                  ((DoubleExpression)valueArg).getDouble(), Order.DESCENDING);
-              this.sortOrderAscending = false;
-            } else {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat64(br,
-                  ((DoubleExpression)valueArg).getDouble(), Order.ASCENDING);
+          case "INT_BE":
+          case "INT":
+          case "UINT_BE":
+          case "UINT":
+          case "UINT4_BE":
+          case "UINT4":
+            if (valueArg instanceof IntExpression
+                && (isEqualityFn || encodingType.startsWith("U"))) {
+              bb = newByteBuf(4, encodingType.endsWith("_BE"));
+              bb.writeInt(((IntExpression) valueArg).getInt());
+            }
+            break;
+          case "BIGINT_BE":
+          case "BIGINT":
+          case "UINT8_BE":
+          case "UINT8":
+            if (valueArg instanceof LongExpression
+                && (isEqualityFn || encodingType.startsWith("U"))) {
+              bb = newByteBuf(8, encodingType.endsWith("_BE"));
+              bb.writeLong(((LongExpression) valueArg).getLong());
+            }
+            break;
+          case "FLOAT":
+            if (valueArg instanceof FloatExpression && isEqualityFn) {
+              bb = newByteBuf(4, true);
+              bb.writeFloat(((FloatExpression) valueArg).getFloat());
+            }
+            break;
+          case "DOUBLE":
+            if (valueArg instanceof DoubleExpression && isEqualityFn) {
+              bb = newByteBuf(8, true);
+              bb.writeDouble(((DoubleExpression) valueArg).getDouble());
             }
-          }
-          break;
-        case "FLOAT_OB":
-        case "FLOAT_OBD":
-          if (valueArg instanceof FloatExpression) {
-            bb = newByteBuf(5, true);
-            PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 5);
-            if (encodingType.endsWith("_OBD")) {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat32(br,
-                  ((FloatExpression)valueArg).getFloat(), Order.DESCENDING);
-              this.sortOrderAscending = false;
-            } else {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat32(br,
-                        ((FloatExpression)valueArg).getFloat(), 
Order.ASCENDING);
+            break;
+          case "TIME_EPOCH":
+          case "TIME_EPOCH_BE":
+            if (valueArg instanceof TimeExpression) {
+              bb = newByteBuf(8, encodingType.endsWith("_BE"));
+              bb.writeLong(((TimeExpression) valueArg).getTime());
             }
-          }
-          break;
-        case "BIGINT_OB":
-        case "BIGINT_OBD":
-          if (valueArg instanceof LongExpression) {
-            bb = newByteBuf(9, true);
-            PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 9);
-            if (encodingType.endsWith("_OBD")) {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeInt64(br,
-                        ((LongExpression)valueArg).getLong(), 
Order.DESCENDING);
-              this.sortOrderAscending = false;
-            } else {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeInt64(br,
-                  ((LongExpression)valueArg).getLong(), Order.ASCENDING);
+            break;
+          case "DATE_EPOCH":
+          case "DATE_EPOCH_BE":
+            if (valueArg instanceof DateExpression) {
+              bb = newByteBuf(8, encodingType.endsWith("_BE"));
+              bb.writeLong(((DateExpression) valueArg).getDate());
             }
-          }
-          break;
-        case "INT_OB":
-        case "INT_OBD":
-          if (valueArg instanceof IntExpression) {
-            bb = newByteBuf(5, true);
-            PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 5);
-            if (encodingType.endsWith("_OBD")) {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeInt32(br,
-                  ((IntExpression)valueArg).getInt(), Order.DESCENDING);
-              this.sortOrderAscending = false;
-            } else {
-              org.apache.hadoop.hbase.util.OrderedBytes.encodeInt32(br,
-                        ((IntExpression)valueArg).getInt(), Order.ASCENDING);
+            break;
+          case "BOOLEAN_BYTE":
+            if (valueArg instanceof BooleanExpression) {
+              bb = newByteBuf(1, false /* does not matter */);
+              bb.writeByte(((BooleanExpression) valueArg).getBoolean() ? 1 : 
0);
             }
-          }
-          break;
-        case "UTF8":
-          // let visitSchemaPath() handle this.
-          return e.getInput().accept(this, valueArg);
+            break;
+          case "DOUBLE_OB":
+          case "DOUBLE_OBD":
+            if (valueArg instanceof DoubleExpression) {
+              bb = newByteBuf(9, true);
+              PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 9);
+              if (encodingType.endsWith("_OBD")) {
+                org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat64(br,
+                  ((DoubleExpression) valueArg).getDouble(), Order.DESCENDING);
+                this.sortOrderAscending = false;
+              } else {
+                org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat64(br,
+                  ((DoubleExpression) valueArg).getDouble(), Order.ASCENDING);
+              }
+            }
+            break;
+          case "FLOAT_OB":
+          case "FLOAT_OBD":
+            if (valueArg instanceof FloatExpression) {
+              bb = newByteBuf(5, true);
+              PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 5);
+              if (encodingType.endsWith("_OBD")) {
+                org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat32(br,
+                  ((FloatExpression) valueArg).getFloat(), Order.DESCENDING);
+                this.sortOrderAscending = false;
+              } else {
+                org.apache.hadoop.hbase.util.OrderedBytes.encodeFloat32(br,
+                          ((FloatExpression) valueArg).getFloat(), 
Order.ASCENDING);
+              }
+            }
+            break;
+          case "BIGINT_OB":
+          case "BIGINT_OBD":
+            if (valueArg instanceof LongExpression) {
+              bb = newByteBuf(9, true);
+              PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 9);
+              if (encodingType.endsWith("_OBD")) {
+                org.apache.hadoop.hbase.util.OrderedBytes.encodeInt64(br,
+                          ((LongExpression) valueArg).getLong(), 
Order.DESCENDING);
+                this.sortOrderAscending = false;
+              } else {
+                org.apache.hadoop.hbase.util.OrderedBytes.encodeInt64(br,
+                  ((LongExpression) valueArg).getLong(), Order.ASCENDING);
+              }
+            }
+            break;
+          case "INT_OB":
+          case "INT_OBD":
+            if (valueArg instanceof IntExpression) {
+              bb = newByteBuf(5, true);
+              PositionedByteRange br = new 
SimplePositionedMutableByteRange(bb.array(), 0, 5);
+              if (encodingType.endsWith("_OBD")) {
+                org.apache.hadoop.hbase.util.OrderedBytes.encodeInt32(br,
+                  ((IntExpression) valueArg).getInt(), Order.DESCENDING);
+                this.sortOrderAscending = false;
+              } else {
+                org.apache.hadoop.hbase.util.OrderedBytes.encodeInt32(br,
+                          ((IntExpression) valueArg).getInt(), 
Order.ASCENDING);
+              }
+            }
+            break;
+          case "UTF8":
+            // let visitSchemaPath() handle this.
+            return e.getInput().accept(this, valueArg);
+          default:
+            bb = getByteBuf(valueArg, encodingType);
         }
 
         if (bb != null) {
@@ -325,53 +349,57 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
     return false;
   }
 
+  protected ByteBuf getByteBuf(LogicalExpression valueArg, String 
encodingType) {
+    return null;
+  }
+
   private Boolean visitRowKeyPrefixConvertExpression(ConvertExpression e,
-    int prefixLength, LogicalExpression valueArg) {
+                                                     int prefixLength, 
LogicalExpression valueArg) {
     String encodingType = e.getEncodingType();
     rowKeyPrefixStartRow = HConstants.EMPTY_START_ROW;
-    rowKeyPrefixStopRow  = HConstants.EMPTY_START_ROW;
-    rowKeyPrefixFilter   = null;
+    rowKeyPrefixStopRow = HConstants.EMPTY_START_ROW;
+    rowKeyPrefixFilter = null;
 
-    if ((encodingType.compareTo("UINT4_BE") == 0) ||
-        (encodingType.compareTo("UINT_BE") == 0)) {
+    if ((encodingType.compareTo("UINT4_BE") == 0)
+        || (encodingType.compareTo("UINT_BE") == 0)) {
       if (prefixLength != 4) {
         throw new RuntimeException("Invalid length(" + prefixLength + ") of 
row-key prefix");
       }
 
       int val;
-      if ((valueArg instanceof IntExpression) == false) {
+      if (!(valueArg instanceof IntExpression)) {
         return false;
       }
 
-      val = ((IntExpression)valueArg).getInt();
+      val = ((IntExpression) valueArg).getInt();
 
       // For TIME_EPOCH_BE/BIGINT_BE encoding, the operators that we push-down 
are =, <>, <, <=, >, >=
       switch (functionName) {
-      case "equal":
-      rowKeyPrefixFilter = new 
PrefixFilter(ByteBuffer.allocate(4).putInt(val).array());
-      rowKeyPrefixStartRow = ByteBuffer.allocate(4).putInt(val).array();
-      rowKeyPrefixStopRow = ByteBuffer.allocate(4).putInt(val + 1).array();
-      return true;
-    case "greater_than_or_equal_to":
-      rowKeyPrefixStartRow = ByteBuffer.allocate(4).putInt(val).array();
-        return true;
-      case "greater_than":
-      rowKeyPrefixStartRow = ByteBuffer.allocate(4).putInt(val + 1).array();
-        return true;
-      case "less_than_or_equal_to":
-        rowKeyPrefixStopRow = ByteBuffer.allocate(4).putInt(val + 1).array();
-        return true;
-      case "less_than":
-        rowKeyPrefixStopRow = ByteBuffer.allocate(4).putInt(val).array();
-        return true;
+        case "equal":
+          rowKeyPrefixFilter = new 
PrefixFilter(ByteBuffer.allocate(4).putInt(val).array());
+          rowKeyPrefixStartRow = ByteBuffer.allocate(4).putInt(val).array();
+          rowKeyPrefixStopRow = ByteBuffer.allocate(4).putInt(val + 1).array();
+          return true;
+        case "greater_than_or_equal_to":
+          rowKeyPrefixStartRow = ByteBuffer.allocate(4).putInt(val).array();
+          return true;
+        case "greater_than":
+          rowKeyPrefixStartRow = ByteBuffer.allocate(4).putInt(val + 
1).array();
+          return true;
+        case "less_than_or_equal_to":
+          rowKeyPrefixStopRow = ByteBuffer.allocate(4).putInt(val + 1).array();
+          return true;
+        case "less_than":
+          rowKeyPrefixStopRow = ByteBuffer.allocate(4).putInt(val).array();
+          return true;
       }
 
       return false;
     }
 
-    if ((encodingType.compareTo("TIMESTAMP_EPOCH_BE") == 0) ||
-        (encodingType.compareTo("TIME_EPOCH_BE") == 0) ||
-        (encodingType.compareTo("UINT8_BE") == 0)) {
+    if ((encodingType.compareTo("TIMESTAMP_EPOCH_BE") == 0)
+        || (encodingType.compareTo("TIME_EPOCH_BE") == 0)
+        || (encodingType.compareTo("UINT8_BE") == 0)) {
 
       if (prefixLength != 8) {
         throw new RuntimeException("Invalid length(" + prefixLength + ") of 
row-key prefix");
@@ -379,23 +407,23 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
 
       long val;
       if (encodingType.compareTo("TIME_EPOCH_BE") == 0) {
-        if ((valueArg instanceof TimeExpression) == false) {
+        if (!(valueArg instanceof TimeExpression)) {
           return false;
         }
 
-        val = ((TimeExpression)valueArg).getTime();
-      } else if (encodingType.compareTo("UINT8_BE") == 0){
-        if ((valueArg instanceof LongExpression) == false) {
+        val = ((TimeExpression) valueArg).getTime();
+      } else if (encodingType.compareTo("UINT8_BE") == 0) {
+        if (!(valueArg instanceof LongExpression)) {
           return false;
         }
 
-        val = ((LongExpression)valueArg).getLong();
+        val = ((LongExpression) valueArg).getLong();
       } else if (encodingType.compareTo("TIMESTAMP_EPOCH_BE") == 0) {
-        if ((valueArg instanceof TimeStampExpression) == false) {
+        if (!(valueArg instanceof TimeStampExpression)) {
           return false;
         }
 
-        val = ((TimeStampExpression)valueArg).getTimeStamp();
+        val = ((TimeStampExpression) valueArg).getTimeStamp();
       } else {
         // Should not reach here.
         return false;
@@ -403,30 +431,30 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
 
       // For TIME_EPOCH_BE/BIGINT_BE encoding, the operators that we push-down 
are =, <>, <, <=, >, >=
       switch (functionName) {
-      case "equal":
-        rowKeyPrefixFilter = new 
PrefixFilter(ByteBuffer.allocate(8).putLong(val).array());
-        rowKeyPrefixStartRow = ByteBuffer.allocate(8).putLong(val).array();
-        rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(val + 1).array();
-        return true;
-      case "greater_than_or_equal_to":
-        rowKeyPrefixStartRow = ByteBuffer.allocate(8).putLong(val).array();
-        return true;
-      case "greater_than":
-        rowKeyPrefixStartRow = ByteBuffer.allocate(8).putLong(val + 1).array();
-        return true;
-      case "less_than_or_equal_to":
-        rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(val + 1).array();
-        return true;
-      case "less_than":
-        rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(val).array();
-        return true;
+        case "equal":
+          rowKeyPrefixFilter = new 
PrefixFilter(ByteBuffer.allocate(8).putLong(val).array());
+          rowKeyPrefixStartRow = ByteBuffer.allocate(8).putLong(val).array();
+          rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(val + 
1).array();
+          return true;
+        case "greater_than_or_equal_to":
+          rowKeyPrefixStartRow = ByteBuffer.allocate(8).putLong(val).array();
+          return true;
+        case "greater_than":
+          rowKeyPrefixStartRow = ByteBuffer.allocate(8).putLong(val + 
1).array();
+          return true;
+        case "less_than_or_equal_to":
+          rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(val + 
1).array();
+          return true;
+        case "less_than":
+          rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(val).array();
+          return true;
       }
 
       return false;
     }
 
     if (encodingType.compareTo("DATE_EPOCH_BE") == 0) {
-      if ((valueArg instanceof DateExpression) == false) {
+      if (!(valueArg instanceof DateExpression)) {
         return false;
       }
 
@@ -434,41 +462,41 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
         throw new RuntimeException("Invalid length(" + prefixLength + ") of 
row-key prefix");
       }
 
-      final long MILLISECONDS_IN_A_DAY  = (long)1000 * 60 * 60 * 24;
+      final long MILLISECONDS_IN_A_DAY = 1000 * 60 * 60 * 24;
       long dateToSet;
       // For DATE encoding, the operators that we push-down are =, <>, <, <=, 
>, >=
       switch (functionName) {
-      case "equal":
-        long startDate = ((DateExpression)valueArg).getDate();
-        rowKeyPrefixStartRow = 
ByteBuffer.allocate(8).putLong(startDate).array();
-        long stopDate  = ((DateExpression)valueArg).getDate() + 
MILLISECONDS_IN_A_DAY;
-        rowKeyPrefixStopRow = ByteBuffer.allocate(8).putLong(stopDate).array();
-        return true;
-      case "greater_than_or_equal_to":
-        dateToSet = ((DateExpression)valueArg).getDate();
-        rowKeyPrefixStartRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
-        return true;
-      case "greater_than":
-        dateToSet = ((DateExpression)valueArg).getDate() + 
MILLISECONDS_IN_A_DAY;
-        rowKeyPrefixStartRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
-        return true;
-      case "less_than_or_equal_to":
-        dateToSet = ((DateExpression)valueArg).getDate() + 
MILLISECONDS_IN_A_DAY;
-        rowKeyPrefixStopRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
-        return true;
-      case "less_than":
-        dateToSet = ((DateExpression)valueArg).getDate();
-        rowKeyPrefixStopRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
-        return true;
+        case "equal":
+          long startDate = ((DateExpression) valueArg).getDate();
+          rowKeyPrefixStartRow = 
ByteBuffer.allocate(8).putLong(startDate).array();
+          long stopDate = ((DateExpression) valueArg).getDate() + 
MILLISECONDS_IN_A_DAY;
+          rowKeyPrefixStopRow = 
ByteBuffer.allocate(8).putLong(stopDate).array();
+          return true;
+        case "greater_than_or_equal_to":
+          dateToSet = ((DateExpression) valueArg).getDate();
+          rowKeyPrefixStartRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
+          return true;
+        case "greater_than":
+          dateToSet = ((DateExpression) valueArg).getDate() + 
MILLISECONDS_IN_A_DAY;
+          rowKeyPrefixStartRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
+          return true;
+        case "less_than_or_equal_to":
+          dateToSet = ((DateExpression) valueArg).getDate() + 
MILLISECONDS_IN_A_DAY;
+          rowKeyPrefixStopRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
+          return true;
+        case "less_than":
+          dateToSet = ((DateExpression) valueArg).getDate();
+          rowKeyPrefixStopRow = 
ByteBuffer.allocate(8).putLong(dateToSet).array();
+          return true;
       }
 
       return false;
-  }
+    }
 
-  return false;
-}
+    return false;
+  }
 
-@Override
+  @Override
   public Boolean visitUnknown(LogicalExpression e, LogicalExpression valueArg) 
throws RuntimeException {
     return false;
   }
@@ -483,7 +511,7 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
     return false;
   }
 
-  private static ByteBuf newByteBuf(int size, boolean bigEndian) {
+  protected static ByteBuf newByteBuf(int size, boolean bigEndian) {
     return Unpooled.wrappedBuffer(new byte[size])
         .order(bigEndian ? ByteOrder.BIG_ENDIAN : ByteOrder.LITTLE_ENDIAN)
         .writerIndex(0);
@@ -525,5 +553,4 @@ class CompareFunctionsProcessor extends 
AbstractExprVisitor<Boolean, LogicalExpr
         .put("less_than", "greater_than")
         .build();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseFilterBuilder.java
----------------------------------------------------------------------
diff --git 
a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseFilterBuilder.java
 
b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseFilterBuilder.java
index 394e128..8d2e8ff 100644
--- 
a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseFilterBuilder.java
+++ 
b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseFilterBuilder.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -99,7 +99,7 @@ public class HBaseFilterBuilder extends 
AbstractExprVisitor<HBaseScanSpec, Void,
         nullComparatorSupported = 
groupScan.getHBaseConf().getBoolean("drill.hbase.supports.null.comparator", 
false);
       }
 
-      CompareFunctionsProcessor processor = 
CompareFunctionsProcessor.process(call, nullComparatorSupported);
+      CompareFunctionsProcessor processor = 
CompareFunctionsProcessor.createFunctionsProcessorInstance(call, 
nullComparatorSupported);
       if (processor.isSuccess()) {
         nodeScanSpec = createHBaseScanSpec(call, processor);
       }
@@ -155,7 +155,7 @@ public class HBaseFilterBuilder extends 
AbstractExprVisitor<HBaseScanSpec, Void,
     SchemaPath field = processor.getPath();
     byte[] fieldValue = processor.getValue();
     boolean sortOrderAscending = processor.isSortOrderAscending();
-    boolean isRowKey = field.getAsUnescapedPath().equals(ROW_KEY);
+    boolean isRowKey = field.getRootSegmentPath().equals(ROW_KEY);
     if (!(isRowKey
         || (!field.getRootSegment().isLastPath()
             && field.getRootSegment().getChild().isLastPath()

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
----------------------------------------------------------------------
diff --git 
a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
 
b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
index bb59600..b1b966a 100644
--- 
a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
+++ 
b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -233,7 +233,7 @@ public class ConvertHiveParquetScanToDrillParquetScan 
extends StoragePluginOptim
     // unlike above where we expanded the '*'. HiveScan and related (subscan) 
can handle '*'.
     final List<SchemaPath> nativeScanCols = Lists.newArrayList();
     for(SchemaPath colName : hiveScanRel.getColumns()) {
-      final String partitionCol = 
partitionColMapping.get(colName.getAsUnescapedPath());
+      final String partitionCol = 
partitionColMapping.get(colName.getRootSegmentPath());
       if (partitionCol != null) {
         nativeScanCols.add(SchemaPath.getSimplePath(partitionCol));
       } else {

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
----------------------------------------------------------------------
diff --git 
a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
 
b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
index 66f41e2..6c10d25 100644
--- 
a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
+++ 
b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -80,10 +80,10 @@ public class HiveDrillNativeScanBatchCreator implements 
BatchCreator<HiveDrillNa
       newColumns = Lists.newArrayList();
       Pattern pattern = Pattern.compile(String.format("%s[0-9]+", 
partitionDesignator));
       for (SchemaPath column : columns) {
-        Matcher m = pattern.matcher(column.getAsUnescapedPath());
+        Matcher m = pattern.matcher(column.getRootSegmentPath());
         if (m.matches()) {
           selectedPartitionColumns.add(
-              
Integer.parseInt(column.getAsUnescapedPath().substring(partitionDesignator.length())));
+              
Integer.parseInt(column.getRootSegmentPath().substring(partitionDesignator.length())));
         } else {
           newColumns.add(column);
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
----------------------------------------------------------------------
diff --git 
a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
 
b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
index ef7efcf..b9a7bf9 100644
--- 
a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
+++ 
b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -106,7 +106,7 @@ public class KuduRecordReader extends AbstractRecordReader {
       if (!isStarQuery()) {
         List<String> colNames = Lists.newArrayList();
         for (SchemaPath p : this.getColumns()) {
-          colNames.add(p.getAsUnescapedPath());
+          colNames.add(p.getRootSegmentPath());
         }
         builder.setProjectedColumnNames(colNames);
       }
@@ -199,7 +199,7 @@ public class KuduRecordReader extends AbstractRecordReader {
         majorType = Types.required(minorType);
       }
       MaterializedField field = MaterializedField.create(name, majorType);
-      final Class<? extends ValueVector> clazz = (Class<? extends 
ValueVector>) TypeHelper.getValueVectorClass(
+      final Class<? extends ValueVector> clazz = 
TypeHelper.getValueVectorClass(
           minorType, majorType.getMode());
       ValueVector vector = output.addField(field, clazz);
       vector.allocateNew();

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordWriterImpl.java
----------------------------------------------------------------------
diff --git 
a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordWriterImpl.java
 
b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordWriterImpl.java
index 2e40acf..9b98ccd 100644
--- 
a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordWriterImpl.java
+++ 
b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordWriterImpl.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -76,7 +76,7 @@ public class KuduRecordWriterImpl extends KuduRecordWriter {
       if (!checkForTable(name)) {
         List<ColumnSchema> columns = new ArrayList<>();
         for (MaterializedField f : schema) {
-          columns.add(new ColumnSchema.ColumnSchemaBuilder(f.getLastName(), 
getType(f.getType()))
+          columns.add(new ColumnSchema.ColumnSchemaBuilder(f.getName(), 
getType(f.getType()))
               .nullable(f.getType().getMode() == DataMode.OPTIONAL)
               .key(i == 0).build());
           i++;

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoFilterBuilder.java
----------------------------------------------------------------------
diff --git 
a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoFilterBuilder.java
 
b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoFilterBuilder.java
index 379f449..70e9a1a 100644
--- 
a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoFilterBuilder.java
+++ 
b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoFilterBuilder.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -165,7 +165,7 @@ public class MongoFilterBuilder extends
       SchemaPath field, Object fieldValue) throws ClassNotFoundException,
       IOException {
     // extract the field name
-    String fieldName = field.getAsUnescapedPath();
+    String fieldName = field.getRootSegmentPath();
     MongoCompareOp compareOp = null;
     switch (functionName) {
     case "equal":

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java 
b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
index a2428a7..b88d4a1 100644
--- a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -82,11 +82,11 @@ public class EventBasedRecordWriter {
     try {
       int fieldId = 0;
       for (VectorWrapper w : batch) {
-        if 
(w.getField().getPath().equalsIgnoreCase(WriterPrel.PARTITION_COMPARATOR_FIELD))
 {
+        if 
(w.getField().getName().equalsIgnoreCase(WriterPrel.PARTITION_COMPARATOR_FIELD))
 {
           continue;
         }
         FieldReader reader = w.getValueVector().getReader();
-        FieldConverter converter = getConverter(recordWriter, fieldId++, 
w.getField().getLastName(), reader);
+        FieldConverter converter = getConverter(recordWriter, fieldId++, 
w.getField().getName(), reader);
         fieldConverters.add(converter);
       }
     } catch(Exception e) {

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java 
b/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java
index 6d92a2f..70c699e 100644
--- a/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -64,7 +64,7 @@ public abstract class StringOutputRecordWriter extends 
AbstractRecordWriter {
     BatchSchema schema = batch.getSchema();
     List<String> columnNames = Lists.newArrayList();
     for (int i=0; i < schema.getFieldCount(); i++) {
-      columnNames.add(schema.getColumn(i).getLastName());
+      columnNames.add(schema.getColumn(i).getName());
     }
 
     startNewSchema(columnNames);

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java 
b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
index cdbd3b4..6c85921 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -100,16 +100,16 @@ public class DumpCat {
    *  Options as input to JCommander.
    */
   static class Options {
-    @Parameter(names = {"-f"}, description = "file containing dump", 
required=true)
+    @Parameter(names = {"-f"}, description = "file containing dump", required 
= true)
     public String location = null;
 
-    @Parameter(names = {"-batch"}, description = "id of batch to show", 
required=false, validateWith = BatchNumValidator.class)
+    @Parameter(names = {"-batch"}, description = "id of batch to show", 
required = false, validateWith = BatchNumValidator.class)
     public int batch = -1;
 
-    @Parameter(names = {"-include-headers"}, description = "whether include 
header of batch", required=false)
+    @Parameter(names = {"-include-headers"}, description = "whether include 
header of batch", required = false)
     public boolean include_headers = false;
 
-    @Parameter(names = {"-h", "-help", "--help"}, description = "show usage", 
help=true)
+    @Parameter(names = {"-h", "-help", "--help"}, description = "show usage", 
help = true)
     public boolean help = false;
    }
 
@@ -138,7 +138,7 @@ public class DumpCat {
 
     @Override
     public String toString() {
-      String avgRecSizeStr = null;
+      String avgRecSizeStr;
       if (this.rows > 0) {
         avgRecSizeStr = String.format("Average Record Size : %d ", 
this.dataSize/this.rows);
       } else {
@@ -175,7 +175,7 @@ public class DumpCat {
     while (input.available() > 0) {
       final VectorAccessibleSerializable vcSerializable = new 
VectorAccessibleSerializable(DumpCat.allocator);
       vcSerializable.readFromStream(input);
-      final VectorContainer vectorContainer = (VectorContainer) 
vcSerializable.get();
+      final VectorContainer vectorContainer = vcSerializable.get();
 
       aggBatchMetaInfo.add(getBatchMetaInfo(vcSerializable));
 
@@ -224,7 +224,7 @@ public class DumpCat {
       vcSerializable.readFromStream(input);
 
       if (batchNum != targetBatchNum) {
-        final VectorContainer vectorContainer = (VectorContainer) 
vcSerializable.get();
+        final VectorContainer vectorContainer = vcSerializable.get();
         vectorContainer.zeroVectors();
       }
     }
@@ -237,13 +237,13 @@ public class DumpCat {
 
     if (vcSerializable != null) {
       showSingleBatch(vcSerializable, showHeader);
-      final VectorContainer vectorContainer = (VectorContainer) 
vcSerializable.get();
+      final VectorContainer vectorContainer = vcSerializable.get();
       vectorContainer.zeroVectors();
     }
   }
 
   private void showSingleBatch (VectorAccessibleSerializable vcSerializable, 
boolean showHeader) {
-    final VectorContainer vectorContainer = (VectorContainer) 
vcSerializable.get();
+    final VectorContainer vectorContainer = vcSerializable.get();
 
     /* show the header of the batch */
     if (showHeader) {
@@ -253,7 +253,7 @@ public class DumpCat {
       for (final VectorWrapper w : vectorContainer) {
         final MaterializedField field = w.getValueVector().getField();
         System.out.println (String.format("name : %s, minor_type : %s, 
data_mode : %s",
-                                          field.getPath(),
+                                          field.getName(),
                                           
field.getType().getMinorType().toString(),
                                           field.isNullable() ? 
"nullable":"non-nullable"
                           ));
@@ -268,8 +268,8 @@ public class DumpCat {
   private BatchMetaInfo getBatchMetaInfo(VectorAccessibleSerializable 
vcSerializable) {
     final VectorAccessible vectorContainer = vcSerializable.get();
 
-    int rows = 0;
-    int selectedRows = 0;
+    int rows;
+    int selectedRows;
     int totalDataSize = 0;
 
     rows = vectorContainer.getRecordCount();

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 4218069..803bd48 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -359,19 +359,19 @@ public class ScanBatch implements CloseableRecordBatch {
     public <T extends ValueVector> T addField(MaterializedField field,
                                               Class<T> clazz) throws 
SchemaChangeException {
       // Check if the field exists.
-      ValueVector v = fieldVectorMap.get(field.getPath());
+      ValueVector v = fieldVectorMap.get(field.getName());
       if (v == null || v.getClass() != clazz) {
         // Field does not exist--add it to the map and the output container.
         v = TypeHelper.getNewVector(field, allocator, callBack);
         if (!clazz.isAssignableFrom(v.getClass())) {
           throw new SchemaChangeException(
-              String.format(
-                  "The class that was provided, %s, does not correspond to the 
"
-                  + "expected vector type of %s.",
-                  clazz.getSimpleName(), v.getClass().getSimpleName()));
+            String.format(
+              "The class that was provided, %s, does not correspond to the "
+                + "expected vector type of %s.",
+              clazz.getSimpleName(), v.getClass().getSimpleName()));
         }
 
-        final ValueVector old = fieldVectorMap.put(field.getPath(), v);
+        final ValueVector old = fieldVectorMap.put(field.getName(), v);
         if (old != null) {
           old.clear();
           container.remove(old);

http://git-wip-us.apache.org/repos/asf/drill/blob/d105950a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
index ff159cd..b82dfc8 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
@@ -71,7 +71,7 @@ public abstract class PriorityQueueTemplate implements 
PriorityQueue {
     BatchSchema schema = container.getSchema();
     VectorContainer newContainer = new VectorContainer();
     for (MaterializedField field : schema) {
-      int[] ids = 
container.getValueVectorId(SchemaPath.getSimplePath(field.getPath())).getFieldIds();
+      int[] ids = 
container.getValueVectorId(SchemaPath.getSimplePath(field.getName())).getFieldIds();
       newContainer.add(container.getValueAccessorById(field.getValueClass(), 
ids).getValueVectors());
     }
     newContainer.buildSchema(BatchSchema.SelectionVectorMode.FOUR_BYTE);

Reply via email to