You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by la...@apache.org on 2016/06/02 20:22:30 UTC

[11/16] phoenix git commit: PHOENIX-1684 Functional Index using REGEXP_SUBSTR doesn't work correctly

PHOENIX-1684 Functional Index using REGEXP_SUBSTR doesn't work correctly


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

Branch: refs/heads/4.x-HBase-1.x
Commit: a8b27e3f010d15d3f3b519c38fbb052ebb4a6cdb
Parents: 250474d
Author: Thomas <td...@salesforce.com>
Authored: Thu Mar 19 13:57:27 2015 -0700
Committer: Thomas <td...@salesforce.com>
Committed: Tue Mar 24 15:56:25 2015 -0700

----------------------------------------------------------------------
 .../end2end/index/IndexExpressionIT.java        | 161 ++++++++++++++-----
 .../phoenix/compile/PostIndexDDLCompiler.java   |   4 +-
 .../parse/IndexExpressionParseNodeRewriter.java |  30 +---
 .../apache/phoenix/schema/MetaDataClient.java   |   4 +-
 .../org/apache/phoenix/util/StringUtil.java     |   5 +
 .../phoenix/compile/QueryCompilerTest.java      |  22 ++-
 6 files changed, 153 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a8b27e3f/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
index 1e3733b..0203e35 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
@@ -1202,54 +1202,60 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testViewUsesTableIndex() throws Exception {
-        ResultSet rs;
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE t (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, s1 VARCHAR, s2 VARCHAR, s3 VARCHAR, s4 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2))";
-        conn.createStatement().execute(ddl);
-        conn.createStatement().execute("CREATE INDEX i1 ON t(k2, s2, s3, s1)");
-        conn.createStatement().execute("CREATE INDEX i2 ON t(k2, s2||'_'||s3, s1, s4)");
-        
-        ddl = "CREATE VIEW v AS SELECT * FROM t WHERE s1 = 'foo'";
-        conn.createStatement().execute(ddl);
-        conn.createStatement().execute("UPSERT INTO t VALUES(1,1,'foo','abc','cab')");
-        conn.createStatement().execute("UPSERT INTO t VALUES(2,2,'bar','xyz','zyx')");
-        conn.commit();
-        
-        rs = conn.createStatement().executeQuery("SELECT count(*) FROM v");
-        assertTrue(rs.next());
-        assertEquals(1, rs.getLong(1));
-        assertFalse(rs.next());
-        
-        //i2 should be used since it contains s3||'_'||s4 i
-        String query = "SELECT s2||'_'||s3 FROM v WHERE k2=1 AND (s2||'_'||s3)='abc_cab'";
-        rs = conn.createStatement(  ).executeQuery("EXPLAIN " + query);
-        String queryPlan = QueryUtil.getExplainPlan(rs);
-        assertEquals(
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER I2 [1,'abc_cab','foo']\n" + 
-                "    SERVER FILTER BY FIRST KEY ONLY", queryPlan);
-        rs = conn.createStatement().executeQuery(query);
-        assertTrue(rs.next());
-        assertEquals("abc_cab", rs.getString(1));
-        assertFalse(rs.next());
-        
-        conn.createStatement().execute("ALTER VIEW v DROP COLUMN s4");
-        //i2 cannot be used since s4 has been dropped from the view, so i1 will be used 
-        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-        queryPlan = QueryUtil.getExplainPlan(rs);
-        assertEquals(
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER I1 [1]\n" + 
-                "    SERVER FILTER BY FIRST KEY ONLY AND ((\"S2\" || '_' || \"S3\") = 'abc_cab' AND \"S1\" = 'foo')", queryPlan);
-        rs = conn.createStatement().executeQuery(query);
-        assertTrue(rs.next());
-        assertEquals("abc_cab", rs.getString(1));
-        assertFalse(rs.next());    
+        try 
+        {
+        	ResultSet rs;
+	        String ddl = "CREATE TABLE t (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, s1 VARCHAR, s2 VARCHAR, s3 VARCHAR, s4 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2))";
+	        conn.createStatement().execute(ddl);
+	        conn.createStatement().execute("CREATE INDEX i1 ON t(k2, s2, s3, s1)");
+	        conn.createStatement().execute("CREATE INDEX i2 ON t(k2, s2||'_'||s3, s1, s4)");
+	        
+	        ddl = "CREATE VIEW v AS SELECT * FROM t WHERE s1 = 'foo'";
+	        conn.createStatement().execute(ddl);
+	        conn.createStatement().execute("UPSERT INTO t VALUES(1,1,'foo','abc','cab')");
+	        conn.createStatement().execute("UPSERT INTO t VALUES(2,2,'bar','xyz','zyx')");
+	        conn.commit();
+	        
+	        rs = conn.createStatement().executeQuery("SELECT count(*) FROM v");
+	        assertTrue(rs.next());
+	        assertEquals(1, rs.getLong(1));
+	        assertFalse(rs.next());
+	        
+	        //i2 should be used since it contains s3||'_'||s4 i
+	        String query = "SELECT s2||'_'||s3 FROM v WHERE k2=1 AND (s2||'_'||s3)='abc_cab'";
+	        rs = conn.createStatement(  ).executeQuery("EXPLAIN " + query);
+	        String queryPlan = QueryUtil.getExplainPlan(rs);
+	        assertEquals(
+	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER I2 [1,'abc_cab','foo']\n" + 
+	                "    SERVER FILTER BY FIRST KEY ONLY", queryPlan);
+	        rs = conn.createStatement().executeQuery(query);
+	        assertTrue(rs.next());
+	        assertEquals("abc_cab", rs.getString(1));
+	        assertFalse(rs.next());
+	        
+	        conn.createStatement().execute("ALTER VIEW v DROP COLUMN s4");
+	        //i2 cannot be used since s4 has been dropped from the view, so i1 will be used 
+	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+	        queryPlan = QueryUtil.getExplainPlan(rs);
+	        assertEquals(
+	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER I1 [1]\n" + 
+	                "    SERVER FILTER BY FIRST KEY ONLY AND ((\"S2\" || '_' || \"S3\") = 'abc_cab' AND \"S1\" = 'foo')", queryPlan);
+	        rs = conn.createStatement().executeQuery(query);
+	        assertTrue(rs.next());
+	        assertEquals("abc_cab", rs.getString(1));
+	        assertFalse(rs.next());    
+        }
+        finally {
+        	conn.close();
+        }
     }
     
 	@Test
 	public void testExpressionThrowsException() throws Exception {
 		Connection conn = DriverManager.getConnection(getUrl());
-		String ddl = "CREATE TABLE t (k1 INTEGER PRIMARY KEY, k2 INTEGER)";
 		try {
+			String ddl = "CREATE TABLE t (k1 INTEGER PRIMARY KEY, k2 INTEGER)";
 			conn.createStatement().execute(ddl);
 			ddl = "CREATE INDEX i on t(k1/k2)";
 			conn.createStatement().execute(ddl);
@@ -1261,6 +1267,79 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 			conn.commit();
 			fail();
 		} catch (CommitException e) {
+		} finally {
+			conn.close();
+		}
+	}
+	
+	@Test
+	public void testImmutableCaseSensitiveFunctionIndex() throws Exception {
+		helpTestCaseSensitiveFunctionIndex(false, false);
+	}
+
+	@Test
+	public void testImmutableLocalCaseSensitiveFunctionIndex() throws Exception {
+		helpTestCaseSensitiveFunctionIndex(false, true);
+	}
+
+	@Test
+	public void testMutableCaseSensitiveFunctionIndex() throws Exception {
+		helpTestCaseSensitiveFunctionIndex(true, false);
+	}
+
+	@Test
+	public void testMutableLocalCaseSensitiveFunctionIndex() throws Exception {
+		helpTestCaseSensitiveFunctionIndex(true, true);
+	}
+
+	protected void helpTestCaseSensitiveFunctionIndex(boolean mutable,
+			boolean localIndex) throws Exception {
+		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+		Connection conn = DriverManager.getConnection(getUrl(), props);
+		try {
+			conn.createStatement().execute(
+					"CREATE TABLE t (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR) "
+							+ (mutable ? "IMMUTABLE_ROWS=true" : ""));
+			String query = "SELECT * FROM t";
+			ResultSet rs = conn.createStatement().executeQuery(query);
+			assertFalse(rs.next());
+			String ddl = "CREATE " + (localIndex ? "LOCAL" : "")
+					+ " INDEX idx ON t (REGEXP_SUBSTR(v,'id:\\\\w+'))";
+			PreparedStatement stmt = conn.prepareStatement(ddl);
+			stmt.execute();
+			query = "SELECT * FROM idx";
+			rs = conn.createStatement().executeQuery(query);
+			assertFalse(rs.next());
+
+			stmt = conn.prepareStatement("UPSERT INTO t VALUES(?,?)");
+			stmt.setString(1, "k1");
+			stmt.setString(2, "{id:id1}");
+			stmt.execute();
+			stmt.setString(1, "k2");
+			stmt.setString(2, "{id:id2}");
+			stmt.execute();
+			conn.commit();
+			
+			query = "SELECT k FROM t WHERE REGEXP_SUBSTR(v,'id:\\\\w+') = 'id:id1'";
+			rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+			if (localIndex) {
+				assertEquals(
+						"CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_T [-32768,'id:id1']\n"
+								+ "    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT",
+						QueryUtil.getExplainPlan(rs));
+			} else {
+				assertEquals(
+						"CLIENT PARALLEL 1-WAY RANGE SCAN OVER IDX ['id:id1']\n"
+								+ "    SERVER FILTER BY FIRST KEY ONLY",
+						QueryUtil.getExplainPlan(rs));
+			}
+
+			rs = conn.createStatement().executeQuery(query);
+			assertTrue(rs.next());
+			assertEquals("k1", rs.getString(1));
+			assertFalse(rs.next());
+		} finally {
+			conn.close();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a8b27e3f/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
index 5836b99..9f99f1c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
@@ -27,6 +27,7 @@ import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.StringUtil;
 
 import com.google.common.collect.Lists;
 
@@ -73,7 +74,8 @@ public class PostIndexDDLCompiler {
         for (int i = posOffset; i < nIndexPKColumns; i++) {
             PColumn col = indexPKColumns.get(i);
             String indexColName = col.getName().getString();
-            String dataColName = col.getExpressionStr();
+            // need to escape backslash as this used in the SELECT statement
+            String dataColName = StringUtil.escapeBackslash(col.getExpressionStr());
             dataColumns.append(dataColName).append(",");
             indexColumns.append('"').append(indexColName).append("\",");
             indexColumnNames.add(indexColName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a8b27e3f/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexExpressionParseNodeRewriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexExpressionParseNodeRewriter.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexExpressionParseNodeRewriter.java
index 43cb9f3..0273041 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexExpressionParseNodeRewriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexExpressionParseNodeRewriter.java
@@ -37,26 +37,6 @@ public class IndexExpressionParseNodeRewriter extends ParseNodeRewriter {
 
     private final Map<ParseNode, ParseNode> indexedParseNodeToColumnParseNodeMap;
     
-    private static class ColumnParseNodeVisitor extends StatelessTraverseAllParseNodeVisitor {
-        
-        private boolean isParseNodeCaseSensitive;
-        
-        public void reset() {
-            this.isParseNodeCaseSensitive = false;
-        }
-        
-        @Override
-        public Void visit(ColumnParseNode node) throws SQLException {
-            isParseNodeCaseSensitive = isParseNodeCaseSensitive  || node.isCaseSensitive() || node.isTableNameCaseSensitive();
-            return null;
-        }
-        
-        public boolean isParseNodeCaseSensitive() {
-            return isParseNodeCaseSensitive;
-        }
-        
-    }
-
     public IndexExpressionParseNodeRewriter(PTable index, PhoenixConnection connection) throws SQLException {
         indexedParseNodeToColumnParseNodeMap = Maps.newHashMapWithExpectedSize(index.getColumns().size());
         NamedTableNode tableNode = NamedTableNode.create(null,
@@ -66,21 +46,13 @@ public class IndexExpressionParseNodeRewriter extends ParseNodeRewriter {
         StatementContext context = new StatementContext(new PhoenixStatement(connection), dataResolver);
         IndexStatementRewriter rewriter = new IndexStatementRewriter(dataResolver, null);
         ExpressionCompiler expressionCompiler = new ExpressionCompiler(context);
-        ColumnParseNodeVisitor columnParseNodeVisitor = new ColumnParseNodeVisitor();
         int indexPosOffset = (index.getBucketNum() == null ? 0 : 1) + (index.isMultiTenant() ? 1 : 0) + (index.getViewIndexId() == null ? 0 : 1);
         List<PColumn> pkColumns = index.getPKColumns();
 		for (int i=indexPosOffset; i<pkColumns.size(); ++i) {
         	PColumn column = pkColumns.get(i);
         	String expressionStr = IndexUtil.getIndexColumnExpressionStr(column);
             ParseNode expressionParseNode  = SQLParser.parseCondition(expressionStr);
-            columnParseNodeVisitor.reset();
-            expressionParseNode.accept(columnParseNodeVisitor);
-            String colName = column.getName().getString();
-            if (columnParseNodeVisitor.isParseNodeCaseSensitive()) {
-                // force column name to be case sensitive name by surround with double quotes
-                colName = "\"" + colName + "\"";
-            }
-            
+            String colName = "\"" + column.getName().getString() + "\"";
             Expression dataExpression = expressionParseNode.accept(expressionCompiler);
             PDataType expressionDataType = dataExpression.getDataType();
             ParseNode indexedParseNode = expressionParseNode.accept(rewriter);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a8b27e3f/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 2ba0cde..e414039 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -164,6 +164,7 @@ import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -1036,7 +1037,8 @@ public class MetaDataClient {
                     // can lose information during compilation
                     StringBuilder buf = new StringBuilder();
                     parseNode.toSQL(resolver, buf);
-                    String expressionStr = buf.toString();
+                    // need to escape backslash as this expression will be re-parsed later
+                    String expressionStr = StringUtil.escapeBackslash(buf.toString());
                     
                     ColumnName colName = null;
                     ColumnRef colRef = expressionIndexCompiler.getColumnRef();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a8b27e3f/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
index a83098a..4a7ae38 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
@@ -331,4 +331,9 @@ public class StringUtil {
     public static String escapeStringConstant(String pattern) {
         return StringEscapeUtils.escapeSql(pattern); // Need to escape double quotes
     }   
+    
+    public static String escapeBackslash(String input) {
+    	// see http://stackoverflow.com/questions/4653831/regex-how-to-escape-backslashes-and-special-characters
+    	return input.replaceAll("\\\\","\\\\\\\\");
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a8b27e3f/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index e17c528..83c984b 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -1564,5 +1564,25 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             stmt.close();
         }
     }
-
+    
+    @Test
+    public void testRegex() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        Statement stmt = conn.createStatement();
+        stmt.execute("CREATE TABLE t (k1 INTEGER PRIMARY KEY, v VARCHAR)");
+        
+        //character classes
+        stmt.executeQuery("select * from T where REGEXP_SUBSTR(v, '[abc]') = 'val'");
+        stmt.executeQuery("select * from T where REGEXP_SUBSTR(v, '[^abc]') = 'val'");
+        stmt.executeQuery("select * from T where REGEXP_SUBSTR(v, '[a-zA-Z]') = 'val'");
+        stmt.executeQuery("select * from T where REGEXP_SUBSTR(v, '[a-d[m-p]]') = 'val'");
+        stmt.executeQuery("select * from T where REGEXP_SUBSTR(v, '[a-z&&[def]]') = 'val'");
+        stmt.executeQuery("select * from T where REGEXP_SUBSTR(v, '[a-z&&[^bc]]') = 'val'");
+        stmt.executeQuery("select * from T where REGEXP_SUBSTR(v, '[a-z&&[^m-p]]') = 'val'");
+        
+        // predefined character classes
+        stmt.executeQuery("select * from T where REGEXP_SUBSTR(v, '.\\\\d\\\\D\\\\s\\\\S\\\\w\\\\W') = 'val'");
+    }
+    
+   
 }