@Override
  public Option<MapStatus> stop(boolean success) {
    try {
      // Update task metrics from accumulators (null in UnsafeShuffleWriterSuite)
      Map<String, Accumulator<Object>> internalAccumulators =
          taskContext.internalMetricsToAccumulators();
      if (internalAccumulators != null) {
        internalAccumulators
            .apply(InternalAccumulator.PEAK_EXECUTION_MEMORY())
            .add(getPeakMemoryUsedBytes());
      }

      if (stopping) {
        return Option.apply(null);
      } else {
        stopping = true;
        if (success) {
          if (mapStatus == null) {
            throw new IllegalStateException("Cannot call stop(true) without having called write()");
          }
          return Option.apply(mapStatus);
        } else {
          // The map task failed, so delete our output data.
          shuffleBlockResolver.removeDataByMap(shuffleId, mapId);
          return Option.apply(null);
        }
      }
    } finally {
      if (sorter != null) {
        // If sorter is non-null, then this implies that we called stop() in response to an error,
        // so we need to clean up memory and spill files created by the sorter
        sorter.cleanupResources();
      }
    }
  }
 @Override
 public void write(scala.collection.Iterator<Product2<K, V>> records) throws IOException {
   // Keep track of success so we know if we encountered an exception
   // We do this rather than a standard try/catch/re-throw to handle
   // generic throwables.
   boolean success = false;
   try {
     while (records.hasNext()) {
       insertRecordIntoSorter(records.next());
     }
     closeAndWriteOutput();
     success = true;
   } finally {
     if (sorter != null) {
       try {
         sorter.cleanupResources();
       } catch (Exception e) {
         // Only throw this error if we won't be masking another
         // error.
         if (success) {
           throw e;
         } else {
           logger.error(
               "In addition to a failure during writing, we failed during " + "cleanup.", e);
         }
       }
     }
   }
 }
 private void updatePeakMemoryUsed() {
   // sorter can be null if this writer is closed
   if (sorter != null) {
     long mem = sorter.getPeakMemoryUsedBytes();
     if (mem > peakMemoryUsedBytes) {
       peakMemoryUsedBytes = mem;
     }
   }
 }
  @VisibleForTesting
  void insertRecordIntoSorter(Product2<K, V> record) throws IOException {
    assert (sorter != null);
    final K key = record._1();
    final int partitionId = partitioner.getPartition(key);
    serBuffer.reset();
    serOutputStream.writeKey(key, OBJECT_CLASS_TAG);
    serOutputStream.writeValue(record._2(), OBJECT_CLASS_TAG);
    serOutputStream.flush();

    final int serializedRecordSize = serBuffer.size();
    assert (serializedRecordSize > 0);

    sorter.insertRecord(
        serBuffer.getBuf(), Platform.BYTE_ARRAY_OFFSET, serializedRecordSize, partitionId);
  }
 @VisibleForTesting
 void closeAndWriteOutput() throws IOException {
   assert (sorter != null);
   updatePeakMemoryUsed();
   serBuffer = null;
   serOutputStream = null;
   final SpillInfo[] spills = sorter.closeAndGetSpills();
   sorter = null;
   final long[] partitionLengths;
   try {
     partitionLengths = mergeSpills(spills);
   } finally {
     for (SpillInfo spill : spills) {
       if (spill.file.exists() && !spill.file.delete()) {
         logger.error("Error while deleting spill file {}", spill.file.getPath());
       }
     }
   }
   shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths);
   mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths);
 }
 @VisibleForTesting
 void forceSorterToSpill() throws IOException {
   assert (sorter != null);
   sorter.spill();
 }