You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2016/02/02 04:59:57 UTC

[23/50] [abbrv] phoenix git commit: PHOENIX-2613 Infinite loop in SkipScan when seeking past null value in non leading primary key column

PHOENIX-2613 Infinite loop in SkipScan when seeking past null value in non leading primary key column


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

Branch: refs/heads/calcite
Commit: 462509625853bf85d0fdeb33fe3cf36bf5c94bec
Parents: 6251f8f
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Jan 21 17:58:20 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Jan 21 17:58:20 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/SkipScanQueryIT.java | 22 +++++++
 .../apache/phoenix/filter/SkipScanFilter.java   | 24 ++++----
 .../phoenix/filter/SkipScanFilterTest.java      | 65 +++++++++++++-------
 3 files changed, 77 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/46250962/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
index 9f73550..31994bb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
@@ -434,4 +434,26 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
             conn.close();
         }
     }
+    
+    @Test
+    public void testNullInfiniteLoop() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.setAutoCommit(true);
+            conn.createStatement().execute(
+              "create table NULL_TEST"+
+              "("+
+                     "CREATETIME VARCHAR,"+
+                     "ACCOUNTID VARCHAR,"+
+                     "SERVICENAME VARCHAR,"+
+                     "SPAN.APPID VARCHAR,"+
+                     "CONSTRAINT pk PRIMARY KEY(CREATETIME,ACCOUNTID,SERVICENAME)"+
+              ")");
+
+            conn.createStatement().execute("upsert into NULL_TEST(CREATETIME,SERVICENAME,SPAN.APPID) values('20160116141006','servlet','android')");
+            conn.createStatement().execute("upsert into NULL_TEST(CREATETIME,ACCOUNTID,SERVICENAME,SPAN.APPID) values('20160116151006','2404787','jdbc','ios')");
+            ResultSet rs = conn.createStatement().executeQuery("select * from NULL_TEST where CREATETIME>='20160116121006' and  CREATETIME<='20160116181006' and ACCOUNTID='2404787'");
+            assertTrue(rs.next());
+            assertFalse(rs.next());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/46250962/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
index 77b4cf6..00320ce 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.io.Writable;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.KeyRange.Bound;
 import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.ValueSchema.Field;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ScanUtil.BytesComparator;
@@ -161,8 +162,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
                                 .getRowOffset(), previousCellHint.getRowLength()) > 0;
         if (!isHintAfterPrevious) {
             String msg = "The next hint must come after previous hint (prev=" + previousCellHint + ", next=" + nextCellHint + ", kv=" + kv + ")";
-            assert isHintAfterPrevious : msg;
-            logger.warn(msg);
+            throw new IllegalStateException(msg);
         }
     }
     
@@ -243,11 +243,6 @@ public class SkipScanFilter extends FilterBase implements Writable {
         }
         // Short circuit out if we only have a single set of keys
         if (slots.size() == 1) {
-//            int offset = slots.get(0).get(endPos).compareLowerToUpperBound(upperExclusiveKey) < 0 ? 1 : 0;
-//            if (endPos + offset <= startPos) {
-//                return false;
-//            }
-//            List<KeyRange> newRanges = slots.get(0).subList(startPos, endPos + offset);
             if (newSlots != null) {
                 List<KeyRange> newRanges = slots.get(0).subList(startPos, endPos+1);
                 newSlots.add(newRanges);
@@ -439,7 +434,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
                     setStartKey();
                     schema.reposition(ptr, ScanUtil.getRowKeyPosition(slotSpan, i), ScanUtil.getRowKeyPosition(slotSpan, j), minOffset, maxOffset, slotSpan[j]);
                 } else {
-                    int currentLength = setStartKey(ptr, minOffset, j+1, nSlots);
+                    int currentLength = setStartKey(ptr, minOffset, j+1, nSlots, false);
                     // From here on, we use startKey as our buffer (resetting minOffset and maxOffset)
                     // We've copied the part of the current key above that we need into startKey
                     // Reinitialize the iterator to be positioned at previous slot position
@@ -454,7 +449,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
             } else if (slots.get(i).get(position[i]).compareLowerToUpperBound(ptr, comparator) > 0) {
                 // Our current key is less than the lower range of the current position in the current slot.
                 // Seek to the lower range, since it's bigger than the current key
-                setStartKey(ptr, minOffset, i, nSlots);
+                setStartKey(ptr, minOffset, i, nSlots, false);
                 return ReturnCode.SEEK_NEXT_USING_HINT;
             } else { // We're in range, check the next slot
                 if (!slots.get(i).get(position[i]).isSingleKey() && i < earliestRangeIndex) {
@@ -477,7 +472,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
                         break;
                     }
                     // Otherwise we seek to the next start key because we're before it now
-                    setStartKey(ptr, minOffset, i, nSlots);
+                    setStartKey(ptr, minOffset, i, nSlots, true);
                     return ReturnCode.SEEK_NEXT_USING_HINT;
                 }
             }
@@ -521,13 +516,16 @@ public class SkipScanFilter extends FilterBase implements Writable {
         startKeyLength = setKey(Bound.LOWER, startKey, 0, 0);
     }
 
-    private int setStartKey(ImmutableBytesWritable ptr, int offset, int i, int nSlots) {
+    private int setStartKey(ImmutableBytesWritable ptr, int offset, int i, int nSlots, boolean atEndOfKey) {
         int length = ptr.getOffset() - offset;
         startKey = copyKey(startKey, length + this.maxKeyLength, ptr.get(), offset, length);
         startKeyLength = length;
         // Add separator byte if we're at end of the key, since trailing separator bytes are stripped
-        if (ptr.getLength() == 0 && i > 0 && i-1 < nSlots && !schema.getField(i-1).getDataType().isFixedWidth()) {
-            startKey[startKeyLength++] = SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), ptr.getLength()==0, schema.getField(i-1));
+        if (atEndOfKey && i > 0 && i-1 < nSlots) {
+            Field field = schema.getField(i-1);
+            if (!field.getDataType().isFixedWidth()) {
+                startKey[startKeyLength++] = SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), true, field);
+            }
         }
         startKeyLength += setKey(Bound.LOWER, startKey, startKeyLength, i);
         return length;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/46250962/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
index 4cb71ff..8ed0add 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
@@ -106,36 +106,59 @@ public class SkipScanFilterTest extends TestCase {
     public static Collection<Object> data() {
         List<Object> testCases = Lists.newArrayList();
         // Variable length tests
+//        testCases.addAll(
+//                foreach(new KeyRange[][]{{
+//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("e"), true, Bytes.toBytes("e"), true),
+//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("f"), true, Bytes.toBytes("f"), true),
+//                },
+//                {
+//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+//                },
+//                {
+//                    KeyRange.EVERYTHING_RANGE,
+//                },
+//                {
+//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+//                }},
+//                new int[4],
+//                new Include(ByteUtil.concat(Bytes.toBytes("a"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+//                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                            QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                            Bytes.toBytes("1") ) ),
+//                new SeekNext(ByteUtil.concat(Bytes.toBytes("e.f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+//                                             Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                             QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                             Bytes.toBytes("1") ), 
+//                            ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+//                                            Bytes.toBytes("b") )),
+//                new Include(ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+//                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                            QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                            Bytes.toBytes("1") ) ) )
+//        );
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("e"), true, Bytes.toBytes("e"), true),
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("f"), true, Bytes.toBytes("f"), true),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("20160116121006"), true, Bytes.toBytes("20160116181006"), true),
                 },
                 {
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                },
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("2404787"), true, Bytes.toBytes("2404787"), true),
+                }/*,
                 {
                     KeyRange.EVERYTHING_RANGE,
                 },
                 {
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                }},
+                    KeyRange.EVERYTHING_RANGE,
+                }*/},
                 new int[4],
-                new Include(ByteUtil.concat(Bytes.toBytes("a"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+                new SeekNext(ByteUtil.concat(Bytes.toBytes("20160116141006"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
                                             QueryConstants.SEPARATOR_BYTE_ARRAY,
-                                            Bytes.toBytes("1") ) ),
-                new SeekNext(ByteUtil.concat(Bytes.toBytes("e.f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-                                             Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
-                                             QueryConstants.SEPARATOR_BYTE_ARRAY,
-                                             Bytes.toBytes("1") ), 
-                            ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-                                            Bytes.toBytes("b") )),
-                new Include(ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
-                                            QueryConstants.SEPARATOR_BYTE_ARRAY,
-                                            Bytes.toBytes("1") ) ) )
+                                            Bytes.toBytes("servlet") ),
+                             ByteUtil.concat(Bytes.toBytes("20160116141006"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+                                             Bytes.toBytes("2404787") )),
+                new Include(ByteUtil.concat(Bytes.toBytes("20160116151006"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+                                            Bytes.toBytes("2404787"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+                                            Bytes.toBytes("jdbc"), QueryConstants.SEPARATOR_BYTE_ARRAY ) ) )
         );
         // Fixed length tests
         testCases.addAll(
@@ -342,7 +365,7 @@ public class SkipScanFilterTest extends TestCase {
         );
         return testCases;
     }
-
+    
     private static Collection<?> foreach(KeyRange[][] ranges, int[] widths, Expectation... expectations) {
         List<List<KeyRange>> cnf = Lists.transform(Lists.newArrayList(ranges), ARRAY_TO_LIST);
         List<Object> ret = Lists.newArrayList();