Uploaded image for project: 'Hadoop Map/Reduce'
  1. Hadoop Map/Reduce
  2. MAPREDUCE-4467

IndexCache failures due to missing synchronization

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • 0.23.2
    • 0.23.3, 2.0.2-alpha
    • nodemanager
    • None

    Description

      TestMRJobs.testSleepJob fails randomly due to synchronization error in IndexCache:

      2012-07-20 19:32:34,627 ERROR [New I/O server worker #2-1] mapred.ShuffleHandler (ShuffleHandler.java:exceptionCaught(528)) - Shuffle error: 
      java.lang.IllegalMonitorStateException
      	at java.lang.Object.wait(Native Method)
      	at org.apache.hadoop.mapred.IndexCache.getIndexInformation(IndexCache.java:74)
      	at org.apache.hadoop.mapred.ShuffleHandler$Shuffle.sendMapOutput(ShuffleHandler.java:471)
      	at org.apache.hadoop.mapred.ShuffleHandler$Shuffle.messageReceived(ShuffleHandler.java:397)
      	at org.jboss.netty.handler.stream.ChunkedWriteHandler.handleUpstream(ChunkedWriteHandler.java:148)
      	at org.jboss.netty.handler.codec.http.HttpChunkAggregator.messageReceived(HttpChunkAggregator.java:116)
      	at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:302)
      	at org.jboss.netty.handler.codec.replay.ReplayingDecoder.unfoldAndfireMessageReceived(ReplayingDecoder.java:522)
      	at org.jboss.netty.handler.codec.replay.ReplayingDecoder.callDecode(ReplayingDecoder.java:506)
      	at org.jboss.netty.handler.codec.replay.ReplayingDecoder.messageReceived(ReplayingDecoder.java:443)
      	at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:274)
      	at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:261)
      	at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:349)
      	at org.jboss.netty.channel.socket.nio.NioWorker.processSelectedKeys(NioWorker.java:280)
      	at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:200)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
      	at java.lang.Thread.run(Thread.java:662)
      

      A related issue is MAPREDUCE-4384. The change introduced there removed "synchronized" keyword and hence "info.wait()" call fails. Tbis needs to be wrapped into a "synchronized" block.

      Attachments

        1. mapreduce-4467.patch.txt
          2 kB
          Kihwal Lee
        2. mapreduce-4467.patch.txt
          2 kB
          Kihwal Lee

        Activity

          People

            kihwal Kihwal Lee
            aklochkov Andrey Klochkov
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: