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

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

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


##########
phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java:
##########
@@ -371,12 +425,32 @@ private static class UnorderedGroupByRegionScanner 
extends BaseRegionScanner {
         private final long pageSizeMs;
         private RegionScanner regionScanner = null;
         private final GroupByCache groupByCache;
+        private final Scan scan;
+        private final byte[] scanStartRowKey;
+        private final boolean includeStartRowKey;
+        private final byte[] prevScanStartRowKey;
+        private final Boolean prevScanIncludeStartRowKey;
+        private boolean firstScan = true;
+        private boolean updateScannerBasedOnPrevRowKey = false;
+        private byte[] lastReturnedRowKey = null;
 
         private UnorderedGroupByRegionScanner(final 
ObserverContext<RegionCoprocessorEnvironment> c,
                                               final Scan scan, final 
RegionScanner scanner, final List<Expression> expressions,
                                               final ServerAggregators 
aggregators, final long limit, final long pageSizeMs) {
             super(scanner);
             this.region = c.getEnvironment().getRegion();
+            this.scan = scan;
+            // If scan start rowkey is empty, use region boundaries. Reverse 
region boundaries
+            // for reverse scan.
+            scanStartRowKey = scan.getStartRow().length > 0 ? 
scan.getStartRow() :
+                    (scan.isReversed() ? region.getRegionInfo().getEndKey() :
+                            region.getRegionInfo().getStartKey());
+            includeStartRowKey = scan.includeStartRow();
+            // Retrieve start rowkey of the previous scan. This would be 
different than
+            // current scan start rowkey if the region has recently moved or 
split or merged.
+            this.prevScanStartRowKey =
+                    
scan.getAttribute(BaseScannerRegionObserverConstants.SCAN_ACTUAL_START_ROW);
+            this.prevScanIncludeStartRowKey = true;

Review Comment:
   The reason why it is kept true here is because everytime when we set 
`_ScanActualStartRow` scan attribute, we always use `scan#withStartRow` 
function which internally sets include start row as true. So, yes this never 
changes, this is meant to be true only, even across region moves after scanner 
resets.





> 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