Github user tejasapatil commented on a diff in the pull request:
https://github.com/apache/spark/pull/16909#discussion_r100821291
--- Diff:
sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala
---
@@ -0,0 +1,179 @@
+/*
+ * 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.util.ConcurrentModificationException
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{SparkEnv, TaskContext}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.util.collection.unsafe.sort.{UnsafeExternalSorter,
UnsafeSorterIterator}
+
+/**
+ * An append-only array for [[UnsafeRow]]s that spills content to disk
when there is insufficient
+ * space for it to grow.
+ *
+ * Setting spill threshold faces following trade-off:
+ *
+ * - If the spill threshold is too high, the in-memory array may occupy
more memory than is
+ * available, resulting in OOM.
+ * - If the spill threshold is too low, we spill frequently and incur
unnecessary disk writes.
+ * This may lead to a performance regression compared to the normal case
of using an
+ * [[ArrayBuffer]] or [[Array]].
+ */
+private[sql] class ExternalAppendOnlyUnsafeRowArray(numRowsSpillThreshold:
Int) extends Logging {
+ private val inMemoryBuffer: ArrayBuffer[UnsafeRow] =
ArrayBuffer.empty[UnsafeRow]
--- End diff --
My intention behind not using raw `Array` is to avoid holding that memory
(if we go this route, one would have to set the spill threshold to a relatively
lower value to avoid potential wastage of memory).
Before this PR:
- Initial size:
- `SortMergeJoin` started off with`ArrayBuffer` of default size (ie. 16)
- `WindowExec` started off with empty `ArrayBuffer`
- For both the cases, there was no shrinking of the array so memory is not
reclaimed until the operator finishes.
Proposed change:
- I am switching to `new ArrayBuffer(128)` for both cases in order to init
with decent size and not start with an empty array. Allocating space for 128
entries upfront is trivial memory footprint.
- Keeping the "no shrinking" behavior same. A part of me thinks I could do
something smarter by shrinking based on running average of actual lengths of
the array, but it might be over-optimization. I will first focus on getting the
basic stuff in.
---
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]