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


##########
phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java:
##########
@@ -411,6 +487,115 @@ private UnorderedGroupByRegionScanner(final 
ObserverContext<RegionCoprocessorEnv
 
         @Override
         public boolean next(List<Cell> resultsToReturn) throws IOException {
+            if (firstScan && prevScanStartRowKey != null && 
prevScanIncludeStartRowKey != null) {
+                firstScan = false;
+                if (scanStartRowKey.length > 0 && 
!ScanUtil.isLocalIndex(scan)) {
+                    if (Bytes.compareTo(prevScanStartRowKey, scanStartRowKey) 
!= 0 ||
+                            prevScanIncludeStartRowKey != includeStartRowKey) {
+                        LOGGER.info("Region has moved.. Prev scan start rowkey 
{} is not same as" +
+                                        " current scan start rowkey {}",
+                                Bytes.toStringBinary(prevScanStartRowKey),
+                                Bytes.toStringBinary(scanStartRowKey));
+                        // If region has moved in the middle of the scan 
operation, after resetting
+                        // the scanner, hbase client uses (latest received 
rowkey + \x00) as new
+                        // start rowkey for resuming the scan operation on the 
new scanner.
+                        if (Bytes.compareTo(
+                                ByteUtil.concat(prevScanStartRowKey, 
Bytes.toBytesBinary("\\x00")),
+                                scanStartRowKey) == 0) {
+                            
scan.setAttribute(QueryServices.PHOENIX_PAGING_NEW_SCAN_START_ROWKEY,
+                                    prevScanStartRowKey);
+                            scan.setAttribute(
+                                    
QueryServices.PHOENIX_PAGING_NEW_SCAN_START_ROWKEY_INCLUDE,
+                                    Bytes.toBytes(prevScanIncludeStartRowKey));
+                        } else {
+                            // This happens when the server side scanner has 
already sent some
+                            // rows back to the client and region has moved, 
so now we need to
+                            // use updateScannerBasedOnPrevRowKey flag and 
also reset the scanner
+                            // at paging region scanner level to re-read the 
previously sent
+                            // values in order to re-compute the aggregation 
and then return
+                            // only the next rowkey that was not yet sent back 
to the client.
+                            updateScannerBasedOnPrevRowKey = true;
+                            
scan.setAttribute(QueryServices.PHOENIX_PAGING_NEW_SCAN_START_ROWKEY,
+                                    prevScanStartRowKey);
+                            scan.setAttribute(
+                                    
QueryServices.PHOENIX_PAGING_NEW_SCAN_START_ROWKEY_INCLUDE,
+                                    Bytes.toBytes(prevScanIncludeStartRowKey));

Review Comment:
   OK, I got it. The first condition is true when we haven't returned any valid 
rows yet. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to