Github user tejasapatil commented on a diff in the pull request:
https://github.com/apache/spark/pull/16909#discussion_r104950862
--- Diff:
sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala
---
@@ -0,0 +1,243 @@
+/*
+ * 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.memory.TaskMemoryManager
+import org.apache.spark.serializer.SerializerManager
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import
org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray.DefaultInitialSizeOfInMemoryBuffer
+import org.apache.spark.storage.BlockManager
+import org.apache.spark.util.collection.unsafe.sort.{UnsafeExternalSorter,
UnsafeSorterIterator}
+
+/**
+ * An append-only array for [[UnsafeRow]]s that spills content to disk
when there a predefined
+ * threshold of rows is reached.
+ *
+ * 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(
+ taskMemoryManager: TaskMemoryManager,
+ blockManager: BlockManager,
+ serializerManager: SerializerManager,
+ taskContext: TaskContext,
+ initialSize: Int,
+ pageSizeBytes: Long,
+ numRowsSpillThreshold: Int) extends Logging {
+
+ def this(numRowsSpillThreshold: Int) {
+ this(
+ TaskContext.get().taskMemoryManager(),
+ SparkEnv.get.blockManager,
+ SparkEnv.get.serializerManager,
+ TaskContext.get(),
+ 1024,
+ SparkEnv.get.memoryManager.pageSizeBytes,
+ numRowsSpillThreshold)
+ }
+
+ private val initialSizeOfInMemoryBuffer =
+ Math.min(DefaultInitialSizeOfInMemoryBuffer, numRowsSpillThreshold)
+
+ private val inMemoryBuffer = if (initialSizeOfInMemoryBuffer > 0) {
+ new ArrayBuffer[UnsafeRow](initialSizeOfInMemoryBuffer)
+ } else {
+ null
+ }
+
+ private var spillableArray: UnsafeExternalSorter = _
+ private var numRows = 0
+
+ // A counter to keep track of total modifications done to this array
since its creation.
+ // This helps to invalidate iterators when there are changes done to the
backing array.
+ private var modificationsCount: Long = 0
--- End diff --
> this is to detect modifications in the same thread right?
Yes
> When does this happen?
It can happen when a client generates an iterator but has not iterated the
whole array. Later, there are new entries being added to the array OR the array
is cleared. Any attempt to use the old iterator now can give inconsistent view
of the array and this counter helps with invalidation of the iterator.
In my opinion, none of the existing places where this array is used hits
this scenario. But having said that I added this to make this data structure
robust to the new usages would be protected against such usage (or even if
there are bugs introduced in the existing usages which would otherwise silently
work).
---
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]