Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/6444#discussion_r34100165
  
    --- Diff: 
core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
 ---
    @@ -0,0 +1,281 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.collection.unsafe.sort;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.SparkConf;
    +import org.apache.spark.TaskContext;
    +import org.apache.spark.executor.ShuffleWriteMetrics;
    +import org.apache.spark.shuffle.ShuffleMemoryManager;
    +import org.apache.spark.storage.BlockManager;
    +import org.apache.spark.unsafe.PlatformDependent;
    +import org.apache.spark.unsafe.memory.MemoryBlock;
    +import org.apache.spark.unsafe.memory.TaskMemoryManager;
    +import org.apache.spark.util.Utils;
    +
    +/**
    + * External sorter based on {@link UnsafeInMemorySorter}.
    + */
    +public final class UnsafeExternalSorter {
    +
    +  private final Logger logger = 
LoggerFactory.getLogger(UnsafeExternalSorter.class);
    +
    +  private static final int PAGE_SIZE = 1 << 27;  // 128 megabytes
    +  @VisibleForTesting
    +  static final int MAX_RECORD_SIZE = PAGE_SIZE - 4;
    +
    +  private final PrefixComparator prefixComparator;
    +  private final RecordComparator recordComparator;
    +  private final int initialSize;
    +  private final TaskMemoryManager memoryManager;
    +  private final ShuffleMemoryManager shuffleMemoryManager;
    +  private final BlockManager blockManager;
    +  private final TaskContext taskContext;
    +  private ShuffleWriteMetrics writeMetrics;
    +
    +  /** The buffer size to use when writing spills using 
DiskBlockObjectWriter */
    +  private final int fileBufferSizeBytes;
    +
    +  /**
    +   * Memory pages that hold the records being sorted. The pages in this 
list are freed when
    +   * spilling, although in principle we could recycle these pages across 
spills (on the other hand,
    +   * this might not be necessary if we maintained a pool of re-usable 
pages in the TaskMemoryManager
    +   * itself).
    +   */
    +  private final LinkedList<MemoryBlock> allocatedPages = new 
LinkedList<MemoryBlock>();
    +
    +  // These variables are reset after spilling:
    +  private UnsafeInMemorySorter sorter;
    +  private MemoryBlock currentPage = null;
    +  private long currentPagePosition = -1;
    +  private long freeSpaceInCurrentPage = 0;
    +
    +  private final LinkedList<UnsafeSorterSpillWriter> spillWriters = new 
LinkedList<>();
    +
    +  public UnsafeExternalSorter(
    +      TaskMemoryManager memoryManager,
    +      ShuffleMemoryManager shuffleMemoryManager,
    +      BlockManager blockManager,
    +      TaskContext taskContext,
    +      RecordComparator recordComparator,
    +      PrefixComparator prefixComparator,
    +      int initialSize,
    +      SparkConf conf) throws IOException {
    +    this.memoryManager = memoryManager;
    +    this.shuffleMemoryManager = shuffleMemoryManager;
    +    this.blockManager = blockManager;
    +    this.taskContext = taskContext;
    +    this.recordComparator = recordComparator;
    +    this.prefixComparator = prefixComparator;
    +    this.initialSize = initialSize;
    +    // Use getSizeAsKb (not bytes) to maintain backwards compatibility for 
units
    +    this.fileBufferSizeBytes = (int) 
conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024;
    +    initializeForWriting();
    +  }
    +
    +  // TODO: metrics tracking + integration with shuffle write metrics
    +  // need to connect the write metrics to task metrics so we count the 
spill IO somewhere.
    +
    +  /**
    +   * Allocates new sort data structures. Called when creating the sorter 
and after each spill.
    +   */
    +  private void initializeForWriting() throws IOException {
    +    this.writeMetrics = new ShuffleWriteMetrics();
    +    // TODO: move this sizing calculation logic into a static method of 
sorter:
    +    final long memoryRequested = initialSize * 8L * 2;
    +    final long memoryAcquired = 
shuffleMemoryManager.tryToAcquire(memoryRequested);
    +    if (memoryAcquired != memoryRequested) {
    +      shuffleMemoryManager.release(memoryAcquired);
    +      throw new IOException("Could not acquire " + memoryRequested + " 
bytes of memory");
    +    }
    +
    +    this.sorter =
    +      new UnsafeInMemorySorter(memoryManager, recordComparator, 
prefixComparator, initialSize);
    +  }
    +
    +  /**
    +   * Sort and spill the current records in response to memory pressure.
    +   */
    +  @VisibleForTesting
    +  public void spill() throws IOException {
    +    logger.info("Thread {} spilling sort data of {} to disk ({} {} so 
far)",
    +      Thread.currentThread().getId(),
    +      Utils.bytesToString(getMemoryUsage()),
    +      spillWriters.size(),
    +      spillWriters.size() > 1 ? " times" : " time");
    +
    +    final UnsafeSorterSpillWriter spillWriter =
    +      new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, 
writeMetrics,
    +        sorter.numRecords());
    +    spillWriters.add(spillWriter);
    +    final UnsafeSorterIterator sortedRecords = sorter.getSortedIterator();
    +    while (sortedRecords.hasNext()) {
    +      sortedRecords.loadNext();
    +      final Object baseObject = sortedRecords.getBaseObject();
    +      final long baseOffset = sortedRecords.getBaseOffset();
    +      final int recordLength = sortedRecords.getRecordLength();
    +      spillWriter.write(baseObject, baseOffset, recordLength, 
sortedRecords.getKeyPrefix());
    +    }
    +    spillWriter.close();
    +    final long sorterMemoryUsage = sorter.getMemoryUsage();
    +    sorter = null;
    +    shuffleMemoryManager.release(sorterMemoryUsage);
    +    final long spillSize = freeMemory();
    +    taskContext.taskMetrics().incMemoryBytesSpilled(spillSize);
    +    initializeForWriting();
    +  }
    +
    +  private long getMemoryUsage() {
    +    return sorter.getMemoryUsage() + (allocatedPages.size() * (long) 
PAGE_SIZE);
    +  }
    +
    +  public long freeMemory() {
    --- End diff --
    
    Where do we call this if any exception happens during sorting?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to