openinx opened a new pull request #10314: [FLINK-14941][hbase] The 
AbstractTableInputFormat#nextRecord in hbase connector will handle the same 
rowkey twice once encountered any exception
URL: https://github.com/apache/flink/pull/10314
 
 
   ## What is the purpose of the change
   
   In the [mail 
list](http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/DataSet-API-HBase-ScannerTimeoutException-and-double-Result-processing-td31174.html).
 The user complain that it will see the same row twice if encountered any HBase 
exception.
   The problem is here:
   
   ```java
   public T nextRecord(T reuse) throws IOException {
                if (resultScanner == null) {
                        throw new IOException("No table result scanner 
provided!");
                }
                try {
                        Result res = resultScanner.next();
                        if (res != null) {
                                scannedRows++;
                                currentRow = res.getRow();
                                return mapResultToOutType(res);
                        }
                } catch (Exception e) {
                        resultScanner.close();
                        //workaround for timeout on scan
                        LOG.warn("Error after scan of " + scannedRows + " rows. 
Retry with a new scanner...", e);
                        scan.setStartRow(currentRow);
                        resultScanner = table.getScanner(scan);
                        Result res = resultScanner.next();
                        if (res != null) {
                                scannedRows++;
                                currentRow = res.getRow();
                                return mapResultToOutType(res);
                        }
                }
   
                endReached = true;
                return null;
   }
   ```
   
   We will set the startRow of the new scan to the currentRow which has been 
seen, that means the currentRow will be seen twice. Actually, we should replace 
the scan.setStartRow(currentRow) as scan.withStartRow(currentRow, false) , the 
false means exclude the currentRow.
   
   
   
   ## Brief change log
   
   - 12e35abc0a [FLINK-14941][hbase] The AbstractTableInputFormat#nextRecord in 
hbase connector will handle the same rowkey twice once encountered any exception
   
   
   ## Verifying this change
   
   This change does not have a test.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not documented)
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to