You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ri...@apache.org on 2023/07/26 07:27:37 UTC

[phoenix] branch master updated: PHOENIX-6999 Point lookups fail with reverse scan (#1646)

This is an automated email from the ASF dual-hosted git repository.

richardantal pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/master by this push:
     new 694f8444ce PHOENIX-6999 Point lookups fail with reverse scan (#1646)
694f8444ce is described below

commit 694f8444ce3bd67e4fb0e2deb40a8d36f82dfd60
Author: Istvan Toth <st...@apache.org>
AuthorDate: Wed Jul 26 09:27:30 2023 +0200

    PHOENIX-6999 Point lookups fail with reverse scan (#1646)
---
 .../org/apache/phoenix/end2end/BaseOrderByIT.java  | 55 +++++++++++++++++++++-
 .../java/org/apache/phoenix/util/ScanUtil.java     | 38 ++-------------
 2 files changed, 58 insertions(+), 35 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOrderByIT.java
index b9de4ee2b7..a0ef9bdb72 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOrderByIT.java
@@ -38,6 +38,7 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.Properties;
 
 import org.apache.phoenix.compile.ExplainPlan;
@@ -46,6 +47,8 @@ import org.apache.phoenix.jdbc.PhoenixPreparedStatement;
 import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryBuilder;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
 
@@ -86,7 +89,6 @@ public abstract class BaseOrderByIT extends ParallelStatsDisabledIT {
         }
     }
 
-
     @Test
     public void testDescMultiOrderByExpr() throws Exception {
         String tenantId = getOrganizationId();
@@ -996,4 +998,55 @@ public abstract class BaseOrderByIT extends ParallelStatsDisabledIT {
         }
     }
 
+    @Test
+    public void testPhoenix6999() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateUniqueName();
+        String descTableName = "TBL_" + generateUniqueName();
+
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullDescTableName =
+                SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, descTableName);
+
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);
+                Statement stmt = conn.createStatement()) {
+            conn.setAutoCommit(false);
+            String ddl =
+                    "CREATE TABLE " + fullTableName
+                            + "(k1 varchar primary key, v1 varchar, v2 varchar)";
+            stmt.execute(ddl);
+            ddl =
+                    "CREATE TABLE " + fullDescTableName
+                            + "(k1 varchar primary key desc, v1 varchar, v2 varchar)";
+            stmt.execute(ddl);
+            stmt.execute("upsert into " + fullTableName + " values ('a','a','a')");
+            stmt.execute("upsert into " + fullTableName + " values ('b','b','b')");
+            stmt.execute("upsert into " + fullTableName + " values ('c','c','c')");
+            stmt.execute("upsert into " + fullDescTableName + " values ('a','a','a')");
+            stmt.execute("upsert into " + fullDescTableName + " values ('b','b','b')");
+            stmt.execute("upsert into " + fullDescTableName + " values ('c','c','c')");
+            conn.commit();
+
+            String query = "SELECT  *  from " + fullTableName + " where k1='b' order by k1 asc";
+            ResultSet rs = stmt.executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+
+            query = "SELECT  *  from " + fullTableName + " where k1='b' order by k1 desc";
+            rs = stmt.executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+
+            query = "SELECT  *  from " + fullDescTableName + " where k1='b' order by k1 asc";
+            rs = stmt.executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+
+            query = "SELECT  *  from " + fullDescTableName + " where k1='b' order by k1 desc";
+            rs = stmt.executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+        }
+    }
+
 }
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
index c02acd9379..658d48402b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
@@ -121,14 +121,6 @@ public class ScanUtil {
     public static final int[] SINGLE_COLUMN_SLOT_SPAN = new int[1];
     public static final int UNKNOWN_CLIENT_VERSION = VersionUtil.encodeVersion(4, 4, 0);
 
-    /*
-     * Max length that we fill our key when we turn an inclusive key
-     * into a exclusive key.
-     */
-    private static final byte[] MAX_FILL_LENGTH_FOR_PREVIOUS_KEY = new byte[16];
-    static {
-        Arrays.fill(MAX_FILL_LENGTH_FOR_PREVIOUS_KEY, (byte)-1);
-    }
     private static final byte[] ZERO_BYTE_ARRAY = new byte[1024];
 
     private ScanUtil() {
@@ -685,35 +677,13 @@ public class ScanUtil {
         scan.setLoadColumnFamiliesOnDemand(true);
     }
 
-    private static byte[] getReversedRow(byte[] startRow) {
-        /*
-         * Must get previous key because this is going from an inclusive start key to an exclusive stop key, and we need
-         * the start key to be included. We get the previous key by decrementing the last byte by one. However, with
-         * variable length data types, we need to fill with the max byte value, otherwise, if the start key is 'ab', we
-         * lower it to 'aa' which would cause 'aab' to be included (which isn't correct). So we fill with a 0xFF byte to
-         * prevent this. A single 0xFF would be enough for our primitive types (as that byte wouldn't occur), but for an
-         * arbitrary VARBINARY key we can't know how many bytes to tack on. It's lame of HBase to force us to do this.
-         */
-        byte[] newStartRow = startRow;
-        if (startRow.length != 0) {
-            newStartRow = Arrays.copyOf(startRow, startRow.length + MAX_FILL_LENGTH_FOR_PREVIOUS_KEY.length);
-            if (ByteUtil.previousKey(newStartRow, startRow.length)) {
-                System.arraycopy(MAX_FILL_LENGTH_FOR_PREVIOUS_KEY, 0, newStartRow, startRow.length,
-                        MAX_FILL_LENGTH_FOR_PREVIOUS_KEY.length);
-            } else {
-                newStartRow = HConstants.EMPTY_START_ROW;
-            }
-        }
-        return newStartRow;
-    }
-
     // Start/stop row must be swapped if scan is being done in reverse
     public static void setupReverseScan(Scan scan) {
         if (isReversed(scan) && !scan.isReversed()) {
-            byte[] newStartRow = getReversedRow(scan.getStartRow());
-            byte[] newStopRow = getReversedRow(scan.getStopRow());
-            scan.withStartRow(newStopRow);
-            scan.withStopRow(newStartRow);
+            byte[] tmpStartRow = scan.getStartRow();
+            boolean tmpIncludeStartRow = scan.includeStartRow();
+            scan.withStartRow(scan.getStopRow(), scan.includeStopRow());
+            scan.withStopRow(tmpStartRow, tmpIncludeStartRow);
             scan.setReversed(true);
         }
     }