private static Environment getMockEnvironment(File[] tempDirs) {
    IOManager ioMan = mock(IOManager.class);
    when(ioMan.getSpillingDirectories()).thenReturn(tempDirs);

    Environment env = mock(Environment.class);
    when(env.getJobID()).thenReturn(new JobID());
    when(env.getUserClassLoader()).thenReturn(RocksDBStateBackendConfigTest.class.getClassLoader());
    when(env.getIOManager()).thenReturn(ioMan);
    return env;
  }
  @After
  public void afterTest() throws Exception {
    ioManager.shutdown();
    Assert.assertTrue("IO Manager has not properly shut down.", ioManager.isProperlyShutDown());

    Assert.assertTrue(
        "Not all memory was returned to the memory manager in the test.", memManager.verifyEmpty());
    memManager.shutdown();
    memManager = null;
  }
  SpilledSubpartitionViewAsyncIO(
      ResultSubpartition parent,
      BufferProvider bufferProvider,
      IOManager ioManager,
      FileIOChannel.ID channelId,
      long initialSeekPosition,
      int readBatchSize)
      throws IOException {

    checkArgument(initialSeekPosition >= 0, "Initial seek position is < 0.");
    checkArgument(readBatchSize >= 1, "Batch read size < 1.");

    this.parent = checkNotNull(parent);
    this.bufferProvider = checkNotNull(bufferProvider);
    this.bufferAvailabilityListener = new BufferProviderCallback(this);

    this.asyncFileReader = ioManager.createBufferFileReader(channelId, new IOThreadCallback(this));

    if (initialSeekPosition > 0) {
      asyncFileReader.seekToPosition(initialSeekPosition);
    }

    this.readBatchSize = readBatchSize;

    this.fileSize = asyncFileReader.getSize();

    // Trigger the initial read requests
    readNextBatchAsync();
  }
  private static Environment getMockEnvironment(File[] tempDirs) {
    IOManager ioMan = mock(IOManager.class);
    when(ioMan.getSpillingDirectories()).thenReturn(tempDirs);

    Environment env = mock(Environment.class);
    when(env.getJobID()).thenReturn(new JobID());
    when(env.getUserClassLoader()).thenReturn(RocksDBStateBackendConfigTest.class.getClassLoader());
    when(env.getIOManager()).thenReturn(ioMan);
    when(env.getTaskKvStateRegistry())
        .thenReturn(new KvStateRegistry().createTaskRegistry(new JobID(), new JobVertexID()));

    TaskInfo taskInfo = mock(TaskInfo.class);
    when(env.getTaskInfo()).thenReturn(taskInfo);

    when(taskInfo.getIndexOfThisSubtask()).thenReturn(0);
    return env;
  }
  private final void testChannelWithSegments(int numSegments) throws Exception {
    final List<MemorySegment> memory = this.memManager.allocatePages(memoryOwner, numSegments);
    final Channel.ID channel = this.ioManager.createChannel();

    BlockChannelWriter writer = null;
    BlockChannelReader reader = null;

    try {
      writer = this.ioManager.createBlockChannelWriter(channel);
      final ChannelWriterOutputView out =
          new ChannelWriterOutputView(writer, memory, this.memManager.getPageSize());

      long writeStart = System.currentTimeMillis();

      int valsLeft = NUM_INTS_WRITTEN;
      while (valsLeft-- > 0) {
        out.writeInt(valsLeft);
      }

      out.close();
      final int numBlocks = out.getBlockCount();
      writer.close();
      writer = null;

      long writeElapsed = System.currentTimeMillis() - writeStart;

      // ----------------------------------------------------------------

      reader = ioManager.createBlockChannelReader(channel);
      final ChannelReaderInputView in =
          new ChannelReaderInputView(reader, memory, numBlocks, false);

      long readStart = System.currentTimeMillis();

      valsLeft = NUM_INTS_WRITTEN;
      while (valsLeft-- > 0) {
        in.readInt();
        //				Assert.assertTrue(rec.getValue() == valsLeft);
      }

      in.close();
      reader.close();

      long readElapsed = System.currentTimeMillis() - readStart;

      reader.deleteChannel();
      reader = null;

      LOG.info(
          "IOManager with "
              + numSegments
              + " mem segments: write "
              + writeElapsed
              + " msecs, read "
              + readElapsed
              + " msecs.");

      memManager.release(memory);
    } finally {
      if (reader != null) {
        reader.closeAndDelete();
      }
      if (writer != null) {
        writer.closeAndDelete();
      }
    }
  }
  @SuppressWarnings("resource")
  private final void speedTestNIO(int bufferSize, boolean direct) throws IOException {
    final Channel.ID tmpChannel = ioManager.createChannel();

    File tempFile = null;
    FileChannel fs = null;

    try {
      tempFile = new File(tmpChannel.getPath());

      RandomAccessFile raf = new RandomAccessFile(tempFile, "rw");
      fs = raf.getChannel();

      ByteBuffer buf =
          direct ? ByteBuffer.allocateDirect(bufferSize) : ByteBuffer.allocate(bufferSize);

      long writeStart = System.currentTimeMillis();

      int valsLeft = NUM_INTS_WRITTEN;
      while (valsLeft-- > 0) {
        if (buf.remaining() < 4) {
          buf.flip();
          fs.write(buf);
          buf.clear();
        }
        buf.putInt(valsLeft);
      }

      if (buf.position() > 0) {
        buf.flip();
        fs.write(buf);
      }

      fs.close();
      raf.close();
      fs = null;

      long writeElapsed = System.currentTimeMillis() - writeStart;

      // ----------------------------------------------------------------

      raf = new RandomAccessFile(tempFile, "r");
      fs = raf.getChannel();
      buf.clear();

      long readStart = System.currentTimeMillis();

      fs.read(buf);
      buf.flip();

      valsLeft = NUM_INTS_WRITTEN;
      while (valsLeft-- > 0) {
        if (buf.remaining() < 4) {
          buf.compact();
          fs.read(buf);
          buf.flip();
        }
        if (buf.getInt() != valsLeft) {
          throw new IOException();
        }
      }

      fs.close();
      raf.close();

      long readElapsed = System.currentTimeMillis() - readStart;

      LOG.info(
          "NIO Channel with buffer "
              + bufferSize
              + ": write "
              + writeElapsed
              + " msecs, read "
              + readElapsed
              + " msecs.");
    } finally {
      // close if possible
      if (fs != null) {
        fs.close();
        fs = null;
      }
      // try to delete the file
      if (tempFile != null) {
        tempFile.delete();
      }
    }
  }
  private final void speedTestStream(int bufferSize) throws IOException {
    final Channel.ID tmpChannel = ioManager.createChannel();
    final IntegerRecord rec = new IntegerRecord(0);

    File tempFile = null;
    DataOutputStream daos = null;
    DataInputStream dais = null;

    try {
      tempFile = new File(tmpChannel.getPath());

      FileOutputStream fos = new FileOutputStream(tempFile);
      daos = new DataOutputStream(new BufferedOutputStream(fos, bufferSize));

      long writeStart = System.currentTimeMillis();

      int valsLeft = NUM_INTS_WRITTEN;
      while (valsLeft-- > 0) {
        rec.setValue(valsLeft);
        rec.write(new OutputViewDataOutputStreamWrapper(daos));
      }
      daos.close();
      daos = null;

      long writeElapsed = System.currentTimeMillis() - writeStart;

      // ----------------------------------------------------------------

      FileInputStream fis = new FileInputStream(tempFile);
      dais = new DataInputStream(new BufferedInputStream(fis, bufferSize));

      long readStart = System.currentTimeMillis();

      valsLeft = NUM_INTS_WRITTEN;
      while (valsLeft-- > 0) {
        rec.read(new InputViewDataInputStreamWrapper(dais));
      }
      dais.close();
      dais = null;

      long readElapsed = System.currentTimeMillis() - readStart;

      LOG.info(
          "File-Stream with buffer "
              + bufferSize
              + ": write "
              + writeElapsed
              + " msecs, read "
              + readElapsed
              + " msecs.");
    } finally {
      // close if possible
      if (daos != null) {
        daos.close();
      }
      if (dais != null) {
        dais.close();
      }
      // try to delete the file
      if (tempFile != null) {
        tempFile.delete();
      }
    }
  }
  /**
   * 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 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);
  }
  @SuppressWarnings("unchecked")
  public long addRecord(T record) throws IOException {

    if (recordsOutFile == null) {

      if (closed) {
        throw new IllegalStateException("The large record handler has been closed.");
      }
      if (recordsReader != null) {
        throw new IllegalStateException("The handler has already switched to sorting.");
      }

      LOG.debug("Initializing the large record spilling...");

      // initialize the utilities
      {
        final TypeComparator<?>[] keyComps = comparator.getFlatComparators();
        numKeyFields = keyComps.length;
        Object[] keyHolder = new Object[numKeyFields];

        comparator.extractKeys(record, keyHolder, 0);

        TypeSerializer<?>[] keySers = new TypeSerializer<?>[numKeyFields];
        TypeSerializer<?>[] tupleSers = new TypeSerializer<?>[numKeyFields + 1];

        int[] keyPos = new int[numKeyFields];

        for (int i = 0; i < numKeyFields; i++) {
          keyPos[i] = i;
          keySers[i] = createSerializer(keyHolder[i], i);
          tupleSers[i] = keySers[i];
        }
        // add the long serializer for the offset
        tupleSers[numKeyFields] = LongSerializer.INSTANCE;

        keySerializer =
            new TupleSerializer<Tuple>(
                (Class<Tuple>) Tuple.getTupleClass(numKeyFields + 1), tupleSers);
        keyComparator = new TupleComparator<Tuple>(keyPos, keyComps, keySers);

        keySerializerFactory =
            new RuntimeSerializerFactory<Tuple>(keySerializer, keySerializer.getTupleClass());

        keyTuple = keySerializer.createInstance();
      }

      // initialize the spilling
      final int totalNumSegments = memory.size();
      final int segmentsForKeys =
          (totalNumSegments >= 2 * MAX_SEGMENTS_FOR_KEY_SPILLING)
              ? MAX_SEGMENTS_FOR_KEY_SPILLING
              : Math.max(
                  MIN_SEGMENTS_FOR_KEY_SPILLING, totalNumSegments - MAX_SEGMENTS_FOR_KEY_SPILLING);

      List<MemorySegment> recordsMemory = new ArrayList<MemorySegment>();
      List<MemorySegment> keysMemory = new ArrayList<MemorySegment>();

      for (int i = 0; i < segmentsForKeys; i++) {
        keysMemory.add(memory.get(i));
      }
      for (int i = segmentsForKeys; i < totalNumSegments; i++) {
        recordsMemory.add(memory.get(i));
      }

      recordsChannel = ioManager.createChannel();
      keysChannel = ioManager.createChannel();

      recordsOutFile =
          new FileChannelOutputView(
              ioManager.createBlockChannelWriter(recordsChannel),
              memManager,
              recordsMemory,
              memManager.getPageSize());

      keysOutFile =
          new FileChannelOutputView(
              ioManager.createBlockChannelWriter(keysChannel),
              memManager,
              keysMemory,
              memManager.getPageSize());
    }

    final long offset = recordsOutFile.getWriteOffset();
    if (offset < 0) {
      throw new RuntimeException("wrong offset");
    }

    Object[] keyHolder = new Object[numKeyFields];

    comparator.extractKeys(record, keyHolder, 0);
    for (int i = 0; i < numKeyFields; i++) {
      keyTuple.setField(keyHolder[i], i);
    }
    keyTuple.setField(offset, numKeyFields);

    keySerializer.serialize(keyTuple, keysOutFile);
    serializer.serialize(record, recordsOutFile);

    recordCounter++;

    return offset;
  }