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

    https://github.com/apache/spark/pull/6444#discussion_r34105095
  
    --- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
 ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.sql.execution;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +
    +import scala.collection.Iterator;
    +import scala.math.Ordering;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +
    +import org.apache.spark.SparkEnv;
    +import org.apache.spark.TaskContext;
    +import org.apache.spark.sql.AbstractScalaRowIterator;
    +import org.apache.spark.sql.catalyst.InternalRow;
    +import org.apache.spark.sql.catalyst.expressions.ObjectUnsafeColumnWriter;
    +import org.apache.spark.sql.catalyst.expressions.UnsafeColumnWriter;
    +import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
    +import org.apache.spark.sql.catalyst.expressions.UnsafeRowConverter;
    +import org.apache.spark.sql.catalyst.util.ObjectPool;
    +import org.apache.spark.sql.types.StructField;
    +import org.apache.spark.sql.types.StructType;
    +import org.apache.spark.unsafe.PlatformDependent;
    +import org.apache.spark.util.collection.unsafe.sort.PrefixComparator;
    +import org.apache.spark.util.collection.unsafe.sort.RecordComparator;
    +import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter;
    +import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterIterator;
    +
    +final class UnsafeExternalRowSorter {
    +
    +  /**
    +   * If positive, forces records to be spilled to disk at the given 
frequency (measured in numbers
    +   * of records). This is only intended to be used in tests.
    +   */
    +  private int testSpillFrequency = 0;
    +
    +  private long numRowsInserted = 0;
    +
    +  private final StructType schema;
    +  private final UnsafeRowConverter rowConverter;
    +  private final PrefixComputer prefixComputer;
    +  private final ObjectPool objPool = new ObjectPool(128);
    +  private final UnsafeExternalSorter sorter;
    +  private byte[] rowConversionBuffer = new byte[1024 * 8];
    +
    +  public static abstract class PrefixComputer {
    +    abstract long computePrefix(InternalRow row);
    +  }
    +
    +  public UnsafeExternalRowSorter(
    +      StructType schema,
    +      Ordering<InternalRow> ordering,
    +      PrefixComparator prefixComparator,
    +      PrefixComputer prefixComputer) throws IOException {
    +    this.schema = schema;
    +    this.rowConverter = new UnsafeRowConverter(schema);
    +    this.prefixComputer = prefixComputer;
    +    final SparkEnv sparkEnv = SparkEnv.get();
    +    final TaskContext taskContext = TaskContext.get();
    +    sorter = new UnsafeExternalSorter(
    +      taskContext.taskMemoryManager(),
    +      sparkEnv.shuffleMemoryManager(),
    +      sparkEnv.blockManager(),
    +      taskContext,
    +      new RowComparator(ordering, schema.length(), objPool),
    +      prefixComparator,
    +      4096,
    +      sparkEnv.conf()
    +    );
    +  }
    +
    +  /**
    +   * Forces spills to occur every `frequency` records. Only for use in 
tests.
    +   */
    +  @VisibleForTesting
    +  void setTestSpillFrequency(int frequency) {
    +    assert frequency > 0 : "Frequency must be positive";
    +    testSpillFrequency = frequency;
    +  }
    +
    +  @VisibleForTesting
    +  void insertRow(InternalRow row) throws IOException {
    +    final int sizeRequirement = rowConverter.getSizeRequirement(row);
    +    if (sizeRequirement > rowConversionBuffer.length) {
    +      rowConversionBuffer = new byte[sizeRequirement];
    +    } else {
    +      // Zero out the buffer that's used to hold the current row. This is 
necessary in order
    +      // to ensure that rows hash properly, since garbage data from the 
previous row could
    +      // otherwise end up as padding in this row. As a performance 
optimization, we only zero
    +      // out the portion of the buffer that we'll actually write to.
    +      Arrays.fill(rowConversionBuffer, 0, sizeRequirement, (byte) 0);
    +    }
    +    final int bytesWritten = rowConverter.writeRow(
    +      row, rowConversionBuffer, PlatformDependent.BYTE_ARRAY_OFFSET, 
objPool);
    +    assert (bytesWritten == sizeRequirement);
    +    final long prefix = prefixComputer.computePrefix(row);
    +    sorter.insertRecord(
    +      rowConversionBuffer,
    +      PlatformDependent.BYTE_ARRAY_OFFSET,
    +      sizeRequirement,
    +      prefix
    +    );
    +    numRowsInserted++;
    +    if (testSpillFrequency > 0 && (numRowsInserted % testSpillFrequency) 
== 0) {
    +      spill();
    +    }
    +  }
    +
    +  @VisibleForTesting
    +  void spill() throws IOException {
    +    sorter.spill();
    +  }
    +
    +  private void cleanupResources() {
    +    sorter.freeMemory();
    +  }
    +
    +  @VisibleForTesting
    +  Iterator<InternalRow> sort() throws IOException {
    +    try {
    +      final UnsafeSorterIterator sortedIterator = 
sorter.getSortedIterator();
    +      if (!sortedIterator.hasNext()) {
    +        // Since we won't ever call next() on an empty iterator, we need 
to clean up resources
    +        // here in order to prevent memory leaks.
    +        cleanupResources();
    +      }
    +      return new AbstractScalaRowIterator() {
    +
    +        private final int numFields = schema.length();
    +        private final UnsafeRow row = new UnsafeRow();
    +
    +        @Override
    +        public boolean hasNext() {
    +          return sortedIterator.hasNext();
    +        }
    +
    +        @Override
    +        public InternalRow next() {
    +          try {
    +            sortedIterator.loadNext();
    +            if (hasNext()) {
    +              row.pointTo(
    +                sortedIterator.getBaseObject(), 
sortedIterator.getBaseOffset(), numFields, objPool);
    +              return row;
    +            } else {
    +              final byte[] rowDataCopy = new 
byte[sortedIterator.getRecordLength()];
    +              PlatformDependent.copyMemory(
    +                sortedIterator.getBaseObject(),
    +                sortedIterator.getBaseOffset(),
    +                rowDataCopy,
    +                PlatformDependent.BYTE_ARRAY_OFFSET,
    +                sortedIterator.getRecordLength()
    +              );
    +              row.backingArray = rowDataCopy;
    --- End diff --
    
    Do we really need this? row will hold rowDataCopy as `baseObject` anyway.


---
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