|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.execution; |
| 19 | + |
| 20 | +import java.io.IOException; |
| 21 | +import java.util.Arrays; |
| 22 | + |
| 23 | +import scala.Function1; |
| 24 | +import scala.collection.AbstractIterator; |
| 25 | +import scala.collection.Iterator; |
| 26 | +import scala.math.Ordering; |
| 27 | + |
| 28 | +import org.apache.spark.SparkEnv; |
| 29 | +import org.apache.spark.TaskContext; |
| 30 | +import org.apache.spark.sql.Row; |
| 31 | +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; |
| 32 | +import org.apache.spark.sql.catalyst.expressions.UnsafeRowConverter; |
| 33 | +import org.apache.spark.sql.types.StructType; |
| 34 | +import org.apache.spark.unsafe.PlatformDependent; |
| 35 | +import org.apache.spark.util.collection.unsafe.sort.PrefixComparator; |
| 36 | +import org.apache.spark.util.collection.unsafe.sort.RecordComparator; |
| 37 | +import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter; |
| 38 | +import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterIterator; |
| 39 | + |
| 40 | +final class UnsafeExternalRowSorter { |
| 41 | + |
| 42 | + private final StructType schema; |
| 43 | + private final UnsafeRowConverter rowConverter; |
| 44 | + private final RowComparator rowComparator; |
| 45 | + private final PrefixComparator prefixComparator; |
| 46 | + private final Function1<Row, Long> prefixComputer; |
| 47 | + |
| 48 | + public UnsafeExternalRowSorter( |
| 49 | + StructType schema, |
| 50 | + Ordering<Row> ordering, |
| 51 | + PrefixComparator prefixComparator, |
| 52 | + // TODO: if possible, avoid this boxing of the return value |
| 53 | + Function1<Row, Long> prefixComputer) { |
| 54 | + this.schema = schema; |
| 55 | + this.rowConverter = new UnsafeRowConverter(schema); |
| 56 | + this.rowComparator = new RowComparator(ordering, schema); |
| 57 | + this.prefixComparator = prefixComparator; |
| 58 | + this.prefixComputer = prefixComputer; |
| 59 | + } |
| 60 | + |
| 61 | + public Iterator<Row> sort(Iterator<Row> inputIterator) throws IOException { |
| 62 | + final SparkEnv sparkEnv = SparkEnv.get(); |
| 63 | + final TaskContext taskContext = TaskContext.get(); |
| 64 | + byte[] rowConversionBuffer = new byte[1024 * 8]; |
| 65 | + final UnsafeExternalSorter sorter = new UnsafeExternalSorter( |
| 66 | + taskContext.taskMemoryManager(), |
| 67 | + sparkEnv.shuffleMemoryManager(), |
| 68 | + sparkEnv.blockManager(), |
| 69 | + taskContext, |
| 70 | + rowComparator, |
| 71 | + prefixComparator, |
| 72 | + 4096, |
| 73 | + sparkEnv.conf() |
| 74 | + ); |
| 75 | + try { |
| 76 | + while (inputIterator.hasNext()) { |
| 77 | + final Row row = inputIterator.next(); |
| 78 | + final int sizeRequirement = rowConverter.getSizeRequirement(row); |
| 79 | + if (sizeRequirement > rowConversionBuffer.length) { |
| 80 | + rowConversionBuffer = new byte[sizeRequirement]; |
| 81 | + } else { |
| 82 | + // Zero out the buffer that's used to hold the current row. This is necessary in order |
| 83 | + // to ensure that rows hash properly, since garbage data from the previous row could |
| 84 | + // otherwise end up as padding in this row. As a performance optimization, we only zero |
| 85 | + // out the portion of the buffer that we'll actually write to. |
| 86 | + Arrays.fill(rowConversionBuffer, 0, sizeRequirement, (byte) 0); |
| 87 | + } |
| 88 | + final int bytesWritten = |
| 89 | + rowConverter.writeRow(row, rowConversionBuffer, PlatformDependent.BYTE_ARRAY_OFFSET); |
| 90 | + assert (bytesWritten == sizeRequirement); |
| 91 | + final long prefix = prefixComputer.apply(row); |
| 92 | + sorter.insertRecord( |
| 93 | + rowConversionBuffer, |
| 94 | + PlatformDependent.BYTE_ARRAY_OFFSET, |
| 95 | + sizeRequirement, |
| 96 | + prefix |
| 97 | + ); |
| 98 | + } |
| 99 | + final UnsafeSorterIterator sortedIterator = sorter.getSortedIterator(); |
| 100 | + return new AbstractIterator<Row>() { |
| 101 | + |
| 102 | + private final int numFields = schema.length(); |
| 103 | + private final UnsafeRow row = new UnsafeRow(); |
| 104 | + |
| 105 | + @Override |
| 106 | + public boolean hasNext() { |
| 107 | + return sortedIterator.hasNext(); |
| 108 | + } |
| 109 | + |
| 110 | + @Override |
| 111 | + public Row next() { |
| 112 | + try { |
| 113 | + sortedIterator.loadNext(); |
| 114 | + if (hasNext()) { |
| 115 | + row.pointTo( |
| 116 | + sortedIterator.getBaseObject(), sortedIterator.getBaseOffset(), numFields, schema); |
| 117 | + return row; |
| 118 | + } else { |
| 119 | + final byte[] rowDataCopy = new byte[sortedIterator.getRecordLength()]; |
| 120 | + PlatformDependent.copyMemory( |
| 121 | + sortedIterator.getBaseObject(), |
| 122 | + sortedIterator.getBaseOffset(), |
| 123 | + rowDataCopy, |
| 124 | + PlatformDependent.BYTE_ARRAY_OFFSET, |
| 125 | + sortedIterator.getRecordLength() |
| 126 | + ); |
| 127 | + row.backingArray = rowDataCopy; |
| 128 | + row.pointTo(rowDataCopy, PlatformDependent.BYTE_ARRAY_OFFSET, numFields, schema); |
| 129 | + sorter.freeMemory(); |
| 130 | + return row; |
| 131 | + } |
| 132 | + } catch (IOException e) { |
| 133 | + // TODO: we need to ensure that files are cleaned properly after an exception, |
| 134 | + // so we need better cleanup methods than freeMemory(). |
| 135 | + sorter.freeMemory(); |
| 136 | + // Scala iterators don't declare any checked exceptions, so we need to use this hack |
| 137 | + // to re-throw the exception: |
| 138 | + PlatformDependent.throwException(e); |
| 139 | + } |
| 140 | + throw new RuntimeException("Exception should have been re-thrown in next()"); |
| 141 | + }; |
| 142 | + }; |
| 143 | + } catch (IOException e) { |
| 144 | + // TODO: we need to ensure that files are cleaned properly after an exception, |
| 145 | + // so we need better cleanup methods than freeMemory(). |
| 146 | + sorter.freeMemory(); |
| 147 | + throw e; |
| 148 | + } |
| 149 | + } |
| 150 | + |
| 151 | + private static final class RowComparator extends RecordComparator { |
| 152 | + private final StructType schema; |
| 153 | + private final Ordering<Row> ordering; |
| 154 | + private final int numFields; |
| 155 | + private final UnsafeRow row1 = new UnsafeRow(); |
| 156 | + private final UnsafeRow row2 = new UnsafeRow(); |
| 157 | + |
| 158 | + public RowComparator(Ordering<Row> ordering, StructType schema) { |
| 159 | + this.schema = schema; |
| 160 | + this.numFields = schema.length(); |
| 161 | + this.ordering = ordering; |
| 162 | + } |
| 163 | + |
| 164 | + @Override |
| 165 | + public int compare(Object baseObj1, long baseOff1, Object baseObj2, long baseOff2) { |
| 166 | + row1.pointTo(baseObj1, baseOff1, numFields, schema); |
| 167 | + row2.pointTo(baseObj2, baseOff2, numFields, schema); |
| 168 | + return ordering.compare(row1, row2); |
| 169 | + } |
| 170 | + } |
| 171 | +} |
0 commit comments