phoenix-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (PHOENIX-2628) Ensure split when iterating through results handled correctly
Date Wed, 06 Apr 2016 00:44:25 GMT

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

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

Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/156#discussion_r58637402
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
---
    @@ -107,8 +127,37 @@ public synchronized void close() throws SQLException {
         @Override
         public synchronized Tuple next() throws SQLException {
             initScanner();
    -        Tuple t = scanIterator.next();
    -        return t;
    +        try {
    +            lastTuple = scanIterator.next();
    +            if (lastTuple != null) {
    +                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    +                lastTuple.getKey(ptr);
    +            }
    +        } catch (SQLException e) {
    +            try {
    +                throw ServerUtil.parseServerException(e);
    +            } catch(StaleRegionBoundaryCacheException e1) {
    +                if(scan.getAttribute(NON_AGGREGATE_QUERY)!=null) {
    +                    Scan newScan = ScanUtil.newScan(scan);
    +                    if(lastTuple != null) {
    +                        lastTuple.getKey(ptr);
    +                        byte[] startRowSuffix = ByteUtil.copyKeyBytesIfNecessary(ptr);
    +                        if(ScanUtil.isLocalIndex(newScan)) {
    +                            newScan.setAttribute(SCAN_START_ROW_SUFFIX, ByteUtil.nextKey(startRowSuffix));
    +                        } else {
    +                            newScan.setStartRow(ByteUtil.nextKey(startRowSuffix));
    +                        }
    +                    }
    +                    plan.getContext().getConnection().getQueryServices().clearTableRegionCache(htable.getTableName());
    +                    this.scanIterator =
    +                            plan.iterator(DefaultParallelScanGrouper.getInstance(), newScan);
    --- End diff --
    
    So we're re-running the scan, only if it's an aggregate scan? Does this apply only for
an "ordered" group by (which is grouping by the leading part of the pk and doing so in-place)?
In the unordered case, each scan is done during the post open and the results are cached on
the RS, so a split won't impact the results. There may be a race condition between the pre
open (where we check if we're in the region) and the post open (where we start the region
operation), but I think that could be solved by doing the in-region check again). In these
cases, we already handle this through the exception catching code in BaseResultIterators.


> Ensure split when iterating through results handled correctly
> -------------------------------------------------------------
>
>                 Key: PHOENIX-2628
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-2628
>             Project: Phoenix
>          Issue Type: Bug
>            Reporter: James Taylor
>            Assignee: Rajeshbabu Chintaguntla
>             Fix For: 4.8.0
>
>         Attachments: PHOENIX-2628-wip.patch, PHOENIX-2628.patch, PHOENIX-2628_v7.patch,
PHOENIX-2628_v8.patch
>
>
> We should start with a test case to ensure this works correctly, both for scans and aggregates.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message