Uploaded image for project: 'Hadoop Common'
  1. Hadoop Common
  2. HADOOP-3821

SequenceFile's Reader.decompressorPool or Writer.decompressorPool gets into an inconsistent state when calling close() more than once

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 0.17.0, 0.17.1, 0.18.0, 0.19.0
    • 0.18.1
    • io
    • None
    • Reviewed

    Description

      SequenceFile.Reader uses a decompressorPool to reuse Decompressor instances. The Reader obtains such an instance from the pool on object creation and returns it back to the pool it when close() is called.

      SequenceFile.Reader implements the java.io.Closable interface and it's spec on the close() method says:

      Closes this stream and releases any system resources associated
      with it. If the stream is already closed then invoking this
      method has no effect.

      This spec is violated by the Reader implementation, because calling close() multiple times has really bad implications.
      When you call close() twice, one and the same Decompressor instances will be returned to the pool two times and the pool would now maintain duplicated references to the same Decompressor instances. When other Readers now request instances from the pool it might happen that two Readers get the same Decompressor instance.

      The correct behavior would be to just ignore a second call to close().

      The exact same issue applies to the SequenceFile.Writer as well.

      We were having big trouble with this, because we were observing sporadic exceptions from merge operations. The strange thing was that executing the same merge again usually succeeded. But sometimes it took multiple attempts to complete a merge successfully. It was very hard to debug that the root cause was some duplicated Decompressor references in the decompressorPool.

      Exceptions that we observed in production looked like this (we were using hadoop 0.17.0):

      java.io.IOException: unknown compression method
      at org.apache.hadoop.io.compress.zlib.BuiltInZlibInflater.decompress(BuiltInZlibInflater.java:47)
      at org.apache.hadoop.io.compress.DecompressorStream.decompress(DecompressorStream.java:80)
      at org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:74)
      at java.io.DataInputStream.readFully(DataInputStream.java:178)
      at org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:56)
      at org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:90)
      at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1995)
      at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:3002)
      at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2760)
      at org.apache.hadoop.io.SequenceFile$Sorter.writeFile(SequenceFile.java:2625)
      at org.apache.hadoop.io.SequenceFile$Sorter.merge(SequenceFile.java:2644)
      

      or

      java.io.IOException: zero length keys not allowed
      at org.apache.hadoop.io.SequenceFile$BlockCompressWriter.appendRaw(SequenceFile.java:1340)
      at org.apache.hadoop.io.SequenceFile$Sorter.writeFile(SequenceFile.java:2626)
      at org.apache.hadoop.io.SequenceFile$Sorter.merge(SequenceFile.java:2644)
      

      The following snippet reproduces the problem:

          public void testCodecPool() throws IOException {
              Configuration conf = new Configuration();
              LocalFileSystem fs = new LocalFileSystem();
              fs.setConf(conf);
              fs.getRawFileSystem().setConf(conf);
      
              // create a sequence file
              Path path = new Path("target/seqFile");
              SequenceFile.Writer writer = SequenceFile.createWriter(fs, conf, path, Text.class, NullWritable.class, CompressionType.BLOCK);
              writer.append(new Text("key1"), NullWritable.get());
              writer.append(new Text("key2"), NullWritable.get());
              writer.close();
      
              // Create a reader which uses 4 BuiltInZLibInflater instances
              SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
              // Returns the 4 BuiltInZLibInflater instances to the CodecPool
              reader.close();
              // The second close erroneously returns the same 4 BuiltInZLibInflater instances to the CodecPool again
              reader.close();
      
              // The first reader gets 4 BuiltInZLibInflater instances from the CodecPool
              SequenceFile.Reader reader1 = new SequenceFile.Reader(fs, path, conf);
              // read first value from reader1
              Text text = new Text();
              reader1.next(text);
              assertEquals("key1", text.toString());
              // The second reader gets the same 4 BuiltInZLibInflater instances from the CodePool as reader1
              SequenceFile.Reader reader2 = new SequenceFile.Reader(fs, path, conf);
              // read first value from reader2
              reader2.next(text);
              assertEquals("key1", text.toString());
              // read second value from reader1
              reader1.next(text);
              assertEquals("key2", text.toString());
              // read second value from reader2 (this throws an exception)
              reader2.next(text);
              assertEquals("key2", text.toString());
              
              assertFalse(reader1.next(text));
              assertFalse(reader2.next(text));
          }
      

      It fails with the exception:

      java.io.EOFException
      	at java.io.DataInputStream.readByte(DataInputStream.java:243)
      	at org.apache.hadoop.io.WritableUtils.readVLong(WritableUtils.java:324)
      	at org.apache.hadoop.io.WritableUtils.readVInt(WritableUtils.java:345)
      	at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1835)
      	at CodecPoolTest.testCodecPool(CodecPoolTest.java:56)
      

      But this is just a very simple test that shows the problem. Much more weired things can happen when running in a complex production environment. Esp. heavy concurrency makes the behavior much more exciting.

      Attachments

        1. HADOOP-3821_0_20080724.patch
          2 kB
          Arun Murthy
        2. HADOOP-3821_test1_20080805.patch
          2 kB
          Peter Voss
        3. HADOOP-3821_1_20080821.patch
          5 kB
          Arun Murthy
        4. HADOOP-3821_1_20080821.patch
          5 kB
          Arun Murthy

        Activity

          People

            acmurthy Arun Murthy
            pvoss Peter Voss
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: