You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2017/10/03 17:37:20 UTC

phoenix git commit: PHOENIX-4265 NPE when ROW_TIMESTAMP is SQL timestamp column

Repository: phoenix
Updated Branches:
  refs/heads/master aaa41a33d -> 119f86e0c


PHOENIX-4265 NPE when ROW_TIMESTAMP is SQL timestamp column


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

Branch: refs/heads/master
Commit: 119f86e0c29ed6331df35028d37f6964393f122b
Parents: aaa41a3
Author: Samarth Jain <sa...@apache.org>
Authored: Tue Oct 3 10:37:15 2017 -0700
Committer: Samarth Jain <sa...@apache.org>
Committed: Tue Oct 3 10:37:15 2017 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/UpsertSelectIT.java  | 66 ++++++++++++++++++++
 .../org/apache/phoenix/compile/ScanRanges.java  | 14 +++--
 .../org/apache/phoenix/schema/types/PLong.java  |  6 +-
 3 files changed, 80 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/119f86e0/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
index fef5a25..275d72d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
@@ -42,10 +42,12 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Timestamp;
 import java.util.Properties;
 
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
@@ -1420,6 +1422,70 @@ public class UpsertSelectIT extends ParallelStatsDisabledIT {
         conn.close();
     }
 
+    @Test // See https://issues.apache.org/jira/browse/PHOENIX-4265
+    public void testLongCodecUsedForRowTimestamp() throws Exception {
+        String tableName = generateUniqueName();
+        String indexName = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute("CREATE IMMUTABLE TABLE " + tableName
+                    + " (k1 TIMESTAMP not null, k2 bigint not null, v bigint, constraint pk primary key (k1 row_timestamp, k2)) SALT_BUCKETS = 9");
+            conn.createStatement().execute(
+                "CREATE INDEX " + indexName + " ON " + tableName + " (v) INCLUDE (k2)");
+            PreparedStatement stmt =
+                    conn.prepareStatement("UPSERT INTO " + tableName + " VALUES (?, ?, ?) ");
+            stmt.setTimestamp(1, new Timestamp(1000));
+            stmt.setLong(2, 2000);
+            stmt.setLong(3, 1000);
+            stmt.executeUpdate();
+            stmt.setTimestamp(1, new Timestamp(2000));
+            stmt.setLong(2, 5000);
+            stmt.setLong(3, 5);
+            stmt.executeUpdate();
+            stmt.setTimestamp(1, new Timestamp(3000));
+            stmt.setLong(2, 5000);
+            stmt.setLong(3, 5);
+            stmt.executeUpdate();
+            stmt.setTimestamp(1, new Timestamp(4000));
+            stmt.setLong(2, 5000);
+            stmt.setLong(3, 5);
+            stmt.executeUpdate();
+            stmt.setTimestamp(1, new Timestamp(5000));
+            stmt.setLong(2, 2000);
+            stmt.setLong(3, 10);
+            stmt.executeUpdate();
+            stmt.setTimestamp(1, new Timestamp(6000));
+            stmt.setLong(2, 2000);
+            stmt.setLong(3, 20);
+            stmt.executeUpdate();
+            conn.commit();
+            ResultSet rs = conn.createStatement().executeQuery("SELECT " +
+                    " K2 FROM " + tableName + " WHERE V = 5");
+            assertTrue("Index " + indexName + " should have been used",
+                rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan().getTableRef()
+                        .getTable().getName().getString().equals(indexName));
+            assertTrue(rs.next());
+            assertEquals(5000, rs.getLong("k2"));
+            assertTrue(rs.next());
+            assertEquals(5000, rs.getLong("k2"));
+            assertTrue(rs.next());
+            assertEquals(5000, rs.getLong("k2"));
+            assertFalse(rs.next());
+            rs =
+                    conn.createStatement().executeQuery("SELECT /*+ INDEX(" + tableName + " "
+                            + indexName + ") */ " + " K2 FROM " + tableName + " WHERE V = 5");
+            assertTrue("Index " + indexName + " should have been used",
+                rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan().getTableRef()
+                        .getTable().getName().getString().equals(indexName));
+            assertTrue(rs.next());
+            assertEquals(5000, rs.getLong("k2"));
+            assertTrue(rs.next());
+            assertEquals(5000, rs.getLong("k2"));
+            assertTrue(rs.next());
+            assertEquals(5000, rs.getLong("k2"));
+            assertFalse(rs.next());
+        }
+    }
+
     private static Connection getTenantConnection(String tenantId) throws Exception {
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
         props.setProperty(TENANT_ID_ATTRIB, tenantId);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/119f86e0/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 a7909fb..79bf3b9 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
@@ -41,7 +41,10 @@ import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.ValueSchema.Field;
+import org.apache.phoenix.schema.types.PDataType.PDataCodec;
+import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ScanUtil.BytesComparator;
 import org.apache.phoenix.util.SchemaUtil;
@@ -668,16 +671,17 @@ public class ScanRanges {
             throws IOException {
         long low;
         long high;
+        PDataCodec codec = PLong.INSTANCE.getCodec();
         if (lowestRange.lowerUnbound()) {
             low = 0;
         } else {
-            long lowerRange = f.getDataType().getCodec().decodeLong(lowestRange.getLowerRange(), 0, SortOrder.ASC);
+            long lowerRange = codec.decodeLong(lowestRange.getLowerRange(), 0, SortOrder.ASC);
             low = lowestRange.isLowerInclusive() ? lowerRange : safelyIncrement(lowerRange);
         }
         if (highestRange.upperUnbound()) {
             high = HConstants.LATEST_TIMESTAMP;
         } else {
-            long upperRange = f.getDataType().getCodec().decodeLong(highestRange.getUpperRange(), 0, SortOrder.ASC);
+            long upperRange = codec.decodeLong(highestRange.getUpperRange(), 0, SortOrder.ASC);
             if (highestRange.isUpperInclusive()) {
                 high = safelyIncrement(upperRange);
             } else {
@@ -692,9 +696,9 @@ public class ScanRanges {
         boolean lowerInclusive = lowestKeyRange.isLowerInclusive();
         boolean upperUnbound = highestKeyRange.upperUnbound();
         boolean upperInclusive = highestKeyRange.isUpperInclusive();
-
-        long low = lowerUnbound ? -1 : f.getDataType().getCodec().decodeLong(lowestKeyRange.getLowerRange(), 0, SortOrder.DESC);
-        long high = upperUnbound ? -1 : f.getDataType().getCodec().decodeLong(highestKeyRange.getUpperRange(), 0, SortOrder.DESC);
+        PDataCodec codec = PLong.INSTANCE.getCodec();
+        long low = lowerUnbound ? -1 : codec.decodeLong(lowestKeyRange.getLowerRange(), 0, SortOrder.DESC);
+        long high = upperUnbound ? -1 : codec.decodeLong(highestKeyRange.getUpperRange(), 0, SortOrder.DESC);
         long newHigh;
         long newLow;
         if (!lowerUnbound && !upperUnbound) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/119f86e0/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLong.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLong.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLong.java
index 80f3de4..0402c6e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLong.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PLong.java
@@ -230,7 +230,11 @@ public class PLong extends PWholeNumber<Long> {
 
     @Override
     public Object getSampleValue(Integer maxLength, Integer arrayLength) {
-        return RANDOM.get().nextLong();
+        long val = RANDOM.get().nextLong();
+        if (val == Long.MIN_VALUE) {
+            return Long.MAX_VALUE;
+        }
+        return Math.abs(val);
     }
 
     static class LongCodec extends BaseCodec {