Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
59 commits
Select commit Hold shift + click to select a range
480a74a
Initial import of code from Databricks unsafe utils repo.
JoshRosen Apr 17, 2015
ab68e08
Begin merging the UTF8String implementations.
JoshRosen Apr 18, 2015
f03e9c1
Play around with Unsafe implementations of more string methods.
JoshRosen Apr 18, 2015
5d55cef
Add skeleton for Row implementation.
JoshRosen Apr 18, 2015
8a8f9df
Add skeleton for GeneratedAggregate integration.
JoshRosen Apr 18, 2015
1ff814d
Add reminder to free memory on iterator completion
JoshRosen Apr 18, 2015
53ba9b7
Start prototyping Java Row -> UnsafeRow converters
JoshRosen Apr 19, 2015
fc4c3a8
Sketch how the converters will be used in UnsafeGeneratedAggregate
JoshRosen Apr 19, 2015
1a483c5
First version that passes some aggregation tests:
JoshRosen Apr 19, 2015
079f1bf
Some clarification of the BytesToBytesMap.lookup() / set() contract.
JoshRosen Apr 19, 2015
f764d13
Simplify address + length calculation in Location.
JoshRosen Apr 19, 2015
c754ae1
Now that the store*() contract has been stregthened, we can remove an…
JoshRosen Apr 19, 2015
ae39694
Add finalizer as "cleanup method of last resort"
JoshRosen Apr 19, 2015
c7f0b56
Reuse UnsafeRow pointer in UnsafeRowConverter
JoshRosen Apr 20, 2015
62ab054
Optimize for fact that get() is only called on String columns.
JoshRosen Apr 20, 2015
c55bf66
Free buffer once iterator has been fully consumed.
JoshRosen Apr 20, 2015
738fa33
Add feature flag to guard UnsafeGeneratedAggregate
JoshRosen Apr 20, 2015
c1b3813
Fix bug in UnsafeMemoryAllocator.free():
JoshRosen Apr 20, 2015
7df6008
Optimizations related to zeroing out memory:
JoshRosen Apr 21, 2015
58ac393
Use UNSAFE allocator in GeneratedAggregate (TODO: make this configura…
JoshRosen Apr 21, 2015
d2bb986
Update to implement new Row methods added upstream
JoshRosen Apr 22, 2015
b3eaccd
Extract aggregation map into its own class.
JoshRosen Apr 22, 2015
bade966
Comment update (bumping to refresh GitHub cache...)
JoshRosen Apr 22, 2015
d85eeff
Add basic sanity test for UnsafeFixedWidthAggregationMap
JoshRosen Apr 22, 2015
1f4b716
Merge Unsafe code into the regular GeneratedAggregate, guarded by a
JoshRosen Apr 22, 2015
92d5a06
Address a number of minor code review comments.
JoshRosen Apr 23, 2015
628f936
Use ints intead of longs for indexing.
JoshRosen Apr 23, 2015
23a440a
Bump up default hash map size
JoshRosen Apr 23, 2015
765243d
Enable optional performance metrics for hash map.
JoshRosen Apr 23, 2015
b26f1d3
Fix bug in murmur hash implementation.
JoshRosen Apr 23, 2015
49aed30
More long -> int conversion.
JoshRosen Apr 23, 2015
29a7575
Remove debug logging
JoshRosen Apr 24, 2015
ef6b3d3
Fix a bunch of FindBugs and IntelliJ inspections
JoshRosen Apr 24, 2015
06e929d
More warning cleanup
JoshRosen Apr 24, 2015
854201a
Import and comment cleanup
JoshRosen Apr 24, 2015
f3dcbfe
More mod replacement
JoshRosen Apr 24, 2015
afe8dca
Some Javadoc cleanup
JoshRosen Apr 24, 2015
a95291e
Cleanups to string handling code
JoshRosen Apr 24, 2015
31eaabc
Lots of TODO and doc cleanup.
JoshRosen Apr 24, 2015
6ffdaa1
Null handling improvements in UnsafeRow.
JoshRosen Apr 24, 2015
9c19fc0
Add configuration options for heap vs. offheap
JoshRosen Apr 24, 2015
cde4132
Add missing pom.xml
JoshRosen Apr 26, 2015
0925847
Disable MiMa checks for new unsafe module
JoshRosen Apr 27, 2015
a8e4a3f
Introduce MemoryManager interface; add to SparkEnv.
JoshRosen Apr 28, 2015
b45f070
Don't redundantly store the offset from key to value, since we can co…
JoshRosen Apr 28, 2015
162caf7
Fix test compilation
JoshRosen Apr 28, 2015
3ca84b2
Only zero the used portion of groupingKeyConversionScratchSpace
JoshRosen Apr 28, 2015
529e571
Measure timeSpentResizing in nanoseconds instead of milliseconds.
JoshRosen Apr 28, 2015
ce3c565
More comments, formatting, and code cleanup.
JoshRosen Apr 28, 2015
78a5b84
Add logging to MemoryManager
JoshRosen Apr 28, 2015
a19e066
Rename unsafe Java test suites to match Scala test naming convention.
JoshRosen Apr 28, 2015
de5e001
Fix debug vs. trace in logging message.
JoshRosen Apr 28, 2015
6e4b192
Remove an unused method from ByteArrayMethods.
JoshRosen Apr 28, 2015
70a39e4
Split MemoryManager into ExecutorMemoryManager and TaskMemoryManager:
JoshRosen Apr 28, 2015
50e9671
Throw memory leak warning even in case of error; add warning about co…
JoshRosen Apr 29, 2015
017b2dc
Remove BytesToBytesMap.finalize()
JoshRosen Apr 29, 2015
1bc36cc
Refactor UnsafeRowConverter to avoid unnecessary boxing.
JoshRosen Apr 29, 2015
81f34f8
Follow 'place children last' convention for GeneratedAggregate
JoshRosen Apr 29, 2015
eeee512
Add converters for Null, Boolean, Byte, and Short columns.
JoshRosen Apr 29, 2015
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Introduce MemoryManager interface; add to SparkEnv.
The configuration of HEAP vs UNSAFE is now done at the Spark core level.

The translation of encoded 64-bit addresses into base object + offset pairs
is now handled by MemoryManager, allowing this pointers to be safely passed
between operators that exchange data pages.
  • Loading branch information
JoshRosen committed Apr 28, 2015
commit a8e4a3fe40574c3a609beeb4794b11bd720a31e7
12 changes: 12 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinato
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager}
import org.apache.spark.storage._
import org.apache.spark.unsafe.memory.{MemoryManager => UnsafeMemoryManager, MemoryAllocator}
import org.apache.spark.util.{RpcUtils, Utils}

/**
Expand Down Expand Up @@ -69,6 +70,7 @@ class SparkEnv (
val sparkFilesDir: String,
val metricsSystem: MetricsSystem,
val shuffleMemoryManager: ShuffleMemoryManager,
val unsafeMemoryManager: UnsafeMemoryManager,
val outputCommitCoordinator: OutputCommitCoordinator,
val conf: SparkConf) extends Logging {

Expand Down Expand Up @@ -382,6 +384,15 @@ object SparkEnv extends Logging {
new OutputCommitCoordinatorEndpoint(rpcEnv, outputCommitCoordinator))
outputCommitCoordinator.coordinatorRef = Some(outputCommitCoordinatorRef)

val unsafeMemoryManager: UnsafeMemoryManager = {
val allocator = if (conf.getBoolean("spark.unsafe.offHeap", false)) {
MemoryAllocator.UNSAFE
} else {
MemoryAllocator.HEAP
}
new UnsafeMemoryManager(allocator)
}

val envInstance = new SparkEnv(
executorId,
rpcEnv,
Expand All @@ -398,6 +409,7 @@ object SparkEnv extends Logging {
sparkFilesDir,
metricsSystem,
shuffleMemoryManager,
unsafeMemoryManager,
outputCommitCoordinator,
conf)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@
import org.apache.spark.sql.types.StructType;
import org.apache.spark.unsafe.PlatformDependent;
import org.apache.spark.unsafe.map.BytesToBytesMap;
import org.apache.spark.unsafe.memory.MemoryAllocator;
import org.apache.spark.unsafe.memory.MemoryLocation;
import org.apache.spark.unsafe.memory.MemoryManager;

/**
* Unsafe-based HashMap for performing aggregations where the aggregated values are fixed-width.
Expand Down Expand Up @@ -102,23 +102,23 @@ public static boolean supportsAggregationBufferSchema(StructType schema) {
* @param emptyAggregationBuffer the default value for new keys (a "zero" of the agg. function)
* @param aggregationBufferSchema the schema of the aggregation buffer, used for row conversion.
* @param groupingKeySchema the schema of the grouping key, used for row conversion.
* @param allocator the memory allocator used to allocate our Unsafe memory structures.
* @param groupingKeySchema the memory manager used to allocate our Unsafe memory structures.
* @param initialCapacity the initial capacity of the map (a sizing hint to avoid re-hashing).
* @param enablePerfMetrics if true, performance metrics will be recorded (has minor perf impact)
*/
public UnsafeFixedWidthAggregationMap(
Row emptyAggregationBuffer,
StructType aggregationBufferSchema,
StructType groupingKeySchema,
MemoryAllocator allocator,
MemoryManager memoryManager,
int initialCapacity,
boolean enablePerfMetrics) {
this.emptyAggregationBuffer =
convertToUnsafeRow(emptyAggregationBuffer, aggregationBufferSchema);
this.aggregationBufferSchema = aggregationBufferSchema;
this.groupingKeyToUnsafeRowConverter = new UnsafeRowConverter(groupingKeySchema);
this.groupingKeySchema = groupingKeySchema;
this.map = new BytesToBytesMap(allocator, initialCapacity, enablePerfMetrics);
this.map = new BytesToBytesMap(memoryManager, initialCapacity, enablePerfMetrics);
this.enablePerfMetrics = enablePerfMetrics;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,19 +17,32 @@

package org.apache.spark.sql.catalyst.expressions

import org.apache.spark.unsafe.memory.MemoryAllocator
import org.scalatest.{FunSuite, Matchers}
import org.apache.spark.unsafe.memory.{MemoryManager, MemoryAllocator}
import org.scalatest.{BeforeAndAfterEach, FunSuite, Matchers}

import org.apache.spark.sql.types._

class UnsafeFixedWidthAggregationMapSuite extends FunSuite with Matchers {
class UnsafeFixedWidthAggregationMapSuite extends FunSuite with Matchers with BeforeAndAfterEach {

import UnsafeFixedWidthAggregationMap._

private val groupKeySchema = StructType(StructField("product", StringType) :: Nil)
private val aggBufferSchema = StructType(StructField("salePrice", IntegerType) :: Nil)
private def emptyAggregationBuffer: Row = new GenericRow(Array[Any](0))

private var memoryManager: MemoryManager = null

override def beforeEach(): Unit = {
memoryManager = new MemoryManager(true)
}

override def afterEach(): Unit = {
if (memoryManager != null) {
memoryManager.cleanUpAllPages()
memoryManager = null
}
}

test("supported schemas") {
assert(!supportsAggregationBufferSchema(StructType(StructField("x", StringType) :: Nil)))
assert(supportsGroupKeySchema(StructType(StructField("x", StringType) :: Nil)))
Expand All @@ -45,7 +58,7 @@ class UnsafeFixedWidthAggregationMapSuite extends FunSuite with Matchers {
emptyAggregationBuffer,
aggBufferSchema,
groupKeySchema,
MemoryAllocator.HEAP,
memoryManager,
1024,
false
)
Expand All @@ -58,7 +71,7 @@ class UnsafeFixedWidthAggregationMapSuite extends FunSuite with Matchers {
emptyAggregationBuffer,
aggBufferSchema,
groupKeySchema,
MemoryAllocator.HEAP,
memoryManager,
1024,
false
)
Expand Down
8 changes: 0 additions & 8 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@ private[spark] object SQLConf {
val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions"
val CODEGEN_ENABLED = "spark.sql.codegen"
val UNSAFE_ENABLED = "spark.sql.unsafe.enabled"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

unless we think there are going to be other options under unsafe, we might consider getting rid of enabled to match codegen

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There was an option for choosing heap vs. offheap, but that option got moved into Spark core. I think we use the enabled convention a bit more in core. I don't have strong feelings on this one.

val UNSAFE_USE_OFF_HEAP = "spark.sql.unsafe.offHeap"
val DIALECT = "spark.sql.dialect"

val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString"
Expand Down Expand Up @@ -159,13 +158,6 @@ private[sql] class SQLConf extends Serializable {
*/
private[spark] def unsafeEnabled: Boolean = getConf(UNSAFE_ENABLED, "false").toBoolean

/**
* When set to true, Spark SQL will use off-heap memory allocation for managed memory operations.
*
* Defaults to false.
*/
private[spark] def unsafeUseOffHeap: Boolean = getConf(UNSAFE_USE_OFF_HEAP, "false").toBoolean

private[spark] def useSqlSerializer2: Boolean = getConf(USE_SQL_SERIALIZER2, "true").toBoolean

/**
Expand Down
2 changes: 0 additions & 2 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1013,8 +1013,6 @@ class SQLContext(@transient val sparkContext: SparkContext)

def unsafeEnabled: Boolean = self.conf.unsafeEnabled

def unsafeUseOffHeap: Boolean = self.conf.unsafeUseOffHeap

def numPartitions: Int = self.conf.numShufflePartitions

def strategies: Seq[Strategy] =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.sql.execution

import org.apache.spark.SparkEnv
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.trees._
Expand All @@ -43,16 +44,14 @@ case class AggregateEvaluation(
* @param aggregateExpressions expressions that are computed for each group.
* @param child the input data source.
* @param unsafeEnabled whether to allow Unsafe-based aggregation buffers to be used.
* @param useOffHeap whether to use off-heap allocation (only takes effect if unsafeEnabled=true)
*/
@DeveloperApi
case class GeneratedAggregate(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should consider breaking this into two operators at some point so the explain is more readable.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed. I'd like to avoid duplication of the code generation logic, though.

partial: Boolean,
groupingExpressions: Seq[Expression],
aggregateExpressions: Seq[NamedExpression],
child: SparkPlan,
unsafeEnabled: Boolean,
useOffHeap: Boolean)
unsafeEnabled: Boolean)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: typically place children last (although we aren't great about following this convention)...

extends UnaryNode {

override def requiredChildDistribution: Seq[Distribution] =
Expand Down Expand Up @@ -291,7 +290,7 @@ case class GeneratedAggregate(
newAggregationBuffer(EmptyRow),
aggregationBufferSchema,
groupKeySchema,
if (useOffHeap) MemoryAllocator.UNSAFE else MemoryAllocator.HEAP,
SparkEnv.get.unsafeMemoryManager,
1024 * 16,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

use named argument for this and the next line

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

AFAIK we can't use the Scala named argument syntax because UnsafeFixedWidthAggregationMap is implemented in Java, but I can add a line comment explaining this. Also, I wonder if there's a better choice of default map size here. If we had statistics / estimates from the optimizer, we could make a better choice here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah ok nvm. then let's add a comment like

1024 * 16,  // initial capacity
...

false
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,10 +141,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
groupingExpressions,
partialComputation,
planLater(child),
unsafeEnabled,
unsafeUseOffHeap),
unsafeEnabled,
unsafeUseOffHeap) :: Nil
unsafeEnabled),
unsafeEnabled) :: Nil
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

indent off here


// Cases where some aggregate can not be codegened
case PartialAggregation(
Expand Down
Loading