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 2016/06/14 01:05:39 UTC

[1/4] phoenix git commit: PHOENIX-2961 select on a composite primary key fails when DESC is set in the table

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.0 f2ab43210 -> 587f615b0


PHOENIX-2961 select on a composite primary key fails when DESC is set in the table


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

Branch: refs/heads/4.x-HBase-1.0
Commit: 628b7dcd33ab1ec4c0fdce27dac574562bbefae7
Parents: 710d15d
Author: James Taylor <ja...@apache.org>
Authored: Mon Jun 13 13:46:47 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Mon Jun 13 18:07:02 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/HashJoinMoreIT.java  | 47 ++++++++++++++++++++
 .../org/apache/phoenix/compile/ScanRanges.java  |  2 +-
 2 files changed, 48 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/628b7dcd/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
index 128baf3..481e4af 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
@@ -875,4 +875,51 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
             conn.close();
         }
     }
+    
+    @Test
+    public void testBug2961() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        
+        conn.createStatement().execute("CREATE TABLE test2961 (\n" + 
+                "ACCOUNT_ID VARCHAR NOT NULL,\n" + 
+                "BUCKET_ID VARCHAR NOT NULL,\n" + 
+                "OBJECT_ID VARCHAR NOT NULL,\n" + 
+                "OBJECT_VERSION VARCHAR NOT NULL,\n" + 
+                "LOC VARCHAR,\n" + 
+                "CONSTRAINT PK PRIMARY KEY (ACCOUNT_ID, BUCKET_ID, OBJECT_ID, OBJECT_VERSION DESC))");
+        conn.createStatement().execute("UPSERT INTO test2961  (ACCOUNT_ID, BUCKET_ID, OBJECT_ID, OBJECT_VERSION, LOC) VALUES ('acct1', 'bucket1', 'obj1', '1111', 'loc1')");
+        ResultSet rs = conn.createStatement().executeQuery("select ACCOUNT_ID, BUCKET_ID, OBJECT_VERSION  from test2961  WHERE ACCOUNT_ID = 'acct1' and BUCKET_ID = 'bucket1' and OBJECT_VERSION = '1111'");
+        assertTrue(rs.next());
+        rs = conn.createStatement().executeQuery("select ACCOUNT_ID, BUCKET_ID, OBJECT_VERSION  from test2961  WHERE ACCOUNT_ID = 'acct1' and BUCKET_ID = 'bucket1' and OBJECT_ID = 'obj1'");
+        assertTrue(rs.next());
+        rs = conn.createStatement().executeQuery("select ACCOUNT_ID, BUCKET_ID, OBJECT_VERSION  from test2961  WHERE ACCOUNT_ID = 'acct1' and BUCKET_ID = 'bucket1' and OBJECT_VERSION = '1111'  and OBJECT_ID = 'obj1'");
+        assertTrue(rs.next());
+
+        conn.createStatement().execute("UPSERT INTO test2961  (ACCOUNT_ID, BUCKET_ID, OBJECT_ID, OBJECT_VERSION, LOC) VALUES ('acct1', 'bucket1', 'obj1', '2222', 'loc1')");
+        rs = conn.createStatement().executeQuery("SELECT  OBJ.ACCOUNT_ID, OBJ.BUCKET_ID, OBJ.OBJECT_ID, OBJ.OBJECT_VERSION, OBJ.LOC "
+                + "FROM ( SELECT ACCOUNT_ID, BUCKET_ID, OBJECT_ID, MAX(OBJECT_VERSION) AS MAXVER"
+                + "       FROM test2961 GROUP BY ACCOUNT_ID, BUCKET_ID, OBJECT_ID) AS X "
+                + "       INNER JOIN test2961 AS OBJ ON X.ACCOUNT_ID = OBJ.ACCOUNT_ID AND X.BUCKET_ID = OBJ.BUCKET_ID AND X.OBJECT_ID = OBJ.OBJECT_ID AND X.MAXVER = OBJ.OBJECT_VERSION");
+        assertTrue(rs.next());
+        assertEquals("2222", rs.getString(4));
+        assertFalse(rs.next());
+        
+        rs = conn.createStatement().executeQuery("SELECT  OBJ.ACCOUNT_ID, OBJ.BUCKET_ID, OBJ.OBJECT_ID, OBJ.OBJECT_VERSION, OBJ.LOC "
+                + "FROM ( SELECT ACCOUNT_ID, BUCKET_ID, OBJECT_ID, MAX(OBJECT_VERSION) AS MAXVER "
+                + "       FROM test2961 GROUP BY ACCOUNT_ID, BUCKET_ID, OBJECT_ID) AS X "
+                + "       INNER JOIN test2961 AS OBJ ON X.ACCOUNT_ID = OBJ.ACCOUNT_ID AND X.OBJECT_ID = OBJ.OBJECT_ID AND X.MAXVER = OBJ.OBJECT_VERSION");
+        assertTrue(rs.next());
+        assertEquals("2222", rs.getString(4));
+        assertFalse(rs.next());
+        
+        rs = conn.createStatement().executeQuery("SELECT  OBJ.ACCOUNT_ID, OBJ.BUCKET_ID, OBJ.OBJECT_ID, OBJ.OBJECT_VERSION, OBJ.LOC "
+                + "FROM ( SELECT ACCOUNT_ID, BUCKET_ID, OBJECT_ID, MAX(OBJECT_VERSION) AS MAXVER "
+                + "       FROM test2961 GROUP BY ACCOUNT_ID, BUCKET_ID, OBJECT_ID) AS X "
+                + "       INNER JOIN test2961 AS OBJ ON X.ACCOUNT_ID = OBJ.ACCOUNT_ID AND X.BUCKET_ID = OBJ.BUCKET_ID AND  X.MAXVER = OBJ.OBJECT_VERSION");
+        assertTrue(rs.next());
+        assertEquals("2222", rs.getString(4));
+        assertFalse(rs.next());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/628b7dcd/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
index 5edaead..6f0adb0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
@@ -103,7 +103,7 @@ public class ScanRanges {
             useSkipScan = keyRanges.size() > 1;
             // Treat as binary if descending because we've got a separator byte at the end
             // which is not part of the value.
-            if (keys.size() > 1 || SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), false, schema.getField(0)) == QueryConstants.DESC_SEPARATOR_BYTE) {
+            if (keys.size() > 1 || SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), false, schema.getField(schema.getFieldCount()-1)) == QueryConstants.DESC_SEPARATOR_BYTE) {
                 schema = SchemaUtil.VAR_BINARY_SCHEMA;
                 slotSpan = ScanUtil.SINGLE_COLUMN_SLOT_SPAN;
             } else {


[2/4] phoenix git commit: PHOENIX-2983 ClassCastException on auto coerce of BIGINT to DECIMAL

Posted by ja...@apache.org.
PHOENIX-2983 ClassCastException on auto coerce of BIGINT to DECIMAL


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

Branch: refs/heads/4.x-HBase-1.0
Commit: 710d15db972971b021d8c7453ae45ff196097e33
Parents: f2ab432
Author: James Taylor <ja...@apache.org>
Authored: Sun Jun 12 22:10:39 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Mon Jun 13 18:07:02 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/UpsertValuesIT.java   | 19 +++++++++++++++++++
 .../apache/phoenix/schema/types/PDataType.java   | 12 ++++++++++++
 .../apache/phoenix/schema/types/PDecimal.java    |  8 ++++++++
 3 files changed, 39 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/710d15db/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
index 5cefd7d..9bbe23e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
@@ -925,6 +925,25 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         }
     }
     
+    @Test
+    public void testAutoCastLongToBigDecimal() throws Exception {
+        long ts = nextTimestamp();
+        try (Connection conn = getConnection(ts)) {
+            conn.createStatement().execute("CREATE TABLE LONG_BUG (NAME VARCHAR PRIMARY KEY, AMOUNT DECIMAL)");
+        }
+        try (Connection conn = getConnection(ts + 10)) {
+            conn.createStatement().execute("UPSERT INTO LONG_BUG (NAME, AMOUNT) VALUES('HELLO1', -50000)");
+            conn.commit();
+        }
+        try (Connection conn = getConnection(ts + 20)) {
+            ResultSet rs = conn.createStatement().executeQuery("SELECT NAME, AMOUNT FROM LONG_BUG");
+            assertTrue(rs.next());
+            assertEquals("HELLO1", rs.getString(1));
+            assertTrue(new BigDecimal(-50000).compareTo(rs.getBigDecimal(2)) == 0);
+            assertFalse(rs.next());
+        }
+    }
+    
     private static Connection getConnection(long ts) throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/710d15db/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
index 1b5b695..0f0328a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
@@ -730,6 +730,18 @@ public abstract class PDataType<T> implements DataType<T>, Comparable<PDataType<
         return isCoercibleTo(targetType);
     }
 
+    /**
+     * Checks whether or not the value represented by value (or ptr if value is null) is compatible in terms
+     * of size with the desired max length and scale. The srcType must be coercible to this type.
+     * @param ptr bytes pointer for the value
+     * @param value object representation of the value. May be null in which case ptr will be used
+     * @param srcType the type of the value
+     * @param maxLength the max length of the source value or null if not applicable
+     * @param scale the scale of the source value or null if not applicable
+     * @param desiredMaxLength the desired max length for the value to be coerced
+     * @param desiredScale the desired scale for the value to be coerced 
+     * @return true if the value may be coerced without losing precision and false otherwise.
+     */
     public boolean isSizeCompatible(ImmutableBytesWritable ptr, Object value, PDataType srcType, Integer maxLength,
             Integer scale, Integer desiredMaxLength, Integer desiredScale) {
         return true;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/710d15db/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
index 228aef1..ff6e186 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.NumberUtil;
 
@@ -304,6 +305,13 @@ public class PDecimal extends PRealNumber<BigDecimal> {
     if (ptr.getLength() == 0) {
       return true;
     }
+    // Any numeric type fits into a DECIMAL
+    if (srcType != PDecimal.INSTANCE) {
+        if(!srcType.isCoercibleTo(this)) {
+            throw new IllegalArgumentException(TypeMismatchException.newException(srcType, this));
+        }
+        return true;
+    }
     // Use the scale from the value if provided, as it prevents a deserialization.
     // The maxLength and scale for the underlying expression are ignored, because they
     // are not relevant in this case: for example a DECIMAL(10,2) may be assigned to a


[4/4] phoenix git commit: PHOENIX-2920 Incorrect Queries on Multi-tenant tables with WHERE clause containing Row Value Constructor

Posted by ja...@apache.org.
PHOENIX-2920 Incorrect Queries on Multi-tenant tables with WHERE clause containing Row Value Constructor


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

Branch: refs/heads/4.x-HBase-1.0
Commit: 587f615b0900ab190aff5b750514493385c79d4f
Parents: 26d1d6c
Author: James Taylor <ja...@apache.org>
Authored: Mon Jun 13 09:08:47 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Mon Jun 13 18:07:03 2016 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/RowValueConstructorIT.java  | 48 ++++++++++++++++++++
 .../apache/phoenix/compile/WhereOptimizer.java  | 27 +++++++++--
 .../phoenix/compile/WhereOptimizerTest.java     | 33 ++++++++++++++
 3 files changed, 103 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/587f615b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
index ea91f4f..ce38fcf 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
@@ -42,6 +42,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.math.BigDecimal;
 import java.sql.Connection;
 import java.sql.Date;
 import java.sql.DriverManager;
@@ -1652,4 +1653,51 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
 		assertEquals("403", rs.getString(4));
 	}
 
+    @Test
+    public void testMultiTenantRVC() throws Exception {
+        Connection conn = nextConnection(getUrl());
+        String tableName = "mtRVC";
+        String ddl = "CREATE TABLE " + tableName 
+                + " (\n" + 
+                "    pk1 VARCHAR NOT NULL,\n" + 
+                "    pk2 DECIMAL NOT NULL,\n" + 
+                "    v1 VARCHAR\n" + 
+                "    CONSTRAINT PK PRIMARY KEY \n" + 
+                "    (\n" + 
+                "        pk1,\n" + 
+                "        pk2\n" + 
+                "    )\n" + 
+                ") MULTI_TENANT=true,IMMUTABLE_ROWS=true";
+        conn.createStatement().execute(ddl);
+
+        conn = nextConnection(getUrl());
+        ddl = "CREATE INDEX  " + tableName + "_idx"
+                + " ON " + tableName + " (v1)";
+        conn.createStatement().execute(ddl);
+        
+        conn = nextConnection(getUrl());
+        String upsert = "UPSERT INTO " + tableName + " VALUES(?, ?, ?)";
+        PreparedStatement stmt = conn.prepareStatement(upsert);
+        stmt.setString(1, "a");
+        stmt.setInt(2, 1);
+        stmt.setString(3, "value");
+        stmt.execute();
+        stmt.setString(1, "a");
+        stmt.setInt(2, 2);
+        stmt.setString(3, "value");
+        stmt.execute();
+        conn.commit();
+
+        conn = nextConnection(getUrl());
+        String query = "SELECT pk1, pk2, v1 FROM " + tableName + " WHERE pk1 = 'a' AND\n" + 
+                "(pk1, pk2) > ('a', 1)\n" + 
+                "ORDER BY PK1, PK2\n" + 
+                "LIMIT 2";
+        ResultSet rs = conn.createStatement().executeQuery(query);
+        assertTrue(rs.next());
+        assertEquals("a", rs.getString(1));
+        assertTrue(new BigDecimal("2").compareTo(rs.getBigDecimal(2)) == 0);
+        assertEquals("value", rs.getString(3));
+        assertFalse(rs.next());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/587f615b/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 0122712..e1e3c53 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
@@ -1330,11 +1330,28 @@ public class WhereOptimizer {
                     // For example: a < (1,2) is true if a = 1, so we need to switch
                     // the compare op to <= like this: a <= 1. Since we strip trailing nulls
                     // in the rvc, we don't need to worry about the a < (1,null) case.
-                    if (usedAllOfLHS && rvc.getChildren().size() < rhs.getChildren().size()) {
-                        if (op == CompareOp.LESS) {
-                            op = CompareOp.LESS_OR_EQUAL;
-                        } else if (op == CompareOp.GREATER_OR_EQUAL) {
-                            op = CompareOp.GREATER;
+                    if (usedAllOfLHS) {
+                        if (rvc.getChildren().size() < rhs.getChildren().size()) {
+                            if (op == CompareOp.LESS) {
+                                op = CompareOp.LESS_OR_EQUAL;
+                            } else if (op == CompareOp.GREATER_OR_EQUAL) {
+                                op = CompareOp.GREATER;
+                            }
+                        }
+                    } else {
+                        // If we're not using all of the LHS, we need to expand the range on either
+                        // side to take into account the rest of the LHS. For example:
+                        // WHERE (pk1, pk3) > ('a',1) AND pk1 = 'a'. In this case, we'll end up
+                        // only using (pk1) and ('a'), so if we use a > operator the expression
+                        // would end up as degenerate since we'd have a non inclusive range for
+                        // ('a'). By switching the operator to extend the range, we end up with
+                        // an ('a') inclusive range which is correct.
+                        if (rvc.getChildren().size() < rhs.getChildren().size()) {
+                            if (op == CompareOp.LESS) {
+                                op = CompareOp.LESS_OR_EQUAL;
+                            } else if (op == CompareOp.GREATER) {
+                                op = CompareOp.GREATER_OR_EQUAL;
+                            }
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/587f615b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
index 9076ea5..f259373 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
@@ -1625,6 +1625,39 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
     }
     
     @Test
+    public void testQueryMoreRVC() throws SQLException {
+        String tenantId = "000000000000001";
+        String parentId = "000000000000008";
+        
+        String ddl = "CREATE TABLE rvcTestIdx "
+                + " (\n" + 
+                "    pk1 VARCHAR NOT NULL,\n" + 
+                "    v1 VARCHAR,\n" + 
+                "    pk2 DECIMAL NOT NULL,\n" + 
+                "    CONSTRAINT PK PRIMARY KEY \n" + 
+                "    (\n" + 
+                "        pk1,\n" + 
+                "        v1,\n" + 
+                "        pk2\n" + 
+                "    )\n" + 
+                ") MULTI_TENANT=true,IMMUTABLE_ROWS=true";
+        Connection conn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES));
+        conn.createStatement().execute(ddl);
+        String query = "SELECT pk1, pk2, v1 FROM rvcTestIdx WHERE pk1 = 'a' AND\n" + 
+                "(pk1, pk2) > ('a', 1)\n" + 
+                "ORDER BY PK1, PK2\n" + 
+                "LIMIT 2";
+        StatementContext context = compileStatement(query, 2);
+        Scan scan = context.getScan();
+        Filter filter = scan.getFilter();
+        assertNotNull(filter);
+        byte[] startRow = Bytes.toBytes("a");
+        byte[] stopRow = ByteUtil.concat(startRow, ByteUtil.nextKey(QueryConstants.SEPARATOR_BYTE_ARRAY));
+        assertArrayEquals(startRow, scan.getStartRow());
+        assertArrayEquals(stopRow, scan.getStopRow());
+    }
+    
+    @Test
     public void testCombiningRVCUsingOr() throws SQLException {
         String firstTenantId = "000000000000001";
         String secondTenantId = "000000000000005";


[3/4] phoenix git commit: PHOENIX-2994 Coalesce function may return null

Posted by ja...@apache.org.
PHOENIX-2994 Coalesce function may return null


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

Branch: refs/heads/4.x-HBase-1.0
Commit: 26d1d6c241608dd550f17657184aa315336bfe35
Parents: 628b7dc
Author: James Taylor <ja...@apache.org>
Authored: Mon Jun 13 16:46:03 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Mon Jun 13 18:07:03 2016 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/CoalesceFunctionIT.java     | 23 ++++++++++++++++++++
 .../expression/RowKeyColumnExpression.java      |  6 +++--
 .../expression/function/CoalesceFunction.java   |  6 ++---
 3 files changed, 30 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/26d1d6c2/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
index 92a9376..68165e2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
@@ -293,5 +293,28 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.wasNull());
     }
 
+    @Test
+    public void testCoalesceInRowKeyColumn() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.createStatement().execute("CREATE TABLE coalesceTest(k1 decimal, k2 decimal, constraint pk primary key (k1,k2))");
+        conn.createStatement().execute("UPSERT INTO coalesceTest(k2) VALUES (1)");
+        conn.createStatement().execute("UPSERT INTO coalesceTest VALUES (2,2)");
+        conn.createStatement().execute("UPSERT INTO coalesceTest VALUES (3,3)");
+        conn.commit();
+        
+        ResultSet rs = conn.createStatement().executeQuery("SELECT coalesce(k1, 1) ,k2 FROM coalesceTest");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(1, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals(2, rs.getInt(1));
+        assertEquals(2, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals(3, rs.getInt(1));
+        assertEquals(3, rs.getInt(2));
+        assertFalse(rs.next());
+    }
+
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/26d1d6c2/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
index 03cc51f..cfa027f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
@@ -129,9 +129,11 @@ public class RowKeyColumnExpression  extends ColumnExpression {
             } else {
                 ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
             }
-            return true;
+        } else {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
         }
-        return false;
+        // Always return true because we're always able to evaluate a row key column
+        return true;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/26d1d6c2/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CoalesceFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CoalesceFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CoalesceFunction.java
index 773120a..cf1fcd1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CoalesceFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CoalesceFunction.java
@@ -28,8 +28,8 @@ import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.parse.FunctionParseNode.Argument;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
-import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.ExpressionUtil;
 
 
@@ -81,10 +81,10 @@ public class CoalesceFunction extends ScalarFunction {
     @Override
     public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
         boolean evaluated = children.get(0).evaluate(tuple, ptr);
-        if (evaluated) {
+        if (evaluated && ptr.getLength() > 0) {
             return true;
         }
-        if (tuple.isImmutable()) {
+        if (evaluated || tuple.isImmutable()) {
             Expression secondChild = children.get(1);
             if (secondChild.evaluate(tuple, ptr)) {
                 // Coerce the type of the second child to the type of the first child