HBase
  1. HBase
  2. HBASE-11402

Scanner performs redundand datanode requests

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Duplicate
    • Affects Version/s: None
    • Fix Version/s: None
    • Component/s: HFile, Scanners
    • Labels:
      None

      Description

      Using hbase 0.94.6 I found duplicate datanode requests of this sort:

      2014-06-09 14:12:22,039 INFO org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: /10.103.0.73:50010, dest: /10.103.0.38:57897, bytes: 1056768, op: HDFS_READ, cliID: DFSClient_NONMAPREDUCE_1702752887_26, offset: 35840, srvID: DS-504316153-10.103.0.73-50010-1342437562377, blockid: BP-404551095-10.103.0.38-1376045452213:blk_3541255952831727320_613837, duration: 109928797000
      2014-06-09 14:12:22,080 INFO org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: /10.103.0.73:50010, dest: /10.103.0.38:57910, bytes: 1056768, op: HDFS_READ, cliID: DFSClient_NONMAPREDUCE_1702752887_26, offset: 0, srvID: DS-504316153-10.103.0.73-50010-1342437562377, blockid: BP-404551095-10.103.0.38-1376045452213:blk_3541255952831727320_613837, duration: 38250000
      

      After short investigation, I found the source of such behaviour:

      • StoreScanner in constructor calls StoreFileScanner::seek, which (after several levels of calls) is calling HFileBlock::readBlockDataInternal which reads block and pre-reads header of the next block.
      • This pre-readed header is stored in ThreadLocal<PrefetchedHeader> variable and stream is left in a position right behind the header of next block.
      • After constructor finished, scanner code does scanning, and, after pre-readed block data finished, it calls HFileReaderV2::readNextDataBlock, which again calls HFileBlock::readBlockDataInternal, but this call occured from different thread and there is nothing usefull in ThreadLocal variable
      • Due to this, stream is asked to seek backwards, and this cause duplicate DN request.

      As far as I understood from trunk code, the problem hasn't fixed yet.

      Log of calls with process above:

      2014-06-18 14:55:36,616 INFO org.apache.hadoop.hbase.io.hfile.HFileBlockIndex: loadDataBlockWithScanInfo: entered
      2014-06-18 14:55:36,616 INFO org.apache.hadoop.hbase.io.hfile.HFileReaderV2: seekTo: readBlock, ofs = 0, size = -1
      2014-06-18 14:55:36,617 INFO org.apache.hadoop.hbase.io.hfile.HFileReaderV2: Before block read: path = hdfs://tsthdp1.p:9000/hbase/webpagesII/ba16051997b1272f00bed5f65094dc63/p/c866b7b0eded4b
      2014-06-18 14:55:36,617 INFO org.apache.hadoop.hbase.io.hfile.HFile: readBlockDataInternal. Ofs = 0, is.pos = 137257042, ondDiskSizeWithHeader = -1
      2014-06-18 14:55:36,617 INFO org.apache.hadoop.hbase.io.hfile.HFile: readBlockDataInternal: prefetchHeader.ofs = -1, thread = 48
      2014-06-18 14:55:36,617 INFO org.apache.hadoop.hbase.io.hfile.HFile: FSReaderV2: readAtOffset: size = 24, offset = 0, peekNext = false
      2014-06-18 14:55:36,617 INFO org.apache.hadoop.hdfs.DFSClient: seek: targetPos = 0, pos = 137257042, blockEnd = 137257229
      2014-06-18 14:55:36,617 INFO org.apache.hadoop.hdfs.DFSClient: seek: not done, blockEnd = -1
      2014-06-18 14:55:36,617 INFO org.apache.hadoop.hdfs.DFSClient: readWithStrategy: before seek, pos = 0, blockEnd = -1, currentNode = 10.103.0.73:50010
      2014-06-18 14:55:36,618 INFO org.apache.hadoop.hdfs.DFSClient: getBlockAt: blockEnd updated to 137257229
      2014-06-18 14:55:36,618 INFO org.apache.hadoop.hdfs.DFSClient: blockSeekTo: loop, target = 0
      2014-06-18 14:55:36,618 INFO org.apache.hadoop.hdfs.DFSClient: getBlockReader: dn = tsthdp2.p, file = /hbase/webpagesII/ba16051997b1272f00bed5f65094dc63/p/c866b7b0eded4b42bc40aa9e18ac8a4b, bl
      2014-06-18 14:55:36,627 INFO org.apache.hadoop.hdfs.DFSClient: readBuffer: ofs = 0, len = 24
      2014-06-18 14:55:36,627 INFO org.apache.hadoop.hdfs.DFSClient: readBuffer: try to read
      2014-06-18 14:55:36,641 INFO org.apache.hadoop.hdfs.DFSClient: readBuffer: done, len = 24
      2014-06-18 14:55:36,641 INFO org.apache.hadoop.hbase.io.hfile.HFile: FSReaderV2: readAtOffset: size = 35899, offset = 24, peekNext = true
      2014-06-18 14:55:36,641 INFO org.apache.hadoop.hdfs.DFSClient: seek: targetPos = 24, pos = 24, blockEnd = 137257229
      2014-06-18 14:55:36,641 INFO org.apache.hadoop.hdfs.DFSClient: seek: check that we cat skip diff = 0
      2014-06-18 14:55:36,641 INFO org.apache.hadoop.hdfs.DFSClient: seek: try to fast-forward on diff = 0, pos = 24
      2014-06-18 14:55:36,641 INFO org.apache.hadoop.hdfs.DFSClient: seek: pos after = 24
      2014-06-18 14:55:36,641 INFO org.apache.hadoop.hdfs.DFSClient: readBuffer: ofs = 24, len = 35923
      2014-06-18 14:55:36,641 INFO org.apache.hadoop.hdfs.DFSClient: readBuffer: try to read
      2014-06-18 14:55:36,641 INFO org.apache.hadoop.hdfs.DFSClient: readBuffer: done, len = 35923
      2014-06-18 14:55:36,642 INFO org.apache.hadoop.hbase.io.hfile.HFileReaderV2: Block data read
      2014-06-18 14:55:36,642 INFO org.apache.hadoop.hbase.io.hfile.HFileReaderV2: After block read, ms = 25191000
      2014-06-18 14:55:36,670 INFO org.apache.hadoop.hbase.io.hfile.HFileReaderV2: readNextDataBlock: entry
      2014-06-18 14:55:36,671 INFO org.apache.hadoop.hbase.io.hfile.HFileReaderV2: readNextDataBlock: readBlock, ofs = 0, with header = 35923, size = 22486
      2014-06-18 14:55:36,671 INFO org.apache.hadoop.hbase.io.hfile.HFileReaderV2: Before block read: path = hdfs://tsthdp1.p:9000/hbase/webpagesII/ba16051997b1272f00bed5f65094dc63/p/c866b7b0eded4b
      2014-06-18 14:55:36,671 INFO org.apache.hadoop.hbase.io.hfile.HFile: readBlockDataInternal. Ofs = 35923, is.pos = 35947, ondDiskSizeWithHeader = 22486
      2014-06-18 14:55:36,671 INFO org.apache.hadoop.hbase.io.hfile.HFile: Inefficient readBlockDataInternal. Read at 35923, stream pos = 35947
      2014-06-18 14:55:36,671 INFO org.apache.hadoop.hbase.io.hfile.HFile: readBlockDataInternal: prefetchHeader.ofs = -1, thread = 50
      2014-06-18 14:55:36,671 INFO org.apache.hadoop.hbase.io.hfile.HFile: FSReaderV2: readAtOffset: size = 22486, offset = 35923, peekNext = true
      2014-06-18 14:55:36,671 INFO org.apache.hadoop.hdfs.DFSClient: seek: targetPos = 35923, pos = 35947, blockEnd = 137257229
      2014-06-18 14:55:36,671 INFO org.apache.hadoop.hdfs.DFSClient: seek: not done, blockEnd = -1
      2014-06-18 14:55:36,671 INFO org.apache.hadoop.hdfs.DFSClient: readWithStrategy: before seek, pos = 35923, blockEnd = -1, currentNode = 10.103.0.73:50010
      2014-06-18 14:55:36,671 INFO org.apache.hadoop.hdfs.DFSClient: getBlockAt: blockEnd updated to 137257229
      2014-06-18 14:55:36,672 INFO org.apache.hadoop.hdfs.DFSClient: blockSeekTo: loop, target = 35923
      2014-06-18 14:55:36,672 INFO org.apache.hadoop.hdfs.DFSClient: getBlockReader: dn = tsthdp2.p, file = /hbase/webpagesII/ba16051997b1272f00bed5f65094dc63/p/c866b7b0eded4b42bc40aa9e18ac8a4b, bl
      2014-06-18 14:55:36,675 INFO org.apache.hadoop.hdfs.DFSClient: readBuffer: ofs = 0, len = 22510
      2014-06-18 14:55:36,675 INFO org.apache.hadoop.hdfs.DFSClient: readBuffer: try to read
      2014-06-18 14:55:36,687 INFO org.apache.hadoop.hdfs.DFSClient: readBuffer: done, len = 22510
      2014-06-18 14:55:36,688 INFO org.apache.hadoop.hbase.io.hfile.HFileReaderV2: Block data read
      2014-06-18 14:55:36,688 INFO org.apache.hadoop.hbase.io.hfile.HFileReaderV2: After block read, ms = 17250000
      

      Currently, I see two possible fixes for that, but not sure that I'm not overlooking something:

      • remove ThreadLocal from block's header cache variable
      • postpone call to StoreFileScanner::seek to cause it to be called by the correct thread

        Issue Links

          Activity

          Hide
          Anoop Sam John added a comment -

          Already fixed by some other jira. Not remembering the jira id.

          Show
          Anoop Sam John added a comment - Already fixed by some other jira. Not remembering the jira id.
          Hide
          Lars Hofhansl added a comment -

          Not yet fixed. See here: HBASE-10676. I guess I dropped that ball on that one.

          Show
          Lars Hofhansl added a comment - Not yet fixed. See here: HBASE-10676 . I guess I dropped that ball on that one.
          Hide
          Anoop Sam John added a comment -

          Oh. Didn't follow up with that issue. Though it was already done. Thanks for the correction Lars.

          Show
          Anoop Sam John added a comment - Oh. Didn't follow up with that issue. Though it was already done. Thanks for the correction Lars.
          Hide
          stack added a comment -

          Resolving as duplicated by HBASE-10676

          Do not take my closing of this issue as a discounting of the work done in here Max Lapan; rather, your work has been carried over into the duplicate and later in HBASE-17072. This issue is still a problem. You identified the useless seek. The threadlocal in turn is super problematic.

          Show
          stack added a comment - Resolving as duplicated by HBASE-10676 Do not take my closing of this issue as a discounting of the work done in here Max Lapan ; rather, your work has been carried over into the duplicate and later in HBASE-17072 . This issue is still a problem. You identified the useless seek. The threadlocal in turn is super problematic.

            People

            • Assignee:
              Unassigned
              Reporter:
              Max Lapan
            • Votes:
              0 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development