Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-1873

Race condition around HRegionServer -> HMaster communication

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Invalid
    • 0.20.0
    • None
    • None
    • None

    Description

      HRegionServer on lines 459 - 463 (part of run()) accesses outboundMsgs in a synchronized fashion, but other uses of the object are not synchronized.

      Specifically, the code is

      459          synchronized(this.outboundMsgs) {
      460            outboundArray =
      461              this.outboundMsgs.toArray(new HMsg[outboundMsgs.size()]); 
      462            this.outboundMsgs.clear();
      463          }
      

      Whereas things are added to this list from calls like

        private void reportOpen(HRegionInfo region) {
          outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, region));
        }
      

      And

        void reportSplit(HRegionInfo oldRegion, HRegionInfo newRegionA,
            HRegionInfo newRegionB) {
          outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_SPLIT, oldRegion,
            ("Daughters; " +
              newRegionA.getRegionNameAsString() + ", " +
              newRegionB.getRegionNameAsString()).getBytes()));
          outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionA));
          outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionB));
        }
      

      It looks like the object is initialized as

        private final List<HMsg> outboundMsgs =
          Collections.synchronizedList(new ArrayList<HMsg>());
      

      Which would appear to provide security, but it doesn't actually prevent an insert from happening between lines 461 and 462, which would subsequently get removed from the call to clear(). At least, from the Sun HotSpot source code, it looks like Collections.synchronizedList() does the right thing and synchronizes on an inner mutex object instead of synchronizing on the externally visible list object itself. That means, however, that the synchronized() on line 459 is largely meaningless.

      I'm not sure how often this race condition would occur in the wild, but every thread waiting on the mutex around the toArray() call increases the probability that the next person to get the mutex is someone who wants to add something to the List, rather than the thread calling clear().

      Simple fix would be to do external synchronization around all accesses to the List. Barring that, perhaps a SynchronizedList implementation with a "emptyToArray()" method that encapsulates the toArray() and subsequent clear().

      Attachments

        Activity

          People

            Unassigned Unassigned
            cheddar Eric Tschetter
            Votes:
            0 Vote for this issue
            Watchers:
            0 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: