[ 
https://issues.apache.org/jira/browse/PHOENIX-7106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807581#comment-17807581
 ] 

ASF GitHub Bot commented on PHOENIX-7106:
-----------------------------------------

virajjasani commented on code in PR #1736:
URL: https://github.com/apache/phoenix/pull/1736#discussion_r1454739552


##########
phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UncoveredIndexRegionScanner.java:
##########
@@ -404,4 +421,37 @@ public boolean next(List<Cell> result) throws IOException {
             region.closeRegionOperation();
         }
     }
+
+    /**
+     * Add dummy cell to the result list based on either the previous rowkey 
returned to the
+     * client or the start rowkey and start rowkey include params.
+     *
+     * @param result result to add the dummy cell to.
+     * @param initStartRowKey scan start rowkey.
+     * @param includeInitStartRowKey scan start rowkey included.
+     * @param scan scan object.
+     */
+    private void updateDummyWithPrevRowKey(List<Cell> result, byte[] 
initStartRowKey,
+                                           boolean includeInitStartRowKey, 
Scan scan) {
+        result.clear();
+        if (previousResultRowKey != null) {
+            getDummyResult(previousResultRowKey, result);
+        } else {
+            if (includeInitStartRowKey && initStartRowKey.length > 0) {
+                byte[] prevKey;
+                if (Bytes.compareTo(initStartRowKey, initStartRowKey.length - 
1,
+                        1, Bytes.toBytesBinary("\\x00"), 0, 1) == 0) {
+                    prevKey = new byte[initStartRowKey.length - 1];
+                    System.arraycopy(initStartRowKey, 0, prevKey, 0, 
prevKey.length);
+                } else {
+                    prevKey = 
ByteUtil.previousKeyWithLength(ByteUtil.concat(initStartRowKey,
+                            new byte[10]), initStartRowKey.length + 10);

Review Comment:
   Addressing this with max row length logic turned out to be much complicated 
that what i thought earlier. Very few tests are able to cover the scenario 
where hbase client has to lookup the given row in meta after meta cache expires 
at the client, and when they do, meta lookup fails because meta lookup scan 
start rowkey becomes much larger by combining table name, the dummy rowkey (max 
length - 1) and some constant values added by client.
   
   i have addressed this in the latest commit: 
https://github.com/apache/phoenix/pull/1736/commits/fcbec0f676b08ce757036d33cf4687b797e2c781
   
   awaiting the build result before moving forward.





> Data Integrity issues due to invalid rowkeys returned by various coprocessors
> -----------------------------------------------------------------------------
>
>                 Key: PHOENIX-7106
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-7106
>             Project: Phoenix
>          Issue Type: Improvement
>    Affects Versions: 5.2.0, 5.1.4
>            Reporter: Viraj Jasani
>            Assignee: Viraj Jasani
>            Priority: Blocker
>             Fix For: 5.2.0, 5.1.4
>
>
> HBase scanner interface expects server to perform scan of the cells from 
> HFile or Block cache and return consistent data i.e. rowkey of the cells 
> returned should stay in the range of the scan boundaries. When a region moves 
> and scanner needs reset, or if the current row is too large and the server 
> returns partial row, the subsequent scanner#next is supposed to return 
> remaining cells. When this happens, cell rowkeys returned by servers i.e. any 
> coprocessors is expected to be in the scan boundary range so that server can 
> reliably perform its validation and return remaining cells as expected.
> Phoenix client initiates serial or parallel scans from the aggregators based 
> on the region boundaries and the scan boundaries are sometimes adjusted based 
> on where optimizer provided key ranges, to include tenant boundaries, salt 
> boundaries etc. After the client opens the scanner and performs scan 
> operation, some of the coprocs return invalid rowkey for the following cases:
>  # Grouped aggregate queries
>  # Some Ungrouped aggregate queries
>  # Offset queries
>  # Dummy cells returned with empty rowkey
>  # Update statistics queries
>  # Uncovered Index queries
>  # Ordered results at server side
>  # ORDER BY DESC on rowkey
>  # Global Index read-repair
>  # Paging region scanner with HBase scanner reopen
>  # ORDER BY on non-pk column(s) with/without paging
>  # GROUP BY on non-pk column(s) with/without paging
> Since many of these cases return reserved rowkeys, they are likely not going 
> to match scan or region boundaries. It has potential to cause data integrity 
> issues in certain scenarios as explained above. Empty rowkey returned by 
> server can be treated as end of the region scan by HBase client.
> With the paging feature enabled, if the page size is kept low, we have higher 
> chances of scanners returning dummy cell, resulting in increased num of RPC 
> calls for better latency and timeouts. We should return only valid rowkey in 
> the scan range for all the cases where we perform above mentioned operations 
> like complex aggregate or offset queries etc.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to