/**
   * Closes all structures and deletes all temporary files. Even in the presence of failures, this
   * method will try and continue closing files and deleting temporary files.
   *
   * @throws IOException Thrown if an error occurred while closing/deleting the files.
   */
  public void close() throws IOException {

    // we go on closing and deleting files in the presence of failures.
    // we remember the first exception to occur and re-throw it later
    Throwable ex = null;

    synchronized (this) {
      if (closed) {
        return;
      }
      closed = true;

      // close the writers
      if (recordsOutFile != null) {
        try {
          recordsOutFile.close();
          recordsOutFile = null;
        } catch (Throwable t) {
          LOG.error("Cannot close the large records spill file.", t);
          ex = ex == null ? t : ex;
        }
      }
      if (keysOutFile != null) {
        try {
          keysOutFile.close();
          keysOutFile = null;
        } catch (Throwable t) {
          LOG.error("Cannot close the large records key spill file.", t);
          ex = ex == null ? t : ex;
        }
      }

      // close the readers
      if (recordsReader != null) {
        try {
          recordsReader.close();
          recordsReader = null;
        } catch (Throwable t) {
          LOG.error("Cannot close the large records reader.", t);
          ex = ex == null ? t : ex;
        }
      }
      if (keysReader != null) {
        try {
          keysReader.close();
          keysReader = null;
        } catch (Throwable t) {
          LOG.error("Cannot close the large records key reader.", t);
          ex = ex == null ? t : ex;
        }
      }

      // delete the spill files
      if (recordsChannel != null) {
        try {
          ioManager.deleteChannel(recordsChannel);
          recordsChannel = null;
        } catch (Throwable t) {
          LOG.error("Cannot delete the large records spill file.", t);
          ex = ex == null ? t : ex;
        }
      }
      if (keysChannel != null) {
        try {
          ioManager.deleteChannel(keysChannel);
          keysChannel = null;
        } catch (Throwable t) {
          LOG.error("Cannot delete the large records key spill file.", t);
          ex = ex == null ? t : ex;
        }
      }

      // close the key sorter
      if (keySorter != null) {
        try {
          keySorter.close();
          keySorter = null;
        } catch (Throwable t) {
          LOG.error("Cannot properly dispose the key sorter and clean up its temporary files.", t);
          ex = ex == null ? t : ex;
        }
      }

      memManager.release(memory);

      recordCounter = 0;
    }

    // re-throw the exception, if necessary
    if (ex != null) {
      throw new IOException(
          "An error occurred cleaning up spill files in the large record handler.", ex);
    }
  }
  public void testStringSorting() {
    File input = null;
    File sorted = null;

    try {
      // the source file
      input = generateFileWithStrings(300000, "http://some-uri.com/that/is/a/common/prefix/to/all");

      // the sorted file
      sorted = File.createTempFile("sorted_strings", "txt");

      String[] command = {
        "/bin/bash",
        "-c",
        "export LC_ALL=\"C\" && cat \""
            + input.getAbsolutePath()
            + "\" | sort > \""
            + sorted.getAbsolutePath()
            + "\""
      };

      Process p = null;
      try {
        p = Runtime.getRuntime().exec(command);
        int retCode = p.waitFor();
        if (retCode != 0) {
          throw new Exception("Command failed with return code " + retCode);
        }
        p = null;
      } finally {
        if (p != null) {
          p.destroy();
        }
      }

      // sort the data
      UnilateralSortMerger<String> sorter = null;
      BufferedReader reader = null;
      BufferedReader verifyReader = null;

      try {
        MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
        IOManager ioMan = new IOManager();

        TypeSerializer<String> serializer = StringSerializer.INSTANCE;
        TypeComparator<String> comparator = new StringComparator(true);

        reader = new BufferedReader(new FileReader(input));
        MutableObjectIterator<String> inputIterator = new StringReaderMutableObjectIterator(reader);

        sorter =
            new UnilateralSortMerger<String>(
                mm,
                ioMan,
                inputIterator,
                new DummyInvokable(),
                new RuntimeStatelessSerializerFactory<String>(serializer, String.class),
                comparator,
                1.0,
                4,
                0.8f);

        MutableObjectIterator<String> sortedData = sorter.getIterator();

        reader.close();

        // verify
        verifyReader = new BufferedReader(new FileReader(sorted));
        String next;

        while ((next = verifyReader.readLine()) != null) {
          String nextFromStratoSort = sortedData.next("");

          Assert.assertNotNull(nextFromStratoSort);
          Assert.assertEquals(next, nextFromStratoSort);
        }
      } finally {
        if (reader != null) {
          reader.close();
        }
        if (verifyReader != null) {
          verifyReader.close();
        }
        if (sorter != null) {
          sorter.close();
        }
      }
    } catch (Exception e) {
      System.err.println(e.getMessage());
      e.printStackTrace();
      Assert.fail(e.getMessage());
    } finally {
      if (input != null) {
        input.delete();
      }
      if (sorted != null) {
        sorted.delete();
      }
    }
  }
  public MutableObjectIterator<T> finishWriteAndSortKeys(List<MemorySegment> memory)
      throws IOException {
    if (recordsOutFile == null || keysOutFile == null) {
      throw new IllegalStateException("The LargeRecordHandler has not spilled any records");
    }

    // close the writers and
    final int lastBlockBytesKeys;
    final int lastBlockBytesRecords;

    recordsOutFile.close();
    keysOutFile.close();
    lastBlockBytesKeys = keysOutFile.getBytesInLatestSegment();
    lastBlockBytesRecords = recordsOutFile.getBytesInLatestSegment();
    recordsOutFile = null;
    keysOutFile = null;

    final int pagesForReaders =
        Math.max(
            3 * MIN_SEGMENTS_FOR_KEY_SPILLING,
            Math.min(2 * MAX_SEGMENTS_FOR_KEY_SPILLING, memory.size() / 50));
    final int pagesForKeyReader =
        Math.min(pagesForReaders - MIN_SEGMENTS_FOR_KEY_SPILLING, MAX_SEGMENTS_FOR_KEY_SPILLING);
    final int pagesForRecordReader = pagesForReaders - pagesForKeyReader;

    // grab memory for the record reader
    ArrayList<MemorySegment> memForRecordReader = new ArrayList<MemorySegment>();
    ArrayList<MemorySegment> memForKeysReader = new ArrayList<MemorySegment>();

    for (int i = 0; i < pagesForRecordReader; i++) {
      memForRecordReader.add(memory.remove(memory.size() - 1));
    }
    for (int i = 0; i < pagesForKeyReader; i++) {
      memForKeysReader.add(memory.remove(memory.size() - 1));
    }

    keysReader =
        new FileChannelInputView(
            ioManager.createBlockChannelReader(keysChannel),
            memManager,
            memForKeysReader,
            lastBlockBytesKeys);
    InputViewIterator<Tuple> keyIterator = new InputViewIterator<Tuple>(keysReader, keySerializer);

    keySorter =
        new UnilateralSortMerger<Tuple>(
            memManager,
            memory,
            ioManager,
            keyIterator,
            memoryOwner,
            keySerializerFactory,
            keyComparator,
            1,
            maxFilehandles,
            1.0f,
            false);

    // wait for the sorter to sort the keys
    MutableObjectIterator<Tuple> result;
    try {
      result = keySorter.getIterator();
    } catch (InterruptedException e) {
      throw new IOException(e);
    }

    recordsReader =
        new SeekableFileChannelInputView(
            ioManager, recordsChannel, memManager, memForRecordReader, lastBlockBytesRecords);

    return new FetchingIterator<T>(serializer, result, recordsReader, keySerializer, numKeyFields);
  }