Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-14941

The AbstractTableInputFormat#nextRecord in hbase connector will handle the same rowkey twice once encountered any exception

    XMLWordPrintableJSON

Details

    Description

      In the mail list [1]. The user complain that it will see the same row twice if encountered any HBase exception.
      The problem is here:

      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.

      [1]. http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/DataSet-API-HBase-ScannerTimeoutException-and-double-Result-processing-td31174.html

      Attachments

        Issue Links

          Activity

            People

              openinx Zheng Hu
              openinx Zheng Hu
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 20m
                  20m