You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2014/09/25 03:43:04 UTC

git commit: PHOENIX-1273 Phoenix should provide ILIKE keyword (Jean-Marc Spaggiari)

Repository: phoenix
Updated Branches:
  refs/heads/master 70319eabe -> 6908c90b5


PHOENIX-1273 Phoenix should provide ILIKE keyword (Jean-Marc Spaggiari)


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

Branch: refs/heads/master
Commit: 6908c90b5bfc5b5cc4a98d4dd32557a075d87a45
Parents: 70319ea
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Sep 24 18:45:26 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Sep 24 18:47:21 2014 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/VariableLengthPKIT.java     | 137 +++++++++++++++++++
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   5 +-
 .../phoenix/compile/ExpressionCompiler.java     |  31 +++--
 .../apache/phoenix/compile/WhereOptimizer.java  |   5 +-
 .../phoenix/expression/LikeExpression.java      |  65 ++++++---
 .../org/apache/phoenix/parse/LikeParseNode.java |  13 +-
 .../apache/phoenix/parse/ParseNodeFactory.java  |  71 +++++-----
 .../apache/phoenix/parse/ParseNodeRewriter.java |  78 +++++------
 .../phoenix/expression/ILikeExpressionTest.java |  62 +++++++++
 .../phoenix/expression/LikeExpressionTest.java  |  39 ++++--
 .../java/org/apache/phoenix/util/TestUtil.java  |   9 +-
 11 files changed, 392 insertions(+), 123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
index 5fe71a8..6800379 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
@@ -1592,6 +1592,143 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
             rs = statement.executeQuery();
             assertFalse(rs.next());
 
+            // Test 5
+            statement = conn.prepareStatement("SELECT INST FROM PTSDB WHERE 'abcdef' LIKE '%bCd%'");
+            rs = statement.executeQuery();
+            assertFalse(rs.next());
+
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testILikeOnColumn() throws Exception {
+        long ts = nextTimestamp();
+        ensureTableCreated(getUrl(),PTSDB_NAME,null, ts-2);
+
+        // Insert all rows at ts
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        PreparedStatement stmt = conn.prepareStatement("upsert into PTSDB(INST, HOST, DATE, VAL, PATTERN VARCHAR) VALUES (?, ?, ?, 0.5, 'x_Z%')");
+        stmt.setDate(3, D1);
+
+        stmt.setString(1, "a");
+        stmt.setString(2, "a");
+        stmt.execute();
+
+        stmt.setString(1, "x");
+        stmt.setString(2, "a");
+        stmt.execute();
+
+        stmt.setString(1, "xy");
+        stmt.setString(2, "b");
+        stmt.execute();
+
+        stmt.setString(1, "xyz");
+        stmt.setString(2, "c");
+        stmt.execute();
+
+        stmt.setString(1, "xyza");
+        stmt.setString(2, "d");
+        stmt.execute();
+
+        stmt.setString(1, "xyzab");
+        stmt.setString(2, "e");
+        stmt.execute();
+
+        stmt.setString(1, "z");
+        stmt.setString(2, "e");
+        stmt.execute();
+
+        conn.commit();
+        conn.close();
+
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        conn = DriverManager.getConnection(url, props);
+        PreparedStatement statement;
+        ResultSet rs;
+        try {
+            // Test 1
+            statement = conn.prepareStatement("SELECT INST FROM PTSDB WHERE INST ILIKE 'x%'");
+            rs = statement.executeQuery();
+
+            assertTrue(rs.next());
+            assertEquals("x", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("xy", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("xyz", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("xyza", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("xyzab", rs.getString(1));
+
+            assertFalse(rs.next());
+
+            // Test 2
+            statement = conn.prepareStatement("SELECT INST FROM PTSDB WHERE INST ILIKE 'xy_a%'");
+            rs = statement.executeQuery();
+
+            assertTrue(rs.next());
+            assertEquals("xyza", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("xyzab", rs.getString(1));
+
+            assertFalse(rs.next());
+
+            // Test 3
+            statement = conn.prepareStatement("SELECT INST FROM PTSDB WHERE INST NOT ILIKE 'xy_a%'");
+            rs = statement.executeQuery();
+
+            assertTrue(rs.next());
+            assertEquals("a", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("x", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("xy", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("xyz", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("z", rs.getString(1));
+
+            assertFalse(rs.next());
+
+            // Test 4
+            statement = conn.prepareStatement("SELECT INST FROM PTSDB WHERE 'xzabc' ILIKE 'xy_a%'");
+            rs = statement.executeQuery();
+            assertFalse(rs.next());
+
+            // Test 5
+            statement = conn.prepareStatement("SELECT INST FROM PTSDB WHERE 'abcdef' ILIKE '%bCd%'");
+            rs = statement.executeQuery();
+            assertTrue(rs.next());
+
+            // Test 5
+            statement = conn.prepareStatement("SELECT INST FROM PTSDB(PATTERN VARCHAR) WHERE INST ILIKE PATTERN");
+            rs = statement.executeQuery();
+
+            assertTrue(rs.next());
+            assertEquals("xyz", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("xyza", rs.getString(1));
+
+            assertTrue(rs.next());
+            assertEquals("xyzab", rs.getString(1));
+
+            assertFalse(rs.next());
+
         } finally {
             conn.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index 6264648..8432f6e 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -29,6 +29,7 @@ tokens
     TRUE='true';
     FALSE='false';
     LIKE='like';
+    ILIKE='ilike';
     AS='as';
     OUTER='outer';
     ON='on';
@@ -149,6 +150,7 @@ import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
 }
 
 @lexer::header {
@@ -713,7 +715,8 @@ comparison_op returns [CompareOp ret]
 boolean_expression returns [ParseNode ret]
     :   l=value_expression ((op=comparison_op (r=value_expression | ((all=ALL | any=ANY) LPAREN r=value_expression RPAREN)) {$ret = all != null ? factory.wrapInAll(op, l, r) : any != null ? factory.wrapInAny(op, l, r) : factory.comparison(op,l,r); } )
                   |  (IS n=NOT? NULL {$ret = factory.isNull(l,n!=null); } )
-                  |  ( n=NOT? ((LIKE r=value_expression {$ret = factory.like(l,r,n!=null); } )
+                  |  ( n=NOT? ((LIKE r=value_expression {$ret = factory.like(l,r,n!=null,LikeType.CASE_SENSITIVE); } )
+                      |        (ILIKE r=value_expression {$ret = factory.like(l,r,n!=null,LikeType.CASE_INSENSITIVE); } )
                       |        (EXISTS LPAREN r=subquery_expression RPAREN {$ret = factory.exists(l,r,n!=null);} )
                       |        (BETWEEN r1=value_expression AND r2=value_expression {$ret = factory.between(l,r1,r2,n!=null); } )
                       |        ((IN ((r=bind_expression {$ret = factory.inList(Arrays.asList(l,r),n!=null);} )

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
index e7883a0..cb3bc74 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
@@ -80,6 +80,7 @@ import org.apache.phoenix.parse.ComparisonParseNode;
 import org.apache.phoenix.parse.DivideParseNode;
 import org.apache.phoenix.parse.FunctionParseNode;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
 import org.apache.phoenix.parse.InListParseNode;
 import org.apache.phoenix.parse.IsNullParseNode;
 import org.apache.phoenix.parse.LikeParseNode;
@@ -122,7 +123,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
     protected final GroupBy groupBy;
     private int nodeCount;
     private final boolean resolveViewConstants;
-    
+
     ExpressionCompiler(StatementContext context) {
         this(context,GroupBy.EMPTY_GROUP_BY, false);
     }
@@ -148,7 +149,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
     public boolean isTopLevel() {
         return nodeCount == 0;
     }
-    
+
     public void reset() {
         this.isAggregate = false;
         this.nodeCount = 0;
@@ -158,7 +159,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
     public boolean visitEnter(ComparisonParseNode node) {
         return true;
     }
-    
+
     private void addBindParamMetaData(ParseNode lhsNode, ParseNode rhsNode, Expression lhsExpr, Expression rhsExpr) throws SQLException {
         if (lhsNode instanceof BindParseNode) {
             context.getBindManager().addParamMetaData((BindParseNode)lhsNode, rhsExpr);
@@ -167,7 +168,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
             context.getBindManager().addParamMetaData((BindParseNode)rhsNode, lhsExpr);
         }
     }
-    
+
     @Override
     public Expression visitLeave(ComparisonParseNode node, List<Expression> children) throws SQLException {
         ParseNode lhsNode = node.getChildren().get(0);
@@ -175,7 +176,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
         Expression lhsExpr = children.get(0);
         Expression rhsExpr = children.get(1);
         CompareOp op = node.getFilterOp();
-        
+
         if (lhsNode instanceof RowValueConstructorParseNode && rhsNode instanceof RowValueConstructorParseNode) {
             int i = 0;
             for (; i < Math.min(lhsExpr.getChildren().size(),rhsExpr.getChildren().size()); i++) {
@@ -284,7 +285,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
     protected Expression addExpression(Expression expression) {
         return context.getExpressionManager().addIfAbsent(expression);
     }
-   
+
     @Override
     /**
      * @param node a function expression node
@@ -422,7 +423,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
         }
         return true;
     }
-    
+
     @Override
     public Expression visitLeave(CaseParseNode node, List<Expression> l) throws SQLException {
         final CaseExpression caseExpression = new CaseExpression(l);
@@ -484,16 +485,18 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
                 if (lhsMaxLength != null && lhsMaxLength != rhsLiteral.length()) {
                     return LiteralExpression.newConstant(false, rhs.isDeterministic());
                 }
-                CompareOp op = node.isNegate() ? CompareOp.NOT_EQUAL : CompareOp.EQUAL;
-                if (pattern.equals(rhsLiteral)) {
-                    return new ComparisonExpression(op, children);
-                } else {
-                    rhs = LiteralExpression.newConstant(rhsLiteral, PDataType.CHAR, rhs.isDeterministic());
-                    return new ComparisonExpression(op, Arrays.asList(lhs,rhs));
+                if (node.getLikeType() == LikeType.CASE_SENSITIVE) {
+                  CompareOp op = node.isNegate() ? CompareOp.NOT_EQUAL : CompareOp.EQUAL;
+                  if (pattern.equals(rhsLiteral)) {
+                      return new ComparisonExpression(op, children);
+                  } else {
+                      rhs = LiteralExpression.newConstant(rhsLiteral, PDataType.CHAR, rhs.isDeterministic());
+                      return new ComparisonExpression(op, Arrays.asList(lhs,rhs));
+                  }
                 }
             }
         }
-        Expression expression = new LikeExpression(children);
+        Expression expression = new LikeExpression(children, node.getLikeType());
         if (ExpressionUtil.isConstant(expression)) {
             ImmutableBytesWritable ptr = context.getTempPtr();
             if (!expression.evaluate(null, ptr)) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
index ab92a14..2b529cb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
@@ -50,6 +50,7 @@ import org.apache.phoenix.expression.function.ScalarFunction;
 import org.apache.phoenix.expression.visitor.TraverseNoExpressionVisitor;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.HintNode.Hint;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PColumn;
@@ -877,7 +878,8 @@ public class WhereOptimizer {
         @Override
         public Iterator<Expression> visitEnter(LikeExpression node) {
             // TODO: can we optimize something that starts with '_' like this: foo LIKE '_a%' ?
-            if (! (node.getChildren().get(1) instanceof LiteralExpression) || node.startsWithWildcard()) {
+            if (node.getLikeType() == LikeType.CASE_INSENSITIVE || // TODO: remove this when we optimize ILIKE
+                ! (node.getChildren().get(1) instanceof LiteralExpression) || node.startsWithWildcard()) {
                 return Iterators.emptyIterator();
             }
 
@@ -886,6 +888,7 @@ public class WhereOptimizer {
 
         @Override
         public KeySlots visitLeave(LikeExpression node, List<KeySlots> childParts) {
+            // TODO: optimize ILIKE by creating two ranges for the literal prefix: one with lower case, one with upper case
             if (childParts.isEmpty()) {
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
index 0cb9212..282fe41 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
@@ -18,15 +18,18 @@
 package org.apache.phoenix.expression;
 
 import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.EOFException;
 import java.io.IOException;
 import java.util.List;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.StringUtil;
@@ -46,10 +49,15 @@ import org.apache.phoenix.util.StringUtil;
  */
 public class LikeExpression extends BaseCompoundExpression {
     private static final Logger logger = LoggerFactory.getLogger(LikeExpression.class);
-    
+
     private static final String ZERO_OR_MORE = "\\E.*\\Q";
     private static final String ANY_ONE = "\\E.\\Q";
-    
+
+    /**
+     * Store whether this like expression has to be case sensitive or not.
+     */
+    private LikeType likeType;
+
     public static String unescapeLike(String s) {
         return StringUtil.replace(s, StringUtil.LIKE_ESCAPE_SEQS, StringUtil.LIKE_UNESCAPED_SEQS);
     }
@@ -67,7 +75,7 @@ public class LikeExpression extends BaseCompoundExpression {
     public static boolean hasWildcards(String s) {
         return indexOfWildcard(s) != -1;
     }
-    
+
     /**
      * Replace unescaped '*' and '?' in s with '%' and '_' respectively
      * such that the returned string may be used in a LIKE expression.
@@ -95,7 +103,7 @@ public class LikeExpression extends BaseCompoundExpression {
             if (underPos != -1 && (i == -1 || underPos < i)) {
                 i = underPos;
             }
-            
+
             if (i > 0 && s.charAt(i - 1) == '\\') {
                 // If we found protection then keep looking
                 buf.append(s.substring(j,i-1));
@@ -108,7 +116,7 @@ public class LikeExpression extends BaseCompoundExpression {
             j = ++i;
         }
     }
-    
+
     public static int indexOfWildcard(String s) {
         // Look for another unprotected % or _ in the middle
         if (s == null) {
@@ -125,7 +133,7 @@ public class LikeExpression extends BaseCompoundExpression {
             if (underPos != -1 && (i == -1 || underPos < i)) {
                 i = underPos;
             }
-            
+
             if (i > 0 && s.charAt(i - 1) == '\\') {
                 // If we found protection then keep looking
                 i++;
@@ -186,28 +194,40 @@ public class LikeExpression extends BaseCompoundExpression {
 //    }
 
     private Pattern pattern;
-    
+
     public LikeExpression() {
     }
 
-    public LikeExpression(List<Expression> children) {
+    public LikeExpression(List<Expression> children, LikeType likeType) {
         super(children);
+        this.likeType = likeType;
         init();
     }
-    
+
+    public LikeType getLikeType () {
+      return likeType;
+    }
+
     public boolean startsWithWildcard() {
         return pattern != null && pattern.pattern().startsWith("\\Q\\E");
     }
-    
+
     private void init() {
         Expression e = getPatternExpression();
         if (e instanceof LiteralExpression) {
             LiteralExpression patternExpression = (LiteralExpression)e;
             String value = (String)patternExpression.getValue();
-            pattern = Pattern.compile(toPattern(value));
+            pattern = compilePattern(value);
         }
     }
 
+    protected Pattern compilePattern (String value) {
+        if (likeType == LikeType.CASE_SENSITIVE)
+            return Pattern.compile(toPattern(value));
+        else
+            return Pattern.compile("(?i)" + toPattern(value));
+    }
+
     private Expression getStrExpression() {
         return children.get(0);
     }
@@ -227,12 +247,12 @@ public class LikeExpression extends BaseCompoundExpression {
                 return false;
             }
             String value = (String)PDataType.VARCHAR.toObject(ptr, getPatternExpression().getSortOrder());
-            pattern = Pattern.compile(toPattern(value));
+            pattern = compilePattern(value);
             if (logger.isDebugEnabled()) {
                 logger.debug("LIKE pattern is expression: " + pattern.pattern());
             }
         }
-        
+
         if (!getStrExpression().evaluate(tuple, ptr)) {
             if (logger.isDebugEnabled()) {
                 logger.debug("LIKE is FALSE: child expression is null");
@@ -242,7 +262,7 @@ public class LikeExpression extends BaseCompoundExpression {
         if (ptr.getLength() == 0) {
             return true;
         }
-        
+
         String value = (String)PDataType.VARCHAR.toObject(ptr, getStrExpression().getSortOrder());
         boolean matched = pattern.matcher(value).matches();
         ptr.set(matched ? PDataType.TRUE_BYTES : PDataType.FALSE_BYTES);
@@ -256,13 +276,24 @@ public class LikeExpression extends BaseCompoundExpression {
     public void readFields(DataInput input) throws IOException {
         super.readFields(input);
         init();
+        try {
+            likeType = LikeType.values()[WritableUtils.readVInt(input)];
+        } catch (EOFException e) {
+            likeType = LikeType.CASE_SENSITIVE;
+        }
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeVInt(output, likeType.ordinal());
     }
 
     @Override
     public PDataType getDataType() {
         return PDataType.BOOLEAN;
     }
-    
+
     @Override
     public final <T> T accept(ExpressionVisitor<T> visitor) {
         List<T> l = acceptChildren(visitor, visitor.visitEnter(this));
@@ -292,7 +323,7 @@ public class LikeExpression extends BaseCompoundExpression {
         pattern.lastIndexOf(ANY_ONE, pattern.length() - endsWith.length() - 1) == -1 &&
         pattern.lastIndexOf(ZERO_OR_MORE, pattern.length() - endsWith.length() - 1) == -1;
     }
-    
+
     @Override
     public String toString() {
         return (children.get(0) + " LIKE " + children.get(1));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
index 27e5acc..9cec70e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
@@ -31,17 +31,26 @@ import java.util.List;
  * @since 0.1
  */
 public class LikeParseNode extends BinaryParseNode {
+
+    public enum LikeType {CASE_SENSITIVE, CASE_INSENSITIVE}
+
     private final boolean negate;
+    private final LikeType likeType;
 
-    LikeParseNode(ParseNode lhs, ParseNode rhs, boolean negate) {
+    LikeParseNode(ParseNode lhs, ParseNode rhs, boolean negate, LikeType likeType) {
         super(lhs, rhs);
         this.negate = negate;
+        this.likeType = likeType;
     }
-    
+
     public boolean isNegate() {
         return negate;
     }
 
+    public LikeType getLikeType() {
+      return likeType;
+    }
+
     @Override
     public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
         List<T> l = Collections.emptyList();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index 16423e0..4af65b9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -39,6 +39,7 @@ import org.apache.phoenix.expression.function.FunctionExpression;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable.IndexType;
@@ -86,12 +87,12 @@ public class ParseNodeFactory {
             this.upperName = lowerName;
             this.argCount = argCount;
         }
-        
+
         @Override
         public String toString() {
             return upperName;
         }
-        
+
         @Override
         public int hashCode() {
             final int prime = 31;
@@ -187,7 +188,7 @@ public class ParseNodeFactory {
     public AliasedNode aliasedNode(String alias, ParseNode expression) {
     	return new AliasedNode(alias, expression);
     }
-    
+
     public AddParseNode add(List<ParseNode> children) {
         return new AddParseNode(children);
     }
@@ -199,7 +200,7 @@ public class ParseNodeFactory {
     public MultiplyParseNode multiply(List<ParseNode> children) {
         return new MultiplyParseNode(children);
     }
-    
+
     public ModulusParseNode modulus(List<ParseNode> children) {
         return new ModulusParseNode(children);
     }
@@ -207,11 +208,11 @@ public class ParseNodeFactory {
     public AndParseNode and(List<ParseNode> children) {
         return new AndParseNode(children);
     }
-    
+
     public FamilyWildcardParseNode family(String familyName){
     	    return new FamilyWildcardParseNode(familyName, false);
     }
-    
+
     public TableWildcardParseNode tableWildcard(TableName tableName) {
         return new TableWildcardParseNode(tableName, false);
     }
@@ -235,7 +236,7 @@ public class ParseNodeFactory {
     public ColumnParseNode column(TableName tableName, String name, String alias) {
         return new ColumnParseNode(tableName,name,alias);
     }
-    
+
     public ColumnName columnName(String columnName) {
         return new ColumnName(columnName);
     }
@@ -255,7 +256,7 @@ public class ParseNodeFactory {
     public ColumnDef columnDef(ColumnName columnDefName, String sqlTypeName, boolean isNull, Integer maxLength, Integer scale, boolean isPK, SortOrder sortOrder) {
         return new ColumnDef(columnDefName, sqlTypeName, isNull, maxLength, scale, isPK, sortOrder);
     }
-    
+
     public ColumnDef columnDef(ColumnName columnDefName, String sqlTypeName, boolean isArray, Integer arrSize, Boolean isNull, Integer maxLength, Integer scale, boolean isPK, 
         	SortOrder sortOrder) {
         return new ColumnDef(columnDefName, sqlTypeName, isArray, arrSize, isNull, maxLength, scale, isPK, sortOrder);
@@ -264,54 +265,54 @@ public class ParseNodeFactory {
     public PrimaryKeyConstraint primaryKey(String name, List<Pair<ColumnName, SortOrder>> columnNameAndSortOrder) {
         return new PrimaryKeyConstraint(name, columnNameAndSortOrder);
     }
-    
+
     public CreateTableStatement createTable(TableName tableName, ListMultimap<String,Pair<String,Object>> props, List<ColumnDef> columns, PrimaryKeyConstraint pkConstraint, List<ParseNode> splits, PTableType tableType, boolean ifNotExists, TableName baseTableName, ParseNode tableTypeIdNode, int bindCount) {
         return new CreateTableStatement(tableName, props, columns, pkConstraint, splits, tableType, ifNotExists, baseTableName, tableTypeIdNode, bindCount);
     }
-    
+
     public CreateIndexStatement createIndex(NamedNode indexName, NamedTableNode dataTable, PrimaryKeyConstraint pkConstraint, List<ColumnName> includeColumns, List<ParseNode> splits, ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, IndexType indexType, int bindCount) {
         return new CreateIndexStatement(indexName, dataTable, pkConstraint, includeColumns, splits, props, ifNotExists, indexType, bindCount);
     }
-    
+
     public CreateSequenceStatement createSequence(TableName tableName, ParseNode startsWith,
             ParseNode incrementBy, ParseNode cacheSize, ParseNode minValue, ParseNode maxValue,
             boolean cycle, boolean ifNotExits, int bindCount) {
         return new CreateSequenceStatement(tableName, startsWith, incrementBy, cacheSize, minValue,
                 maxValue, cycle, ifNotExits, bindCount);
     }
-    
+
     public DropSequenceStatement dropSequence(TableName tableName, boolean ifExits, int bindCount){
         return new DropSequenceStatement(tableName, ifExits, bindCount);
     }
-    
-	public SequenceValueParseNode currentValueFor(TableName tableName) {
-		return new SequenceValueParseNode(tableName, SequenceValueParseNode.Op.CURRENT_VALUE);
-	}
-    
+
+    public SequenceValueParseNode currentValueFor(TableName tableName) {
+        return new SequenceValueParseNode(tableName, SequenceValueParseNode.Op.CURRENT_VALUE);
+    }
+
     public SequenceValueParseNode nextValueFor(TableName tableName) {
         return new SequenceValueParseNode(tableName, SequenceValueParseNode.Op.NEXT_VALUE);
     }
-    
+
     public AddColumnStatement addColumn(NamedTableNode table,  PTableType tableType, List<ColumnDef> columnDefs, boolean ifNotExists, Map<String,Object> props) {
         return new AddColumnStatement(table, tableType, columnDefs, ifNotExists, props);
     }
-    
+
     public DropColumnStatement dropColumn(NamedTableNode table,  PTableType tableType, List<ColumnName> columnNodes, boolean ifExists) {
         return new DropColumnStatement(table, tableType, columnNodes, ifExists);
     }
-    
+
     public DropTableStatement dropTable(TableName tableName, PTableType tableType, boolean ifExists, boolean cascade) {
         return new DropTableStatement(tableName, tableType, ifExists, cascade);
     }
-    
+
     public DropIndexStatement dropIndex(NamedNode indexName, TableName tableName, boolean ifExists) {
         return new DropIndexStatement(indexName, tableName, ifExists);
     }
-    
+
     public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
         return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state);
     }
-    
+
     public TableName table(String schemaName, String tableName) {
         return TableName.createNormalized(schemaName,tableName);
     }
@@ -339,7 +340,7 @@ public class ParseNodeFactory {
     public DivideParseNode divide(List<ParseNode> children) {
         return new DivideParseNode(children);
     }
-    
+
     public UpdateStatisticsStatement updateStatistics(NamedTableNode table) {
       return new UpdateStatisticsStatement(table);
     }
@@ -354,7 +355,7 @@ public class ParseNodeFactory {
             throw new UnsupportedOperationException("DISTINCT not supported with " + name);
         }
     }
-    
+
     public FunctionParseNode arrayElemRef(List<ParseNode> args) {
     	return function(ARRAY_ELEM, args);
     }
@@ -410,10 +411,9 @@ public class ParseNodeFactory {
         for (JoinPartNode part : parts) {
             table = new JoinTableNode(part.getType(), table, part.getTable(), part.getOnNode());
         }
-        
         return table;
     }
-    
+
     JoinPartNode joinPart(JoinType type, ParseNode onNode, TableNode table) {
         return new JoinPartNode(type, onNode, table);
     }
@@ -426,11 +426,10 @@ public class ParseNodeFactory {
         return new DerivedTableNode(alias, select);
     }
 
-    public LikeParseNode like(ParseNode lhs, ParseNode rhs, boolean negate) {
-        return new LikeParseNode(lhs, rhs, negate);
+    public LikeParseNode like(ParseNode lhs, ParseNode rhs, boolean negate, LikeType likeType) {
+        return new LikeParseNode(lhs, rhs, negate, likeType);
     }
 
-
     public LiteralParseNode literal(Object value) {
         return new LiteralParseNode(value);
     }
@@ -450,11 +449,11 @@ public class ParseNodeFactory {
     public CastParseNode cast(ParseNode expression, String dataType, Integer maxLength, Integer scale, boolean arr) {
         return new CastParseNode(expression, dataType, maxLength, scale, arr);
     }
-    
+
     public ParseNode rowValueConstructor(List<ParseNode> l) {
         return new RowValueConstructorParseNode(l);
     }
-    
+
     private void checkTypeMatch (PDataType expectedType, PDataType actualType) throws SQLException {
         if (!expectedType.isCoercibleTo(actualType)) {
             throw TypeMismatchException.newException(expectedType, actualType);
@@ -509,7 +508,7 @@ public class ParseNodeFactory {
     public ArrayAllComparisonNode wrapInAll(CompareOp op, ParseNode lhs, ParseNode rhs) {
         return new ArrayAllComparisonNode(rhs, comparison(op, lhs, elementRef(Arrays.<ParseNode>asList(rhs, literal(1)))));
     }
-    
+
     public ArrayElemRefNode elementRef(List<ParseNode> parseNode) {
         return new ArrayElemRefNode(parseNode);
     }
@@ -566,18 +565,18 @@ public class ParseNodeFactory {
     public OuterJoinParseNode outer(ParseNode node) {
         return new OuterJoinParseNode(node);
     }
-    
+
     public SelectStatement select(List<? extends TableNode> from, HintNode hint, boolean isDistinct, List<AliasedNode> select, ParseNode where,
             List<ParseNode> groupBy, ParseNode having, List<OrderByNode> orderBy, LimitNode limit, int bindCount, boolean isAggregate, boolean hasSequence) {
 
         return new SelectStatement(from, hint, isDistinct, select, where, groupBy == null ? Collections.<ParseNode>emptyList() : groupBy, having,
                 orderBy == null ? Collections.<OrderByNode>emptyList() : orderBy, limit, bindCount, isAggregate, hasSequence);
     }
-    
+
     public UpsertStatement upsert(NamedTableNode table, HintNode hint, List<ColumnName> columns, List<ParseNode> values, SelectStatement select, int bindCount) {
         return new UpsertStatement(table, hint, columns, values, select, bindCount);
     }
-    
+
     public DeleteStatement delete(NamedTableNode table, HintNode hint, ParseNode node, List<OrderByNode> orderBy, LimitNode limit, int bindCount) {
         return new DeleteStatement(table, hint, node, orderBy, limit, bindCount);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
index d96850c..34d22d6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
@@ -37,7 +37,7 @@ import com.google.common.collect.Maps;
  * @since 0.1
  */
 public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
-    
+
     protected static final ParseNodeFactory NODE_FACTORY = new ParseNodeFactory();
 
     public static ParseNode rewrite(ParseNode where, ParseNodeRewriter rewriter) throws SQLException {
@@ -47,7 +47,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
         rewriter.reset();
         return where.accept(rewriter);
     }
-    
+
     /**
      * Rewrite the select statement by switching any constants to the right hand side
      * of the expression.
@@ -118,7 +118,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
                 }
             }
         }
-        
+
         List<ParseNode> groupByNodes = statement.getGroupBy();
         List<ParseNode> normGroupByNodes = groupByNodes;
         for (int i = 0; i < groupByNodes.size(); i++) {
@@ -154,7 +154,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
             normOrderByNodes.add(NODE_FACTORY.orderBy(normNode, orderByNode.isNullsLast(), orderByNode.isAscending()));
         }
-        
+
         // Return new SELECT statement with updated WHERE clause
         if (normFrom == from && 
         		normWhere == where && 
@@ -168,7 +168,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
                 normSelectNodes, normWhere, normGroupByNodes, normHaving, normOrderByNodes,
                 statement.getLimit(), statement.getBindCount(), statement.isAggregate(), statement.hasSequence());
     }
-    
+
     private Map<String, ParseNode> getAliasMap() {
         return aliasMap;
     }
@@ -176,38 +176,38 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
     private final ColumnResolver resolver;
     private final Map<String, ParseNode> aliasMap;
     private int nodeCount;
-    
+
     public boolean isTopLevel() {
         return nodeCount == 0;
     }
-    
+
     protected ParseNodeRewriter() {
         this.resolver = null;
         this.aliasMap = null;
     }
-    
+
     protected ParseNodeRewriter(ColumnResolver resolver) {
         this.resolver = resolver;
         this.aliasMap = null;
     }
-    
+
     protected ParseNodeRewriter(ColumnResolver resolver, int maxAliasCount) {
         this.resolver = resolver;
         this.aliasMap = Maps.newHashMapWithExpectedSize(maxAliasCount);
     }
-    
+
     protected ColumnResolver getResolver() {
         return resolver;
     }
-    
+
     protected void reset() {
         this.nodeCount = 0;
     }
-    
+
     private static interface CompoundNodeFactory {
         ParseNode createNode(List<ParseNode> children);
     }
-    
+
     private ParseNode leaveCompoundNode(CompoundParseNode node, List<ParseNode> children, CompoundNodeFactory factory) {
         if (children.equals(node.getChildren())) {
             return node;
@@ -215,7 +215,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             return factory.createNode(children);
         }
     }
-    
+
     @Override
     public ParseNode visitLeave(AndParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -235,7 +235,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(SubtractParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -245,7 +245,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(AddParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -255,7 +255,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(MultiplyParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -265,7 +265,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(DivideParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -275,7 +275,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(ModulusParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -285,7 +285,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(final FunctionParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -295,7 +295,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(CaseParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -311,11 +311,11 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
             @Override
             public ParseNode createNode(List<ParseNode> children) {
-                return NODE_FACTORY.like(children.get(0),children.get(1),node.isNegate());
+                return NODE_FACTORY.like(children.get(0),children.get(1),node.isNegate(), node.getLikeType());
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(NotParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -325,7 +325,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(final CastParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -335,7 +335,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(final InListParseNode node, List<ParseNode> nodes) throws SQLException {
         ParseNode normNode = leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -346,7 +346,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
         });
         return normNode;
     }
-    
+
     @Override
     public ParseNode visitLeave(final IsNullParseNode node, List<ParseNode> nodes) throws SQLException {
         return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -356,7 +356,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visitLeave(final ComparisonParseNode node, List<ParseNode> nodes) throws SQLException {
         ParseNode normNode = leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
@@ -381,7 +381,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             }
         });
     }
-    
+
     @Override
     public ParseNode visit(ColumnParseNode node) throws SQLException {
         // If we're resolving aliases and we have an unqualified ColumnParseNode,
@@ -407,33 +407,33 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
     public ParseNode visit(LiteralParseNode node) throws SQLException {
         return node;
     }
-    
+
     @Override
     public ParseNode visit(BindParseNode node) throws SQLException {
         return node;
     }
-    
+
     @Override
     public ParseNode visit(WildcardParseNode node) throws SQLException {
         return node;
     }
-    
+
     @Override
     public ParseNode visit(TableWildcardParseNode node) throws SQLException {
         return node;
     }
-    
+
     @Override
     public ParseNode visit(FamilyWildcardParseNode node) throws SQLException {
         return node;
     }
-    
+
     @Override
     public List<ParseNode> newElementList(int size) {
         nodeCount += size;
         return new ArrayList<ParseNode>(size);
     }
-    
+
     @Override
     public ParseNode visitLeave(StringConcatParseNode node, List<ParseNode> l) throws SQLException {
         return leaveCompoundNode(node, l, new CompoundNodeFactory() {
@@ -480,7 +480,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
                 flattenedChildren.addAll(child.getChildren());
             }
         }
-        
+
         return leaveCompoundNode(node, flattenedChildren, new CompoundNodeFactory() {
             @Override
             public ParseNode createNode(List<ParseNode> children) {
@@ -506,11 +506,11 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
 	
 	private static class TableNodeRewriter implements TableNodeVisitor<TableNode> {
 	    private final ParseNodeRewriter parseNodeRewriter;
-	    
+
 	    public TableNodeRewriter(ParseNodeRewriter parseNodeRewriter) {
 	        this.parseNodeRewriter = parseNodeRewriter;
 	    }
-	    
+
 	    public void reset() {
 	    }
 
@@ -530,21 +530,19 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
             ParseNode normOnNode = onNode.accept(parseNodeRewriter);
             if (lhsNode == normLhsNode && rhsNode == normRhsNode && onNode == normOnNode)
                 return joinNode;
-            
+
             return NODE_FACTORY.join(joinNode.getType(), normLhsNode, normRhsNode, normOnNode);
         }
 
         @Override
         public TableNode visit(NamedTableNode namedTableNode) throws SQLException {
             return namedTableNode;
-            
         }
 
         @Override
         public TableNode visit(DerivedTableNode subselectNode) throws SQLException {
             return subselectNode;
         }
-	    
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/test/java/org/apache/phoenix/expression/ILikeExpressionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/ILikeExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/ILikeExpressionTest.java
new file mode 100644
index 0000000..2c4af74
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/ILikeExpressionTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.phoenix.expression;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
+import org.junit.Test;
+
+public class ILikeExpressionTest {
+    public boolean testExpression (String value, String expression) {
+      LiteralExpression v = LiteralExpression.newConstant(value);
+      LiteralExpression p = LiteralExpression.newConstant(expression);
+      List<Expression> children = Arrays.<Expression>asList(v,p);
+      LikeExpression e = new LikeExpression(children, LikeType.CASE_INSENSITIVE);
+      ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+      boolean evaluated = e.evaluate(null, ptr);
+      Boolean result = (Boolean)e.getDataType().toObject(ptr);
+      assertTrue(evaluated);
+      return result;
+    }
+
+    @Test
+    public void testStartWildcard() throws Exception {
+        assertEquals(Boolean.FALSE, testExpression ("149na7-app1-2-", "%-w"));
+        assertEquals(Boolean.TRUE, testExpression ("149na7-app1-2-", "%-2%"));
+        assertEquals(Boolean.TRUE, testExpression ("149na7-app1-2-", "%4%7%2%"));
+        assertEquals(Boolean.FALSE, testExpression ("149na7-app1-2-", "%9%4%2%"));
+    }
+
+    @Test
+    public void testCaseSensitive() throws Exception {
+        assertEquals(Boolean.TRUE, testExpression ("test", "test"));
+        assertEquals(Boolean.TRUE, testExpression ("test", "teSt"));
+    }
+
+    @Test
+    public void testStartWildcardAndCaseInsensitive() throws Exception {
+        assertEquals(Boolean.TRUE, testExpression ("test", "%s%"));
+        assertEquals(Boolean.TRUE, testExpression ("test", "%S%"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/test/java/org/apache/phoenix/expression/LikeExpressionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/LikeExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/LikeExpressionTest.java
index e36c187..81b7124 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/LikeExpressionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/LikeExpressionTest.java
@@ -24,19 +24,38 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
 import org.junit.Test;
 
 public class LikeExpressionTest {
+    public boolean testExpression (String value, String expression) {
+      LiteralExpression v = LiteralExpression.newConstant(value);
+      LiteralExpression p = LiteralExpression.newConstant(expression);
+      List<Expression> children = Arrays.<Expression>asList(v,p);
+      LikeExpression e = new LikeExpression(children, LikeType.CASE_SENSITIVE);
+      ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+      boolean evaluated = e.evaluate(null, ptr);
+      Boolean result = (Boolean)e.getDataType().toObject(ptr);
+      assertTrue(evaluated);
+      return result;
+    }
     @Test
     public void testStartWildcard() throws Exception {
-        LiteralExpression v = LiteralExpression.newConstant("149na7-app1-2-");
-        LiteralExpression p = LiteralExpression.newConstant("%-w");
-        List<Expression> children = Arrays.<Expression>asList(v,p);
-        LikeExpression e = new LikeExpression(children);
-        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-        boolean evaluated = e.evaluate(null, ptr);
-        Boolean result = (Boolean)e.getDataType().toObject(ptr);
-        assertTrue(evaluated);
-        assertEquals(Boolean.FALSE,result);
+        assertEquals(Boolean.FALSE, testExpression ("149na7-app1-2-", "%-w"));
+        assertEquals(Boolean.TRUE, testExpression ("149na7-app1-2-", "%-2%"));
+        assertEquals(Boolean.TRUE, testExpression ("149na7-app1-2-", "%4%7%2%"));
+        assertEquals(Boolean.FALSE, testExpression ("149na7-app1-2-", "%9%4%2%"));
+    }
+
+    @Test
+    public void testCaseSensitive() throws Exception {
+        assertEquals(Boolean.TRUE, testExpression ("test", "test"));
+        assertEquals(Boolean.FALSE, testExpression ("test", "teSt"));
+    }
+
+    @Test
+    public void testStartWildcardAndCaseInsensitive() throws Exception {
+        assertEquals(Boolean.TRUE, testExpression ("test", "%s%"));
+        assertEquals(Boolean.FALSE, testExpression ("test", "%S%"));
     }
-}
+ }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6908c90b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index 17a5e4f..cb072ed 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
@@ -72,6 +72,7 @@ import org.apache.phoenix.filter.SingleKeyValueComparisonFilter;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixPreparedStatement;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PColumn;
@@ -188,7 +189,7 @@ public class TestUtil {
     public static final String JOIN_CUSTOMER_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_CUSTOMER_TABLE;
     public static final String JOIN_ITEM_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_ITEM_TABLE;
     public static final String JOIN_SUPPLIER_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_SUPPLIER_TABLE;
- 
+
     /**
      * Read-only properties used by all tests
      */
@@ -249,9 +250,13 @@ public class TestUtil {
     }
 
     public static Expression like(Expression e, Object o) {
-        return  new LikeExpression(Arrays.asList(e, LiteralExpression.newConstant(o)));
+        return  new LikeExpression(Arrays.asList(e, LiteralExpression.newConstant(o)), LikeType.CASE_SENSITIVE);
     }
 
+    public static Expression ilike(Expression e, Object o) {
+      return  new LikeExpression(Arrays.asList(e, LiteralExpression.newConstant(o)), LikeType.CASE_INSENSITIVE);
+  }
+
     public static Expression substr(Expression e, Object offset, Object length) {
         return  new SubstrFunction(Arrays.asList(e, LiteralExpression.newConstant(offset), LiteralExpression.newConstant(length)));
     }