From 1516833233fb054b1897ba40ca10afd7698c2314 Mon Sep 17 00:00:00 2001 From: Yan Ma Date: Tue, 25 May 2021 11:24:54 +0800 Subject: [PATCH] [PMEM-SPILL-34][POAE7-1119]Port RDD cache to Spark 3.1.1 as separate module (#41) --- RDD-Cache/pom.xml | 145 ++ .../org/apache/spark/memory/MemoryMode.java | 0 .../apache/spark/memory/MemoryManager.scala | 74 +- .../spark/memory/StorageMemoryPool.scala | 0 .../spark/memory/UnifiedMemoryManager.scala | 9 - .../apache/spark/storage/BlockManager.scala | 53 +- .../org/apache/spark/storage/NvmStore.scala | 0 .../apache/spark/storage/StorageLevel.scala | 0 .../spark/storage/memory/MemoryStore.scala | 0 .../spark/memory/TestMemoryManager.scala | 18 +- pom.xml | 150 +- .../apache/spark/memory/MemoryConsumer.java | 165 -- .../spark/memory/PMemManagerInitializer.java | 88 - .../spark/memory/TaskMemoryManager.java | 619 ------ .../spark/unsafe/map/BytesToBytesMap.java | 936 --------- .../memory/ExtendedMemoryAllocator.java | 21 - .../spark/unsafe/memory/MemoryAllocator.java | 46 - .../spark/unsafe/memory/MemoryBlock.java | 103 - .../collection/unsafe/sort/PMemReader.java | 87 - .../PMemReaderForUnsafeExternalSorter.java | 141 -- .../unsafe/sort/PMemSpillWriterFactory.java | 99 - .../unsafe/sort/PMemSpillWriterType.java | 25 - .../collection/unsafe/sort/PMemWriter.java | 242 --- .../unsafe/sort/SortedIteratorForSpills.java | 138 -- .../sort/SortedPMemPageSpillWriter.java | 286 --- .../sort/SpillWriterForUnsafeSorter.java | 34 - .../unsafe/sort/UnsafeExternalSorter.java | 741 ------- .../unsafe/sort/UnsafeInMemorySorter.java | 435 ---- .../sort/UnsafeSorterPMemSpillWriter.java | 85 - .../unsafe/sort/UnsafeSorterSpillReader.java | 159 -- .../unsafe/sort/UnsafeSorterSpillWriter.java | 249 --- .../sort/UnsafeSorterStreamSpillReader.java | 57 - .../sort/UnsafeSorterStreamSpillWriter.java | 62 - .../scala/org/apache/spark/SparkEnv.scala | 510 ----- .../spark/internal/config/package.scala | 1869 ----------------- .../spark/memory/ExtendedMemoryPool.scala | 111 - .../spark/storage/PMemBlockObjectWriter.scala | 295 --- 37 files changed, 199 insertions(+), 7853 deletions(-) create mode 100644 RDD-Cache/pom.xml rename {src => RDD-Cache/src}/main/java/org/apache/spark/memory/MemoryMode.java (100%) rename {src => RDD-Cache/src}/main/scala/org/apache/spark/memory/MemoryManager.scala (83%) rename {src => RDD-Cache/src}/main/scala/org/apache/spark/memory/StorageMemoryPool.scala (100%) rename {src => RDD-Cache/src}/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala (97%) rename {src => RDD-Cache/src}/main/scala/org/apache/spark/storage/BlockManager.scala (98%) rename {src => RDD-Cache/src}/main/scala/org/apache/spark/storage/NvmStore.scala (100%) rename {src => RDD-Cache/src}/main/scala/org/apache/spark/storage/StorageLevel.scala (100%) rename {src => RDD-Cache/src}/main/scala/org/apache/spark/storage/memory/MemoryStore.scala (100%) rename {src => RDD-Cache/src}/test/scala/org/apache/spark/memory/TestMemoryManager.scala (84%) delete mode 100644 src/main/java/org/apache/spark/memory/MemoryConsumer.java delete mode 100644 src/main/java/org/apache/spark/memory/PMemManagerInitializer.java delete mode 100644 src/main/java/org/apache/spark/memory/TaskMemoryManager.java delete mode 100644 src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java delete mode 100644 src/main/java/org/apache/spark/unsafe/memory/ExtendedMemoryAllocator.java delete mode 100644 src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java delete mode 100644 src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemReader.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemReaderForUnsafeExternalSorter.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemSpillWriterFactory.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemSpillWriterType.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemWriter.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/SortedIteratorForSpills.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/SortedPMemPageSpillWriter.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/SpillWriterForUnsafeSorter.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterPMemSpillWriter.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterStreamSpillReader.java delete mode 100644 src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterStreamSpillWriter.java delete mode 100644 src/main/scala/org/apache/spark/SparkEnv.scala delete mode 100644 src/main/scala/org/apache/spark/internal/config/package.scala delete mode 100644 src/main/scala/org/apache/spark/memory/ExtendedMemoryPool.scala delete mode 100644 src/main/scala/org/apache/spark/storage/PMemBlockObjectWriter.scala diff --git a/RDD-Cache/pom.xml b/RDD-Cache/pom.xml new file mode 100644 index 00000000..55fec53b --- /dev/null +++ b/RDD-Cache/pom.xml @@ -0,0 +1,145 @@ + + + 4.0.0 + + + com.intel.oap + pmem-spill-parent + 1.1.0 + ../pom.xml + + + com.intel.oap + pmem-rdd-cache + 1.1.0 + OAP Project PMem RDD Cache + jar + + + 1.8 + 4.12 + 2.12.10 + 2.12 + 3.1.1 + 1.2 + 2.6 + 9.4.18.v20190429 + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.internal.version} + + + com.intel.oap + pmem-common + ${project.version} + + + io.pmem + libpmemkv-jni + + + + + org.eclipse.jetty + jetty-plus + compile + ${jetty.version} + + + org.eclipse.jetty + jetty-security + compile + ${jetty.version} + + + org.eclipse.jetty + jetty-util + compile + ${jetty.version} + + + org.eclipse.jetty + jetty-server + compile + ${jetty.version} + + + org.eclipse.jetty + jetty-http + compile + ${jetty.version} + + + org.eclipse.jetty + jetty-continuation + compile + ${jetty.version} + + + org.eclipse.jetty + jetty-servlet + compile + ${jetty.version} + + + org.eclipse.jetty + jetty-servlets + compile + ${jetty.version} + + + javax.servlet + javax.servlet-api + 3.1.0 + + + + + ${project.artifactId}-${project.version}-with-spark-${spark.internal.version} + + + net.alchim31.maven + scala-maven-plugin + 3.2.2 + + + scala-compile-first + process-resources + + compile + + + + scala-test-compile-first + process-test-resources + + testCompile + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.3 + + ${java.version} + ${java.version} + UTF-8 + 1024m + true + + -Xlint:all,-serial,-path + + + + + + + diff --git a/src/main/java/org/apache/spark/memory/MemoryMode.java b/RDD-Cache/src/main/java/org/apache/spark/memory/MemoryMode.java similarity index 100% rename from src/main/java/org/apache/spark/memory/MemoryMode.java rename to RDD-Cache/src/main/java/org/apache/spark/memory/MemoryMode.java diff --git a/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/RDD-Cache/src/main/scala/org/apache/spark/memory/MemoryManager.scala similarity index 83% rename from src/main/scala/org/apache/spark/memory/MemoryManager.scala rename to RDD-Cache/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 7c385cd5..7e8c58cf 100644 --- a/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/RDD-Cache/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -19,8 +19,6 @@ package org.apache.spark.memory import javax.annotation.concurrent.GuardedBy -import scala.collection.mutable.ArrayBuffer - import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -28,7 +26,7 @@ import org.apache.spark.storage.BlockId import org.apache.spark.storage.memory.MemoryStore import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.unsafe.memory.{MemoryAllocator, MemoryBlock} +import org.apache.spark.unsafe.memory.MemoryAllocator /** * An abstract memory manager that enforces how memory is shared between execution and storage. @@ -57,8 +55,6 @@ private[spark] abstract class MemoryManager( protected val onHeapExecutionMemoryPool = new ExecutionMemoryPool(this, MemoryMode.ON_HEAP) @GuardedBy("this") protected val offHeapExecutionMemoryPool = new ExecutionMemoryPool(this, MemoryMode.OFF_HEAP) - @GuardedBy("this") - protected val extendedMemoryPool = new ExtendedMemoryPool(this) onHeapStorageMemoryPool.incrementPoolSize(onHeapStorageMemory) onHeapExecutionMemoryPool.incrementPoolSize(onHeapExecutionMemory) @@ -75,14 +71,7 @@ private[spark] abstract class MemoryManager( protected[this] val pmemStorageMemory = (pmemInitialSize * pmemUsableRatio).toLong pmemStorageMemoryPool.incrementPoolSize(pmemStorageMemory) - protected[this] val extendedMemorySize = conf.get(MEMORY_EXTENDED_SIZE) - extendedMemoryPool.incrementPoolSize((extendedMemorySize * 0.9).toLong) - private[memory] var _pMemPages = new ArrayBuffer[MemoryBlock]; - - private[memory] def pMemPages: ArrayBuffer[MemoryBlock] = { - _pMemPages - } /** * Total available on heap memory for storage, in bytes. This amount can vary over time, * depending on the MemoryManager implementation. @@ -97,9 +86,9 @@ private[spark] abstract class MemoryManager( def maxOffHeapStorageMemory: Long /** - * Total available pmem memory for storage, in bytes. This amount can vary over time, - * depending on the MemoryManager implementation. - */ + * Total available pmem memory for storage, in bytes. This amount can vary over time, + * depending on the MemoryManager implementation. + */ def maxPMemStorageMemory: Long /** @@ -145,18 +134,6 @@ private[spark] abstract class MemoryManager( taskAttemptId: Long, memoryMode: MemoryMode): Long - /** - * try to acquire numBytes of extended memory for current task and return the number - * of number of bytes obtained, or 0 if non can be allocated. - * @param numBytes - * @param taskAttemptId - * @return - */ - private[memory] - def acquireExtendedMemory( - numBytes: Long, - taskAttemptId: Long): Long - /** * Release numBytes of execution memory belonging to the given task. */ @@ -208,47 +185,6 @@ private[spark] abstract class MemoryManager( releaseStorageMemory(numBytes, memoryMode) } - /** - * release extended memory of given task - * @param numBytes - * @param taskAttemptId - */ - def releaseExtendedMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { - extendedMemoryPool.releaseMemory(numBytes, taskAttemptId) - } - - /** - * release all extended memory occupied by given task - * @param taskAttemptId - * @return - */ - def releaseAllExtendedMemoryForTask(taskAttemptId: Long): Long = synchronized { - extendedMemoryPool.releaseAllMemoryForTask(taskAttemptId) - } - - def addPMemPages(pMemPage: MemoryBlock): Unit = synchronized { - pMemPages.append(pMemPage); - } - - def freeAllPMemPages(): Unit = synchronized { - for (pMemPage <- pMemPages) { - extendedMemoryAllocator.free(pMemPage); - } - } - - /** - * @param size size of current page request - * @return PMem Page that suits for current page request - */ - def getUsablePMemPage(size : Long): MemoryBlock = synchronized { - for (pMemPage <- pMemPages) { - if (pMemPage.pageNumber == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER && - pMemPage.size() == size) { - return pMemPage; - } - } - return null; - } /** * Execution memory currently in use, in bytes. */ @@ -349,6 +285,4 @@ private[spark] abstract class MemoryManager( case MemoryMode.OFF_HEAP => MemoryAllocator.UNSAFE } } - - private[memory] final val extendedMemoryAllocator = MemoryAllocator.EXTENDED } diff --git a/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/RDD-Cache/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala similarity index 100% rename from src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala rename to RDD-Cache/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala diff --git a/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/RDD-Cache/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala similarity index 97% rename from src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala rename to RDD-Cache/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 8bfcb360..6686082d 100644 --- a/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/RDD-Cache/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -154,15 +154,6 @@ private[spark] class UnifiedMemoryManager( numBytes, taskAttemptId, maybeGrowExecutionPool, () => computeMaxExecutionPoolSize) } - override def acquireExtendedMemory(numBytes: Long, taskAttemptId: Long): Long = synchronized { - if (numBytes > extendedMemoryPool.memoryFree) { - logInfo(s"No PMem Space left, allocation fails.") - return 0; - } - extendedMemoryPool.acquireMemory(numBytes, taskAttemptId); - return numBytes - } - override def acquireStorageMemory( blockId: BlockId, numBytes: Long, diff --git a/src/main/scala/org/apache/spark/storage/BlockManager.scala b/RDD-Cache/src/main/scala/org/apache/spark/storage/BlockManager.scala similarity index 98% rename from src/main/scala/org/apache/spark/storage/BlockManager.scala rename to RDD-Cache/src/main/scala/org/apache/spark/storage/BlockManager.scala index 5b3379c8..e7f7b828 100644 --- a/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/RDD-Cache/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -18,7 +18,7 @@ package org.apache.spark.storage import java.io._ -import java.lang.ref.{ReferenceQueue => JReferenceQueue, WeakReference} +import java.lang.ref.{WeakReference, ReferenceQueue => JReferenceQueue} import java.nio.ByteBuffer import java.nio.channels.Channels import java.util.Collections @@ -31,13 +31,10 @@ import scala.concurrent.duration._ import scala.reflect.ClassTag import scala.util.{Failure, Random, Success, Try} import scala.util.control.NonFatal - import com.codahale.metrics.{MetricRegistry, MetricSet} import com.google.common.cache.CacheBuilder import com.intel.oap.common.unsafe.PersistentMemoryPlatform - import org.apache.commons.io.IOUtils - import org.apache.spark._ import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.Logging @@ -55,7 +52,7 @@ import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.serializer.{SerializerInstance, SerializerManager} -import org.apache.spark.shuffle.{ShuffleManager, ShuffleWriteMetricsReporter} +import org.apache.spark.shuffle.{MigratableResolver, ShuffleManager, ShuffleWriteMetricsReporter} import org.apache.spark.storage.memory._ import org.apache.spark.unsafe.Platform import org.apache.spark.util._ @@ -189,7 +186,7 @@ private[spark] class BlockManager( val pmemMode = conf.get("spark.memory.pmem.mode", "AppDirect") val numNum = conf.getInt("spark.yarn.numa.num", 2) - if (memExtensionEnabled && pmemMode.equals("AppDirect")) { + if (pmemMode.equals("AppDirect")) { if (!isDriver && pmemInitialPaths.size >= 1) { if (numaNodeId == -1) { numaNodeId = executorId.toInt @@ -213,7 +210,7 @@ private[spark] class BlockManager( PersistentMemoryPlatform.initialize(file.getAbsolutePath, pmemInitialSize, 0) logInfo(s"Intel Optane PMem initialized with path: ${file.getAbsolutePath}, size: ${pmemInitialSize} ") } - } else if (memExtensionEnabled && pmemMode.equals("KMemDax")) { + } else if (pmemMode.equals("KMemDax")) { if (!isDriver) { if (numaNodeId == -1) { numaNodeId = (executorId.toInt + 1) % 2 @@ -273,7 +270,7 @@ private[spark] class BlockManager( private val slaveEndpoint = rpcEnv.setupEndpoint( "BlockManagerEndpoint" + BlockManager.ID_GENERATOR.next, - new BlockManagerSlaveEndpoint(rpcEnv, this, mapOutputTracker)) + new BlockManagerStorageEndpoint(rpcEnv, this, mapOutputTracker)) // Pending re-registration action being executed asynchronously or null if none is pending. // Accesses should synchronize on asyncReregisterLock. @@ -287,6 +284,10 @@ private[spark] class BlockManager( private var blockReplicationPolicy: BlockReplicationPolicy = _ + // visible for test + // This is volatile since if it's defined we should not accept remote blocks. + @volatile private[spark] var decommissioner: Option[BlockManagerDecommissioner] = None + // A DownloadFileManager used to track all the files of remote blocks which are above the // specified memory threshold. Files will be deleted automatically based on weak reference. // Exposed for test @@ -296,6 +297,26 @@ private[spark] class BlockManager( var hostLocalDirManager: Option[HostLocalDirManager] = None + @inline final private def isDecommissioning() = { + decommissioner.isDefined + } + + @inline final private def checkShouldStore(blockId: BlockId) = { + // Don't reject broadcast blocks since they may be stored during task exec and + // don't need to be migrated. + if (isDecommissioning() && !blockId.isBroadcast) { + throw new BlockSavedOnDecommissionedBlockManagerException(blockId) + } + } + + // This is a lazy val so someone can migrating RDDs even if they don't have a MigratableResolver + // for shuffles. Used in BlockManagerDecommissioner & block puts. + private[storage] lazy val migratableResolver: MigratableResolver = { + shuffleManager.shuffleBlockResolver.asInstanceOf[MigratableResolver] + } + + override def getLocalDiskDirs: Array[String] = diskBlockManager.localDirsString + /** * Abstraction for storing blocks from bytes, whether they start in memory or on disk. * @@ -1289,22 +1310,6 @@ private[spark] class BlockManager( syncWrites, writeMetrics, blockId) } - /** - * A short circuited method to get a PMem writer that can write data directly to PMem. - * The Block will be appended to the PMem stream specified by filename. Callers should handle - * error cases. - */ - def getPMemWriter( - blockId: BlockId, - file: File, - serializerInstance: SerializerInstance, - bufferSize: Int, - writeMetrics: ShuffleWriteMetricsReporter): PMemBlockObjectWriter = { - val syncWrites = conf.getBoolean("spark.shuffle.sync", false) - new PMemBlockObjectWriter(file, serializerManager, serializerInstance, bufferSize, - syncWrites, writeMetrics, blockId) - } - /** * Put a new block of serialized bytes to the block manager. * diff --git a/src/main/scala/org/apache/spark/storage/NvmStore.scala b/RDD-Cache/src/main/scala/org/apache/spark/storage/NvmStore.scala similarity index 100% rename from src/main/scala/org/apache/spark/storage/NvmStore.scala rename to RDD-Cache/src/main/scala/org/apache/spark/storage/NvmStore.scala diff --git a/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/RDD-Cache/src/main/scala/org/apache/spark/storage/StorageLevel.scala similarity index 100% rename from src/main/scala/org/apache/spark/storage/StorageLevel.scala rename to RDD-Cache/src/main/scala/org/apache/spark/storage/StorageLevel.scala diff --git a/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/RDD-Cache/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala similarity index 100% rename from src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala rename to RDD-Cache/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala diff --git a/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/RDD-Cache/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala similarity index 84% rename from src/test/scala/org/apache/spark/memory/TestMemoryManager.scala rename to RDD-Cache/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala index 2910dbae..b9bb4274 100644 --- a/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala +++ b/RDD-Cache/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala @@ -17,12 +17,11 @@ package org.apache.spark.memory -import com.intel.oap.common.unsafe.PersistentMemoryPlatform import javax.annotation.concurrent.GuardedBy + import scala.collection.mutable import org.apache.spark.SparkConf -import org.apache.spark.internal.config import org.apache.spark.storage.BlockId class TestMemoryManager(conf: SparkConf) @@ -34,16 +33,11 @@ class TestMemoryManager(conf: SparkConf) private var available = Long.MaxValue @GuardedBy("this") private val memoryForTask = mutable.HashMap[Long, Long]().withDefaultValue(0L) - private var extendedMemoryInitialized = false override private[memory] def acquireExecutionMemory( numBytes: Long, taskAttemptId: Long, memoryMode: MemoryMode): Long = synchronized { - if (conf.get(config.MEMORY_SPILL_PMEM_ENABLED) && extendedMemoryInitialized == false) { - PersistentMemoryPlatform.initialize("/dev/shm", 64 * 1024 * 1024, 0) - extendedMemoryInitialized = true - } require(numBytes >= 0) val acquired = { if (consequentOOM > 0) { @@ -81,16 +75,6 @@ class TestMemoryManager(conf: SparkConf) memoryForTask.remove(taskAttemptId).getOrElse(0L) } - override private[memory] def acquireExtendedMemory( - numBytes: Long, - taskAttemptId: Long): Long = synchronized { - if (extendedMemoryInitialized == false) { - PersistentMemoryPlatform.initialize("/dev/shm", 64 * 1024 * 1024, 0) - extendedMemoryInitialized = true - } - return extendedMemoryPool.acquireMemory(numBytes, taskAttemptId) - } - override private[memory] def getExecutionMemoryUsageForTask(taskAttemptId: Long): Long = { memoryForTask.getOrElse(taskAttemptId, 0L) } diff --git a/pom.xml b/pom.xml index cc236bec..2874f379 100644 --- a/pom.xml +++ b/pom.xml @@ -1,138 +1,28 @@ + - 4.0.0 + 4.0.0 - com.intel.oap - pmem-spill - 1.1.0 - OAP Project PMem Spill - jar + com.intel.oap + pmem-spill-parent + 1.1.0 + OAP Project PMem Spill + pom - - 1.8 - 4.12 - 2.12.10 - 2.12 - 3.0.0 - 1.2 - 2.6 - 9.4.39.v20210325 - - - - - org.apache.spark - spark-core_${scala.binary.version} - ${spark.internal.version} - - - com.intel.oap - pmem-common - ${project.version} - - - io.pmem - libpmemkv-jni - - - - - org.eclipse.jetty - jetty-plus - compile - ${jetty.version} - - - org.eclipse.jetty - jetty-security - compile - ${jetty.version} - - - org.eclipse.jetty - jetty-util - compile - ${jetty.version} - - - org.eclipse.jetty - jetty-server - compile - ${jetty.version} - - - org.eclipse.jetty - jetty-http - compile - ${jetty.version} - - - org.eclipse.jetty - jetty-continuation - compile - ${jetty.version} - - - org.eclipse.jetty - jetty-servlet - compile - ${jetty.version} - - - org.eclipse.jetty - jetty-servlets - compile - ${jetty.version} - - - javax.servlet - javax.servlet-api - 3.1.0 - - - - - ${project.artifactId}-${project.version}-with-spark-${spark.internal.version} - - - net.alchim31.maven - scala-maven-plugin - 3.2.2 - - - scala-compile-first - process-resources - - compile - - - - scala-test-compile-first - process-test-resources - - testCompile - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.3 - - ${java.version} - ${java.version} - UTF-8 - 1024m - true - - -Xlint:all,-serial,-path - - - - - + + RDD-Cache + diff --git a/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/src/main/java/org/apache/spark/memory/MemoryConsumer.java deleted file mode 100644 index 12e360ac..00000000 --- a/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * 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.memory; - -import java.io.IOException; - -import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.unsafe.memory.MemoryBlock; - -/** - * A memory consumer of {@link TaskMemoryManager} that supports spilling. - * - * Note: this only supports allocation / spilling of Tungsten memory. - */ -public abstract class MemoryConsumer { - - protected final TaskMemoryManager taskMemoryManager; - private final long pageSize; - private final MemoryMode mode; - protected long used; - - protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize, MemoryMode mode) { - this.taskMemoryManager = taskMemoryManager; - this.pageSize = pageSize; - this.mode = mode; - } - - protected MemoryConsumer(TaskMemoryManager taskMemoryManager) { - this(taskMemoryManager, taskMemoryManager.pageSizeBytes(), MemoryMode.ON_HEAP); - } - - /** - * Returns the memory mode, {@link MemoryMode#ON_HEAP} or {@link MemoryMode#OFF_HEAP}. - */ - public MemoryMode getMode() { - return mode; - } - - /** - * Returns the size of used memory in bytes. - */ - public long getUsed() { - return used; - } - - /** - * Force spill during building. - */ - public void spill() throws IOException { - spill(Long.MAX_VALUE, this); - } - - /** - * Spill some data to disk to release memory, which will be called by TaskMemoryManager - * when there is not enough memory for the task. - * - * This should be implemented by subclass. - * - * Note: In order to avoid possible deadlock, should not call acquireMemory() from spill(). - * - * Note: today, this only frees Tungsten-managed pages. - * - * @param size the amount of memory should be released - * @param trigger the MemoryConsumer that trigger this spilling - * @return the amount of released memory in bytes - */ - public abstract long spill(long size, MemoryConsumer trigger) throws IOException; - - /** - * Allocates a LongArray of `size`. Note that this method may throw `SparkOutOfMemoryError` - * if Spark doesn't have enough memory for this allocation, or throw `TooLargePageException` - * if this `LongArray` is too large to fit in a single page. The caller side should take care of - * these two exceptions, or make sure the `size` is small enough that won't trigger exceptions. - * - * @throws SparkOutOfMemoryError - * @throws TooLargePageException - */ - public LongArray allocateArray(long size) { - long required = size * 8L; - MemoryBlock page = taskMemoryManager.allocatePage(required, this); - if (page == null || page.size() < required) { - throwOom(page, required); - } - used += required; - return new LongArray(page); - } - - /** - * Frees a LongArray. - */ - public void freeArray(LongArray array) { - freePage(array.memoryBlock()); - } - - /** - * Allocate a memory block with at least `required` bytes. - * - * @throws SparkOutOfMemoryError - */ - protected MemoryBlock allocatePage(long required) { - MemoryBlock page = taskMemoryManager.allocatePage(Math.max(pageSize, required), this); - if (page == null || page.size() < required) { - throwOom(page, required); - } - used += page.size(); - return page; - } - - /** - * Free a memory block. - */ - protected void freePage(MemoryBlock page) { - used -= page.size(); - taskMemoryManager.freePage(page, this); - } - - /** - * Allocates memory of `size`. - */ - public long acquireMemory(long size) { - long granted = taskMemoryManager.acquireExecutionMemory(size, this); - used += granted; - return granted; - } - - /** - * Release N bytes of memory. - */ - public void freeMemory(long size) { - taskMemoryManager.releaseExecutionMemory(size, this); - used -= size; - } - - private void throwOom(final MemoryBlock page, final long required) { - long got = 0; - if (page != null) { - got = page.size(); - taskMemoryManager.freePage(page, this); - } - taskMemoryManager.showMemoryUsage(); - // checkstyle.off: RegexpSinglelineJava - throw new SparkOutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + - got); - // checkstyle.on: RegexpSinglelineJava - } - - public TaskMemoryManager getTaskMemoryManager() { - return taskMemoryManager; - } -} diff --git a/src/main/java/org/apache/spark/memory/PMemManagerInitializer.java b/src/main/java/org/apache/spark/memory/PMemManagerInitializer.java deleted file mode 100644 index 8b042f27..00000000 --- a/src/main/java/org/apache/spark/memory/PMemManagerInitializer.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.memory; - -import com.intel.oap.common.storage.stream.PMemManager; - -import org.apache.spark.SparkEnv; -import org.apache.spark.internal.config.ConfigEntry; -import org.apache.spark.internal.config.package$; -import org.apache.spark.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.BufferedInputStream; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.Properties; - -public class PMemManagerInitializer { - private static final Logger logger = LoggerFactory.getLogger(PMemManagerInitializer.class); - private static PMemManager pMemManager; - private static Properties properties; - - public static Properties getProperties() { - if (properties == null) { - synchronized (Properties.class) { - if (properties == null) { - ConfigEntry pMemPropertiesFile = - package$.MODULE$.PMEM_PROPERTY_FILE(); - String filePath = SparkEnv.get() == null ? pMemPropertiesFile.defaultValue().get() : SparkEnv.get().conf().get(pMemPropertiesFile); - logger.debug("PMem Property file: " + filePath); - Properties pps = new Properties(); - InputStream in = null; - try { - in = Utils.getSparkClassLoader().getResourceAsStream(filePath); - if (in == null) { - in = new BufferedInputStream(new FileInputStream(filePath)); - } - assert(in != null); - pps.load(in); - pps.setProperty("chunkSize", String.valueOf( - Utils.byteStringAsBytes(pps.getProperty("chunkSize")))); - pps.setProperty("totalSize", String.valueOf( - Utils.byteStringAsBytes(pps.getProperty("totalSize")))); - pps.setProperty("initialSize", String.valueOf( - Utils.byteStringAsBytes(pps.getProperty("initialSize")))); - } catch (IOException e) { - e.printStackTrace(); - } finally { - try { - in.close(); - } catch (IOException e) { - e.printStackTrace(); - } - } - properties = pps; - } - } - } - return properties; - } - - public static PMemManager getPMemManager() { - if (pMemManager == null) { - synchronized (PMemManager.class) { - if (pMemManager == null) { - pMemManager = new PMemManager(getProperties()); - } - } - } - return pMemManager; - } -} diff --git a/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/src/main/java/org/apache/spark/memory/TaskMemoryManager.java deleted file mode 100644 index 6bbe33e0..00000000 --- a/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ /dev/null @@ -1,619 +0,0 @@ -/* - * 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.memory; - -import javax.annotation.concurrent.GuardedBy; -import java.io.IOException; -import java.nio.channels.ClosedByInterruptException; -import java.util.Arrays; -import java.util.ArrayList; -import java.util.BitSet; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - -import com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.util.Utils; - -/** - * Manages the memory allocated by an individual task. - *

- * Most of the complexity in this class deals with encoding of off-heap addresses into 64-bit longs. - * In off-heap mode, memory can be directly addressed with 64-bit longs. In on-heap mode, memory is - * addressed by the combination of a base Object reference and a 64-bit offset within that object. - * This is a problem when we want to store pointers to data structures inside of other structures, - * such as record pointers inside hashmaps or sorting buffers. Even if we decided to use 128 bits - * to address memory, we can't just store the address of the base object since it's not guaranteed - * to remain stable as the heap gets reorganized due to GC. - *

- * Instead, we use the following approach to encode record pointers in 64-bit longs: for off-heap - * mode, just store the raw address, and for on-heap mode use the upper 13 bits of the address to - * store a "page number" and the lower 51 bits to store an offset within this page. These page - * numbers are used to index into a "page table" array inside of the MemoryManager in order to - * retrieve the base object. - *

- * This allows us to address 8192 pages. In on-heap mode, the maximum page size is limited by the - * maximum size of a long[] array, allowing us to address 8192 * (2^31 - 1) * 8 bytes, which is - * approximately 140 terabytes of memory. - */ -public class TaskMemoryManager { - - private static final Logger logger = LoggerFactory.getLogger(TaskMemoryManager.class); - - /** The number of bits used to address the page table. */ - private static final int PAGE_NUMBER_BITS = 13; - - /** The number of bits used to encode offsets in data pages. */ - @VisibleForTesting - static final int OFFSET_BITS = 64 - PAGE_NUMBER_BITS; // 51 - - /** The number of entries in the page table. */ - private static final int PAGE_TABLE_SIZE = 1 << PAGE_NUMBER_BITS; - - /** - * Maximum supported data page size (in bytes). In principle, the maximum addressable page size is - * (1L << OFFSET_BITS) bytes, which is 2+ petabytes. However, the on-heap allocator's - * maximum page size is limited by the maximum amount of data that can be stored in a long[] - * array, which is (2^31 - 1) * 8 bytes (or about 17 gigabytes). Therefore, we cap this at 17 - * gigabytes. - */ - public static final long MAXIMUM_PAGE_SIZE_BYTES = ((1L << 31) - 1) * 8L; - - /** Bit mask for the lower 51 bits of a long. */ - private static final long MASK_LONG_LOWER_51_BITS = 0x7FFFFFFFFFFFFL; - - /** - * Similar to an operating system's page table, this array maps page numbers into base object - * pointers, allowing us to translate between the hashtable's internal 64-bit address - * representation and the baseObject+offset representation which we use to support both on- and - * off-heap addresses. When using an off-heap allocator, every entry in this map will be `null`. - * When using an on-heap allocator, the entries in this map will point to pages' base objects. - * Entries are added to this map as new data pages are allocated. - */ - private final MemoryBlock[] pageTable = new MemoryBlock[PAGE_TABLE_SIZE]; - - /** - * Bitmap for tracking free pages. - */ - private final BitSet allocatedPages = new BitSet(PAGE_TABLE_SIZE); - - private final MemoryManager memoryManager; - - private final long taskAttemptId; - - /** - * Tracks whether we're on-heap or off-heap. For off-heap, we short-circuit most of these methods - * without doing any masking or lookups. Since this branching should be well-predicted by the JIT, - * this extra layer of indirection / abstraction hopefully shouldn't be too expensive. - */ - final MemoryMode tungstenMemoryMode; - - /** - * Tracks spillable memory consumers. - */ - @GuardedBy("this") - private final HashSet consumers; - - /** - * The amount of memory that is acquired but not used. - */ - private volatile long acquiredButNotUsed = 0L; - - /** - * Construct a new TaskMemoryManager. - */ - public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) { - this.tungstenMemoryMode = memoryManager.tungstenMemoryMode(); - this.memoryManager = memoryManager; - this.taskAttemptId = taskAttemptId; - this.consumers = new HashSet<>(); - } - - /** - * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call - * spill() of consumers to release more memory. - * - * @return number of bytes successfully granted (<= N). - */ - public long acquireExecutionMemory(long required, MemoryConsumer consumer) { - assert(required >= 0); - assert(consumer != null); - MemoryMode mode = consumer.getMode(); - // If we are allocating Tungsten pages off-heap and receive a request to allocate on-heap - // memory here, then it may not make sense to spill since that would only end up freeing - // off-heap memory. This is subject to change, though, so it may be risky to make this - // optimization now in case we forget to undo it late when making changes. - synchronized (this) { - long got = memoryManager.acquireExecutionMemory(required, taskAttemptId, mode); - - // Try to release memory from other consumers first, then we can reduce the frequency of - // spilling, avoid to have too many spilled files. - if (got < required) { - // Call spill() on other consumers to release memory - // Sort the consumers according their memory usage. So we avoid spilling the same consumer - // which is just spilled in last few times and re-spilling on it will produce many small - // spill files. - TreeMap> sortedConsumers = new TreeMap<>(); - for (MemoryConsumer c: consumers) { - if (c != consumer && c.getUsed() > 0 && c.getMode() == mode) { - long key = c.getUsed(); - List list = - sortedConsumers.computeIfAbsent(key, k -> new ArrayList<>(1)); - list.add(c); - } - } - while (!sortedConsumers.isEmpty()) { - // Get the consumer using the least memory more than the remaining required memory. - Map.Entry> currentEntry = - sortedConsumers.ceilingEntry(required - got); - // No consumer has used memory more than the remaining required memory. - // Get the consumer of largest used memory. - if (currentEntry == null) { - currentEntry = sortedConsumers.lastEntry(); - } - List cList = currentEntry.getValue(); - MemoryConsumer c = cList.get(cList.size() - 1); - try { - long released = c.spill(required - got, consumer); - if (released > 0) { - logger.debug("Task {} released {} from {} for {}", taskAttemptId, - Utils.bytesToString(released), c, consumer); - got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); - if (got >= required) { - break; - } - } else { - cList.remove(cList.size() - 1); - if (cList.isEmpty()) { - sortedConsumers.remove(currentEntry.getKey()); - } - } - } catch (ClosedByInterruptException e) { - // This called by user to kill a task (e.g: speculative task). - logger.error("error while calling spill() on " + c, e); - throw new RuntimeException(e.getMessage()); - } catch (IOException e) { - logger.error("error while calling spill() on " + c, e); - // checkstyle.off: RegexpSinglelineJava - throw new SparkOutOfMemoryError("error while calling spill() on " + c + " : " - + e.getMessage()); - // checkstyle.on: RegexpSinglelineJava - } - } - } - - // call spill() on itself - if (got < required) { - try { - long released = consumer.spill(required - got, consumer); - if (released > 0) { - logger.debug("Task {} released {} from itself ({})", taskAttemptId, - Utils.bytesToString(released), consumer); - got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); - } - } catch (ClosedByInterruptException e) { - // This called by user to kill a task (e.g: speculative task). - logger.error("error while calling spill() on " + consumer, e); - throw new RuntimeException(e.getMessage()); - } catch (IOException e) { - logger.error("error while calling spill() on " + consumer, e); - // checkstyle.off: RegexpSinglelineJava - throw new SparkOutOfMemoryError("error while calling spill() on " + consumer + " : " - + e.getMessage()); - // checkstyle.on: RegexpSinglelineJava - } - } - - consumers.add(consumer); - logger.debug("Task {} acquired {} for {}", taskAttemptId, Utils.bytesToString(got), consumer); - return got; - } - } - - /** - * Acquire extended memory - * When extended memory is acqurired, spill will not be triggered. - * @param required - * @return - */ - public long acquireExtendedMemory(long required, MemoryConsumer consumer) { - assert(required >= 0); - logger.debug("Task {} acquire {} bytes PMem memory.", taskAttemptId, Utils.bytesToString(required)); - synchronized (this) { - long got = memoryManager.acquireExtendedMemory(required, taskAttemptId); - logger.debug("Task {} got {} bytes PMem memory.", taskAttemptId, Utils.bytesToString(got)); - // The MemoryConsumer which acquired extended memory should be traced in TaskMemoryManagr. - // Not very sure about whether it should be added to the consumers here. Maybe should maintain - // another list for consumers which use extended memory. - consumers.add(consumer); - return got; - } - } - - /** - * Release N bytes of execution memory for a MemoryConsumer. - */ - public void releaseExecutionMemory(long size, MemoryConsumer consumer) { - logger.debug("Task {} release {} from {}", taskAttemptId, Utils.bytesToString(size), consumer); - memoryManager.releaseExecutionMemory(size, taskAttemptId, consumer.getMode()); - } - - public long acquireExtendedMemory(long required) { - assert(required >= 0); - logger.debug("Task {} acquire {} bytes PMem memory.", taskAttemptId, Utils.bytesToString(required)); - synchronized (this) { - long got = memoryManager.acquireExtendedMemory(required, taskAttemptId); - return got; - } - } - - public void releaseExtendedMemory(long size) { - logger.debug("Task {} release {} PMem space.", taskAttemptId, Utils.bytesToString(size)); - memoryManager.releaseExtendedMemory(size, taskAttemptId); - } - /** - * Rlease extended memory - * @param size - */ - public void releaseExtendedMemory(long size, MemoryConsumer consumer) { - logger.debug("Task {} release {} PMem space.", taskAttemptId, Utils.bytesToString(size)); - memoryManager.releaseExtendedMemory(size, taskAttemptId); - } - - /** - * Dump the memory usage of all consumers. - */ - public void showMemoryUsage() { - logger.info("Memory used in task " + taskAttemptId); - synchronized (this) { - long memoryAccountedForByConsumers = 0; - for (MemoryConsumer c: consumers) { - long totalMemUsage = c.getUsed(); - memoryAccountedForByConsumers += totalMemUsage; - if (totalMemUsage > 0) { - logger.info("Acquired by " + c + ": " + Utils.bytesToString(totalMemUsage)); - } - } - long memoryNotAccountedFor = - memoryManager.getExecutionMemoryUsageForTask(taskAttemptId) - memoryAccountedForByConsumers; - logger.info( - "{} bytes of memory were used by task {} but are not associated with specific consumers", - memoryNotAccountedFor, taskAttemptId); - logger.info( - "{} bytes of memory are used for execution and {} bytes of memory are used for storage", - memoryManager.executionMemoryUsed(), memoryManager.storageMemoryUsed()); - } - } - - /** - * Return the page size in bytes. - */ - public long pageSizeBytes() { - return memoryManager.pageSizeBytes(); - } - - /** - * Allocate a block of memory that will be tracked in the MemoryManager's page table; this is - * intended for allocating large blocks of Tungsten memory that will be shared between operators. - * - * Returns `null` if there was not enough memory to allocate the page. May return a page that - * contains fewer bytes than requested, so callers should verify the size of returned pages. - * - * @throws TooLargePageException - */ - public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { - return allocatePage(size, consumer, false); - } - - public MemoryBlock allocatePage(long size, MemoryConsumer consumer, boolean useExtendedMem) { - assert(consumer != null); - assert(consumer.getMode() == tungstenMemoryMode); - if (size > MAXIMUM_PAGE_SIZE_BYTES) { - throw new TooLargePageException(size); - } - - long acquired = 0L; - if (useExtendedMem) { - acquired = acquireExtendedMemory(size, consumer); - } else { - acquired = acquireExecutionMemory(size, consumer); - } - if (acquired <= 0) { - return null; - } - - final int pageNumber; - synchronized (this) { - pageNumber = allocatedPages.nextClearBit(0); - if (pageNumber >= PAGE_TABLE_SIZE) { - if (useExtendedMem) { - releaseExtendedMemory(acquired, consumer); - } else { - releaseExecutionMemory(acquired, consumer); - } - throw new IllegalStateException( - "Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages"); - } - allocatedPages.set(pageNumber); - } - MemoryBlock page = null; - try { - if (useExtendedMem) { - page = memoryManager.extendedMemoryAllocator().allocate(size); - page.isExtendedMemory(true); - } else { - page = memoryManager.tungstenMemoryAllocator().allocate(acquired); - } - } catch (OutOfMemoryError e) { - logger.warn("Failed to allocate a page ({} bytes), try again.", acquired); - // there is no enough memory actually, it means the actual free memory is smaller than - // MemoryManager thought, we should keep the acquired memory. - synchronized (this) { - acquiredButNotUsed += acquired; - allocatedPages.clear(pageNumber); - } - if (useExtendedMem) { - // will not force spill when use extended mem - return null; - } else { - // this could trigger spilling to free some pages. - return allocatePage(size, consumer); - } - } - - page.pageNumber = pageNumber; - pageTable[pageNumber] = page; - if (logger.isTraceEnabled()) { - logger.trace("Allocate page number {} ({} bytes)", pageNumber, acquired); - } - return page; - } - - /** - * Free a block of memory allocated via {@link TaskMemoryManager#allocatePage}. - */ - public void freePage(MemoryBlock page, MemoryConsumer consumer) { - assert (page.pageNumber != MemoryBlock.NO_PAGE_NUMBER) : - "Called freePage() on memory that wasn't allocated with allocatePage()"; - assert (page.pageNumber != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : - "Called freePage() on a memory block that has already been freed"; - assert (page.pageNumber != MemoryBlock.FREED_IN_TMM_PAGE_NUMBER) : - "Called freePage() on a memory block that has already been freed"; - assert(allocatedPages.get(page.pageNumber)); - pageTable[page.pageNumber] = null; - synchronized (this) { - allocatedPages.clear(page.pageNumber); - } - if (logger.isTraceEnabled()) { - logger.trace("Freed page number {} ({} bytes)", page.pageNumber, page.size()); - } - long pageSize = page.size(); - // Clear the page number before passing the block to the MemoryAllocator's free(). - // Doing this allows the MemoryAllocator to detect when a TaskMemoryManager-managed - // page has been inappropriately directly freed without calling TMM.freePage(). - page.pageNumber = MemoryBlock.FREED_IN_TMM_PAGE_NUMBER; - if (page.isExtendedMemory) { - memoryManager.extendedMemoryAllocator().free(page); - releaseExtendedMemory(pageSize, consumer); - } else { - memoryManager.tungstenMemoryAllocator().free(page); - releaseExecutionMemory(pageSize, consumer); - } - } - - /** - * @param size - * @return - */ - public MemoryBlock allocatePMemPage(long size) { - if (size > MAXIMUM_PAGE_SIZE_BYTES) { - throw new TooLargePageException(size); - } - final int pageNumber; - synchronized (this) { - pageNumber = allocatedPages.nextClearBit(0); - if (pageNumber >= PAGE_TABLE_SIZE) { - throw new IllegalStateException( - "Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages"); - } - allocatedPages.set(pageNumber); - } - MemoryBlock page = null; - try { - page = memoryManager.getUsablePMemPage(size); - if (page == null) { - page = memoryManager.extendedMemoryAllocator().allocate(size); - memoryManager.addPMemPages(page); - } else { - logger.debug("reuse pmem page."); - } - } catch (OutOfMemoryError e) { - logger.debug("Failed to allocate a PMem page ({} bytes).", size); - return null; - } - page.isExtendedMemory(true); - page.pageNumber = pageNumber; - pageTable[pageNumber] = page; - if (logger.isTraceEnabled()) { - logger.trace("Allocate page number {} ({} bytes)", pageNumber, size); - } - return page; - - } - - public void freePMemPage(MemoryBlock page, MemoryConsumer consumer) { - assert (page.pageNumber != MemoryBlock.NO_PAGE_NUMBER) : - "Called freePage() on memory that wasn't allocated with allocatePage()"; - assert (page.pageNumber != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : - "Called freePage() on a memory block that has already been freed"; - assert (page.pageNumber != MemoryBlock.FREED_IN_TMM_PAGE_NUMBER) : - "Called freePage() on a memory block that has already been freed"; - assert(allocatedPages.get(page.pageNumber)); - pageTable[page.pageNumber] = null; - synchronized (this) { - allocatedPages.clear(page.pageNumber); - } - if (logger.isTraceEnabled()) { - logger.trace("Freed PMem page number {} ({} bytes)", page.pageNumber, page.size()); - } - // Clear the page number before passing the block to the MemoryAllocator's free(). - // Doing this allows the MemoryAllocator to detect when a TaskMemoryManager-managed - // page has been inappropriately directly freed without calling TMM.freePage(). - - page.pageNumber = MemoryBlock.FREED_IN_TMM_PAGE_NUMBER; - // not really free the PMem page for future page reuse - } - - /** - * Given a memory page and offset within that page, encode this address into a 64-bit long. - * This address will remain valid as long as the corresponding page has not been freed. - * - * @param page a data page allocated by {@link TaskMemoryManager#allocatePage}/ - * @param offsetInPage an offset in this page which incorporates the base offset. In other words, - * this should be the value that you would pass as the base offset into an - * UNSAFE call (e.g. page.baseOffset() + something). - * @return an encoded page address. - */ - public long encodePageNumberAndOffset(MemoryBlock page, long offsetInPage) { - if (tungstenMemoryMode == MemoryMode.OFF_HEAP) { - // In off-heap mode, an offset is an absolute address that may require a full 64 bits to - // encode. Due to our page size limitation, though, we can convert this into an offset that's - // relative to the page's base offset; this relative offset will fit in 51 bits. - offsetInPage -= page.getBaseOffset(); - } - return encodePageNumberAndOffset(page.pageNumber, offsetInPage); - } - - @VisibleForTesting - public static long encodePageNumberAndOffset(int pageNumber, long offsetInPage) { - assert (pageNumber >= 0) : "encodePageNumberAndOffset called with invalid page"; - return (((long) pageNumber) << OFFSET_BITS) | (offsetInPage & MASK_LONG_LOWER_51_BITS); - } - - @VisibleForTesting - public static int decodePageNumber(long pagePlusOffsetAddress) { - return (int) (pagePlusOffsetAddress >>> OFFSET_BITS); - } - - private static long decodeOffset(long pagePlusOffsetAddress) { - return (pagePlusOffsetAddress & MASK_LONG_LOWER_51_BITS); - } - - /** - * Get the page associated with an address encoded by - * {@link TaskMemoryManager#encodePageNumberAndOffset(MemoryBlock, long)} - */ - public Object getPage(long pagePlusOffsetAddress) { - if (tungstenMemoryMode == MemoryMode.ON_HEAP) { - final int pageNumber = decodePageNumber(pagePlusOffsetAddress); - assert (pageNumber >= 0 && pageNumber < PAGE_TABLE_SIZE); - final MemoryBlock page = pageTable[pageNumber]; - assert (page != null); - assert (page.getBaseObject() != null); - return page.getBaseObject(); - } else { - return null; - } - } - - public MemoryBlock getOriginalPage(long pagePlusOffsetAddress) { - if (tungstenMemoryMode == MemoryMode.ON_HEAP) { - final int pageNumber = decodePageNumber(pagePlusOffsetAddress); - assert (pageNumber >= 0 && pageNumber < PAGE_TABLE_SIZE); - final MemoryBlock page = pageTable[pageNumber]; - assert (page != null); - assert (page.getBaseObject() != null); - return page; - } else { - return null; - } - } - - /** - * Get the offset associated with an address encoded by - * {@link TaskMemoryManager#encodePageNumberAndOffset(MemoryBlock, long)} - */ - public long getOffsetInPage(long pagePlusOffsetAddress) { - final long offsetInPage = decodeOffset(pagePlusOffsetAddress); - if (tungstenMemoryMode == MemoryMode.ON_HEAP) { - return offsetInPage; - } else { - // In off-heap mode, an offset is an absolute address. In encodePageNumberAndOffset, we - // converted the absolute address into a relative address. Here, we invert that operation: - final int pageNumber = decodePageNumber(pagePlusOffsetAddress); - assert (pageNumber >= 0 && pageNumber < PAGE_TABLE_SIZE); - final MemoryBlock page = pageTable[pageNumber]; - assert (page != null); - return page.getBaseOffset() + offsetInPage; - } - } - - /** - * Clean up all allocated memory and pages. Returns the number of bytes freed. A non-zero return - * value can be used to detect memory leaks. - */ - public long cleanUpAllAllocatedMemory() { - synchronized (this) { - for (MemoryConsumer c: consumers) { - if (c != null && c.getUsed() > 0) { - // In case of failed task, it's normal to see leaked memory - logger.debug("unreleased " + Utils.bytesToString(c.getUsed()) + " memory from " + c); - } - } - consumers.clear(); - - for (MemoryBlock page : pageTable) { - if (page != null) { - logger.debug("unreleased page: " + page + " in task " + taskAttemptId); - page.pageNumber = MemoryBlock.FREED_IN_TMM_PAGE_NUMBER; - if (!page.isExtendedMemory){ - memoryManager.tungstenMemoryAllocator().free(page); - } else { - memoryManager.extendedMemoryAllocator().free(page); - } - } - } - Arrays.fill(pageTable, null); - } - - // release the memory that is not used by any consumer (acquired for pages in tungsten mode). - memoryManager.releaseExecutionMemory(acquiredButNotUsed, taskAttemptId, tungstenMemoryMode); - memoryManager.releaseAllExtendedMemoryForTask(taskAttemptId); - - return memoryManager.releaseAllExecutionMemoryForTask(taskAttemptId); - } - - /** - * Returns the memory consumption, in bytes, for the current task. - */ - public long getMemoryConsumptionForThisTask() { - return memoryManager.getExecutionMemoryUsageForTask(taskAttemptId); - } - - /** - * Returns Tungsten memory mode - */ - public MemoryMode getTungstenMemoryMode() { - return tungstenMemoryMode; - } -} diff --git a/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java deleted file mode 100644 index e7463908..00000000 --- a/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ /dev/null @@ -1,936 +0,0 @@ -/* - * 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.unsafe.map; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import java.util.LinkedList; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.io.Closeables; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.SparkEnv; -import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.memory.MemoryConsumer; -import org.apache.spark.memory.SparkOutOfMemoryError; -import org.apache.spark.memory.TaskMemoryManager; -import org.apache.spark.serializer.SerializerManager; -import org.apache.spark.storage.BlockManager; -import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.UnsafeAlignedOffset; -import org.apache.spark.unsafe.array.ByteArrayMethods; -import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillReader; -import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillWriter; - -/** - * An append-only hash map where keys and values are contiguous regions of bytes. - * - * This is backed by a power-of-2-sized hash table, using quadratic probing with triangular numbers, - * which is guaranteed to exhaust the space. - * - * The map can support up to 2^29 keys. If the key cardinality is higher than this, you should - * probably be using sorting instead of hashing for better cache locality. - * - * The key and values under the hood are stored together, in the following format: - * First uaoSize bytes: len(k) (key length in bytes) + len(v) (value length in bytes) + uaoSize - * Next uaoSize bytes: len(k) - * Next len(k) bytes: key data - * Next len(v) bytes: value data - * Last 8 bytes: pointer to next pair - * - * It means first uaoSize bytes store the entire record (key + value + uaoSize) length. This format - * is compatible with {@link org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter}, - * so we can pass records from this map directly into the sorter to sort records in place. - */ -public final class BytesToBytesMap extends MemoryConsumer { - - private static final Logger logger = LoggerFactory.getLogger(BytesToBytesMap.class); - - private static final HashMapGrowthStrategy growthStrategy = HashMapGrowthStrategy.DOUBLING; - - private final TaskMemoryManager taskMemoryManager; - - /** - * A linked list for tracking all allocated data pages so that we can free all of our memory. - */ - private final LinkedList dataPages = new LinkedList<>(); - - /** - * The data page that will be used to store keys and values for new hashtable entries. When this - * page becomes full, a new page will be allocated and this pointer will change to point to that - * new page. - */ - private MemoryBlock currentPage = null; - - /** - * Offset into `currentPage` that points to the location where new data can be inserted into - * the page. This does not incorporate the page's base offset. - */ - private long pageCursor = 0; - - /** - * The maximum number of keys that BytesToBytesMap supports. The hash table has to be - * power-of-2-sized and its backing Java array can contain at most (1 << 30) elements, - * since that's the largest power-of-2 that's less than Integer.MAX_VALUE. We need two long array - * entries per key, giving us a maximum capacity of (1 << 29). - */ - public static final int MAX_CAPACITY = (1 << 29); - - // This choice of page table size and page size means that we can address up to 500 gigabytes - // of memory. - - /** - * A single array to store the key and value. - * - * Position {@code 2 * i} in the array is used to track a pointer to the key at index {@code i}, - * while position {@code 2 * i + 1} in the array holds key's full 32-bit hashcode. - */ - @Nullable private LongArray longArray; - // TODO: we're wasting 32 bits of space here; we can probably store fewer bits of the hashcode - // and exploit word-alignment to use fewer bits to hold the address. This might let us store - // only one long per map entry, increasing the chance that this array will fit in cache at the - // expense of maybe performing more lookups if we have hash collisions. Say that we stored only - // 27 bits of the hashcode and 37 bits of the address. 37 bits is enough to address 1 terabyte - // of RAM given word-alignment. If we use 13 bits of this for our page table, that gives us a - // maximum page size of 2^24 * 8 = ~134 megabytes per page. This change will require us to store - // full base addresses in the page table for off-heap mode so that we can reconstruct the full - // absolute memory addresses. - - /** - * Whether or not the longArray can grow. We will not insert more elements if it's false. - */ - private boolean canGrowArray = true; - - private final double loadFactor; - - /** - * The size of the data pages that hold key and value data. Map entries cannot span multiple - * pages, so this limits the maximum entry size. - */ - private final long pageSizeBytes; - - /** - * Number of keys defined in the map. - */ - private int numKeys; - - /** - * Number of values defined in the map. A key could have multiple values. - */ - private int numValues; - - /** - * The map will be expanded once the number of keys exceeds this threshold. - */ - private int growthThreshold; - - /** - * Mask for truncating hashcodes so that they do not exceed the long array's size. - * This is a strength reduction optimization; we're essentially performing a modulus operation, - * but doing so with a bitmask because this is a power-of-2-sized hash map. - */ - private int mask; - - /** - * Return value of {@link BytesToBytesMap#lookup(Object, long, int)}. - */ - private final Location loc; - - private long numProbes = 0L; - - private long numKeyLookups = 0L; - - private long peakMemoryUsedBytes = 0L; - - private final int initialCapacity; - - private final BlockManager blockManager; - private final SerializerManager serializerManager; - private volatile MapIterator destructiveIterator = null; - private LinkedList spillWriters = new LinkedList<>(); - - public BytesToBytesMap( - TaskMemoryManager taskMemoryManager, - BlockManager blockManager, - SerializerManager serializerManager, - int initialCapacity, - double loadFactor, - long pageSizeBytes) { - super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); - this.taskMemoryManager = taskMemoryManager; - this.blockManager = blockManager; - this.serializerManager = serializerManager; - this.loadFactor = loadFactor; - this.loc = new Location(); - this.pageSizeBytes = pageSizeBytes; - if (initialCapacity <= 0) { - throw new IllegalArgumentException("Initial capacity must be greater than 0"); - } - if (initialCapacity > MAX_CAPACITY) { - throw new IllegalArgumentException( - "Initial capacity " + initialCapacity + " exceeds maximum capacity of " + MAX_CAPACITY); - } - if (pageSizeBytes > TaskMemoryManager.MAXIMUM_PAGE_SIZE_BYTES) { - throw new IllegalArgumentException("Page size " + pageSizeBytes + " cannot exceed " + - TaskMemoryManager.MAXIMUM_PAGE_SIZE_BYTES); - } - this.initialCapacity = initialCapacity; - allocate(initialCapacity); - } - - public BytesToBytesMap( - TaskMemoryManager taskMemoryManager, - int initialCapacity, - long pageSizeBytes) { - this( - taskMemoryManager, - SparkEnv.get() != null ? SparkEnv.get().blockManager() : null, - SparkEnv.get() != null ? SparkEnv.get().serializerManager() : null, - initialCapacity, - // In order to re-use the longArray for sorting, the load factor cannot be larger than 0.5. - 0.5, - pageSizeBytes); - } - - /** - * Returns the number of keys defined in the map. - */ - public int numKeys() { return numKeys; } - - /** - * Returns the number of values defined in the map. A key could have multiple values. - */ - public int numValues() { return numValues; } - - public final class MapIterator implements Iterator { - - private int numRecords; - private final Location loc; - - private MemoryBlock currentPage = null; - private int recordsInPage = 0; - private Object pageBaseObject; - private long offsetInPage; - - // If this iterator destructive or not. When it is true, it frees each page as it moves onto - // next one. - private boolean destructive = false; - private UnsafeSorterSpillReader reader = null; - - private MapIterator(int numRecords, Location loc, boolean destructive) { - this.numRecords = numRecords; - this.loc = loc; - this.destructive = destructive; - if (destructive) { - destructiveIterator = this; - // longArray will not be used anymore if destructive is true, release it now. - if (longArray != null) { - freeArray(longArray); - longArray = null; - } - } - } - - private void advanceToNextPage() { - // SPARK-26265: We will first lock this `MapIterator` and then `TaskMemoryManager` when going - // to free a memory page by calling `freePage`. At the same time, it is possibly that another - // memory consumer first locks `TaskMemoryManager` and then this `MapIterator` when it - // acquires memory and causes spilling on this `MapIterator`. To avoid deadlock here, we keep - // reference to the page to free and free it after releasing the lock of `MapIterator`. - MemoryBlock pageToFree = null; - - try { - synchronized (this) { - int nextIdx = dataPages.indexOf(currentPage) + 1; - if (destructive && currentPage != null) { - dataPages.remove(currentPage); - pageToFree = currentPage; - nextIdx--; - } - if (dataPages.size() > nextIdx) { - currentPage = dataPages.get(nextIdx); - pageBaseObject = currentPage.getBaseObject(); - offsetInPage = currentPage.getBaseOffset(); - recordsInPage = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); - offsetInPage += UnsafeAlignedOffset.getUaoSize(); - } else { - currentPage = null; - if (reader != null) { - handleFailedDelete(); - } - try { - Closeables.close(reader, /* swallowIOException = */ false); - reader = spillWriters.getFirst().getReader(serializerManager, null); - recordsInPage = -1; - } catch (IOException e) { - // Scala iterator does not handle exception - Platform.throwException(e); - } - } - } - } finally { - if (pageToFree != null) { - freePage(pageToFree); - } - } - } - - @Override - public boolean hasNext() { - if (numRecords == 0) { - if (reader != null) { - handleFailedDelete(); - } - } - return numRecords > 0; - } - - @Override - public Location next() { - if (recordsInPage == 0) { - advanceToNextPage(); - } - numRecords--; - if (currentPage != null) { - int totalLength = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); - loc.with(currentPage, offsetInPage); - // [total size] [key size] [key] [value] [pointer to next] - offsetInPage += UnsafeAlignedOffset.getUaoSize() + totalLength + 8; - recordsInPage --; - return loc; - } else { - assert(reader != null); - if (!reader.hasNext()) { - advanceToNextPage(); - } - try { - reader.loadNext(); - } catch (IOException e) { - try { - reader.close(); - } catch(IOException e2) { - logger.error("Error while closing spill reader", e2); - } - // Scala iterator does not handle exception - Platform.throwException(e); - } - loc.with(reader.getBaseObject(), reader.getBaseOffset(), reader.getRecordLength()); - return loc; - } - } - - public synchronized long spill(long numBytes) throws IOException { - if (!destructive || dataPages.size() == 1) { - return 0L; - } - - updatePeakMemoryUsed(); - - // TODO: use existing ShuffleWriteMetrics - ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); - - long released = 0L; - while (dataPages.size() > 0) { - MemoryBlock block = dataPages.getLast(); - // The currentPage is used, cannot be released - if (block == currentPage) { - break; - } - - Object base = block.getBaseObject(); - long offset = block.getBaseOffset(); - int numRecords = UnsafeAlignedOffset.getSize(base, offset); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - offset += uaoSize; - final UnsafeSorterSpillWriter writer = - new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords); - while (numRecords > 0) { - int length = UnsafeAlignedOffset.getSize(base, offset); - writer.write(base, offset + uaoSize, length, 0); - offset += uaoSize + length + 8; - numRecords--; - } - writer.close(); - spillWriters.add(writer); - - dataPages.removeLast(); - released += block.size(); - freePage(block); - - if (released >= numBytes) { - break; - } - } - - return released; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - - private void handleFailedDelete() { - // remove the spill file from disk - File file = spillWriters.removeFirst().getFile(); - if (file != null && file.exists() && !file.delete()) { - logger.error("Was unable to delete spill file {}", file.getAbsolutePath()); - } - } - } - - /** - * Returns an iterator for iterating over the entries of this map. - * - * For efficiency, all calls to `next()` will return the same {@link Location} object. - * - * If any other lookups or operations are performed on this map while iterating over it, including - * `lookup()`, the behavior of the returned iterator is undefined. - */ - public MapIterator iterator() { - return new MapIterator(numValues, loc, false); - } - - /** - * Returns a thread safe iterator that iterates of the entries of this map. - */ - public MapIterator safeIterator() { - return new MapIterator(numValues, new Location(), false); - } - - /** - * Returns a destructive iterator for iterating over the entries of this map. It frees each page - * as it moves onto next one. Notice: it is illegal to call any method on the map after - * `destructiveIterator()` has been called. - * - * For efficiency, all calls to `next()` will return the same {@link Location} object. - * - * If any other lookups or operations are performed on this map while iterating over it, including - * `lookup()`, the behavior of the returned iterator is undefined. - */ - public MapIterator destructiveIterator() { - updatePeakMemoryUsed(); - return new MapIterator(numValues, loc, true); - } - - /** - * Looks up a key, and return a {@link Location} handle that can be used to test existence - * and read/write values. - * - * This function always return the same {@link Location} instance to avoid object allocation. - */ - public Location lookup(Object keyBase, long keyOffset, int keyLength) { - safeLookup(keyBase, keyOffset, keyLength, loc, - Murmur3_x86_32.hashUnsafeWords(keyBase, keyOffset, keyLength, 42)); - return loc; - } - - /** - * Looks up a key, and return a {@link Location} handle that can be used to test existence - * and read/write values. - * - * This function always return the same {@link Location} instance to avoid object allocation. - */ - public Location lookup(Object keyBase, long keyOffset, int keyLength, int hash) { - safeLookup(keyBase, keyOffset, keyLength, loc, hash); - return loc; - } - - /** - * Looks up a key, and saves the result in provided `loc`. - * - * This is a thread-safe version of `lookup`, could be used by multiple threads. - */ - public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location loc, int hash) { - assert(longArray != null); - - numKeyLookups++; - - int pos = hash & mask; - int step = 1; - while (true) { - numProbes++; - if (longArray.get(pos * 2) == 0) { - // This is a new key. - loc.with(pos, hash, false); - return; - } else { - long stored = longArray.get(pos * 2 + 1); - if ((int) (stored) == hash) { - // Full hash code matches. Let's compare the keys for equality. - loc.with(pos, hash, true); - if (loc.getKeyLength() == keyLength) { - final boolean areEqual = ByteArrayMethods.arrayEquals( - keyBase, - keyOffset, - loc.getKeyBase(), - loc.getKeyOffset(), - keyLength - ); - if (areEqual) { - return; - } - } - } - } - pos = (pos + step) & mask; - step++; - } - } - - /** - * Handle returned by {@link BytesToBytesMap#lookup(Object, long, int)} function. - */ - public final class Location { - /** An index into the hash map's Long array */ - private int pos; - /** True if this location points to a position where a key is defined, false otherwise */ - private boolean isDefined; - /** - * The hashcode of the most recent key passed to - * {@link BytesToBytesMap#lookup(Object, long, int, int)}. Caching this hashcode here allows us - * to avoid re-hashing the key when storing a value for that key. - */ - private int keyHashcode; - private Object baseObject; // the base object for key and value - private long keyOffset; - private int keyLength; - private long valueOffset; - private int valueLength; - - /** - * Memory page containing the record. Only set if created by {@link BytesToBytesMap#iterator()}. - */ - @Nullable private MemoryBlock memoryPage; - - private void updateAddressesAndSizes(long fullKeyAddress) { - updateAddressesAndSizes( - taskMemoryManager.getPage(fullKeyAddress), - taskMemoryManager.getOffsetInPage(fullKeyAddress)); - } - - private void updateAddressesAndSizes(final Object base, long offset) { - baseObject = base; - final int totalLength = UnsafeAlignedOffset.getSize(base, offset); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - offset += uaoSize; - keyLength = UnsafeAlignedOffset.getSize(base, offset); - offset += uaoSize; - keyOffset = offset; - valueOffset = offset + keyLength; - valueLength = totalLength - keyLength - uaoSize; - } - - private Location with(int pos, int keyHashcode, boolean isDefined) { - assert(longArray != null); - this.pos = pos; - this.isDefined = isDefined; - this.keyHashcode = keyHashcode; - if (isDefined) { - final long fullKeyAddress = longArray.get(pos * 2); - updateAddressesAndSizes(fullKeyAddress); - } - return this; - } - - private Location with(MemoryBlock page, long offsetInPage) { - this.isDefined = true; - this.memoryPage = page; - updateAddressesAndSizes(page.getBaseObject(), offsetInPage); - return this; - } - - /** - * This is only used for spilling - */ - private Location with(Object base, long offset, int length) { - this.isDefined = true; - this.memoryPage = null; - baseObject = base; - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - keyOffset = offset + uaoSize; - keyLength = UnsafeAlignedOffset.getSize(base, offset); - valueOffset = offset + uaoSize + keyLength; - valueLength = length - uaoSize - keyLength; - return this; - } - - /** - * Find the next pair that has the same key as current one. - */ - public boolean nextValue() { - assert isDefined; - long nextAddr = Platform.getLong(baseObject, valueOffset + valueLength); - if (nextAddr == 0) { - return false; - } else { - updateAddressesAndSizes(nextAddr); - return true; - } - } - - /** - * Returns the memory page that contains the current record. - * This is only valid if this is returned by {@link BytesToBytesMap#iterator()}. - */ - public MemoryBlock getMemoryPage() { - return this.memoryPage; - } - - /** - * Returns true if the key is defined at this position, and false otherwise. - */ - public boolean isDefined() { - return isDefined; - } - - /** - * Returns the base object for key. - */ - public Object getKeyBase() { - assert (isDefined); - return baseObject; - } - - /** - * Returns the offset for key. - */ - public long getKeyOffset() { - assert (isDefined); - return keyOffset; - } - - /** - * Returns the base object for value. - */ - public Object getValueBase() { - assert (isDefined); - return baseObject; - } - - /** - * Returns the offset for value. - */ - public long getValueOffset() { - assert (isDefined); - return valueOffset; - } - - /** - * Returns the length of the key defined at this position. - * Unspecified behavior if the key is not defined. - */ - public int getKeyLength() { - assert (isDefined); - return keyLength; - } - - /** - * Returns the length of the value defined at this position. - * Unspecified behavior if the key is not defined. - */ - public int getValueLength() { - assert (isDefined); - return valueLength; - } - - /** - * Append a new value for the key. This method could be called multiple times for a given key. - * The return value indicates whether the put succeeded or whether it failed because additional - * memory could not be acquired. - *

- * It is only valid to call this method immediately after calling `lookup()` using the same key. - *

- *

- * The key and value must be word-aligned (that is, their sizes must be a multiple of 8). - *

- *

- * After calling this method, calls to `get[Key|Value]Address()` and `get[Key|Value]Length` - * will return information on the data stored by this `append` call. - *

- *

- * As an example usage, here's the proper way to store a new key: - *

- *
-     *   Location loc = map.lookup(keyBase, keyOffset, keyLength);
-     *   if (!loc.isDefined()) {
-     *     if (!loc.append(keyBase, keyOffset, keyLength, ...)) {
-     *       // handle failure to grow map (by spilling, for example)
-     *     }
-     *   }
-     * 
- *

- * Unspecified behavior if the key is not defined. - *

- * - * @return true if the put() was successful and false if the put() failed because memory could - * not be acquired. - */ - public boolean append(Object kbase, long koff, int klen, Object vbase, long voff, int vlen) { - assert (klen % 8 == 0); - assert (vlen % 8 == 0); - assert (longArray != null); - - // We should not increase number of keys to be MAX_CAPACITY. The usage pattern of this map is - // lookup + append. If we append key until the number of keys to be MAX_CAPACITY, next time - // the call of lookup will hang forever because it cannot find an empty slot. - if (numKeys == MAX_CAPACITY - 1 - // The map could be reused from last spill (because of no enough memory to grow), - // then we don't try to grow again if hit the `growthThreshold`. - || !canGrowArray && numKeys >= growthThreshold) { - return false; - } - - // Here, we'll copy the data into our data pages. Because we only store a relative offset from - // the key address instead of storing the absolute address of the value, the key and value - // must be stored in the same memory page. - // (total length) (key length) (key) (value) (8 byte pointer to next value) - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - final long recordLength = (2L * uaoSize) + klen + vlen + 8; - if (currentPage == null || currentPage.size() - pageCursor < recordLength) { - if (!acquireNewPage(recordLength + uaoSize)) { - return false; - } - } - - // --- Append the key and value data to the current data page -------------------------------- - final Object base = currentPage.getBaseObject(); - long offset = currentPage.getBaseOffset() + pageCursor; - final long recordOffset = offset; - UnsafeAlignedOffset.putSize(base, offset, klen + vlen + uaoSize); - UnsafeAlignedOffset.putSize(base, offset + uaoSize, klen); - offset += (2L * uaoSize); - Platform.copyMemory(kbase, koff, base, offset, klen); - offset += klen; - Platform.copyMemory(vbase, voff, base, offset, vlen); - offset += vlen; - // put this value at the beginning of the list - Platform.putLong(base, offset, isDefined ? longArray.get(pos * 2) : 0); - - // --- Update bookkeeping data structures ---------------------------------------------------- - offset = currentPage.getBaseOffset(); - UnsafeAlignedOffset.putSize(base, offset, UnsafeAlignedOffset.getSize(base, offset) + 1); - pageCursor += recordLength; - final long storedKeyAddress = taskMemoryManager.encodePageNumberAndOffset( - currentPage, recordOffset); - longArray.set(pos * 2, storedKeyAddress); - updateAddressesAndSizes(storedKeyAddress); - numValues++; - if (!isDefined) { - numKeys++; - longArray.set(pos * 2 + 1, keyHashcode); - isDefined = true; - - // We use two array entries per key, so the array size is twice the capacity. - // We should compare the current capacity of the array, instead of its size. - if (numKeys >= growthThreshold && longArray.size() / 2 < MAX_CAPACITY) { - try { - growAndRehash(); - } catch (SparkOutOfMemoryError oom) { - canGrowArray = false; - } - } - } - return true; - } - } - - /** - * Acquire a new page from the memory manager. - * @return whether there is enough space to allocate the new page. - */ - private boolean acquireNewPage(long required) { - try { - currentPage = allocatePage(required); - } catch (SparkOutOfMemoryError e) { - return false; - } - dataPages.add(currentPage); - UnsafeAlignedOffset.putSize(currentPage.getBaseObject(), currentPage.getBaseOffset(), 0); - pageCursor = UnsafeAlignedOffset.getUaoSize(); - return true; - } - - @Override - public long spill(long size, MemoryConsumer trigger) throws IOException { - if (trigger != this && destructiveIterator != null) { - return destructiveIterator.spill(size); - } - return 0L; - } - - /** - * Allocate new data structures for this map. When calling this outside of the constructor, - * make sure to keep references to the old data structures so that you can free them. - * - * @param capacity the new map capacity - */ - private void allocate(int capacity) { - assert (capacity >= 0); - capacity = Math.max((int) Math.min(MAX_CAPACITY, ByteArrayMethods.nextPowerOf2(capacity)), 64); - assert (capacity <= MAX_CAPACITY); - longArray = allocateArray(capacity * 2L); - longArray.zeroOut(); - - this.growthThreshold = (int) (capacity * loadFactor); - this.mask = capacity - 1; - } - - /** - * Free all allocated memory associated with this map, including the storage for keys and values - * as well as the hash map array itself. - * - * This method is idempotent and can be called multiple times. - */ - public void free() { - updatePeakMemoryUsed(); - if (longArray != null) { - freeArray(longArray); - longArray = null; - } - Iterator dataPagesIterator = dataPages.iterator(); - while (dataPagesIterator.hasNext()) { - MemoryBlock dataPage = dataPagesIterator.next(); - dataPagesIterator.remove(); - freePage(dataPage); - } - assert(dataPages.isEmpty()); - - while (!spillWriters.isEmpty()) { - File file = spillWriters.removeFirst().getFile(); - if (file != null && file.exists()) { - if (!file.delete()) { - logger.error("Was unable to delete spill file {}", file.getAbsolutePath()); - } - } - } - } - - public TaskMemoryManager getTaskMemoryManager() { - return taskMemoryManager; - } - - public long getPageSizeBytes() { - return pageSizeBytes; - } - - /** - * Returns the total amount of memory, in bytes, consumed by this map's managed structures. - */ - public long getTotalMemoryConsumption() { - long totalDataPagesSize = 0L; - for (MemoryBlock dataPage : dataPages) { - totalDataPagesSize += dataPage.size(); - } - return totalDataPagesSize + ((longArray != null) ? longArray.memoryBlock().size() : 0L); - } - - private void updatePeakMemoryUsed() { - long mem = getTotalMemoryConsumption(); - if (mem > peakMemoryUsedBytes) { - peakMemoryUsedBytes = mem; - } - } - - /** - * Return the peak memory used so far, in bytes. - */ - public long getPeakMemoryUsedBytes() { - updatePeakMemoryUsed(); - return peakMemoryUsedBytes; - } - - /** - * Returns the average number of probes per key lookup. - */ - public double getAvgHashProbeBucketListIterations() { - return (1.0 * numProbes) / numKeyLookups; - } - - @VisibleForTesting - public int getNumDataPages() { - return dataPages.size(); - } - - /** - * Returns the underline long[] of longArray. - */ - public LongArray getArray() { - assert(longArray != null); - return longArray; - } - - /** - * Reset this map to initialized state. - */ - public void reset() { - updatePeakMemoryUsed(); - numKeys = 0; - numValues = 0; - freeArray(longArray); - longArray = null; - while (dataPages.size() > 0) { - MemoryBlock dataPage = dataPages.removeLast(); - freePage(dataPage); - } - allocate(initialCapacity); - canGrowArray = true; - currentPage = null; - pageCursor = 0; - } - - /** - * Grows the size of the hash table and re-hash everything. - */ - @VisibleForTesting - void growAndRehash() { - assert(longArray != null); - - // Store references to the old data structures to be used when we re-hash - final LongArray oldLongArray = longArray; - final int oldCapacity = (int) oldLongArray.size() / 2; - - // Allocate the new data structures - allocate(Math.min(growthStrategy.nextCapacity(oldCapacity), MAX_CAPACITY)); - - // Re-mask (we don't recompute the hashcode because we stored all 32 bits of it) - for (int i = 0; i < oldLongArray.size(); i += 2) { - final long keyPointer = oldLongArray.get(i); - if (keyPointer == 0) { - continue; - } - final int hashcode = (int) oldLongArray.get(i + 1); - int newPos = hashcode & mask; - int step = 1; - while (longArray.get(newPos * 2) != 0) { - newPos = (newPos + step) & mask; - step++; - } - longArray.set(newPos * 2, keyPointer); - longArray.set(newPos * 2 + 1, hashcode); - } - freeArray(oldLongArray); - } -} diff --git a/src/main/java/org/apache/spark/unsafe/memory/ExtendedMemoryAllocator.java b/src/main/java/org/apache/spark/unsafe/memory/ExtendedMemoryAllocator.java deleted file mode 100644 index 7223068a..00000000 --- a/src/main/java/org/apache/spark/unsafe/memory/ExtendedMemoryAllocator.java +++ /dev/null @@ -1,21 +0,0 @@ - -package org.apache.spark.unsafe.memory; -import com.intel.oap.common.unsafe.PersistentMemoryPlatform; - -public class ExtendedMemoryAllocator implements MemoryAllocator{ - - @Override - public MemoryBlock allocate(long size) throws OutOfMemoryError { - long address = PersistentMemoryPlatform.allocateVolatileMemory(size); - MemoryBlock memoryBlock = new MemoryBlock(null, address, size); - - return memoryBlock; - } - - @Override - public void free(MemoryBlock memoryBlock) { - assert (memoryBlock.getBaseObject() == null) : - "baseObject not null; are you trying to use the AEP-heap allocator to free on-heap memory?"; - PersistentMemoryPlatform.freeMemory(memoryBlock.getBaseOffset()); - } -} \ No newline at end of file diff --git a/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java b/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java deleted file mode 100644 index b47370f4..00000000 --- a/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.unsafe.memory; - -public interface MemoryAllocator { - - /** - * Whether to fill newly allocated and deallocated memory with 0xa5 and 0x5a bytes respectively. - * This helps catch misuse of uninitialized or freed memory, but imposes some overhead. - */ - boolean MEMORY_DEBUG_FILL_ENABLED = Boolean.parseBoolean( - System.getProperty("spark.memory.debugFill", "false")); - - // Same as jemalloc's debug fill values. - byte MEMORY_DEBUG_FILL_CLEAN_VALUE = (byte)0xa5; - byte MEMORY_DEBUG_FILL_FREED_VALUE = (byte)0x5a; - - /** - * Allocates a contiguous block of memory. Note that the allocated memory is not guaranteed - * to be zeroed out (call `fill(0)` on the result if this is necessary). - */ - MemoryBlock allocate(long size) throws OutOfMemoryError; - - void free(MemoryBlock memory); - - MemoryAllocator UNSAFE = new UnsafeMemoryAllocator(); - - MemoryAllocator HEAP = new HeapMemoryAllocator(); - - MemoryAllocator EXTENDED = new ExtendedMemoryAllocator(); -} diff --git a/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java deleted file mode 100644 index 7517b3a0..00000000 --- a/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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.unsafe.memory; - -import javax.annotation.Nullable; - -import org.apache.spark.unsafe.Platform; - -/** - * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. - */ -public class MemoryBlock extends MemoryLocation { - - /** Special `pageNumber` value for pages which were not allocated by TaskMemoryManagers */ - public static final int NO_PAGE_NUMBER = -1; - - /** - * Special `pageNumber` value for marking pages that have been freed in the TaskMemoryManager. - * We set `pageNumber` to this value in TaskMemoryManager.freePage() so that MemoryAllocator - * can detect if pages which were allocated by TaskMemoryManager have been freed in the TMM - * before being passed to MemoryAllocator.free() (it is an error to allocate a page in - * TaskMemoryManager and then directly free it in a MemoryAllocator without going through - * the TMM freePage() call). - */ - public static final int FREED_IN_TMM_PAGE_NUMBER = -2; - - /** - * Special `pageNumber` value for pages that have been freed by the MemoryAllocator. This allows - * us to detect double-frees. - */ - public static final int FREED_IN_ALLOCATOR_PAGE_NUMBER = -3; - - private final long length; - - /** - * Optional page number; used when this MemoryBlock represents a page allocated by a - * TaskMemoryManager. This field is public so that it can be modified by the TaskMemoryManager, - * which lives in a different package. - */ - public int pageNumber = NO_PAGE_NUMBER; - - /** - * Indicate the memory block is on extended memory or not. - */ - public boolean isExtendedMemory = false; - - public MemoryBlock(@Nullable Object obj, long offset, long length) { - super(obj, offset); - this.length = length; - } - - /** - * Returns the size of the memory block. - */ - public long size() { - return length; - } - - /** - * Creates a memory block pointing to the memory used by the long array. - */ - public static MemoryBlock fromLongArray(final long[] array) { - return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8L); - } - - /** - * Fills the memory block with the specified byte value. - */ - public void fill(byte value) { - Platform.setMemory(obj, offset, length, value); - } - - /** - * Whether this memory block is on extended memory. - * @return - */ - public boolean isExtendedMemory() { - return isExtendedMemory; - } - - /** - * set whether this memory block is on extended memory. - * @param isExtended - */ - public void isExtendedMemory(boolean isExtended) { - this.isExtendedMemory = isExtended; - } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemReader.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemReader.java deleted file mode 100644 index 1d2b1ee3..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemReader.java +++ /dev/null @@ -1,87 +0,0 @@ -package org.apache.spark.util.collection.unsafe.sort; - -import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.memory.MemoryBlock; - -import java.io.Closeable; -import java.util.LinkedList; - -public final class PMemReader extends UnsafeSorterIterator implements Closeable { - private int recordLength; - private long keyPrefix; - private int numRecordsRemaining; - private int numRecords; - private LinkedList pMemPages; - private MemoryBlock pMemPage = null; - private int readingPageIndex = 0; - private int readedRecordsInCurrentPage = 0; - private int numRecordsInpage = 0; - private long offset = 0; - private byte[] arr = new byte[1024 * 1024]; - private Object baseObject = arr; - public PMemReader(LinkedList pMemPages, int numRecords) { - this.pMemPages = pMemPages; - this.numRecordsRemaining = this.numRecords = numRecords; - } - @Override - public void loadNext() { - assert (readingPageIndex <= pMemPages.size()) - : "Illegal state: Pages finished read but hasNext() is true."; - if(pMemPage == null || readedRecordsInCurrentPage == numRecordsInpage) { - // read records from each page - pMemPage = pMemPages.get(readingPageIndex++); - readedRecordsInCurrentPage = 0; - numRecordsInpage = Platform.getInt(null, pMemPage.getBaseOffset()); - offset = pMemPage.getBaseOffset() + 4; - } - // record: BaseOffSet, record length, KeyPrefix, record value - keyPrefix = Platform.getLong(null, offset); - offset += 8; - recordLength = Platform.getInt(null, offset); - offset += 4; - if (recordLength > arr.length) { - arr = new byte[recordLength]; - baseObject = arr; - } - Platform.copyMemory(null, offset , baseObject, Platform.BYTE_ARRAY_OFFSET, recordLength); - offset += recordLength; - readedRecordsInCurrentPage ++; - numRecordsRemaining --; - - - } - @Override - public int getNumRecords() { - return numRecords; - } - - @Override - public boolean hasNext() { - return (numRecordsRemaining > 0); - } - - @Override - public Object getBaseObject() { - return baseObject; - } - - @Override - public long getBaseOffset() { - return Platform.BYTE_ARRAY_OFFSET; - } - - @Override - public int getRecordLength() { - return recordLength; - } - - @Override - public long getKeyPrefix() { - return keyPrefix; - } - - @Override - public void close() { - // do nothing here - } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemReaderForUnsafeExternalSorter.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemReaderForUnsafeExternalSorter.java deleted file mode 100644 index 4a39648b..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemReaderForUnsafeExternalSorter.java +++ /dev/null @@ -1,141 +0,0 @@ -package org.apache.spark.util.collection.unsafe.sort; - -import org.apache.spark.SparkEnv; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.internal.config.package$; -import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.util.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.Closeable; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; - -public final class PMemReaderForUnsafeExternalSorter extends UnsafeSorterIterator implements Closeable { - private static final Logger logger = LoggerFactory.getLogger(PMemReaderForUnsafeExternalSorter.class); - private int recordLength; - private long keyPrefix; - private int numRecordsRemaining; - private int numRecords; - private LongArray sortedArray; - private int position; - private byte[] arr = new byte[1024 * 1024]; - private byte[] bytes = new byte[1024 * 1024]; - private Object baseObject = arr; - private TaskMetrics taskMetrics; - private long startTime; - private ByteBuffer byteBuffer; - public PMemReaderForUnsafeExternalSorter( - LongArray sortedArray, int position, int numRecords, TaskMetrics taskMetrics) { - this.sortedArray = sortedArray; - this.position = position; - this.numRecords = numRecords; - this.numRecordsRemaining = numRecords - position/2; - this.taskMetrics = taskMetrics; - int readBufferSize = SparkEnv.get() == null? 8 * 1024 * 1024 : - (int) (long) SparkEnv.get().conf().get(package$.MODULE$.MEMORY_SPILL_PMEM_READ_BUFFERSIZE()); - logger.info("PMem read buffer size is:" + Utils.bytesToString(readBufferSize)); - this.byteBuffer = ByteBuffer.wrap(new byte[readBufferSize]); - byteBuffer.flip(); - byteBuffer.order(ByteOrder.nativeOrder()); - } - - @Override - public void loadNext() { - if (!byteBuffer.hasRemaining()) { - boolean refilled = refill(); - if (!refilled) { - logger.error("Illegal status: records finished read but hasNext() is true."); - } - } - keyPrefix = byteBuffer.getLong(); - recordLength = byteBuffer.getInt(); - if (recordLength > arr.length) { - arr = new byte[recordLength]; - baseObject = arr; - } - byteBuffer.get(arr, 0, recordLength); - numRecordsRemaining --; - } - - @Override - public int getNumRecords() { - return numRecords; - } - - /** - * load more PMem records in the buffer - */ - private boolean refill() { - byteBuffer.clear(); - int nRead = loadData(); - byteBuffer.flip(); - if (nRead <= 0) { - return false; - } - return true; - } - - private int loadData() { - // no records remaining to read - if (position >= numRecords * 2) - return -1; - int bufferPos = 0; - int capacity = byteBuffer.capacity(); - while (bufferPos < capacity && position < numRecords * 2) { - long curRecordAddress = sortedArray.get(position); - int recordLen = Platform.getInt(null, curRecordAddress); - // length + keyprefix + record length - int length = Integer.BYTES + Long.BYTES + recordLen; - if (length > capacity) { - logger.error("single record size exceeds PMem read buffer. Please increase buffer size."); - } - if (bufferPos + length <= capacity) { - long curKeyPrefix = sortedArray.get(position + 1); - if (length > bytes.length) { - bytes = new byte[length]; - } - Platform.putLong(bytes, Platform.BYTE_ARRAY_OFFSET, curKeyPrefix); - Platform.copyMemory(null, curRecordAddress, bytes, Platform.BYTE_ARRAY_OFFSET + Long.BYTES, length - Long.BYTES); - byteBuffer.put(bytes, 0, length); - bufferPos += length; - position += 2; - } else { - break; - } - } - return bufferPos; - } - - @Override - public boolean hasNext() { - return (numRecordsRemaining > 0); - } - - @Override - public Object getBaseObject() { - return baseObject; - } - - @Override - public long getBaseOffset() { - return Platform.BYTE_ARRAY_OFFSET; - } - - @Override - public int getRecordLength() { - return recordLength; - } - - @Override - public long getKeyPrefix() { - return keyPrefix; - } - - @Override - public void close() { - // do nothing here - } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemSpillWriterFactory.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemSpillWriterFactory.java deleted file mode 100644 index 97d7c61d..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemSpillWriterFactory.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -import org.apache.spark.memory.MemoryConsumer; -import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.serializer.SerializerManager; -import org.apache.spark.storage.BlockManager; - -import java.io.IOException; - -public class PMemSpillWriterFactory { - public static SpillWriterForUnsafeSorter getSpillWriter( - PMemSpillWriterType writerType, - UnsafeExternalSorter externalSorter, - UnsafeSorterIterator sortedIterator, - int numberOfRecordsToWritten, - SerializerManager serializerManager, - BlockManager blockManager, - int fileBufferSize, - ShuffleWriteMetrics writeMetrics, - TaskMetrics taskMetrics, - boolean spillToPMEMEnabled, - boolean isSorted) throws IOException { - if (spillToPMEMEnabled && writerType == PMemSpillWriterType.MEM_COPY_ALL_DATA_PAGES_TO_PMEM_WITHLONGARRAY){ - SortedIteratorForSpills sortedSpillIte = SortedIteratorForSpills.createFromExistingSorterIte( - (UnsafeInMemorySorter.SortedIterator)sortedIterator, - externalSorter.getInMemSorter()); - return new PMemWriter( - externalSorter, - sortedSpillIte, - isSorted, - numberOfRecordsToWritten, - serializerManager, - blockManager, - fileBufferSize, - writeMetrics, - taskMetrics); - } else { - if (sortedIterator == null) { - sortedIterator = externalSorter.getInMemSorter().getSortedIterator(); - } - if (spillToPMEMEnabled && sortedIterator instanceof UnsafeInMemorySorter.SortedIterator){ - - if (writerType == PMemSpillWriterType.WRITE_SORTED_RECORDS_TO_PMEM) { - SortedIteratorForSpills sortedSpillIte = SortedIteratorForSpills.createFromExistingSorterIte( - (UnsafeInMemorySorter.SortedIterator)sortedIterator, - externalSorter.getInMemSorter()); - return new SortedPMemPageSpillWriter( - externalSorter, - sortedSpillIte, - numberOfRecordsToWritten, - serializerManager, - blockManager, - fileBufferSize, - writeMetrics, - taskMetrics); - } - if (spillToPMEMEnabled && writerType == PMemSpillWriterType.STREAM_SPILL_TO_PMEM) { - return new UnsafeSorterStreamSpillWriter( - blockManager, - fileBufferSize, - sortedIterator, - numberOfRecordsToWritten, - serializerManager, - writeMetrics, - taskMetrics); - } - } else { - return new UnsafeSorterSpillWriter( - blockManager, - fileBufferSize, - sortedIterator, - numberOfRecordsToWritten, - serializerManager, - writeMetrics, - taskMetrics); - } - - } - return null; - } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemSpillWriterType.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemSpillWriterType.java deleted file mode 100644 index e6bad61d..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemSpillWriterType.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -public enum PMemSpillWriterType { - STREAM_SPILL_TO_PMEM, - MEM_COPY_ALL_DATA_PAGES_TO_PMEM, - MEM_COPY_ALL_DATA_PAGES_TO_PMEM_WITHLONGARRAY, - WRITE_SORTED_RECORDS_TO_PMEM -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemWriter.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemWriter.java deleted file mode 100644 index 996d8f35..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/PMemWriter.java +++ /dev/null @@ -1,242 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -import com.intel.oap.common.unsafe.PersistentMemoryPlatform; - -import org.apache.spark.SparkEnv; -import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.internal.config.package$; -import org.apache.spark.serializer.SerializerManager; -import org.apache.spark.storage.BlockManager; -import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.unsafe.memory.MemoryBlock; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; - -/** - * In this writer, records page along with LongArray page are both dumped to PMem when spill happens - */ -public final class PMemWriter extends UnsafeSorterPMemSpillWriter { - private static final Logger logger = LoggerFactory.getLogger(PMemWriter.class); - private LongArray sortedArray; - private HashMap pageMap = new HashMap<>(); - private int position; - private LinkedList allocatedDramPages; - private MemoryBlock pMemPageForLongArray; - private UnsafeSorterSpillWriter diskSpillWriter; - private BlockManager blockManager; - private SerializerManager serializerManager; - private int fileBufferSize; - private boolean isSorted; - private int totalRecordsWritten; - private final boolean spillToPMemConcurrently = SparkEnv.get() != null && (boolean) SparkEnv.get().conf().get( - package$.MODULE$.MEMORY_SPILL_PMEM_SORT_BACKGROUND()); - private final boolean pMemClflushEnabled = SparkEnv.get() != null && - (boolean)SparkEnv.get().conf().get(package$.MODULE$.MEMORY_SPILL_PMEM_CLFLUSH_ENABLED()); - - public PMemWriter( - UnsafeExternalSorter externalSorter, - SortedIteratorForSpills sortedIterator, - boolean isSorted, - int numberOfRecordsToWritten, - SerializerManager serializerManager, - BlockManager blockManager, - int fileBufferSize, - ShuffleWriteMetrics writeMetrics, - TaskMetrics taskMetrics) { - // SortedIterator is null or readingIterator from UnsafeExternalSorter. - // But it isn't used in this PMemWriter, only for keep same constructor with other spill writers. - super(externalSorter, sortedIterator, numberOfRecordsToWritten, writeMetrics, taskMetrics); - this.allocatedDramPages = externalSorter.getAllocatedPages(); - this.blockManager = blockManager; - this.serializerManager = serializerManager; - this.fileBufferSize = fileBufferSize; - this.isSorted = isSorted; - // In the case that spill happens when iterator isn't sorted yet, the valid records - // will be [0, inMemsorter.numRecords]. When iterator is sorted, the valid records will be - // [position/2, inMemsorter.numRecords] - this.totalRecordsWritten = externalSorter.getInMemSorter().numRecords(); - } - - @Override - public void write() throws IOException { - // write records based on externalsorter - // try to allocate all needed PMem pages before spill to PMem - UnsafeInMemorySorter inMemSorter = externalSorter.getInMemSorter(); - if (allocatePMemPages(allocatedDramPages, inMemSorter.getArray().memoryBlock())) { - if (spillToPMemConcurrently && !isSorted) { - logger.info("Concurrent PMem write/records sort"); - long writeDuration = 0; - ExecutorService executorService = Executors.newSingleThreadExecutor(); - Future future = executorService.submit(()->dumpPagesToPMem()); - inMemSorter.getSortedIterator(); - try { - writeDuration = future.get(); - } catch (InterruptedException | ExecutionException e) { - logger.error(e.getMessage()); - } - executorService.shutdownNow(); - updateLongArray(inMemSorter.getArray(), totalRecordsWritten, 0); - } else if(!isSorted) { - dumpPagesToPMem(); - // get sorted iterator - inMemSorter.getSortedIterator(); - // update LongArray - updateLongArray(inMemSorter.getArray(), totalRecordsWritten, 0); - } else { - dumpPagesToPMem(); - // get sorted iterator - assert(sortedIterator != null); - updateLongArray(inMemSorter.getArray(), totalRecordsWritten, sortedIterator.getPosition()); - } - } else { - // fallback to disk spill - if (diskSpillWriter == null) { - diskSpillWriter = new UnsafeSorterSpillWriter( - blockManager, - fileBufferSize, - isSorted? sortedIterator : inMemSorter.getSortedIterator(), - numberOfRecordsToWritten, - serializerManager, - writeMetrics, - taskMetrics); - } - diskSpillWriter.write(false); - } - } - - public boolean allocatePMemPages(LinkedList dramPages, MemoryBlock longArrayPage) { - for (MemoryBlock page: dramPages) { - MemoryBlock pMemBlock = taskMemoryManager.allocatePMemPage(page.size()); - if (pMemBlock != null) { - allocatedPMemPages.add(pMemBlock); - pageMap.put(page, pMemBlock); - } else { - freeAllPMemPages(); - pageMap.clear(); - return false; - } - } - pMemPageForLongArray = taskMemoryManager.allocatePMemPage(longArrayPage.size()); - if (pMemPageForLongArray != null) { - allocatedPMemPages.add(pMemPageForLongArray); - pageMap.put(longArrayPage, pMemPageForLongArray); - } else { - freeAllPMemPages(); - pageMap.clear(); - return false; - } - return (allocatedPMemPages.size() == dramPages.size() + 1); - } - - private long dumpPagesToPMem() { - long dumpTime = System.nanoTime(); - for (MemoryBlock page : allocatedDramPages) { - dumpPageToPMem(page); - } - long dumpDuration = System.nanoTime() - dumpTime; - return dumpDuration; - - } - - private void dumpPageToPMem(MemoryBlock page) { - MemoryBlock pMemBlock = pageMap.get(page); - if (pMemBlock == null) { - logger.error("Fail to copy data to PMem as no corresponding PMem page found."); - } else { - PersistentMemoryPlatform.copyMemory( - page.getBaseObject(), page.getBaseOffset(), - null, pMemBlock.getBaseOffset(), page.size(), - pMemClflushEnabled); - writeMetrics.incBytesWritten(page.size()); - } - } - - public void updateLongArray(LongArray sortedArray, int numRecords, int position) { - this.position = position; - while (position < numRecords * 2){ - // update recordPointer in this array - long originalRecordPointer = sortedArray.get(position); - MemoryBlock page = taskMemoryManager.getOriginalPage(originalRecordPointer); - long baseOffset = page == null? 0: page.getBaseOffset(); - long offset = taskMemoryManager.getOffsetInPage(originalRecordPointer) - baseOffset; - MemoryBlock pMemBlock = pageMap.get(page); - if (pMemBlock == null) { - logger.error("No PMem page found corresponding with LongArrayPage"); - } else { - long pMemOffset = pMemBlock.getBaseOffset() + offset; - sortedArray.set(position, pMemOffset); - position += 2; - } - } - // copy the LongArray to PMem - MemoryBlock arrayBlock = sortedArray.memoryBlock(); - MemoryBlock pMemBlock = pageMap.get(arrayBlock); - if (pMemBlock == null) { - logger.error("failed to update LongArray as no pair PMem page found."); - } else { - PersistentMemoryPlatform.copyMemory( - arrayBlock.getBaseObject(), arrayBlock.getBaseOffset(), - null, pMemBlock.getBaseOffset(), arrayBlock.size(), - pMemClflushEnabled); - writeMetrics.incBytesWritten(pMemBlock.size()); - this.sortedArray = new LongArray(pMemBlock); - } - } - - @Override - public UnsafeSorterIterator getSpillReader() throws IOException { - // TODO: consider partial spill to PMem + Disk. - if (diskSpillWriter != null) { - return diskSpillWriter.getSpillReader(); - } else { - return new PMemReaderForUnsafeExternalSorter(sortedArray, position, totalRecordsWritten, taskMetrics); - } - } - - public void clearAll() { - freeAllPMemPages(); - if (diskSpillWriter != null) { - diskSpillWriter.clearAll(); - } - } - - @Override - public int recordsSpilled() { - return numberOfRecordsToWritten; - } - - @Override - public void freeAllPMemPages() { - for( MemoryBlock page: allocatedPMemPages) { - taskMemoryManager.freePMemPage(page, externalSorter); - } - allocatedPMemPages.clear(); - } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/SortedIteratorForSpills.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/SortedIteratorForSpills.java deleted file mode 100644 index 5c2961fd..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/SortedIteratorForSpills.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -import org.apache.spark.TaskContext; -import org.apache.spark.memory.TaskMemoryManager; -import org.apache.spark.unsafe.UnsafeAlignedOffset; -import org.apache.spark.unsafe.array.LongArray; - -public class SortedIteratorForSpills extends UnsafeSorterIterator { - private LongArray sortedArray; - private final int numRecords; - private int position; - private int offset; - private Object baseObject; - private long baseOffset; - private long keyPrefix; - private int recordLength; - private long currentPageNumber; - private final TaskContext taskContext = TaskContext.get(); - private final TaskMemoryManager memoryManager; - - /** - * Construct an iterator to read the spill. - * @param array the array here should be already sorted. - * @param numRecords the number of the records recorded in this LongArray - * @param offset - */ - public SortedIteratorForSpills( - final TaskMemoryManager memoryManager, - LongArray array, - int numRecords, - int offset) { - this.memoryManager = memoryManager; - this.sortedArray = array; - this.numRecords = numRecords; - this.position = 0; - this.offset = offset; - } - - public static SortedIteratorForSpills createFromExistingSorterIte( - UnsafeInMemorySorter.SortedIterator sortedIte, - UnsafeInMemorySorter inMemSorter) { - if (sortedIte == null) { - return null; - } - TaskMemoryManager taskMemoryManager = inMemSorter.getTaskMemoryManager(); - LongArray array = inMemSorter.getLongArray(); - int numberRecords = sortedIte.getNumRecords(); - int offset = sortedIte.getOffset(); - int position = sortedIte.getPosition(); - SortedIteratorForSpills spillIte = new SortedIteratorForSpills(taskMemoryManager, array,numberRecords,offset); - spillIte.pointTo(position); - return spillIte; - } - - @Override - public int getNumRecords() { - return numRecords; - } - - @Override - public boolean hasNext() { - return position / 2 < numRecords; - } - - @Override - public void loadNext() { - // Kill the task in case it has been marked as killed. This logic is from - // InterruptibleIterator, but we inline it here instead of wrapping the iterator in order - // to avoid performance overhead. This check is added here in `loadNext()` instead of in - // `hasNext()` because it's technically possible for the caller to be relying on - // `getNumRecords()` instead of `hasNext()` to know when to stop. - if (taskContext != null) { - taskContext.killTaskIfInterrupted(); - } - loadPosition(); - } - /** - * load the record of current position and move to end of the record. - */ - private void loadPosition() { - // This pointer points to a 4-byte record length, followed by the record's bytes - final long recordPointer = sortedArray.get(offset + position); - currentPageNumber = TaskMemoryManager.decodePageNumber(recordPointer); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - baseObject = memoryManager.getPage(recordPointer); - // Skip over record length - baseOffset = memoryManager.getOffsetInPage(recordPointer) + uaoSize; - recordLength = UnsafeAlignedOffset.getSize(baseObject, baseOffset - uaoSize); - keyPrefix = sortedArray.get(offset + position + 1); - position += 2; - } - - /** - * point to a given position. - * @param pos - */ - public void pointTo(int pos) { - if (pos % 2 != 0) { - throw new IllegalArgumentException("Can't point to the middle of a record."); - } - position = pos; - } - - @Override - public Object getBaseObject() { return baseObject; } - - @Override - public long getBaseOffset() { return baseOffset; } - - public long getCurrentPageNumber() { return currentPageNumber; } - - @Override - public int getRecordLength() { return recordLength; } - - @Override - public long getKeyPrefix() { return keyPrefix; } - - public LongArray getLongArray() { return sortedArray; } - - public int getPosition() { return position; } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/SortedPMemPageSpillWriter.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/SortedPMemPageSpillWriter.java deleted file mode 100644 index bd8ca9c8..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/SortedPMemPageSpillWriter.java +++ /dev/null @@ -1,286 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -import com.intel.oap.common.unsafe.PersistentMemoryPlatform; - -import org.apache.spark.SparkEnv; -import org.apache.spark.internal.config.package$; -import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.UnsafeAlignedOffset; -import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.storage.BlockManager; -import org.apache.spark.serializer.SerializerManager; -import java.io.IOException; -import java.util.LinkedHashMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SortedPMemPageSpillWriter extends UnsafeSorterPMemSpillWriter { - private static final Logger sorted_logger = LoggerFactory.getLogger(SortedPMemPageSpillWriter.class); - private MemoryBlock currentPMemPage = null; - private long currentOffsetInPage = 0L; - private int currentNumOfRecordsInPage = 0; - //Page -> record number map - private LinkedHashMap pageNumOfRecMap = new LinkedHashMap(); - private int numRecords = 0; - private int numRecordsOnPMem = 0; - - private BlockManager blockManager; - private SerializerManager serializerManager; - private int fileBufferSize = 0; - private UnsafeSorterSpillWriter diskSpillWriter; - - private final boolean pMemClflushEnabled = SparkEnv.get() != null && - (boolean)SparkEnv.get().conf().get(package$.MODULE$.MEMORY_SPILL_PMEM_CLFLUSH_ENABLED()); - - public SortedPMemPageSpillWriter( - UnsafeExternalSorter externalSorter, - SortedIteratorForSpills sortedIterator, - int numberOfRecordsToWritten, - SerializerManager serializerManager, - BlockManager blockManager, - int fileBufferSize, - ShuffleWriteMetrics writeMetrics, - TaskMetrics taskMetrics) { - super(externalSorter, sortedIterator, numberOfRecordsToWritten, writeMetrics, taskMetrics); - this.blockManager = blockManager; - this.serializerManager = serializerManager; - this.fileBufferSize = fileBufferSize; - } - - @Override - public void write() throws IOException { - boolean allBeWritten = writeToPMem(); - if (!allBeWritten) { - sorted_logger.debug("No more PMEM space available. Write left spills to disk."); - writeToDisk(); - } - } - - /** - * @return if all records have been write to PMem, return true. Otherwise, return false. - * @throws IOException - */ - private boolean writeToPMem() throws IOException { - while (sortedIterator.hasNext()) { - sortedIterator.loadNext(); - final Object baseObject = sortedIterator.getBaseObject(); - final long baseOffset = sortedIterator.getBaseOffset(); - int curRecLen = sortedIterator.getRecordLength(); - long curPrefix = sortedIterator.getKeyPrefix(); - if (needNewPMemPage(curRecLen)) { - currentPMemPage = allocatePMemPage(); - } - if (currentPMemPage != null) { - long pageBaseOffset = currentPMemPage.getBaseOffset(); - long curPMemOffset = pageBaseOffset + currentOffsetInPage; - writeRecordToPMem(baseObject, baseOffset, curRecLen, curPrefix, curPMemOffset); - currentNumOfRecordsInPage ++; - pageNumOfRecMap.put(currentPMemPage, currentNumOfRecordsInPage); - numRecords ++; - } else { - //No more PMem space available, current loaded record can't be written to PMem. - return false; - } - } - //All records have been written to PMem. - return true; - } - - private void writeToDisk() throws IOException{ - int numOfRecLeft = numberOfRecordsToWritten - numRecordsOnPMem; - if (diskSpillWriter == null) { - diskSpillWriter = new UnsafeSorterSpillWriter( - blockManager, - fileBufferSize, - sortedIterator, - numOfRecLeft, - serializerManager, - writeMetrics, - taskMetrics); - } - diskSpillWriter.write(true); - sorted_logger.info("Num of rec {}; Num of rec written to PMem {}; still {} records left; num of rec written to disk {}.", - sortedIterator.getNumRecords(), - numRecordsOnPMem, - numOfRecLeft, - diskSpillWriter.recordsSpilled()); - } - - private boolean needNewPMemPage(int nextRecLen) { - if (allocatedPMemPages.isEmpty()) { - return true; - } - else { - long pageBaseOffset = currentPMemPage.getBaseOffset(); - long leftLenInCurPage = currentPMemPage.size() - currentOffsetInPage; - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - long recSizeRequired = uaoSize + Long.BYTES + nextRecLen; - if (leftLenInCurPage < recSizeRequired) { - return true; - } - } - return false; - } - - private void writeRecordToPMem(Object baseObject, long baseOffset, int recLength, long prefix, long pMemOffset){ - Platform.putInt( - null, - pMemOffset, - recLength); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - long currentOffset = pMemOffset + uaoSize; - Platform.putLong( - null, - currentOffset, - prefix); - currentOffset += Long.BYTES; - PersistentMemoryPlatform.copyMemory( - baseObject, - baseOffset, - null, - currentOffset, - recLength, - pMemClflushEnabled); - numRecordsOnPMem ++; - currentOffsetInPage += uaoSize + Long.BYTES + recLength; - } - - protected MemoryBlock allocatePMemPage() throws IOException{ - currentPMemPage = super.allocatePMemPage(); - currentOffsetInPage = 0; - currentNumOfRecordsInPage = 0; - return currentPMemPage; - } - - @Override - public UnsafeSorterIterator getSpillReader() throws IOException { - return new SortedPMemPageSpillReader(); - } - - @Override - public void clearAll() { - freeAllPMemPages(); - if (diskSpillWriter != null) { - diskSpillWriter.clearAll(); - } - } - - public int recordsSpilled() { - int recordsSpilledOnDisk = 0; - if (diskSpillWriter != null) { - recordsSpilledOnDisk = diskSpillWriter.recordsSpilled(); - } - return numRecordsOnPMem + recordsSpilledOnDisk; - } - - private class SortedPMemPageSpillReader extends UnsafeSorterIterator { - private final Logger sorted_reader_logger = LoggerFactory.getLogger(SortedPMemPageSpillReader.class); - private MemoryBlock curPage = null; - private int curPageIdx = -1; - private int curOffsetInPage = 0; - private int curNumOfRecInPage = 0; - private int curNumOfRec = 0; - private Object baseObject = null; - private long baseOffset = 0; - private int recordLength; - private long keyPrefix; - private UnsafeSorterIterator diskSpillReader; - private int numRecordsOnDisk = 0; - - public SortedPMemPageSpillReader() throws IOException{ - if (diskSpillWriter != null) { - diskSpillReader = diskSpillWriter.getSpillReader(); - numRecordsOnDisk = diskSpillWriter.recordsSpilled(); - } - } - @Override - public boolean hasNext() { - return curNumOfRec < numRecordsOnPMem + numRecordsOnDisk; - } - @Override - public void loadNext() throws IOException { - if(curNumOfRec < numRecordsOnPMem) { - loadNextOnPMem(); - } else { - loadNextOnDisk(); - } - } - - private void loadNextOnPMem() throws IOException { - if (curPage == null || curNumOfRecInPage >= pageNumOfRecMap.get(curPage)) { - moveToNextPMemPage(); - } - long curPageBaseOffset = curPage.getBaseOffset(); - recordLength = UnsafeAlignedOffset.getSize(null, curPageBaseOffset + curOffsetInPage); - curOffsetInPage += UnsafeAlignedOffset.getUaoSize(); - keyPrefix = Platform.getLong(null, curPageBaseOffset + curOffsetInPage); - curOffsetInPage += Long.BYTES; - baseOffset = curPageBaseOffset + curOffsetInPage; - curOffsetInPage += recordLength; - curNumOfRecInPage ++; - curNumOfRec ++; - } - - private void loadNextOnDisk() throws IOException { - if (diskSpillReader != null && diskSpillReader.hasNext()) { - diskSpillReader.loadNext(); - baseObject = diskSpillReader.getBaseObject(); - baseOffset = diskSpillReader.getBaseOffset(); - recordLength = diskSpillReader.getRecordLength(); - keyPrefix = diskSpillReader.getKeyPrefix(); - curNumOfRec ++; - } - } - - private void moveToNextPMemPage() { - curPageIdx++; - curPage = allocatedPMemPages.get(curPageIdx); - curOffsetInPage = 0; - curNumOfRecInPage = 0; - } - - @Override - public Object getBaseObject() { - return baseObject; - } - - @Override - public long getBaseOffset() { - return baseOffset; - } - - @Override - public int getRecordLength() { - return recordLength; - } - - @Override - public long getKeyPrefix() { - return keyPrefix; - } - - @Override - public int getNumRecords() { - return numRecordsOnPMem + numRecordsOnDisk; - } - } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/SpillWriterForUnsafeSorter.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/SpillWriterForUnsafeSorter.java deleted file mode 100644 index 06a670e8..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/SpillWriterForUnsafeSorter.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -import java.io.IOException; - -public interface SpillWriterForUnsafeSorter { - //write down all the spills. - public void write() throws IOException; - - //get reader for the spill maintained by this writer. - public UnsafeSorterIterator getSpillReader() throws IOException; - - //clear all acquired resource after read is done. - public void clearAll(); - - //get spilled record number. - public int recordsSpilled(); -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java deleted file mode 100644 index 9f27c80b..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ /dev/null @@ -1,741 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.LinkedList; -import java.util.Queue; -import java.util.function.Supplier; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.spark.SparkEnv; -import org.apache.spark.internal.config.package$; -import org.apache.spark.memory.SparkOutOfMemoryError; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.SparkEnv; -import org.apache.spark.internal.config.package$; -import org.apache.spark.TaskContext; -import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.memory.MemoryConsumer; -import org.apache.spark.memory.TaskMemoryManager; -import org.apache.spark.memory.TooLargePageException; -import org.apache.spark.serializer.SerializerManager; -import org.apache.spark.storage.BlockManager; -import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.UnsafeAlignedOffset; -import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.util.Utils; - -/** - * External sorter based on {@link UnsafeInMemorySorter}. - */ -public final class UnsafeExternalSorter extends MemoryConsumer { - - private static final Logger logger = LoggerFactory.getLogger(UnsafeExternalSorter.class); - - @Nullable - private final PrefixComparator prefixComparator; - - /** - * {@link RecordComparator} may probably keep the reference to the records they compared last - * time, so we should not keep a {@link RecordComparator} instance inside - * {@link UnsafeExternalSorter}, because {@link UnsafeExternalSorter} is referenced by - * {@link TaskContext} and thus can not be garbage collected until the end of the task. - */ - @Nullable - private final Supplier recordComparatorSupplier; - - private final TaskMemoryManager taskMemoryManager; - private final BlockManager blockManager; - private final SerializerManager serializerManager; - private final TaskContext taskContext; - - /** The buffer size to use when writing spills using DiskBlockObjectWriter */ - private final int fileBufferSizeBytes; - - /** - * Force this sorter to spill when there are this many elements in memory. - */ - private final int numElementsForSpillThreshold; - private final boolean spillToPMemEnabled = SparkEnv.get() != null && (boolean) SparkEnv.get().conf().get( - package$.MODULE$.MEMORY_SPILL_PMEM_ENABLED()); - /** - * spillWriterType - */ - private String spillWriterType = null; - /** - * Memory pages that hold the records being sorted. The pages in this list are freed when - * spilling, although in principle we could recycle these pages across spills (on the other hand, - * this might not be necessary if we maintained a pool of re-usable pages in the TaskMemoryManager - * itself). - */ - private final LinkedList allocatedPages = new LinkedList<>(); - - private final LinkedList spillWriters = new LinkedList<>(); - - // These variables are reset after spilling: - @Nullable private volatile UnsafeInMemorySorter inMemSorter; - - private MemoryBlock currentPage = null; - private long pageCursor = -1; - private long peakMemoryUsedBytes = 0; - private long totalSpillBytes = 0L; - private long totalSortTimeNanos = 0L; - private volatile SpillableIterator readingIterator = null; - - public static UnsafeExternalSorter createWithExistingInMemorySorter( - TaskMemoryManager taskMemoryManager, - BlockManager blockManager, - SerializerManager serializerManager, - TaskContext taskContext, - Supplier recordComparatorSupplier, - PrefixComparator prefixComparator, - int initialSize, - long pageSizeBytes, - int numElementsForSpillThreshold, - UnsafeInMemorySorter inMemorySorter) throws IOException { - UnsafeExternalSorter sorter = new UnsafeExternalSorter(taskMemoryManager, blockManager, - serializerManager, taskContext, recordComparatorSupplier, prefixComparator, initialSize, - pageSizeBytes, numElementsForSpillThreshold, inMemorySorter, false /* ignored */); - sorter.spill(Long.MAX_VALUE, sorter); - // The external sorter will be used to insert records, in-memory sorter is not needed. - sorter.inMemSorter = null; - return sorter; - } - - public static UnsafeExternalSorter create( - TaskMemoryManager taskMemoryManager, - BlockManager blockManager, - SerializerManager serializerManager, - TaskContext taskContext, - Supplier recordComparatorSupplier, - PrefixComparator prefixComparator, - int initialSize, - long pageSizeBytes, - int numElementsForSpillThreshold, - boolean canUseRadixSort) { - return new UnsafeExternalSorter(taskMemoryManager, blockManager, serializerManager, - taskContext, recordComparatorSupplier, prefixComparator, initialSize, pageSizeBytes, - numElementsForSpillThreshold, null, canUseRadixSort); - } - - private UnsafeExternalSorter( - TaskMemoryManager taskMemoryManager, - BlockManager blockManager, - SerializerManager serializerManager, - TaskContext taskContext, - Supplier recordComparatorSupplier, - PrefixComparator prefixComparator, - int initialSize, - long pageSizeBytes, - int numElementsForSpillThreshold, - @Nullable UnsafeInMemorySorter existingInMemorySorter, - boolean canUseRadixSort) { - super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); - this.taskMemoryManager = taskMemoryManager; - this.blockManager = blockManager; - this.serializerManager = serializerManager; - this.taskContext = taskContext; - this.recordComparatorSupplier = recordComparatorSupplier; - this.prefixComparator = prefixComparator; - // Use getSizeAsKb (not bytes) to maintain backwards compatibility for units - // this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024 - this.fileBufferSizeBytes = 32 * 1024; - SparkEnv sparkEnv = SparkEnv.get(); - if (sparkEnv != null && sparkEnv.conf() != null){ - this.spillWriterType = sparkEnv.conf().get(package$.MODULE$.USAFE_EXTERNAL_SORTER_SPILL_WRITE_TYPE()); - } else { - this.spillWriterType = PMemSpillWriterType.WRITE_SORTED_RECORDS_TO_PMEM.toString(); - } - if (existingInMemorySorter == null) { - RecordComparator comparator = null; - if (recordComparatorSupplier != null) { - comparator = recordComparatorSupplier.get(); - } - this.inMemSorter = new UnsafeInMemorySorter( - this, - taskMemoryManager, - comparator, - prefixComparator, - initialSize, - canUseRadixSort); - } else { - this.inMemSorter = existingInMemorySorter; - } - this.peakMemoryUsedBytes = getMemoryUsage(); - this.numElementsForSpillThreshold = numElementsForSpillThreshold; - - // Register a cleanup task with TaskContext to ensure that memory is guaranteed to be freed at - // the end of the task. This is necessary to avoid memory leaks in when the downstream operator - // does not fully consume the sorter's output (e.g. sort followed by limit). - taskContext.addTaskCompletionListener(context -> { - cleanupResources(); - }); - } - - /** - * Marks the current page as no-more-space-available, and as a result, either allocate a - * new page or spill when we see the next record. - */ - @VisibleForTesting - public void closeCurrentPage() { - if (currentPage != null) { - pageCursor = currentPage.getBaseOffset() + currentPage.size(); - } - } - - /** - * Sort and spill the current records in response to memory pressure. - */ - @Override - public long spill(long size, MemoryConsumer trigger) throws IOException { - if (trigger != this) { - if (readingIterator != null) { - return readingIterator.spill(); - } - return 0L; // this should throw exception - } - - if (inMemSorter == null || inMemSorter.numRecords() <= 0) { - return 0L; - } - - logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)", - Thread.currentThread().getId(), - Utils.bytesToString(getMemoryUsage()), - spillWriters.size() , - spillWriters.size() > 1 ? " times" : " time"); - ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); - // Sorting records or not will be handled by different spill writer, here null is given instead. - spillWithWriter(null, inMemSorter.numRecords(), writeMetrics, false); - final long spillSize = freeMemory(); - inMemSorter.reset(); - // Note that this is more-or-less going to be a multiple of the page size, so wasted space in - // pages will currently be counted as memory spilled even though that space isn't actually - // written to disk. This also counts the space needed to store the sorter's pointer array. - // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the - // records. Otherwise, if the task is over allocated memory, then without freeing the memory - // pages, we might not be able to get memory for the pointer array. - taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); - taskContext.taskMetrics().incDiskBytesSpilled(writeMetrics.bytesWritten()); - totalSpillBytes += spillSize; - return spillSize; - } - - //Todo: It's confusing to pass in ShuffleWriteMetrics here. Will reconsider and fix it later - public SpillWriterForUnsafeSorter spillWithWriter( - UnsafeSorterIterator sortedIterator, - int numberOfRecordsToWritten, - ShuffleWriteMetrics writeMetrics, - boolean isSorted) throws IOException { - PMemSpillWriterType writerType = PMemSpillWriterType.valueOf(spillWriterType); - logger.info("PMemSpillWriterType:{}",writerType.toString()); - final SpillWriterForUnsafeSorter spillWriter = PMemSpillWriterFactory.getSpillWriter( - writerType, - this, - sortedIterator, - numberOfRecordsToWritten, - serializerManager, - blockManager, - fileBufferSizeBytes, - writeMetrics, - taskContext.taskMetrics(), - spillToPMemEnabled, - isSorted); - if (spillWriter == null) { - logger.error("failed to get SpillWriter, please check related configuration."); - return null; - } else { - spillWriter.write(); - spillWriters.add(spillWriter); - return spillWriter; - } - } - - /** - * Return the total memory usage of this sorter, including the data pages and the sorter's pointer - * array. - */ - private long getMemoryUsage() { - long totalPageSize = 0; - for (MemoryBlock page : allocatedPages) { - totalPageSize += page.size(); - } - return ((inMemSorter == null) ? 0 : inMemSorter.getMemoryUsage()) + totalPageSize; - } - - private void updatePeakMemoryUsed() { - long mem = getMemoryUsage(); - if (mem > peakMemoryUsedBytes) { - peakMemoryUsedBytes = mem; - } - } - - /** - * Return the peak memory used so far, in bytes. - */ - public long getPeakMemoryUsedBytes() { - updatePeakMemoryUsed(); - return peakMemoryUsedBytes; - } - - /** - * @return the total amount of time spent sorting data (in-memory only). - */ - public long getSortTimeNanos() { - UnsafeInMemorySorter sorter = inMemSorter; - if (sorter != null) { - return sorter.getSortTimeNanos(); - } - return totalSortTimeNanos; - } - - /** - * Return the total number of bytes that has been spilled into disk so far. - */ - public long getSpillSize() { - return totalSpillBytes; - } - - @VisibleForTesting - public int getNumberOfAllocatedPages() { - return allocatedPages.size(); - } - - public LinkedList getAllocatedPages() { - return this.allocatedPages; - } - /** - * Free this sorter's data pages. - * - * @return the number of bytes freed. - */ - private long freeMemory() { - updatePeakMemoryUsed(); - long memoryFreed = 0; - for (MemoryBlock block : allocatedPages) { - memoryFreed += block.size(); - freePage(block); - } - allocatedPages.clear(); - currentPage = null; - pageCursor = 0; - return memoryFreed; - } - - - private void freeSpills() { - for (SpillWriterForUnsafeSorter spillWriter : spillWriters) { - spillWriter.clearAll(); - } - spillWriters.clear(); - } - - /** - * Frees this sorter's in-memory data structures and cleans up its spill files. - */ - public void cleanupResources() { - synchronized (this) { - freeSpills(); - freeMemory(); - if (inMemSorter != null) { - inMemSorter.free(); - inMemSorter = null; - } - } - } - - /** - * Checks whether there is enough space to insert an additional record in to the sort pointer - * array and grows the array if additional space is required. If the required space cannot be - * obtained, then the in-memory data will be spilled to disk. - */ - private void growPointerArrayIfNecessary() throws IOException { - assert(inMemSorter != null); - if (!inMemSorter.hasSpaceForAnotherRecord()) { - long used = inMemSorter.getMemoryUsage(); - LongArray array; - try { - // could trigger spilling - array = allocateArray(used / 8 * 2); - } catch (TooLargePageException e) { - // The pointer array is too big to fix in a single page, spill. - spill(); - return; - } catch (SparkOutOfMemoryError e) { - // should have trigger spilling - if (!inMemSorter.hasSpaceForAnotherRecord()) { - logger.error("Unable to grow the pointer array"); - throw e; - } - return; - } - // check if spilling is triggered or not - if (inMemSorter.hasSpaceForAnotherRecord()) { - freeArray(array); - } else { - inMemSorter.expandPointerArray(array); - } - } - } - - /** - * Allocates more memory in order to insert an additional record. This will request additional - * memory from the memory manager and spill if the requested memory can not be obtained. - * - * @param required the required space in the data page, in bytes, including space for storing - * the record size. This must be less than or equal to the page size (records - * that exceed the page size are handled via a different code path which uses - * special overflow pages). - */ - private void acquireNewPageIfNecessary(int required) { - if (currentPage == null || - pageCursor + required > currentPage.getBaseOffset() + currentPage.size()) { - // TODO: try to find space on previous pages - currentPage = allocatePage(required); - pageCursor = currentPage.getBaseOffset(); - allocatedPages.add(currentPage); - } - } - - /** - * Write a record to the sorter. - */ - public void insertRecord( - Object recordBase, long recordOffset, int length, long prefix, boolean prefixIsNull) - throws IOException { - - assert(inMemSorter != null); - if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { - logger.info("Spilling data because number of spilledRecords crossed the threshold " + - numElementsForSpillThreshold); - spill(); - } - - growPointerArrayIfNecessary(); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - // Need 4 or 8 bytes to store the record length. - final int required = length + uaoSize; - acquireNewPageIfNecessary(required); - - final Object base = currentPage.getBaseObject(); - final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor); - UnsafeAlignedOffset.putSize(base, pageCursor, length); - pageCursor += uaoSize; - Platform.copyMemory(recordBase, recordOffset, base, pageCursor, length); - pageCursor += length; - inMemSorter.insertRecord(recordAddress, prefix, prefixIsNull); - } - - /** - * Write a key-value record to the sorter. The key and value will be put together in-memory, - * using the following format: - * - * record length (4 bytes), key length (4 bytes), key data, value data - * - * record length = key length + value length + 4 - */ - public void insertKVRecord(Object keyBase, long keyOffset, int keyLen, - Object valueBase, long valueOffset, int valueLen, long prefix, boolean prefixIsNull) - throws IOException { - - growPointerArrayIfNecessary(); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - final int required = keyLen + valueLen + (2 * uaoSize); - acquireNewPageIfNecessary(required); - - final Object base = currentPage.getBaseObject(); - final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor); - UnsafeAlignedOffset.putSize(base, pageCursor, keyLen + valueLen + uaoSize); - pageCursor += uaoSize; - UnsafeAlignedOffset.putSize(base, pageCursor, keyLen); - pageCursor += uaoSize; - Platform.copyMemory(keyBase, keyOffset, base, pageCursor, keyLen); - pageCursor += keyLen; - Platform.copyMemory(valueBase, valueOffset, base, pageCursor, valueLen); - pageCursor += valueLen; - assert(inMemSorter != null); - inMemSorter.insertRecord(recordAddress, prefix, prefixIsNull); - } - - /** - * Merges another UnsafeExternalSorters into this one, the other one will be emptied. - */ - public void merge(UnsafeExternalSorter other) throws IOException { - other.spill(); - spillWriters.addAll(other.spillWriters); - // remove them from `spillWriters`, or the files will be deleted in `cleanupResources`. - other.spillWriters.clear(); - other.cleanupResources(); - } - - /** - * Returns a sorted iterator. It is the caller's responsibility to call `cleanupResources()` - * after consuming this iterator. - */ - public UnsafeSorterIterator getSortedIterator() throws IOException { - assert(recordComparatorSupplier != null); - if (spillWriters.isEmpty()) { - assert(inMemSorter != null); - readingIterator = new SpillableIterator(inMemSorter.getSortedIterator()); - return readingIterator; - } else { - final UnsafeSorterSpillMerger spillMerger = new UnsafeSorterSpillMerger( - recordComparatorSupplier.get(), prefixComparator, spillWriters.size()); - for (SpillWriterForUnsafeSorter spillWriter: spillWriters) { - spillMerger.addSpillIfNotEmpty(spillWriter.getSpillReader()); - } - if (inMemSorter != null) { - readingIterator = new SpillableIterator(inMemSorter.getSortedIterator()); - spillMerger.addSpillIfNotEmpty(readingIterator); - } - return spillMerger.getSortedIterator(); - } - } - public UnsafeInMemorySorter getInMemSorter() { return inMemSorter; } - - @VisibleForTesting boolean hasSpaceForAnotherRecord() { - return inMemSorter.hasSpaceForAnotherRecord(); - } - - /** - * An UnsafeSorterIterator that support spilling. - */ - class SpillableIterator extends UnsafeSorterIterator { - private UnsafeSorterIterator upstream; - private UnsafeSorterIterator nextUpstream = null; - private MemoryBlock lastPage = null; - private boolean loaded = false; - private int numRecords = 0; - - SpillableIterator(UnsafeSorterIterator inMemIterator) { - this.upstream = inMemIterator; - this.numRecords = inMemIterator.getNumRecords(); - } - - @Override - public int getNumRecords() { - return numRecords; - } - - public long spill() throws IOException { - synchronized (this) { - if (!(upstream instanceof UnsafeInMemorySorter.SortedIterator && nextUpstream == null - && numRecords > 0)) { - return 0L; - } - UnsafeInMemorySorter.SortedIterator inMemIterator = - ((UnsafeInMemorySorter.SortedIterator) upstream).clone(); - - ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); - long released = 0L; - SpillWriterForUnsafeSorter spillWriter = spillWithWriter(inMemIterator, numRecords, writeMetrics, true); - nextUpstream = spillWriter.getSpillReader(); - assert(nextUpstream != null); - synchronized (UnsafeExternalSorter.this) { - // release the pages except the one that is used. There can still be a caller that - // is accessing the current record. We free this page in that caller's next loadNext() - // call. - for (MemoryBlock page : allocatedPages) { - if (!loaded || page.pageNumber != - ((UnsafeInMemorySorter.SortedIterator)upstream).getCurrentPageNumber()) { - released += page.size(); - freePage(page); - } else { - lastPage = page; - } - } - allocatedPages.clear(); - } - - // in-memory sorter will not be used after spilling - assert(inMemSorter != null); - released += inMemSorter.getMemoryUsage(); - totalSortTimeNanos += inMemSorter.getSortTimeNanos(); - inMemSorter.free(); - inMemSorter = null; - taskContext.taskMetrics().incMemoryBytesSpilled(released); - taskContext.taskMetrics().incDiskBytesSpilled(writeMetrics.bytesWritten()); - totalSpillBytes += released; - return released; - } - } - - @Override - public boolean hasNext() { - return numRecords > 0; - } - - @Override - public void loadNext() throws IOException { - MemoryBlock pageToFree = null; - try { - synchronized (this) { - loaded = true; - if (nextUpstream != null) { - // Just consumed the last record from in memory iterator - if(lastPage != null) { - // Do not free the page here, while we are locking `SpillableIterator`. The `freePage` - // method locks the `TaskMemoryManager`, and it's a bad idea to lock 2 objects in - // sequence. We may hit dead lock if another thread locks `TaskMemoryManager` and - // `SpillableIterator` in sequence, which may happen in - // `TaskMemoryManager.acquireExecutionMemory`. - pageToFree = lastPage; - lastPage = null; - } - upstream = nextUpstream; - nextUpstream = null; - } - numRecords--; - upstream.loadNext(); - } - } finally { - if (pageToFree != null) { - freePage(pageToFree); - } - } - } - - @Override - public Object getBaseObject() { - return upstream.getBaseObject(); - } - - @Override - public long getBaseOffset() { - return upstream.getBaseOffset(); - } - - @Override - public int getRecordLength() { - return upstream.getRecordLength(); - } - - @Override - public long getKeyPrefix() { - return upstream.getKeyPrefix(); - } - } - - /** - * Returns an iterator starts from startIndex, which will return the rows in the order as - * inserted. - * - * It is the caller's responsibility to call `cleanupResources()` - * after consuming this iterator. - * - * TODO: support forced spilling - */ - public UnsafeSorterIterator getIterator(int startIndex) throws IOException { - if (spillWriters.isEmpty()) { - assert(inMemSorter != null); - UnsafeSorterIterator iter = inMemSorter.getSortedIterator(); - moveOver(iter, startIndex); - return iter; - } else { - LinkedList queue = new LinkedList<>(); - int i = 0; - for (SpillWriterForUnsafeSorter spillWriter : spillWriters) { - if (i + spillWriter.recordsSpilled() > startIndex) { - UnsafeSorterIterator iter = spillWriter.getSpillReader(); - moveOver(iter, startIndex - i); - queue.add(iter); - } - i += spillWriter.recordsSpilled(); - } - if (inMemSorter != null) { - UnsafeSorterIterator iter = inMemSorter.getSortedIterator(); - moveOver(iter, startIndex - i); - queue.add(iter); - } - return new ChainedIterator(queue); - } - } - - private void moveOver(UnsafeSorterIterator iter, int steps) - throws IOException { - if (steps > 0) { - for (int i = 0; i < steps; i++) { - if (iter.hasNext()) { - iter.loadNext(); - } else { - throw new ArrayIndexOutOfBoundsException("Failed to move the iterator " + steps + - " steps forward"); - } - } - } - } - - /** - * Chain multiple UnsafeSorterIterator together as single one. - */ - static class ChainedIterator extends UnsafeSorterIterator { - - private final Queue iterators; - private UnsafeSorterIterator current; - private int numRecords; - - ChainedIterator(Queue iterators) { - assert iterators.size() > 0; - this.numRecords = 0; - for (UnsafeSorterIterator iter: iterators) { - this.numRecords += iter.getNumRecords(); - } - this.iterators = iterators; - this.current = iterators.remove(); - } - - @Override - public int getNumRecords() { - return numRecords; - } - - @Override - public boolean hasNext() { - while (!current.hasNext() && !iterators.isEmpty()) { - current = iterators.remove(); - } - return current.hasNext(); - } - - @Override - public void loadNext() throws IOException { - while (!current.hasNext() && !iterators.isEmpty()) { - current = iterators.remove(); - } - current.loadNext(); - } - - @Override - public Object getBaseObject() { return current.getBaseObject(); } - - @Override - public long getBaseOffset() { return current.getBaseOffset(); } - - @Override - public int getRecordLength() { return current.getRecordLength(); } - - @Override - public long getKeyPrefix() { return current.getKeyPrefix(); } - } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java deleted file mode 100644 index b1c1d393..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ /dev/null @@ -1,435 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -import java.util.Comparator; -import java.util.LinkedList; - -import org.apache.avro.reflect.Nullable; - -import org.apache.spark.TaskContext; -import org.apache.spark.memory.MemoryConsumer; -import org.apache.spark.memory.SparkOutOfMemoryError; -import org.apache.spark.memory.TaskMemoryManager; -import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.UnsafeAlignedOffset; -import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.util.collection.Sorter; - -/** - * Sorts records using an AlphaSort-style key-prefix sort. This sort stores pointers to records - * alongside a user-defined prefix of the record's sorting key. When the underlying sort algorithm - * compares records, it will first compare the stored key prefixes; if the prefixes are not equal, - * then we do not need to traverse the record pointers to compare the actual records. Avoiding these - * random memory accesses improves cache hit rates. - */ -public final class UnsafeInMemorySorter { - - private static final class SortComparator implements Comparator { - - private final RecordComparator recordComparator; - private final PrefixComparator prefixComparator; - private final TaskMemoryManager memoryManager; - - SortComparator( - RecordComparator recordComparator, - PrefixComparator prefixComparator, - TaskMemoryManager memoryManager) { - this.recordComparator = recordComparator; - this.prefixComparator = prefixComparator; - this.memoryManager = memoryManager; - } - - @Override - public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) { - final int prefixComparisonResult = prefixComparator.compare(r1.keyPrefix, r2.keyPrefix); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - if (prefixComparisonResult == 0) { - final Object baseObject1 = memoryManager.getPage(r1.recordPointer); - final long baseOffset1 = memoryManager.getOffsetInPage(r1.recordPointer) + uaoSize; - final int baseLength1 = UnsafeAlignedOffset.getSize(baseObject1, baseOffset1 - uaoSize); - final Object baseObject2 = memoryManager.getPage(r2.recordPointer); - final long baseOffset2 = memoryManager.getOffsetInPage(r2.recordPointer) + uaoSize; - final int baseLength2 = UnsafeAlignedOffset.getSize(baseObject2, baseOffset2 - uaoSize); - return recordComparator.compare(baseObject1, baseOffset1, baseLength1, baseObject2, - baseOffset2, baseLength2); - } else { - return prefixComparisonResult; - } - } - } - - private final MemoryConsumer consumer; - private final TaskMemoryManager memoryManager; - @Nullable - private final Comparator sortComparator; - - /** - * If non-null, specifies the radix sort parameters and that radix sort will be used. - */ - @Nullable - private final PrefixComparators.RadixSortSupport radixSortSupport; - - /** - * Within this buffer, position {@code 2 * i} holds a pointer to the record at - * index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix. - * - * Only part of the array will be used to store the pointers, the rest part is preserved as - * temporary buffer for sorting. - */ - private LongArray array; - - /** - * The position in the sort buffer where new records can be inserted. - */ - private int pos = 0; - - /** - * If sorting with radix sort, specifies the starting position in the sort buffer where records - * with non-null prefixes are kept. Positions [0..nullBoundaryPos) will contain null-prefixed - * records, and positions [nullBoundaryPos..pos) non-null prefixed records. This lets us avoid - * radix sorting over null values. - */ - private int nullBoundaryPos = 0; - - /* - * How many records could be inserted, because part of the array should be left for sorting. - */ - private int usableCapacity = 0; - - private long initialSize; - - private long totalSortTimeNanos = 0L; - - public UnsafeInMemorySorter( - final MemoryConsumer consumer, - final TaskMemoryManager memoryManager, - final RecordComparator recordComparator, - final PrefixComparator prefixComparator, - int initialSize, - boolean canUseRadixSort) { - this(consumer, memoryManager, recordComparator, prefixComparator, - consumer.allocateArray(initialSize * 2L), canUseRadixSort); - } - - public UnsafeInMemorySorter( - final MemoryConsumer consumer, - final TaskMemoryManager memoryManager, - final RecordComparator recordComparator, - final PrefixComparator prefixComparator, - LongArray array, - boolean canUseRadixSort) { - this.consumer = consumer; - this.memoryManager = memoryManager; - this.initialSize = array.size(); - if (recordComparator != null) { - this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager); - if (canUseRadixSort && prefixComparator instanceof PrefixComparators.RadixSortSupport) { - this.radixSortSupport = (PrefixComparators.RadixSortSupport)prefixComparator; - } else { - this.radixSortSupport = null; - } - } else { - this.sortComparator = null; - this.radixSortSupport = null; - } - this.array = array; - this.usableCapacity = getUsableCapacity(); - } - - private int getUsableCapacity() { - // Radix sort requires same amount of used memory as buffer, Tim sort requires - // half of the used memory as buffer. - return (int) (array.size() / (radixSortSupport != null ? 2 : 1.5)); - } - - /** - * Free the memory used by pointer array. - */ - public void freeWithoutLongArray() { - if (consumer != null) { - array = null; - } - } - - public void free() { - if (consumer != null) { - if (array != null) { - consumer.freeArray(array); - } - array = null; - } - } - - public void resetWithoutLongArrray() { - if (consumer != null) { - // the call to consumer.allocateArray may trigger a spill which in turn access this instance - // and eventually re-enter this method and try to free the array again. by setting the array - // to null and its length to 0 we effectively make the spill code-path a no-op. setting the - // array to null also indicates that it has already been de-allocated which prevents a double - // de-allocation in free(). - array = null; - usableCapacity = 0; - pos = 0; - nullBoundaryPos = 0; - array = consumer.allocateArray(initialSize); - usableCapacity = getUsableCapacity(); - } - pos = 0; - nullBoundaryPos = 0; - } - - public void reset() { - if (consumer != null) { - consumer.freeArray(array); - // the call to consumer.allocateArray may trigger a spill which in turn access this instance - // and eventually re-enter this method and try to free the array again. by setting the array - // to null and its length to 0 we effectively make the spill code-path a no-op. setting the - // array to null also indicates that it has already been de-allocated which prevents a double - // de-allocation in free(). - array = null; - usableCapacity = 0; - pos = 0; - nullBoundaryPos = 0; - array = consumer.allocateArray(initialSize); - usableCapacity = getUsableCapacity(); - } - pos = 0; - nullBoundaryPos = 0; - } - - /** - * @return the number of records that have been inserted into this sorter. - */ - public int numRecords() { - return pos / 2; - } - - /** - * @return the total amount of time spent sorting data (in-memory only). - */ - public long getSortTimeNanos() { - return totalSortTimeNanos; - } - - public long getMemoryUsage() { - if (array == null) { - return 0L; - } - - return array.size() * 8; - } - - public LongArray getSortedArray() { - getSortedIterator(); - return array; - } - - public LongArray getArray() { - return array; - } - - public boolean hasSpaceForAnotherRecord() { - return pos + 1 < usableCapacity; - } - - public void expandPointerArray(LongArray newArray) { - if (newArray.size() < array.size()) { - // checkstyle.off: RegexpSinglelineJava - throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); - // checkstyle.on: RegexpSinglelineJava - } - Platform.copyMemory( - array.getBaseObject(), - array.getBaseOffset(), - newArray.getBaseObject(), - newArray.getBaseOffset(), - pos * 8L); - consumer.freeArray(array); - array = newArray; - usableCapacity = getUsableCapacity(); - } - - /** - * Inserts a record to be sorted. Assumes that the record pointer points to a record length - * stored as a uaoSize(4 or 8) bytes integer, followed by the record's bytes. - * - * @param recordPointer pointer to a record in a data page, encoded by {@link TaskMemoryManager}. - * @param keyPrefix a user-defined key prefix - */ - public void insertRecord(long recordPointer, long keyPrefix, boolean prefixIsNull) { - if (!hasSpaceForAnotherRecord()) { - throw new IllegalStateException("There is no space for new record"); - } - if (prefixIsNull && radixSortSupport != null) { - // Swap forward a non-null record to make room for this one at the beginning of the array. - array.set(pos, array.get(nullBoundaryPos)); - pos++; - array.set(pos, array.get(nullBoundaryPos + 1)); - pos++; - // Place this record in the vacated position. - array.set(nullBoundaryPos, recordPointer); - nullBoundaryPos++; - array.set(nullBoundaryPos, keyPrefix); - nullBoundaryPos++; - } else { - array.set(pos, recordPointer); - pos++; - array.set(pos, keyPrefix); - pos++; - } - } - - public final class SortedIterator extends UnsafeSorterIterator implements Cloneable { - - private final int numRecords; - private int position; - private int offset; - private Object baseObject; - private long baseOffset; - private long keyPrefix; - private long currentRecordPointer; - private int recordLength; - private long currentPageNumber; - private final TaskContext taskContext = TaskContext.get(); - - private SortedIterator(int numRecords, int offset) { - this.numRecords = numRecords; - this.position = 0; - this.offset = offset; - } - - public SortedIterator clone() { - SortedIterator iter = new SortedIterator(numRecords, offset); - iter.position = position; - iter.baseObject = baseObject; - iter.baseOffset = baseOffset; - iter.keyPrefix = keyPrefix; - iter.recordLength = recordLength; - iter.currentPageNumber = currentPageNumber; - return iter; - } - - @Override - public int getNumRecords() { - return numRecords; - } - - @Override - public boolean hasNext() { - return position / 2 < numRecords; - } - - @Override - public void loadNext() { - // Kill the task in case it has been marked as killed. This logic is from - // InterruptibleIterator, but we inline it here instead of wrapping the iterator in order - // to avoid performance overhead. This check is added here in `loadNext()` instead of in - // `hasNext()` because it's technically possible for the caller to be relying on - // `getNumRecords()` instead of `hasNext()` to know when to stop. - if (taskContext != null) { - taskContext.killTaskIfInterrupted(); - } - // This pointer points to a 4-byte record length, followed by the record's bytes - final long recordPointer = array.get(offset + position); - currentRecordPointer = recordPointer; - currentPageNumber = TaskMemoryManager.decodePageNumber(recordPointer); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - baseObject = memoryManager.getPage(recordPointer); - // Skip over record length - baseOffset = memoryManager.getOffsetInPage(recordPointer) + uaoSize; - recordLength = UnsafeAlignedOffset.getSize(baseObject, baseOffset - uaoSize); - keyPrefix = array.get(offset + position + 1); - position += 2; - } - - @Override - public Object getBaseObject() { return baseObject; } - - @Override - public long getBaseOffset() { return baseOffset; } - - public long getCurrentPageNumber() { - return currentPageNumber; - } - - @Override - public int getRecordLength() { return recordLength; } - - @Override - public long getKeyPrefix() { return keyPrefix; } - - public int getPosition() { return position; } - - public int getOffset() { return offset; } - - public long getCurrentRecordPointer() { return currentRecordPointer; } - } - - /** - * Return an iterator over record pointers in sorted order. For efficiency, all calls to - * {@code next()} will return the same mutable object. - */ - public UnsafeSorterIterator getSortedIterator() { - int offset = 0; - long start = System.nanoTime(); - if (sortComparator != null) { - if (this.radixSortSupport != null) { - offset = RadixSort.sortKeyPrefixArray( - array, nullBoundaryPos, (pos - nullBoundaryPos) / 2L, 0, 7, - radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); - } else { - MemoryBlock unused = new MemoryBlock( - array.getBaseObject(), - array.getBaseOffset() + pos * 8L, - (array.size() - pos) * 8L); - LongArray buffer = new LongArray(unused); - Sorter sorter = - new Sorter<>(new UnsafeSortDataFormat(buffer)); - sorter.sort(array, 0, pos / 2, sortComparator); - } - } - totalSortTimeNanos += System.nanoTime() - start; - if (nullBoundaryPos > 0) { - assert radixSortSupport != null : "Nulls are only stored separately with radix sort"; - LinkedList queue = new LinkedList<>(); - - // The null order is either LAST or FIRST, regardless of sorting direction (ASC|DESC) - if (radixSortSupport.nullsFirst()) { - queue.add(new SortedIterator(nullBoundaryPos / 2, 0)); - queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset)); - } else { - queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset)); - queue.add(new SortedIterator(nullBoundaryPos / 2, 0)); - } - return new UnsafeExternalSorter.ChainedIterator(queue); - } else { - return new SortedIterator(pos / 2, offset); - } - } - - public LongArray getLongArray() { - return array; - } - - public TaskMemoryManager getTaskMemoryManager() { - return memoryManager; - } - -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterPMemSpillWriter.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterPMemSpillWriter.java deleted file mode 100644 index becab97d..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterPMemSpillWriter.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -import org.apache.spark.memory.MemoryConsumer; -import org.apache.spark.memory.TaskMemoryManager; -import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.unsafe.memory.MemoryBlock; - -import java.io.IOException; -import java.util.LinkedList; - -public abstract class UnsafeSorterPMemSpillWriter implements SpillWriterForUnsafeSorter{ - /** - * the memConsumer used to allocate pmem pages - */ - protected UnsafeExternalSorter externalSorter; - - protected SortedIteratorForSpills sortedIterator; - - protected int numberOfRecordsToWritten = 0; - - protected TaskMemoryManager taskMemoryManager; - - //Todo: It's confusing to have ShuffleWriteMetrics here. will reconsider and fix it later. - protected ShuffleWriteMetrics writeMetrics; - - protected TaskMetrics taskMetrics; - - protected LinkedList allocatedPMemPages = new LinkedList(); - - //Page size in bytes. - private static long DEFAULT_PAGE_SIZE = 64*1024*1024; - - public UnsafeSorterPMemSpillWriter( - UnsafeExternalSorter externalSorter, - SortedIteratorForSpills sortedIterator, - int numberOfRecordsToWritten, - ShuffleWriteMetrics writeMetrics, - TaskMetrics taskMetrics) { - this.externalSorter = externalSorter; - this.taskMemoryManager = externalSorter.getTaskMemoryManager(); - this.sortedIterator = sortedIterator; - this.numberOfRecordsToWritten = numberOfRecordsToWritten; - this.writeMetrics = writeMetrics; - this.taskMetrics = taskMetrics; - } - - protected MemoryBlock allocatePMemPage() throws IOException { ; - return allocatePMemPage(DEFAULT_PAGE_SIZE); - } - - protected MemoryBlock allocatePMemPage(long size) { - MemoryBlock page = taskMemoryManager.allocatePage(size, externalSorter, true); - if (page != null) { - allocatedPMemPages.add(page); - } - return page; - } - - protected void freeAllPMemPages() { - for (MemoryBlock page : allocatedPMemPages) { - taskMemoryManager.freePage(page, externalSorter); - } - allocatedPMemPages.clear(); - } - public abstract UnsafeSorterIterator getSpillReader() throws IOException; -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java deleted file mode 100644 index 25ca11fb..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -import com.google.common.io.ByteStreams; -import com.google.common.io.Closeables; -import org.apache.spark.SparkEnv; -import org.apache.spark.TaskContext; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.internal.config.package$; -import org.apache.spark.internal.config.ConfigEntry; -import org.apache.spark.io.NioBufferedFileInputStream; -import org.apache.spark.io.ReadAheadInputStream; -import org.apache.spark.serializer.SerializerManager; -import org.apache.spark.storage.BlockId; -import org.apache.spark.unsafe.Platform; - -import java.io.*; - -/** - * Reads spill files written by {@link UnsafeSorterSpillWriter} (see that class for a description - * of the file format). - */ -public class UnsafeSorterSpillReader extends UnsafeSorterIterator implements Closeable { - public static final int MAX_BUFFER_SIZE_BYTES = 16777216; // 16 mb - - protected InputStream in; - protected DataInputStream din; - - // Variables that change with every record read: - protected int recordLength; - protected long keyPrefix; - protected int numRecords; - protected int numRecordsRemaining; - - protected byte[] arr = new byte[1024 * 1024]; - protected Object baseObject = arr; - protected final TaskContext taskContext = TaskContext.get(); - protected final TaskMetrics taskMetrics; - - public UnsafeSorterSpillReader(TaskMetrics taskMetrics) { - this.taskMetrics = taskMetrics; - } - - public UnsafeSorterSpillReader( - SerializerManager serializerManager, - TaskMetrics taskMetrics, - File file, - BlockId blockId) throws IOException { - assert (file.length() > 0); - this.taskMetrics = taskMetrics; - final ConfigEntry bufferSizeConfigEntry = - package$.MODULE$.UNSAFE_SORTER_SPILL_READER_BUFFER_SIZE(); - // This value must be less than or equal to MAX_BUFFER_SIZE_BYTES. Cast to int is always safe. - final int DEFAULT_BUFFER_SIZE_BYTES = - ((Long) bufferSizeConfigEntry.defaultValue().get()).intValue(); - int bufferSizeBytes = SparkEnv.get() == null ? DEFAULT_BUFFER_SIZE_BYTES : - ((Long) SparkEnv.get().conf().get(bufferSizeConfigEntry)).intValue(); - - final boolean readAheadEnabled = SparkEnv.get() != null && (boolean)SparkEnv.get().conf().get( - package$.MODULE$.UNSAFE_SORTER_SPILL_READ_AHEAD_ENABLED()); - - final InputStream bs = - new NioBufferedFileInputStream(file, bufferSizeBytes); - try { - if (readAheadEnabled) { - this.in = new ReadAheadInputStream(serializerManager.wrapStream(blockId, bs), - bufferSizeBytes); - } else { - this.in = serializerManager.wrapStream(blockId, bs); - } - this.din = new DataInputStream(this.in); - numRecords = numRecordsRemaining = din.readInt(); - } catch (IOException e) { - Closeables.close(bs, /* swallowIOException = */ true); - throw e; - } - } - - @Override - public int getNumRecords() { - return numRecords; - } - - @Override - public boolean hasNext() { - return (numRecordsRemaining > 0); - } - - @Override - public void loadNext() throws IOException { - // Kill the task in case it has been marked as killed. This logic is from - // InterruptibleIterator, but we inline it here instead of wrapping the iterator in order - // to avoid performance overhead. This check is added here in `loadNext()` instead of in - // `hasNext()` because it's technically possible for the caller to be relying on - // `getNumRecords()` instead of `hasNext()` to know when to stop. - if (taskContext != null) { - taskContext.killTaskIfInterrupted(); - } - recordLength = din.readInt(); - keyPrefix = din.readLong(); - if (recordLength > arr.length) { - arr = new byte[recordLength]; - baseObject = arr; - } - ByteStreams.readFully(in, arr, 0, recordLength); - numRecordsRemaining--; - if (numRecordsRemaining == 0) { - close(); - } - } - - @Override - public Object getBaseObject() { - return baseObject; - } - - @Override - public long getBaseOffset() { - return Platform.BYTE_ARRAY_OFFSET; - } - - @Override - public int getRecordLength() { - return recordLength; - } - - @Override - public long getKeyPrefix() { - return keyPrefix; - } - - @Override - public void close() throws IOException { - if (in != null) { - try { - in.close(); - } finally { - in = null; - din = null; - } - } - } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java deleted file mode 100644 index 7ee391d7..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java +++ /dev/null @@ -1,249 +0,0 @@ -/* - * 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.util.collection.unsafe.sort; - -import java.io.File; -import java.io.IOException; - -import org.apache.spark.executor.TaskMetrics; -import scala.Tuple2; - -import org.apache.spark.SparkConf; -import org.apache.spark.serializer.SerializerManager; -import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.serializer.DummySerializerInstance; -import org.apache.spark.storage.BlockId; -import org.apache.spark.storage.BlockManager; -import org.apache.spark.storage.DiskBlockObjectWriter; -import org.apache.spark.storage.TempLocalBlockId; -import org.apache.spark.unsafe.Platform; -import org.apache.spark.internal.config.package$; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Spills a list of sorted records to disk. Spill files have the following format: - * - * [# of records (int)] [[len (int)][prefix (long)][data (bytes)]...] - */ -public class UnsafeSorterSpillWriter implements SpillWriterForUnsafeSorter{ - private static final Logger logger = LoggerFactory.getLogger(UnsafeSorterSpillWriter.class); - - protected final SparkConf conf = new SparkConf(); - - /** - * The buffer size to use when writing the sorted records to an on-disk file, and - * this space used by prefix + len + recordLength must be greater than 4 + 8 bytes. - */ - protected final int diskWriteBufferSize = - (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); - - // Small writes to DiskBlockObjectWriter will be fairly inefficient. Since there doesn't seem to - // be an API to directly transfer bytes from managed memory to the disk writer, we buffer - // data through a byte array. - protected byte[] writeBuffer = new byte[diskWriteBufferSize]; - - protected File file = null; - protected BlockId blockId = null; - protected int numRecordsToWrite = 0; - protected DiskBlockObjectWriter writer; - protected int numRecordsSpilled = 0; - - protected UnsafeSorterIterator inMemIterator; - protected SerializerManager serializerManager; - protected TaskMetrics taskMetrics; - - public UnsafeSorterSpillWriter() {} - public UnsafeSorterSpillWriter( - BlockManager blockManager, - int fileBufferSize, - UnsafeSorterIterator inMemIterator, - int numRecordsToWrite, - SerializerManager serializerManager, - ShuffleWriteMetrics writeMetrics, - TaskMetrics taskMetrics) { - final Tuple2 spilledFileInfo = - blockManager.diskBlockManager().createTempLocalBlock(); - this.file = spilledFileInfo._2(); - this.blockId = spilledFileInfo._1(); - this.numRecordsToWrite = numRecordsToWrite; - this.serializerManager = serializerManager; - this.taskMetrics = taskMetrics; - this.inMemIterator = inMemIterator; - // Unfortunately, we need a serializer instance in order to construct a DiskBlockObjectWriter. - // Our write path doesn't actually use this serializer (since we end up calling the `write()` - // OutputStream methods), but DiskBlockObjectWriter still calls some methods on it. To work - // around this, we pass a dummy no-op serializer. - writer = blockManager.getDiskWriter( - blockId, file, DummySerializerInstance.INSTANCE, fileBufferSize, writeMetrics); - // Write the number of records - writeIntToBuffer(numRecordsToWrite, 0); - writer.write(writeBuffer, 0, 4); - } - - public UnsafeSorterSpillWriter( - BlockManager blockManager, - int fileBufferSize, - ShuffleWriteMetrics writeMetrics, - int numRecordsToWrite) throws IOException { - this(blockManager, - fileBufferSize, - null, - numRecordsToWrite, - null, - writeMetrics, - null); - } - - public UnsafeSorterSpillWriter( - BlockManager blockManager, - int fileBufferSize, - UnsafeSorterIterator inMemIterator, - SerializerManager serializerManager, - ShuffleWriteMetrics writeMetrics, - TaskMetrics taskMetrics) throws IOException { - this(blockManager, - fileBufferSize, - inMemIterator, - inMemIterator.getNumRecords(), - serializerManager, - writeMetrics, - taskMetrics); - } - - // Based on DataOutputStream.writeLong. - private void writeLongToBuffer(long v, int offset) { - writeBuffer[offset + 0] = (byte)(v >>> 56); - writeBuffer[offset + 1] = (byte)(v >>> 48); - writeBuffer[offset + 2] = (byte)(v >>> 40); - writeBuffer[offset + 3] = (byte)(v >>> 32); - writeBuffer[offset + 4] = (byte)(v >>> 24); - writeBuffer[offset + 5] = (byte)(v >>> 16); - writeBuffer[offset + 6] = (byte)(v >>> 8); - writeBuffer[offset + 7] = (byte)(v >>> 0); - } - - // Based on DataOutputStream.writeInt. - protected void writeIntToBuffer(int v, int offset) { - writeBuffer[offset + 0] = (byte)(v >>> 24); - writeBuffer[offset + 1] = (byte)(v >>> 16); - writeBuffer[offset + 2] = (byte)(v >>> 8); - writeBuffer[offset + 3] = (byte)(v >>> 0); - } - - /** - * Write a record to a spill file. - * - * @param baseObject the base object / memory page containing the record - * @param baseOffset the base offset which points directly to the record data. - * @param recordLength the length of the record. - * @param keyPrefix a sort key prefix - */ - public void write( - Object baseObject, - long baseOffset, - int recordLength, - long keyPrefix) throws IOException { - if (numRecordsSpilled == numRecordsToWrite) { - throw new IllegalStateException( - "Number of records written exceeded numRecordsToWrite = " + numRecordsToWrite); - } else { - numRecordsSpilled++; - } - writeIntToBuffer(recordLength, 0); - writeLongToBuffer(keyPrefix, 4); - int dataRemaining = recordLength; - int freeSpaceInWriteBuffer = diskWriteBufferSize - 4 - 8; // space used by prefix + len - long recordReadPosition = baseOffset; - while (dataRemaining > 0) { - final int toTransfer = Math.min(freeSpaceInWriteBuffer, dataRemaining); - Platform.copyMemory( - baseObject, - recordReadPosition, - writeBuffer, - Platform.BYTE_ARRAY_OFFSET + (diskWriteBufferSize - freeSpaceInWriteBuffer), - toTransfer); - writer.write(writeBuffer, 0, (diskWriteBufferSize - freeSpaceInWriteBuffer) + toTransfer); - recordReadPosition += toTransfer; - dataRemaining -= toTransfer; - freeSpaceInWriteBuffer = diskWriteBufferSize; - } - if (freeSpaceInWriteBuffer < diskWriteBufferSize) { - writer.write(writeBuffer, 0, (diskWriteBufferSize - freeSpaceInWriteBuffer)); - } - writer.recordWritten(); - } - - public void close() throws IOException { - writer.commitAndGet(); - writer.close(); - writer = null; - writeBuffer = null; - } - - public File getFile() { - return file; - } - - public int recordsSpilled() { - return numRecordsSpilled; - } - - @Override - public void write() throws IOException { - write(false); - } - - public void write(boolean alreadyLoad) throws IOException { - if (inMemIterator != null) { - if (alreadyLoad) { - final Object baseObject = inMemIterator.getBaseObject(); - final long baseOffset = inMemIterator.getBaseOffset(); - final int recordLength = inMemIterator.getRecordLength(); - write(baseObject, baseOffset, recordLength, inMemIterator.getKeyPrefix()); - } - while (inMemIterator.hasNext()) { - inMemIterator.loadNext(); - final Object baseObject = inMemIterator.getBaseObject(); - final long baseOffset = inMemIterator.getBaseOffset(); - final int recordLength = inMemIterator.getRecordLength(); - write(baseObject, baseOffset, recordLength, inMemIterator.getKeyPrefix()); - } - close(); - } - } - - public UnsafeSorterSpillReader getReader(SerializerManager serializerManager, - TaskMetrics taskMetrics) throws IOException { - return new UnsafeSorterSpillReader(serializerManager, taskMetrics, file, blockId); - } - - @Override - public UnsafeSorterIterator getSpillReader() throws IOException{ - return new UnsafeSorterSpillReader(serializerManager, taskMetrics, file, blockId); - } - - @Override - public void clearAll() { - if (file != null && file.exists()) { - if (!file.delete()) { - logger.error("Was unable to delete spill file {}", file.getAbsolutePath()); - } - } - } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterStreamSpillReader.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterStreamSpillReader.java deleted file mode 100644 index 96438686..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterStreamSpillReader.java +++ /dev/null @@ -1,57 +0,0 @@ -package org.apache.spark.util.collection.unsafe.sort; - -import com.google.common.io.Closeables; -import com.intel.oap.common.storage.stream.ChunkInputStream; -import com.intel.oap.common.storage.stream.DataStore; -import org.apache.spark.SparkEnv; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.internal.config.ConfigEntry; -import org.apache.spark.internal.config.package$; -import org.apache.spark.io.ReadAheadInputStream; -import org.apache.spark.memory.PMemManagerInitializer; -import org.apache.spark.serializer.SerializerManager; -import org.apache.spark.storage.BlockId; - -import java.io.DataInputStream; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; - -public class UnsafeSorterStreamSpillReader extends UnsafeSorterSpillReader { - - protected final ChunkInputStream chunkInputStream; - public UnsafeSorterStreamSpillReader( - SerializerManager serializerManager, - TaskMetrics taskMetrics, - File file, - BlockId blockId) throws IOException { - super(taskMetrics); - final ConfigEntry bufferSizeConfigEntry = - package$.MODULE$.UNSAFE_SORTER_SPILL_READER_BUFFER_SIZE(); - // This value must be less than or equal to MAX_BUFFER_SIZE_BYTES. Cast to int is always safe. - final int DEFAULT_BUFFER_SIZE_BYTES = - ((Long) bufferSizeConfigEntry.defaultValue().get()).intValue(); - int bufferSizeBytes = SparkEnv.get() == null ? DEFAULT_BUFFER_SIZE_BYTES : - ((Long) SparkEnv.get().conf().get(bufferSizeConfigEntry)).intValue(); - final boolean readAheadEnabled = SparkEnv.get() != null && (boolean)SparkEnv.get().conf().get( - package$.MODULE$.UNSAFE_SORTER_SPILL_READ_AHEAD_ENABLED()); - chunkInputStream = - ChunkInputStream.getChunkInputStreamInstance(file.toString(), - new DataStore(PMemManagerInitializer.getPMemManager(), - PMemManagerInitializer.getProperties())); - final InputStream bs = chunkInputStream; - try { - if (readAheadEnabled) { - this.in = new ReadAheadInputStream(serializerManager.wrapStream(blockId, bs), - bufferSizeBytes); - } else { - this.in = serializerManager.wrapStream(blockId, bs); - } - this.din = new DataInputStream(this.in); - numRecords = numRecordsRemaining = din.readInt(); - } catch (IOException e) { - Closeables.close(bs, /* swallowIOException = */ true); - throw e; - } - } -} diff --git a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterStreamSpillWriter.java b/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterStreamSpillWriter.java deleted file mode 100644 index 3676800c..00000000 --- a/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterStreamSpillWriter.java +++ /dev/null @@ -1,62 +0,0 @@ -package org.apache.spark.util.collection.unsafe.sort; - -import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.serializer.DummySerializerInstance; -import org.apache.spark.serializer.SerializerManager; -import org.apache.spark.storage.BlockManager; -import org.apache.spark.storage.TempLocalBlockId; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import scala.Tuple2; - -import java.io.File; -import java.io.IOException; - -public class UnsafeSorterStreamSpillWriter extends UnsafeSorterSpillWriter { - private static final Logger logger = LoggerFactory.getLogger(UnsafeSorterStreamSpillWriter.class); - private UnsafeSorterStreamSpillReader reader; - public UnsafeSorterStreamSpillWriter( - BlockManager blockManager, - int fileBufferSize, - UnsafeSorterIterator inMemIterator, - int numRecordsToWrite, - SerializerManager serializerManager, - ShuffleWriteMetrics writeMetrics, - TaskMetrics taskMetrics) { - super(); - final Tuple2 spilledFileInfo = - blockManager.diskBlockManager().createTempLocalBlock(); - this.file = spilledFileInfo._2(); - this.blockId = spilledFileInfo._1(); - this.numRecordsToWrite = numRecordsToWrite; - this.serializerManager = serializerManager; - this.taskMetrics = taskMetrics; - this.inMemIterator = inMemIterator; - // Unfortunately, we need a serializer instance in order to construct a DiskBlockObjectWriter. - // Our write path doesn't actually use this serializer (since we end up calling the `write()` - // OutputStream methods), but DiskBlockObjectWriter still calls some methods on it. To work - // around this, we pass a dummy no-op serializer. - writer = blockManager.getPMemWriter( - blockId, file, DummySerializerInstance.INSTANCE, fileBufferSize, writeMetrics); - // Write the number of records - writeIntToBuffer(numRecordsToWrite, 0); - writer.write(writeBuffer, 0, 4); - } - - @Override - public UnsafeSorterIterator getSpillReader() throws IOException { - reader = new UnsafeSorterStreamSpillReader(serializerManager, taskMetrics, file, blockId); - return reader; - } - - @Override - public void clearAll() { - assert(reader != null); - try { - reader.chunkInputStream.free(); - } catch (IOException e) { - logger.debug(e.toString()); - } - } -} diff --git a/src/main/scala/org/apache/spark/SparkEnv.scala b/src/main/scala/org/apache/spark/SparkEnv.scala deleted file mode 100644 index da83f262..00000000 --- a/src/main/scala/org/apache/spark/SparkEnv.scala +++ /dev/null @@ -1,510 +0,0 @@ -/* - * 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 - -import java.io.File -import java.net.Socket -import java.util.Locale - -import scala.collection.JavaConverters._ -import scala.collection.concurrent -import scala.collection.mutable -import scala.util.Properties - -import com.google.common.cache.CacheBuilder -import com.intel.oap.common.unsafe.PersistentMemoryPlatform - -import org.apache.hadoop.conf.Configuration - -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.api.python.PythonWorkerFactory -import org.apache.spark.broadcast.BroadcastManager -import org.apache.spark.internal.{config, Logging} -import org.apache.spark.internal.config._ -import org.apache.spark.memory.{MemoryManager, UnifiedMemoryManager} -import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} -import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} -import org.apache.spark.network.shuffle.ExternalBlockStoreClient -import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} -import org.apache.spark.scheduler.{LiveListenerBus, OutputCommitCoordinator} -import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorEndpoint -import org.apache.spark.security.CryptoStreamUtils -import org.apache.spark.serializer.{JavaSerializer, Serializer, SerializerManager} -import org.apache.spark.shuffle.ShuffleManager -import org.apache.spark.storage._ -import org.apache.spark.util.{RpcUtils, Utils} - -/** - * :: DeveloperApi :: - * Holds all the runtime environment objects for a running Spark instance (either master or worker), - * including the serializer, RpcEnv, block manager, map output tracker, etc. Currently - * Spark code finds the SparkEnv through a global variable, so all the threads can access the same - * SparkEnv. It can be accessed by SparkEnv.get (e.g. after creating a SparkContext). - */ -@DeveloperApi -class SparkEnv ( - val executorId: String, - private[spark] val rpcEnv: RpcEnv, - val serializer: Serializer, - val closureSerializer: Serializer, - val serializerManager: SerializerManager, - val mapOutputTracker: MapOutputTracker, - val shuffleManager: ShuffleManager, - val broadcastManager: BroadcastManager, - val blockManager: BlockManager, - val securityManager: SecurityManager, - val metricsSystem: MetricsSystem, - val memoryManager: MemoryManager, - val outputCommitCoordinator: OutputCommitCoordinator, - val conf: SparkConf) extends Logging { - - @volatile private[spark] var isStopped = false - private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() - - // A general, soft-reference map for metadata needed during HadoopRDD split computation - // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). - private[spark] val hadoopJobMetadata = - CacheBuilder.newBuilder().softValues().build[String, AnyRef]().asMap() - - private[spark] var driverTmpDir: Option[String] = None - - private[spark] def stop(): Unit = { - - if (!isStopped) { - isStopped = true - pythonWorkers.values.foreach(_.stop()) - mapOutputTracker.stop() - shuffleManager.stop() - broadcastManager.stop() - blockManager.stop() - blockManager.master.stop() - metricsSystem.stop() - outputCommitCoordinator.stop() - rpcEnv.shutdown() - rpcEnv.awaitTermination() - - // If we only stop sc, but the driver process still run as a services then we need to delete - // the tmp dir, if not, it will create too many tmp dirs. - // We only need to delete the tmp dir create by driver - driverTmpDir match { - case Some(path) => - try { - Utils.deleteRecursively(new File(path)) - } catch { - case e: Exception => - logWarning(s"Exception while deleting Spark temp dir: $path", e) - } - case None => // We just need to delete tmp dir created by driver, so do nothing on executor - } - } - } - - private[spark] - def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = { - synchronized { - val key = (pythonExec, envVars) - pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() - } - } - - private[spark] - def destroyPythonWorker(pythonExec: String, - envVars: Map[String, String], worker: Socket): Unit = { - synchronized { - val key = (pythonExec, envVars) - pythonWorkers.get(key).foreach(_.stopWorker(worker)) - } - } - - private[spark] - def releasePythonWorker(pythonExec: String, - envVars: Map[String, String], worker: Socket): Unit = { - synchronized { - val key = (pythonExec, envVars) - pythonWorkers.get(key).foreach(_.releaseWorker(worker)) - } - } -} - -object SparkEnv extends Logging { - @volatile private var env: SparkEnv = _ - - private[spark] val driverSystemName = "sparkDriver" - private[spark] val executorSystemName = "sparkExecutor" - - def set(e: SparkEnv): Unit = { - env = e - } - - /** - * Returns the SparkEnv. - */ - def get: SparkEnv = { - env - } - - /** - * Create a SparkEnv for the driver. - */ - private[spark] def createDriverEnv( - conf: SparkConf, - isLocal: Boolean, - listenerBus: LiveListenerBus, - numCores: Int, - mockOutputCommitCoordinator: Option[OutputCommitCoordinator] = None): SparkEnv = { - assert(conf.contains(DRIVER_HOST_ADDRESS), - s"${DRIVER_HOST_ADDRESS.key} is not set on the driver!") - assert(conf.contains(DRIVER_PORT), s"${DRIVER_PORT.key} is not set on the driver!") - val bindAddress = conf.get(DRIVER_BIND_ADDRESS) - val advertiseAddress = conf.get(DRIVER_HOST_ADDRESS) - val port = conf.get(DRIVER_PORT) - val ioEncryptionKey = if (conf.get(IO_ENCRYPTION_ENABLED)) { - Some(CryptoStreamUtils.createKey(conf)) - } else { - None - } - create( - conf, - SparkContext.DRIVER_IDENTIFIER, - bindAddress, - advertiseAddress, - Option(port), - isLocal, - numCores, - ioEncryptionKey, - listenerBus = listenerBus, - mockOutputCommitCoordinator = mockOutputCommitCoordinator - ) - } - - /** - * Create a SparkEnv for an executor. - * In coarse-grained mode, the executor provides an RpcEnv that is already instantiated. - */ - private[spark] def createExecutorEnv( - conf: SparkConf, - executorId: String, - bindAddress: String, - hostname: String, - numCores: Int, - ioEncryptionKey: Option[Array[Byte]], - isLocal: Boolean): SparkEnv = { - val env = create( - conf, - executorId, - bindAddress, - hostname, - None, - isLocal, - numCores, - ioEncryptionKey - ) - SparkEnv.set(env) - env - } - - private[spark] def createExecutorEnv( - conf: SparkConf, - executorId: String, - hostname: String, - numCores: Int, - ioEncryptionKey: Option[Array[Byte]], - isLocal: Boolean): SparkEnv = { - createExecutorEnv(conf, executorId, hostname, - hostname, numCores, ioEncryptionKey, isLocal) - } - - /** - * Helper method to create a SparkEnv for a driver or an executor. - */ - private def create( - conf: SparkConf, - executorId: String, - bindAddress: String, - advertiseAddress: String, - port: Option[Int], - isLocal: Boolean, - numUsableCores: Int, - ioEncryptionKey: Option[Array[Byte]], - listenerBus: LiveListenerBus = null, - mockOutputCommitCoordinator: Option[OutputCommitCoordinator] = None): SparkEnv = { - - val isDriver = executorId == SparkContext.DRIVER_IDENTIFIER - - val pMemEnabled = conf.get(MEMORY_SPILL_PMEM_ENABLED); - if (pMemEnabled && !isDriver) { - val pMemInitialPath = conf.get(MEMORY_EXTENDED_PATH) - val pMemInitialSize = conf.get(MEMORY_EXTENDED_SIZE) - PersistentMemoryPlatform.initialize(pMemInitialPath, pMemInitialSize, 0) - logInfo(s"PMem initialize path: ${pMemInitialPath}, size: ${pMemInitialSize} ") - } - - // Listener bus is only used on the driver - if (isDriver) { - assert(listenerBus != null, "Attempted to create driver SparkEnv with null listener bus!") - } - val authSecretFileConf = if (isDriver) AUTH_SECRET_FILE_DRIVER else AUTH_SECRET_FILE_EXECUTOR - val securityManager = new SecurityManager(conf, ioEncryptionKey, authSecretFileConf) - if (isDriver) { - securityManager.initializeAuth() - } - - ioEncryptionKey.foreach { _ => - if (!securityManager.isEncryptionEnabled()) { - logWarning("I/O encryption enabled without RPC encryption: keys will be visible on the " + - "wire.") - } - } - - val systemName = if (isDriver) driverSystemName else executorSystemName - val rpcEnv = RpcEnv.create(systemName, bindAddress, advertiseAddress, port.getOrElse(-1), conf, - securityManager, numUsableCores, !isDriver) - - // Figure out which port RpcEnv actually bound to in case the original port is 0 or occupied. - if (isDriver) { - conf.set(DRIVER_PORT, rpcEnv.address.port) - } - - // Create an instance of the class with the given name, possibly initializing it with our conf - def instantiateClass[T](className: String): T = { - val cls = Utils.classForName(className) - // Look for a constructor taking a SparkConf and a boolean isDriver, then one taking just - // SparkConf, then one taking no arguments - try { - cls.getConstructor(classOf[SparkConf], java.lang.Boolean.TYPE) - .newInstance(conf, java.lang.Boolean.valueOf(isDriver)) - .asInstanceOf[T] - } catch { - case _: NoSuchMethodException => - try { - cls.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[T] - } catch { - case _: NoSuchMethodException => - cls.getConstructor().newInstance().asInstanceOf[T] - } - } - } - - // Create an instance of the class named by the given SparkConf property - // if the property is not set, possibly initializing it with our conf - def instantiateClassFromConf[T](propertyName: ConfigEntry[String]): T = { - instantiateClass[T](conf.get(propertyName)) - } - - val serializer = instantiateClassFromConf[Serializer](SERIALIZER) - logDebug(s"Using serializer: ${serializer.getClass}") - - val serializerManager = new SerializerManager(serializer, conf, ioEncryptionKey) - - val closureSerializer = new JavaSerializer(conf) - - def registerOrLookupEndpoint( - name: String, endpointCreator: => RpcEndpoint): - RpcEndpointRef = { - if (isDriver) { - logInfo("Registering " + name) - rpcEnv.setupEndpoint(name, endpointCreator) - } else { - RpcUtils.makeDriverRef(name, conf, rpcEnv) - } - } - - val broadcastManager = new BroadcastManager(isDriver, conf, securityManager) - - val mapOutputTracker = if (isDriver) { - new MapOutputTrackerMaster(conf, broadcastManager, isLocal) - } else { - new MapOutputTrackerWorker(conf) - } - - // Have to assign trackerEndpoint after initialization as MapOutputTrackerEndpoint - // requires the MapOutputTracker itself - mapOutputTracker.trackerEndpoint = registerOrLookupEndpoint(MapOutputTracker.ENDPOINT_NAME, - new MapOutputTrackerMasterEndpoint( - rpcEnv, mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) - - // Let the user specify short names for shuffle managers - val shortShuffleMgrNames = Map( - "sort" -> classOf[org.apache.spark.shuffle.sort.SortShuffleManager].getName, - "tungsten-sort" -> classOf[org.apache.spark.shuffle.sort.SortShuffleManager].getName) - val shuffleMgrName = conf.get(config.SHUFFLE_MANAGER) - val shuffleMgrClass = - shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase(Locale.ROOT), shuffleMgrName) - val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) - - val memoryManager: MemoryManager = UnifiedMemoryManager(conf, numUsableCores) - - val blockManagerPort = if (isDriver) { - conf.get(DRIVER_BLOCK_MANAGER_PORT) - } else { - conf.get(BLOCK_MANAGER_PORT) - } - - val externalShuffleClient = if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) { - val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores) - Some(new ExternalBlockStoreClient(transConf, securityManager, - securityManager.isAuthenticationEnabled(), conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT))) - } else { - None - } - - // Mapping from block manager id to the block manager's information. - val blockManagerInfo = new concurrent.TrieMap[BlockManagerId, BlockManagerInfo]() - val blockManagerMaster = new BlockManagerMaster( - registerOrLookupEndpoint( - BlockManagerMaster.DRIVER_ENDPOINT_NAME, - new BlockManagerMasterEndpoint( - rpcEnv, - isLocal, - conf, - listenerBus, - if (conf.get(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED)) { - externalShuffleClient - } else { - None - }, blockManagerInfo)), - registerOrLookupEndpoint( - BlockManagerMaster.DRIVER_HEARTBEAT_ENDPOINT_NAME, - new BlockManagerMasterHeartbeatEndpoint(rpcEnv, isLocal, blockManagerInfo)), - conf, - isDriver) - - val blockTransferService = - new NettyBlockTransferService(conf, securityManager, bindAddress, advertiseAddress, - blockManagerPort, numUsableCores, blockManagerMaster.driverEndpoint) - - // NB: blockManager is not valid until initialize() is called later. - val blockManager = new BlockManager( - executorId, - rpcEnv, - blockManagerMaster, - serializerManager, - conf, - memoryManager, - mapOutputTracker, - shuffleManager, - blockTransferService, - securityManager, - externalShuffleClient) - - val metricsSystem = if (isDriver) { - // Don't start metrics system right now for Driver. - // We need to wait for the task scheduler to give us an app ID. - // Then we can start the metrics system. - MetricsSystem.createMetricsSystem(MetricsSystemInstances.DRIVER, conf, securityManager) - } else { - // We need to set the executor ID before the MetricsSystem is created because sources and - // sinks specified in the metrics configuration file will want to incorporate this executor's - // ID into the metrics they report. - conf.set(EXECUTOR_ID, executorId) - val ms = MetricsSystem.createMetricsSystem(MetricsSystemInstances.EXECUTOR, conf, - securityManager) - ms.start(conf.get(METRICS_STATIC_SOURCES_ENABLED)) - ms - } - - val outputCommitCoordinator = mockOutputCommitCoordinator.getOrElse { - new OutputCommitCoordinator(conf, isDriver) - } - val outputCommitCoordinatorRef = registerOrLookupEndpoint("OutputCommitCoordinator", - new OutputCommitCoordinatorEndpoint(rpcEnv, outputCommitCoordinator)) - outputCommitCoordinator.coordinatorRef = Some(outputCommitCoordinatorRef) - - val envInstance = new SparkEnv( - executorId, - rpcEnv, - serializer, - closureSerializer, - serializerManager, - mapOutputTracker, - shuffleManager, - broadcastManager, - blockManager, - securityManager, - metricsSystem, - memoryManager, - outputCommitCoordinator, - conf) - - // Add a reference to tmp dir created by driver, we will delete this tmp dir when stop() is - // called, and we only need to do it for driver. Because driver may run as a service, and if we - // don't delete this tmp dir when sc is stopped, then will create too many tmp dirs. - if (isDriver) { - val sparkFilesDir = Utils.createTempDir(Utils.getLocalDir(conf), "userFiles").getAbsolutePath - envInstance.driverTmpDir = Some(sparkFilesDir) - } - - envInstance - } - - /** - * Return a map representation of jvm information, Spark properties, system properties, and - * class paths. Map keys define the category, and map values represent the corresponding - * attributes as a sequence of KV pairs. This is used mainly for SparkListenerEnvironmentUpdate. - */ - private[spark] - def environmentDetails( - conf: SparkConf, - hadoopConf: Configuration, - schedulingMode: String, - addedJars: Seq[String], - addedFiles: Seq[String]): Map[String, Seq[(String, String)]] = { - - import Properties._ - val jvmInformation = Seq( - ("Java Version", s"$javaVersion ($javaVendor)"), - ("Java Home", javaHome), - ("Scala Version", versionString) - ).sorted - - // Spark properties - // This includes the scheduling mode whether or not it is configured (used by SparkUI) - val schedulerMode = - if (!conf.contains(SCHEDULER_MODE)) { - Seq((SCHEDULER_MODE.key, schedulingMode)) - } else { - Seq.empty[(String, String)] - } - val sparkProperties = (conf.getAll ++ schedulerMode).sorted - - // System properties that are not java classpaths - val systemProperties = Utils.getSystemProperties.toSeq - val otherProperties = systemProperties.filter { case (k, _) => - k != "java.class.path" && !k.startsWith("spark.") - }.sorted - - // Class paths including all added jars and files - val classPathEntries = javaClassPath - .split(File.pathSeparator) - .filterNot(_.isEmpty) - .map((_, "System Classpath")) - val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) - val classPaths = (addedJarsAndFiles ++ classPathEntries).sorted - - // Add Hadoop properties, it will not ignore configs including in Spark. Some spark - // conf starting with "spark.hadoop" may overwrite it. - val hadoopProperties = hadoopConf.asScala - .map(entry => (entry.getKey, entry.getValue)).toSeq.sorted - Map[String, Seq[(String, String)]]( - "JVM Information" -> jvmInformation, - "Spark Properties" -> sparkProperties, - "Hadoop Properties" -> hadoopProperties, - "System Properties" -> otherProperties, - "Classpath Entries" -> classPaths) - } -} diff --git a/src/main/scala/org/apache/spark/internal/config/package.scala b/src/main/scala/org/apache/spark/internal/config/package.scala deleted file mode 100644 index 07661590..00000000 --- a/src/main/scala/org/apache/spark/internal/config/package.scala +++ /dev/null @@ -1,1869 +0,0 @@ -/* - * 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.internal - -import java.util.concurrent.TimeUnit - -import org.apache.spark.launcher.SparkLauncher -import org.apache.spark.metrics.GarbageCollectionMetrics -import org.apache.spark.network.shuffle.Constants -import org.apache.spark.network.util.ByteUnit -import org.apache.spark.scheduler.{EventLoggingListener, SchedulingMode} -import org.apache.spark.shuffle.sort.io.LocalDiskShuffleDataIO -import org.apache.spark.storage.{DefaultTopologyMapper, RandomBlockReplicationPolicy} -import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.util.Utils -import org.apache.spark.util.collection.unsafe.sort.PMemSpillWriterType -import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillReader.MAX_BUFFER_SIZE_BYTES - -package object config { - - private[spark] val SPARK_DRIVER_PREFIX = "spark.driver" - private[spark] val SPARK_EXECUTOR_PREFIX = "spark.executor" - private[spark] val SPARK_TASK_PREFIX = "spark.task" - private[spark] val LISTENER_BUS_EVENT_QUEUE_PREFIX = "spark.scheduler.listenerbus.eventqueue" - - private[spark] val RESOURCES_DISCOVERY_PLUGIN = - ConfigBuilder("spark.resources.discoveryPlugin") - .doc("Comma-separated list of class names implementing" + - "org.apache.spark.api.resource.ResourceDiscoveryPlugin to load into the application." + - "This is for advanced users to replace the resource discovery class with a " + - "custom implementation. Spark will try each class specified until one of them " + - "returns the resource information for that resource. It tries the discovery " + - "script last if none of the plugins return information for that resource.") - .version("3.0.0") - .stringConf - .toSequence - .createWithDefault(Nil) - - private[spark] val DRIVER_RESOURCES_FILE = - ConfigBuilder("spark.driver.resourcesFile") - .internal() - .doc("Path to a file containing the resources allocated to the driver. " + - "The file should be formatted as a JSON array of ResourceAllocation objects. " + - "Only used internally in standalone mode.") - .version("3.0.0") - .stringConf - .createOptional - - private[spark] val DRIVER_CLASS_PATH = - ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH) - .version("1.0.0") - .stringConf - .createOptional - - private[spark] val DRIVER_JAVA_OPTIONS = - ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS) - .withPrepended(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS) - .version("1.0.0") - .stringConf - .createOptional - - private[spark] val DRIVER_LIBRARY_PATH = - ConfigBuilder(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH) - .version("1.0.0") - .stringConf - .createOptional - - private[spark] val DRIVER_USER_CLASS_PATH_FIRST = - ConfigBuilder("spark.driver.userClassPathFirst") - .version("1.3.0") - .booleanConf - .createWithDefault(false) - - private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores") - .doc("Number of cores to use for the driver process, only in cluster mode.") - .version("1.3.0") - .intConf - .createWithDefault(1) - - private[spark] val DRIVER_MEMORY = ConfigBuilder(SparkLauncher.DRIVER_MEMORY) - .doc("Amount of memory to use for the driver process, in MiB unless otherwise specified.") - .version("1.1.1") - .bytesConf(ByteUnit.MiB) - .createWithDefaultString("1g") - - private[spark] val DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.driver.memoryOverhead") - .doc("The amount of non-heap memory to be allocated per driver in cluster mode, " + - "in MiB unless otherwise specified.") - .version("2.3.0") - .bytesConf(ByteUnit.MiB) - .createOptional - - private[spark] val DRIVER_LOG_DFS_DIR = - ConfigBuilder("spark.driver.log.dfsDir").version("3.0.0").stringConf.createOptional - - private[spark] val DRIVER_LOG_LAYOUT = - ConfigBuilder("spark.driver.log.layout") - .version("3.0.0") - .stringConf - .createOptional - - private[spark] val DRIVER_LOG_PERSISTTODFS = - ConfigBuilder("spark.driver.log.persistToDfs.enabled") - .version("3.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val DRIVER_LOG_ALLOW_EC = - ConfigBuilder("spark.driver.log.allowErasureCoding") - .version("3.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_ENABLED = ConfigBuilder("spark.eventLog.enabled") - .version("1.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_DIR = ConfigBuilder("spark.eventLog.dir") - .version("1.0.0") - .stringConf - .createWithDefault(EventLoggingListener.DEFAULT_LOG_DIR) - - private[spark] val EVENT_LOG_COMPRESS = - ConfigBuilder("spark.eventLog.compress") - .version("1.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_BLOCK_UPDATES = - ConfigBuilder("spark.eventLog.logBlockUpdates.enabled") - .version("2.3.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_ALLOW_EC = - ConfigBuilder("spark.eventLog.erasureCoding.enabled") - .version("3.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_TESTING = - ConfigBuilder("spark.eventLog.testing") - .internal() - .version("1.0.1") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_OUTPUT_BUFFER_SIZE = ConfigBuilder("spark.eventLog.buffer.kb") - .doc("Buffer size to use when writing to output streams, in KiB unless otherwise specified.") - .version("1.0.0") - .bytesConf(ByteUnit.KiB) - .createWithDefaultString("100k") - - private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS = - ConfigBuilder("spark.eventLog.logStageExecutorMetrics") - .doc("Whether to write per-stage peaks of executor metrics (for each executor) " + - "to the event log.") - .version("3.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS = - ConfigBuilder("spark.eventLog.gcMetrics.youngGenerationGarbageCollectors") - .doc("Names of supported young generation garbage collector. A name usually is " + - " the return of GarbageCollectorMXBean.getName. The built-in young generation garbage " + - s"collectors are ${GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS}") - .version("3.0.0") - .stringConf - .toSequence - .createWithDefault(GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS) - - private[spark] val EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS = - ConfigBuilder("spark.eventLog.gcMetrics.oldGenerationGarbageCollectors") - .doc("Names of supported old generation garbage collector. A name usually is " + - "the return of GarbageCollectorMXBean.getName. The built-in old generation garbage " + - s"collectors are ${GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS}") - .version("3.0.0") - .stringConf - .toSequence - .createWithDefault(GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS) - - private[spark] val EVENT_LOG_OVERWRITE = - ConfigBuilder("spark.eventLog.overwrite") - .version("1.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_CALLSITE_LONG_FORM = - ConfigBuilder("spark.eventLog.longForm.enabled") - .version("2.4.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_ENABLE_ROLLING = - ConfigBuilder("spark.eventLog.rolling.enabled") - .doc("Whether rolling over event log files is enabled. If set to true, it cuts down " + - "each event log file to the configured size.") - .version("3.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_ROLLING_MAX_FILE_SIZE = - ConfigBuilder("spark.eventLog.rolling.maxFileSize") - .doc(s"When ${EVENT_LOG_ENABLE_ROLLING.key}=true, specifies the max size of event log file" + - " to be rolled over.") - .version("3.0.0") - .bytesConf(ByteUnit.BYTE) - .checkValue(_ >= ByteUnit.MiB.toBytes(10), "Max file size of event log should be " + - "configured to be at least 10 MiB.") - .createWithDefaultString("128m") - - private[spark] val EXECUTOR_ID = - ConfigBuilder("spark.executor.id").version("1.2.0").stringConf.createOptional - - private[spark] val EXECUTOR_CLASS_PATH = - ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH) - .version("1.0.0") - .stringConf - .createOptional - - private[spark] val EXECUTOR_HEARTBEAT_DROP_ZERO_ACCUMULATOR_UPDATES = - ConfigBuilder("spark.executor.heartbeat.dropZeroAccumulatorUpdates") - .internal() - .version("3.0.0") - .booleanConf - .createWithDefault(true) - - private[spark] val EXECUTOR_HEARTBEAT_INTERVAL = - ConfigBuilder("spark.executor.heartbeatInterval") - .version("1.1.0") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString("10s") - - private[spark] val EXECUTOR_HEARTBEAT_MAX_FAILURES = - ConfigBuilder("spark.executor.heartbeat.maxFailures") - .internal() - .version("1.6.2") - .intConf - .createWithDefault(60) - - private[spark] val EXECUTOR_PROCESS_TREE_METRICS_ENABLED = - ConfigBuilder("spark.executor.processTreeMetrics.enabled") - .doc("Whether to collect process tree metrics (from the /proc filesystem) when collecting " + - "executor metrics.") - .version("3.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EXECUTOR_METRICS_POLLING_INTERVAL = - ConfigBuilder("spark.executor.metrics.pollingInterval") - .doc("How often to collect executor metrics (in milliseconds). " + - "If 0, the polling is done on executor heartbeats. " + - "If positive, the polling is done at this interval.") - .version("3.0.0") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString("0") - - private[spark] val EXECUTOR_JAVA_OPTIONS = - ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS) - .withPrepended(SparkLauncher.EXECUTOR_DEFAULT_JAVA_OPTIONS) - .version("1.0.0") - .stringConf - .createOptional - - private[spark] val EXECUTOR_LIBRARY_PATH = - ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH) - .version("1.0.0") - .stringConf - .createOptional - - private[spark] val EXECUTOR_USER_CLASS_PATH_FIRST = - ConfigBuilder("spark.executor.userClassPathFirst") - .version("1.3.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EXECUTOR_CORES = ConfigBuilder(SparkLauncher.EXECUTOR_CORES) - .version("1.0.0") - .intConf - .createWithDefault(1) - - private[spark] val EXECUTOR_MEMORY = ConfigBuilder(SparkLauncher.EXECUTOR_MEMORY) - .doc("Amount of memory to use per executor process, in MiB unless otherwise specified.") - .version("0.7.0") - .bytesConf(ByteUnit.MiB) - .createWithDefaultString("1g") - - private[spark] val EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.executor.memoryOverhead") - .doc("The amount of non-heap memory to be allocated per executor in cluster mode, " + - "in MiB unless otherwise specified.") - .version("2.3.0") - .bytesConf(ByteUnit.MiB) - .createOptional - - private[spark] val CORES_MAX = ConfigBuilder("spark.cores.max") - .doc("When running on a standalone deploy cluster or a Mesos cluster in coarse-grained " + - "sharing mode, the maximum amount of CPU cores to request for the application from across " + - "the cluster (not from each machine). If not set, the default will be " + - "`spark.deploy.defaultCores` on Spark's standalone cluster manager, or infinite " + - "(all available cores) on Mesos.") - .version("0.6.0") - .intConf - .createOptional - - private[spark] val MEMORY_OFFHEAP_ENABLED = ConfigBuilder("spark.memory.offHeap.enabled") - .doc("If true, Spark will attempt to use off-heap memory for certain operations. " + - "If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive.") - .version("1.6.0") - .withAlternative("spark.unsafe.offHeap") - .booleanConf - .createWithDefault(false) - - private[spark] val MEMORY_OFFHEAP_SIZE = ConfigBuilder("spark.memory.offHeap.size") - .doc("The absolute amount of memory which can be used for off-heap allocation, " + - " in bytes unless otherwise specified. " + - "This setting has no impact on heap memory usage, so if your executors' total memory " + - "consumption must fit within some hard limit then be sure to shrink your JVM heap size " + - "accordingly. This must be set to a positive value when spark.memory.offHeap.enabled=true.") - .version("1.6.0") - .bytesConf(ByteUnit.BYTE) - .checkValue(_ >= 0, "The off-heap memory size must not be negative") - .createWithDefault(0) - - val MEMORY_SPILL_PMEM_ENABLED = - ConfigBuilder("spark.memory.spill.pmem.enabled") - .doc("Set memory spill to PMem instead of disk.") - .booleanConf - .createWithDefault(true) - - val MEMORY_EXTENDED_PATH = - ConfigBuilder("spark.memory.extended.path") - .doc("intialize path for extended memory") - .stringConf - .createWithDefault("/mnt/pmem") - - private[spark] val MEMORY_EXTENDED_SIZE = ConfigBuilder("spark.memory.extended.size") - .doc("The absolute amount of memory which can be used for extended memory allocation.") - .version("1.6.0") - .bytesConf(ByteUnit.BYTE) - .checkValue(_ >= 0, "The extended memory size must not be negative") - .createWithDefault(64 * 1024 * 1024) - - val MEMORY_SPILL_PMEM_SORT_BACKGROUND = - ConfigBuilder("spark.memory.spill.pmem.sort.background") - .doc("Doing sort and dump pages to PMem concurrently") - .booleanConf - .createWithDefault(false) - - val MEMORY_SPILL_PMEM_CLFLUSH_ENABLED = - ConfigBuilder("spark.memory.spill.pmem.clflush.enabled") - .doc("Enable clflush when copy to PMEM.") - .booleanConf - .createWithDefault(false) - - val PMEM_PROPERTY_FILE = - ConfigBuilder("spark.memory.spill.pmem.config.file") - .doc("A config file used to config Intel PMem settings for memory extension.") - .stringConf - .createWithDefault("pmem.properties") - - val USAFE_EXTERNAL_SORTER_SPILL_WRITE_TYPE = ConfigBuilder("spark.unsafe.sort.spillwriter.type") - .doc("The spill writer type for UnsafeExteranlSorter") - .stringConf - .createWithDefault(PMemSpillWriterType.WRITE_SORTED_RECORDS_TO_PMEM.toString()) - - private[spark] val MEMORY_SPILL_PMEM_READ_BUFFERSIZE = - ConfigBuilder("spark.memory.spill.pmem.readBufferSize") - .doc("The buffer size, in bytes, to use when reading records from PMem.") - .bytesConf(ByteUnit.BYTE) - .checkValue(v => v > 12 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, - s"The buffer size must be greater than 12 and less than or equal to " + - s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") - .createWithDefaultString("8m") - - private[spark] val MEMORY_STORAGE_FRACTION = ConfigBuilder("spark.memory.storageFraction") - .doc("Amount of storage memory immune to eviction, expressed as a fraction of the " + - "size of the region set aside by spark.memory.fraction. The higher this is, the " + - "less working memory may be available to execution and tasks may spill to disk more " + - "often. Leaving this at the default value is recommended. ") - .version("1.6.0") - .doubleConf - .checkValue(v => v >= 0.0 && v < 1.0, "Storage fraction must be in [0,1)") - .createWithDefault(0.5) - - private[spark] val MEMORY_FRACTION = ConfigBuilder("spark.memory.fraction") - .doc("Fraction of (heap space - 300MB) used for execution and storage. The " + - "lower this is, the more frequently spills and cached data eviction occur. " + - "The purpose of this config is to set aside memory for internal metadata, " + - "user data structures, and imprecise size estimation in the case of sparse, " + - "unusually large records. Leaving this at the default value is recommended. ") - .version("1.6.0") - .doubleConf - .createWithDefault(0.6) - - private[spark] val STORAGE_SAFETY_FRACTION = ConfigBuilder("spark.storage.safetyFraction") - .version("1.1.0") - .doubleConf - .createWithDefault(0.9) - - private[spark] val STORAGE_UNROLL_MEMORY_THRESHOLD = - ConfigBuilder("spark.storage.unrollMemoryThreshold") - .doc("Initial memory to request before unrolling any block") - .version("1.1.0") - .longConf - .createWithDefault(1024 * 1024) - - private[spark] val STORAGE_REPLICATION_PROACTIVE = - ConfigBuilder("spark.storage.replication.proactive") - .doc("Enables proactive block replication for RDD blocks. " + - "Cached RDD block replicas lost due to executor failures are replenished " + - "if there are any existing available replicas. This tries to " + - "get the replication level of the block to the initial number") - .version("2.2.0") - .booleanConf - .createWithDefault(false) - - private[spark] val STORAGE_MEMORY_MAP_THRESHOLD = - ConfigBuilder("spark.storage.memoryMapThreshold") - .doc("Size in bytes of a block above which Spark memory maps when " + - "reading a block from disk. " + - "This prevents Spark from memory mapping very small blocks. " + - "In general, memory mapping has high overhead for blocks close to or below " + - "the page size of the operating system.") - .version("0.9.2") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("2m") - - private[spark] val STORAGE_REPLICATION_POLICY = - ConfigBuilder("spark.storage.replication.policy") - .version("2.1.0") - .stringConf - .createWithDefaultString(classOf[RandomBlockReplicationPolicy].getName) - - private[spark] val STORAGE_REPLICATION_TOPOLOGY_MAPPER = - ConfigBuilder("spark.storage.replication.topologyMapper") - .version("2.1.0") - .stringConf - .createWithDefaultString(classOf[DefaultTopologyMapper].getName) - - private[spark] val STORAGE_CACHED_PEERS_TTL = ConfigBuilder("spark.storage.cachedPeersTtl") - .version("1.1.1") - .intConf - .createWithDefault(60 * 1000) - - private[spark] val STORAGE_MAX_REPLICATION_FAILURE = - ConfigBuilder("spark.storage.maxReplicationFailures") - .version("1.1.1") - .intConf - .createWithDefault(1) - - private[spark] val STORAGE_REPLICATION_TOPOLOGY_FILE = - ConfigBuilder("spark.storage.replication.topologyFile") - .version("2.1.0") - .stringConf - .createOptional - - private[spark] val STORAGE_EXCEPTION_PIN_LEAK = - ConfigBuilder("spark.storage.exceptionOnPinLeak") - .version("1.6.2") - .booleanConf - .createWithDefault(false) - - private[spark] val STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL = - ConfigBuilder("spark.storage.blockManagerTimeoutIntervalMs") - .version("0.7.3") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString("60s") - - private[spark] val STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT = - ConfigBuilder("spark.storage.blockManagerSlaveTimeoutMs") - .version("0.7.0") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString(Network.NETWORK_TIMEOUT.defaultValueString) - - private[spark] val STORAGE_CLEANUP_FILES_AFTER_EXECUTOR_EXIT = - ConfigBuilder("spark.storage.cleanupFilesAfterExecutorExit") - .doc("Whether or not cleanup the files not served by the external shuffle service " + - "on executor exits.") - .version("2.4.0") - .booleanConf - .createWithDefault(true) - - private[spark] val DISKSTORE_SUB_DIRECTORIES = - ConfigBuilder("spark.diskStore.subDirectories") - .doc("Number of subdirectories inside each path listed in spark.local.dir for " + - "hashing Block files into.") - .version("0.6.0") - .intConf - .checkValue(_ > 0, "The number of subdirectories must be positive.") - .createWithDefault(64) - - private[spark] val BLOCK_FAILURES_BEFORE_LOCATION_REFRESH = - ConfigBuilder("spark.block.failures.beforeLocationRefresh") - .doc("Max number of failures before this block manager refreshes " + - "the block locations from the driver.") - .version("2.0.0") - .intConf - .createWithDefault(5) - - private[spark] val IS_PYTHON_APP = - ConfigBuilder("spark.yarn.isPython") - .internal() - .version("1.5.0") - .booleanConf - .createWithDefault(false) - - private[spark] val CPUS_PER_TASK = - ConfigBuilder("spark.task.cpus").version("0.5.0").intConf.createWithDefault(1) - - private[spark] val DYN_ALLOCATION_ENABLED = - ConfigBuilder("spark.dynamicAllocation.enabled") - .version("1.2.0") - .booleanConf - .createWithDefault(false) - - private[spark] val DYN_ALLOCATION_TESTING = - ConfigBuilder("spark.dynamicAllocation.testing") - .version("1.2.0") - .booleanConf - .createWithDefault(false) - - private[spark] val DYN_ALLOCATION_MIN_EXECUTORS = - ConfigBuilder("spark.dynamicAllocation.minExecutors") - .version("1.2.0") - .intConf - .createWithDefault(0) - - private[spark] val DYN_ALLOCATION_INITIAL_EXECUTORS = - ConfigBuilder("spark.dynamicAllocation.initialExecutors") - .version("1.3.0") - .fallbackConf(DYN_ALLOCATION_MIN_EXECUTORS) - - private[spark] val DYN_ALLOCATION_MAX_EXECUTORS = - ConfigBuilder("spark.dynamicAllocation.maxExecutors") - .version("1.2.0") - .intConf - .createWithDefault(Int.MaxValue) - - private[spark] val DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO = - ConfigBuilder("spark.dynamicAllocation.executorAllocationRatio") - .version("2.4.0") - .doubleConf - .createWithDefault(1.0) - - private[spark] val DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT = - ConfigBuilder("spark.dynamicAllocation.cachedExecutorIdleTimeout") - .version("1.4.0") - .timeConf(TimeUnit.SECONDS) - .checkValue(_ >= 0L, "Timeout must be >= 0.") - .createWithDefault(Integer.MAX_VALUE) - - private[spark] val DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT = - ConfigBuilder("spark.dynamicAllocation.executorIdleTimeout") - .version("1.2.0") - .timeConf(TimeUnit.SECONDS) - .checkValue(_ >= 0L, "Timeout must be >= 0.") - .createWithDefault(60) - - private[spark] val DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED = - ConfigBuilder("spark.dynamicAllocation.shuffleTracking.enabled") - .version("3.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val DYN_ALLOCATION_SHUFFLE_TRACKING_TIMEOUT = - ConfigBuilder("spark.dynamicAllocation.shuffleTracking.timeout") - .version("3.0.0") - .timeConf(TimeUnit.MILLISECONDS) - .checkValue(_ >= 0L, "Timeout must be >= 0.") - .createWithDefault(Long.MaxValue) - - private[spark] val DYN_ALLOCATION_SCHEDULER_BACKLOG_TIMEOUT = - ConfigBuilder("spark.dynamicAllocation.schedulerBacklogTimeout") - .version("1.2.0") - .timeConf(TimeUnit.SECONDS).createWithDefault(1) - - private[spark] val DYN_ALLOCATION_SUSTAINED_SCHEDULER_BACKLOG_TIMEOUT = - ConfigBuilder("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout") - .version("1.2.0") - .fallbackConf(DYN_ALLOCATION_SCHEDULER_BACKLOG_TIMEOUT) - - private[spark] val LOCALITY_WAIT = ConfigBuilder("spark.locality.wait") - .version("0.5.0") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString("3s") - - private[spark] val SHUFFLE_SERVICE_ENABLED = - ConfigBuilder("spark.shuffle.service.enabled") - .version("1.2.0") - .booleanConf - .createWithDefault(false) - - private[spark] val SHUFFLE_SERVICE_FETCH_RDD_ENABLED = - ConfigBuilder(Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED) - .doc("Whether to use the ExternalShuffleService for fetching disk persisted RDD blocks. " + - "In case of dynamic allocation if this feature is enabled executors having only disk " + - "persisted blocks are considered idle after " + - "'spark.dynamicAllocation.executorIdleTimeout' and will be released accordingly.") - .version("3.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val SHUFFLE_SERVICE_DB_ENABLED = - ConfigBuilder("spark.shuffle.service.db.enabled") - .doc("Whether to use db in ExternalShuffleService. Note that this only affects " + - "standalone mode.") - .version("3.0.0") - .booleanConf - .createWithDefault(true) - - private[spark] val SHUFFLE_SERVICE_PORT = - ConfigBuilder("spark.shuffle.service.port").version("1.2.0").intConf.createWithDefault(7337) - - private[spark] val KEYTAB = ConfigBuilder("spark.kerberos.keytab") - .doc("Location of user's keytab.") - .version("3.0.0") - .stringConf.createOptional - - private[spark] val PRINCIPAL = ConfigBuilder("spark.kerberos.principal") - .doc("Name of the Kerberos principal.") - .version("3.0.0") - .stringConf - .createOptional - - private[spark] val KERBEROS_RELOGIN_PERIOD = ConfigBuilder("spark.kerberos.relogin.period") - .version("3.0.0") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("1m") - - private[spark] val KERBEROS_RENEWAL_CREDENTIALS = - ConfigBuilder("spark.kerberos.renewal.credentials") - .doc( - "Which credentials to use when renewing delegation tokens for executors. Can be either " + - "'keytab', the default, which requires a keytab to be provided, or 'ccache', which uses " + - "the local credentials cache.") - .version("3.0.0") - .stringConf - .checkValues(Set("keytab", "ccache")) - .createWithDefault("keytab") - - private[spark] val KERBEROS_FILESYSTEMS_TO_ACCESS = - ConfigBuilder("spark.kerberos.access.hadoopFileSystems") - .doc("Extra Hadoop filesystem URLs for which to request delegation tokens. The filesystem " + - "that hosts fs.defaultFS does not need to be listed here.") - .version("3.0.0") - .stringConf - .toSequence - .createWithDefault(Nil) - - private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances") - .version("1.0.0") - .intConf - .createOptional - - private[spark] val PY_FILES = ConfigBuilder("spark.yarn.dist.pyFiles") - .internal() - .version("2.2.1") - .stringConf - .toSequence - .createWithDefault(Nil) - - private[spark] val TASK_MAX_DIRECT_RESULT_SIZE = - ConfigBuilder("spark.task.maxDirectResultSize") - .version("2.0.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefault(1L << 20) - - private[spark] val TASK_MAX_FAILURES = - ConfigBuilder("spark.task.maxFailures") - .version("0.8.0") - .intConf - .createWithDefault(4) - - private[spark] val TASK_REAPER_ENABLED = - ConfigBuilder("spark.task.reaper.enabled") - .version("2.0.3") - .booleanConf - .createWithDefault(false) - - private[spark] val TASK_REAPER_KILL_TIMEOUT = - ConfigBuilder("spark.task.reaper.killTimeout") - .version("2.0.3") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefault(-1) - - private[spark] val TASK_REAPER_POLLING_INTERVAL = - ConfigBuilder("spark.task.reaper.pollingInterval") - .version("2.0.3") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString("10s") - - private[spark] val TASK_REAPER_THREAD_DUMP = - ConfigBuilder("spark.task.reaper.threadDump") - .version("2.0.3") - .booleanConf - .createWithDefault(true) - - // Blacklist confs - private[spark] val BLACKLIST_ENABLED = - ConfigBuilder("spark.blacklist.enabled") - .version("2.1.0") - .booleanConf - .createOptional - - private[spark] val MAX_TASK_ATTEMPTS_PER_EXECUTOR = - ConfigBuilder("spark.blacklist.task.maxTaskAttemptsPerExecutor") - .version("2.1.0") - .intConf - .createWithDefault(1) - - private[spark] val MAX_TASK_ATTEMPTS_PER_NODE = - ConfigBuilder("spark.blacklist.task.maxTaskAttemptsPerNode") - .version("2.1.0") - .intConf - .createWithDefault(2) - - private[spark] val MAX_FAILURES_PER_EXEC = - ConfigBuilder("spark.blacklist.application.maxFailedTasksPerExecutor") - .version("2.2.0") - .intConf - .createWithDefault(2) - - private[spark] val MAX_FAILURES_PER_EXEC_STAGE = - ConfigBuilder("spark.blacklist.stage.maxFailedTasksPerExecutor") - .version("2.1.0") - .intConf - .createWithDefault(2) - - private[spark] val MAX_FAILED_EXEC_PER_NODE = - ConfigBuilder("spark.blacklist.application.maxFailedExecutorsPerNode") - .version("2.2.0") - .intConf - .createWithDefault(2) - - private[spark] val MAX_FAILED_EXEC_PER_NODE_STAGE = - ConfigBuilder("spark.blacklist.stage.maxFailedExecutorsPerNode") - .version("2.1.0") - .intConf - .createWithDefault(2) - - private[spark] val BLACKLIST_TIMEOUT_CONF = - ConfigBuilder("spark.blacklist.timeout") - .version("2.1.0") - .timeConf(TimeUnit.MILLISECONDS) - .createOptional - - private[spark] val BLACKLIST_KILL_ENABLED = - ConfigBuilder("spark.blacklist.killBlacklistedExecutors") - .version("2.2.0") - .booleanConf - .createWithDefault(false) - - private[spark] val BLACKLIST_LEGACY_TIMEOUT_CONF = - ConfigBuilder("spark.scheduler.executorTaskBlacklistTime") - .internal() - .version("1.0.0") - .timeConf(TimeUnit.MILLISECONDS) - .createOptional - - private[spark] val BLACKLIST_FETCH_FAILURE_ENABLED = - ConfigBuilder("spark.blacklist.application.fetchFailure.enabled") - .version("2.3.0") - .booleanConf - .createWithDefault(false) - // End blacklist confs - - private[spark] val UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE = - ConfigBuilder("spark.files.fetchFailure.unRegisterOutputOnHost") - .doc("Whether to un-register all the outputs on the host in condition that we receive " + - " a FetchFailure. This is set default to false, which means, we only un-register the " + - " outputs related to the exact executor(instead of the host) on a FetchFailure.") - .version("2.3.0") - .booleanConf - .createWithDefault(false) - - private[spark] val LISTENER_BUS_EVENT_QUEUE_CAPACITY = - ConfigBuilder("spark.scheduler.listenerbus.eventqueue.capacity") - .doc("The default capacity for event queues. Spark will try to initialize " + - "an event queue using capacity specified by `spark.scheduler.listenerbus" + - ".eventqueue.queueName.capacity` first. If it's not configured, Spark will " + - "use the default capacity specified by this config.") - .version("2.3.0") - .intConf - .checkValue(_ > 0, "The capacity of listener bus event queue must be positive") - .createWithDefault(10000) - - private[spark] val LISTENER_BUS_METRICS_MAX_LISTENER_CLASSES_TIMED = - ConfigBuilder("spark.scheduler.listenerbus.metrics.maxListenerClassesTimed") - .internal() - .version("2.3.0") - .intConf - .createWithDefault(128) - - private[spark] val LISTENER_BUS_LOG_SLOW_EVENT_ENABLED = - ConfigBuilder("spark.scheduler.listenerbus.logSlowEvent") - .internal() - .doc("When enabled, log the event that takes too much time to process. This helps us " + - "discover the event types that cause performance bottlenecks. The time threshold is " + - "controlled by spark.scheduler.listenerbus.logSlowEvent.threshold.") - .version("3.0.0") - .booleanConf - .createWithDefault(true) - - private[spark] val LISTENER_BUS_LOG_SLOW_EVENT_TIME_THRESHOLD = - ConfigBuilder("spark.scheduler.listenerbus.logSlowEvent.threshold") - .internal() - .doc("The time threshold of whether a event is considered to be taking too much time to " + - s"process. Log the event if ${LISTENER_BUS_LOG_SLOW_EVENT_ENABLED.key} is true.") - .version("3.0.0") - .timeConf(TimeUnit.NANOSECONDS) - .createWithDefaultString("1s") - - // This property sets the root namespace for metrics reporting - private[spark] val METRICS_NAMESPACE = ConfigBuilder("spark.metrics.namespace") - .version("2.1.0") - .stringConf - .createOptional - - private[spark] val METRICS_CONF = ConfigBuilder("spark.metrics.conf") - .version("0.8.0") - .stringConf - .createOptional - - private[spark] val METRICS_EXECUTORMETRICS_SOURCE_ENABLED = - ConfigBuilder("spark.metrics.executorMetricsSource.enabled") - .doc("Whether to register the ExecutorMetrics source with the metrics system.") - .version("3.0.0") - .booleanConf - .createWithDefault(true) - - private[spark] val METRICS_STATIC_SOURCES_ENABLED = - ConfigBuilder("spark.metrics.staticSources.enabled") - .doc("Whether to register static sources with the metrics system.") - .version("3.0.0") - .booleanConf - .createWithDefault(true) - - private[spark] val PYSPARK_DRIVER_PYTHON = ConfigBuilder("spark.pyspark.driver.python") - .version("2.1.0") - .stringConf - .createOptional - - private[spark] val PYSPARK_PYTHON = ConfigBuilder("spark.pyspark.python") - .version("2.1.0") - .stringConf - .createOptional - - // To limit how many applications are shown in the History Server summary ui - private[spark] val HISTORY_UI_MAX_APPS = - ConfigBuilder("spark.history.ui.maxApplications") - .version("2.0.1") - .intConf - .createWithDefault(Integer.MAX_VALUE) - - private[spark] val IO_ENCRYPTION_ENABLED = ConfigBuilder("spark.io.encryption.enabled") - .version("2.1.0") - .booleanConf - .createWithDefault(false) - - private[spark] val IO_ENCRYPTION_KEYGEN_ALGORITHM = - ConfigBuilder("spark.io.encryption.keygen.algorithm") - .version("2.1.0") - .stringConf - .createWithDefault("HmacSHA1") - - private[spark] val IO_ENCRYPTION_KEY_SIZE_BITS = ConfigBuilder("spark.io.encryption.keySizeBits") - .version("2.1.0") - .intConf - .checkValues(Set(128, 192, 256)) - .createWithDefault(128) - - private[spark] val IO_CRYPTO_CIPHER_TRANSFORMATION = - ConfigBuilder("spark.io.crypto.cipher.transformation") - .internal() - .version("2.1.0") - .stringConf - .createWithDefaultString("AES/CTR/NoPadding") - - private[spark] val DRIVER_HOST_ADDRESS = ConfigBuilder("spark.driver.host") - .doc("Address of driver endpoints.") - .version("0.7.0") - .stringConf - .createWithDefault(Utils.localCanonicalHostName()) - - private[spark] val DRIVER_PORT = ConfigBuilder("spark.driver.port") - .doc("Port of driver endpoints.") - .version("0.7.0") - .intConf - .createWithDefault(0) - - private[spark] val DRIVER_SUPERVISE = ConfigBuilder("spark.driver.supervise") - .doc("If true, restarts the driver automatically if it fails with a non-zero exit status. " + - "Only has effect in Spark standalone mode or Mesos cluster deploy mode.") - .version("1.3.0") - .booleanConf - .createWithDefault(false) - - private[spark] val DRIVER_BIND_ADDRESS = ConfigBuilder("spark.driver.bindAddress") - .doc("Address where to bind network listen sockets on the driver.") - .version("2.1.0") - .fallbackConf(DRIVER_HOST_ADDRESS) - - private[spark] val BLOCK_MANAGER_PORT = ConfigBuilder("spark.blockManager.port") - .doc("Port to use for the block manager when a more specific setting is not provided.") - .version("1.1.0") - .intConf - .createWithDefault(0) - - private[spark] val DRIVER_BLOCK_MANAGER_PORT = ConfigBuilder("spark.driver.blockManager.port") - .doc("Port to use for the block manager on the driver.") - .version("2.1.0") - .fallbackConf(BLOCK_MANAGER_PORT) - - private[spark] val IGNORE_CORRUPT_FILES = ConfigBuilder("spark.files.ignoreCorruptFiles") - .doc("Whether to ignore corrupt files. If true, the Spark jobs will continue to run when " + - "encountering corrupted or non-existing files and contents that have been read will still " + - "be returned.") - .version("2.1.0") - .booleanConf - .createWithDefault(false) - - private[spark] val IGNORE_MISSING_FILES = ConfigBuilder("spark.files.ignoreMissingFiles") - .doc("Whether to ignore missing files. If true, the Spark jobs will continue to run when " + - "encountering missing files and the contents that have been read will still be returned.") - .version("2.4.0") - .booleanConf - .createWithDefault(false) - - private[spark] val APP_CALLER_CONTEXT = ConfigBuilder("spark.log.callerContext") - .version("2.2.0") - .stringConf - .createOptional - - private[spark] val FILES_MAX_PARTITION_BYTES = ConfigBuilder("spark.files.maxPartitionBytes") - .doc("The maximum number of bytes to pack into a single partition when reading files.") - .version("2.1.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefault(128 * 1024 * 1024) - - private[spark] val FILES_OPEN_COST_IN_BYTES = ConfigBuilder("spark.files.openCostInBytes") - .doc("The estimated cost to open a file, measured by the number of bytes could be scanned in" + - " the same time. This is used when putting multiple files into a partition. It's better to" + - " over estimate, then the partitions with small files will be faster than partitions with" + - " bigger files.") - .version("2.1.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefault(4 * 1024 * 1024) - - private[spark] val HADOOP_RDD_IGNORE_EMPTY_SPLITS = - ConfigBuilder("spark.hadoopRDD.ignoreEmptySplits") - .internal() - .doc("When true, HadoopRDD/NewHadoopRDD will not create partitions for empty input splits.") - .version("2.3.0") - .booleanConf - .createWithDefault(false) - - private[spark] val SECRET_REDACTION_PATTERN = - ConfigBuilder("spark.redaction.regex") - .doc("Regex to decide which Spark configuration properties and environment variables in " + - "driver and executor environments contain sensitive information. When this regex matches " + - "a property key or value, the value is redacted from the environment UI and various logs " + - "like YARN and event logs.") - .version("2.1.2") - .regexConf - .createWithDefault("(?i)secret|password|token".r) - - private[spark] val STRING_REDACTION_PATTERN = - ConfigBuilder("spark.redaction.string.regex") - .doc("Regex to decide which parts of strings produced by Spark contain sensitive " + - "information. When this regex matches a string part, that string part is replaced by a " + - "dummy value. This is currently used to redact the output of SQL explain commands.") - .version("2.2.0") - .regexConf - .createOptional - - private[spark] val AUTH_SECRET = - ConfigBuilder("spark.authenticate.secret") - .version("1.0.0") - .stringConf - .createOptional - - private[spark] val AUTH_SECRET_BIT_LENGTH = - ConfigBuilder("spark.authenticate.secretBitLength") - .version("1.6.0") - .intConf - .createWithDefault(256) - - private[spark] val NETWORK_AUTH_ENABLED = - ConfigBuilder("spark.authenticate") - .version("1.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val SASL_ENCRYPTION_ENABLED = - ConfigBuilder("spark.authenticate.enableSaslEncryption") - .version("1.4.0") - .booleanConf - .createWithDefault(false) - - private[spark] val AUTH_SECRET_FILE = - ConfigBuilder("spark.authenticate.secret.file") - .doc("Path to a file that contains the authentication secret to use. The secret key is " + - "loaded from this path on both the driver and the executors if overrides are not set for " + - "either entity (see below). File-based secret keys are only allowed when using " + - "Kubernetes.") - .version("3.0.0") - .stringConf - .createOptional - - private[spark] val AUTH_SECRET_FILE_DRIVER = - ConfigBuilder("spark.authenticate.secret.driver.file") - .doc("Path to a file that contains the authentication secret to use. Loaded by the " + - "driver. In Kubernetes client mode it is often useful to set a different secret " + - "path for the driver vs. the executors, since the driver may not be running in " + - "a pod unlike the executors. If this is set, an accompanying secret file must " + - "be specified for the executors. The fallback configuration allows the same path to be " + - "used for both the driver and the executors when running in cluster mode. File-based " + - "secret keys are only allowed when using Kubernetes.") - .version("3.0.0") - .fallbackConf(AUTH_SECRET_FILE) - - private[spark] val AUTH_SECRET_FILE_EXECUTOR = - ConfigBuilder("spark.authenticate.secret.executor.file") - .doc("Path to a file that contains the authentication secret to use. Loaded by the " + - "executors only. In Kubernetes client mode it is often useful to set a different " + - "secret path for the driver vs. the executors, since the driver may not be running " + - "in a pod unlike the executors. If this is set, an accompanying secret file must be " + - "specified for the executors. The fallback configuration allows the same path to be " + - "used for both the driver and the executors when running in cluster mode. File-based " + - "secret keys are only allowed when using Kubernetes.") - .version("3.0.0") - .fallbackConf(AUTH_SECRET_FILE) - - private[spark] val BUFFER_WRITE_CHUNK_SIZE = - ConfigBuilder("spark.buffer.write.chunkSize") - .internal() - .doc("The chunk size in bytes during writing out the bytes of ChunkedByteBuffer.") - .version("2.3.0") - .bytesConf(ByteUnit.BYTE) - .checkValue(_ <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, - "The chunk size during writing out the bytes of ChunkedByteBuffer should" + - s" be less than or equal to ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") - .createWithDefault(64 * 1024 * 1024) - - private[spark] val CHECKPOINT_COMPRESS = - ConfigBuilder("spark.checkpoint.compress") - .doc("Whether to compress RDD checkpoints. Generally a good idea. Compression will use " + - "spark.io.compression.codec.") - .version("2.2.0") - .booleanConf - .createWithDefault(false) - - private[spark] val CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME = - ConfigBuilder("spark.rdd.checkpoint.cachePreferredLocsExpireTime") - .internal() - .doc("Expire time in minutes for caching preferred locations of checkpointed RDD." + - "Caching preferred locations can relieve query loading to DFS and save the query " + - "time. The drawback is that the cached locations can be possibly outdated and " + - "lose data locality. If this config is not specified, it will not cache.") - .version("3.0.0") - .timeConf(TimeUnit.MINUTES) - .checkValue(_ > 0, "The expire time for caching preferred locations cannot be non-positive.") - .createOptional - - private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD = - ConfigBuilder("spark.shuffle.accurateBlockThreshold") - .doc("Threshold in bytes above which the size of shuffle blocks in " + - "HighlyCompressedMapStatus is accurately recorded. This helps to prevent OOM " + - "by avoiding underestimating shuffle block size when fetch shuffle blocks.") - .version("2.2.1") - .bytesConf(ByteUnit.BYTE) - .createWithDefault(100 * 1024 * 1024) - - private[spark] val SHUFFLE_REGISTRATION_TIMEOUT = - ConfigBuilder("spark.shuffle.registration.timeout") - .doc("Timeout in milliseconds for registration to the external shuffle service.") - .version("2.3.0") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefault(5000) - - private[spark] val SHUFFLE_REGISTRATION_MAX_ATTEMPTS = - ConfigBuilder("spark.shuffle.registration.maxAttempts") - .doc("When we fail to register to the external shuffle service, we will " + - "retry for maxAttempts times.") - .version("2.3.0") - .intConf - .createWithDefault(3) - - private[spark] val REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS = - ConfigBuilder("spark.reducer.maxBlocksInFlightPerAddress") - .doc("This configuration limits the number of remote blocks being fetched per reduce task " + - "from a given host port. When a large number of blocks are being requested from a given " + - "address in a single fetch or simultaneously, this could crash the serving executor or " + - "Node Manager. This is especially useful to reduce the load on the Node Manager when " + - "external shuffle is enabled. You can mitigate the issue by setting it to a lower value.") - .version("2.2.1") - .intConf - .checkValue(_ > 0, "The max no. of blocks in flight cannot be non-positive.") - .createWithDefault(Int.MaxValue) - - private[spark] val MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM = - ConfigBuilder("spark.network.maxRemoteBlockSizeFetchToMem") - .doc("Remote block will be fetched to disk when size of the block is above this threshold " + - "in bytes. This is to avoid a giant request takes too much memory. Note this " + - "configuration will affect both shuffle fetch and block manager remote block fetch. " + - "For users who enabled external shuffle service, this feature can only work when " + - "external shuffle service is at least 2.3.0.") - .version("3.0.0") - .bytesConf(ByteUnit.BYTE) - // fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB, so we might - // as well use fetch-to-disk in that case. The message includes some metadata in addition - // to the block data itself (in particular UploadBlock has a lot of metadata), so we leave - // extra room. - .checkValue( - _ <= Int.MaxValue - 512, - "maxRemoteBlockSizeFetchToMem cannot be larger than (Int.MaxValue - 512) bytes.") - .createWithDefaultString("200m") - - private[spark] val TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES = - ConfigBuilder("spark.taskMetrics.trackUpdatedBlockStatuses") - .doc("Enable tracking of updatedBlockStatuses in the TaskMetrics. Off by default since " + - "tracking the block statuses can use a lot of memory and its not used anywhere within " + - "spark.") - .version("2.3.0") - .booleanConf - .createWithDefault(false) - - private[spark] val SHUFFLE_IO_PLUGIN_CLASS = - ConfigBuilder("spark.shuffle.sort.io.plugin.class") - .doc("Name of the class to use for shuffle IO.") - .version("3.0.0") - .stringConf - .createWithDefault(classOf[LocalDiskShuffleDataIO].getName) - - private[spark] val SHUFFLE_FILE_BUFFER_SIZE = - ConfigBuilder("spark.shuffle.file.buffer") - .doc("Size of the in-memory buffer for each shuffle file output stream, in KiB unless " + - "otherwise specified. These buffers reduce the number of disk seeks and system calls " + - "made in creating intermediate shuffle files.") - .version("1.4.0") - .bytesConf(ByteUnit.KiB) - .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, - s"The file buffer size must be positive and less than or equal to" + - s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") - .createWithDefaultString("32k") - - private[spark] val SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE = - ConfigBuilder("spark.shuffle.unsafe.file.output.buffer") - .doc("The file system for this buffer size after each partition " + - "is written in unsafe shuffle writer. In KiB unless otherwise specified.") - .version("2.3.0") - .bytesConf(ByteUnit.KiB) - .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, - s"The buffer size must be positive and less than or equal to" + - s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") - .createWithDefaultString("32k") - - private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE = - ConfigBuilder("spark.shuffle.spill.diskWriteBufferSize") - .doc("The buffer size, in bytes, to use when writing the sorted records to an on-disk file.") - .version("2.3.0") - .bytesConf(ByteUnit.BYTE) - .checkValue(v => v > 12 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, - s"The buffer size must be greater than 12 and less than or equal to " + - s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") - .createWithDefault(1024 * 1024) - - private[spark] val UNROLL_MEMORY_CHECK_PERIOD = - ConfigBuilder("spark.storage.unrollMemoryCheckPeriod") - .internal() - .doc("The memory check period is used to determine how often we should check whether " - + "there is a need to request more memory when we try to unroll the given block in memory.") - .version("2.3.0") - .longConf - .createWithDefault(16) - - private[spark] val UNROLL_MEMORY_GROWTH_FACTOR = - ConfigBuilder("spark.storage.unrollMemoryGrowthFactor") - .internal() - .doc("Memory to request as a multiple of the size that used to unroll the block.") - .version("2.3.0") - .doubleConf - .createWithDefault(1.5) - - private[spark] val FORCE_DOWNLOAD_SCHEMES = - ConfigBuilder("spark.yarn.dist.forceDownloadSchemes") - .doc("Comma-separated list of schemes for which resources will be downloaded to the " + - "local disk prior to being added to YARN's distributed cache. For use in cases " + - "where the YARN service does not support schemes that are supported by Spark, like http, " + - "https and ftp, or jars required to be in the local YARN client's classpath. Wildcard " + - "'*' is denoted to download resources for all the schemes.") - .version("2.3.0") - .stringConf - .toSequence - .createWithDefault(Nil) - - private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners") - .doc("Class names of listeners to add to SparkContext during initialization.") - .version("1.3.0") - .stringConf - .toSequence - .createOptional - - private[spark] val SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD = - ConfigBuilder("spark.shuffle.spill.numElementsForceSpillThreshold") - .internal() - .doc("The maximum number of elements in memory before forcing the shuffle sorter to spill. " + - "By default it's Integer.MAX_VALUE, which means we never force the sorter to spill, " + - "until we reach some limitations, like the max page size limitation for the pointer " + - "array in the sorter.") - .version("1.6.0") - .intConf - .createWithDefault(Integer.MAX_VALUE) - - private[spark] val SHUFFLE_MAP_OUTPUT_PARALLEL_AGGREGATION_THRESHOLD = - ConfigBuilder("spark.shuffle.mapOutput.parallelAggregationThreshold") - .internal() - .doc("Multi-thread is used when the number of mappers * shuffle partitions is greater than " + - "or equal to this threshold. Note that the actual parallelism is calculated by number of " + - "mappers * shuffle partitions / this threshold + 1, so this threshold should be positive.") - .version("2.3.0") - .intConf - .checkValue(v => v > 0, "The threshold should be positive.") - .createWithDefault(10000000) - - private[spark] val MAX_RESULT_SIZE = ConfigBuilder("spark.driver.maxResultSize") - .doc("Size limit for results.") - .version("1.2.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("1g") - - private[spark] val CREDENTIALS_RENEWAL_INTERVAL_RATIO = - ConfigBuilder("spark.security.credentials.renewalRatio") - .doc("Ratio of the credential's expiration time when Spark should fetch new credentials.") - .version("2.4.0") - .doubleConf - .createWithDefault(0.75d) - - private[spark] val CREDENTIALS_RENEWAL_RETRY_WAIT = - ConfigBuilder("spark.security.credentials.retryWait") - .doc("How long to wait before retrying to fetch new credentials after a failure.") - .version("2.4.0") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("1h") - - private[spark] val SHUFFLE_SORT_INIT_BUFFER_SIZE = - ConfigBuilder("spark.shuffle.sort.initialBufferSize") - .internal() - .version("2.1.0") - .bytesConf(ByteUnit.BYTE) - .checkValue(v => v > 0 && v <= Int.MaxValue, - s"The buffer size must be greater than 0 and less than or equal to ${Int.MaxValue}.") - .createWithDefault(4096) - - private[spark] val SHUFFLE_COMPRESS = - ConfigBuilder("spark.shuffle.compress") - .doc("Whether to compress shuffle output. Compression will use " + - "spark.io.compression.codec.") - .version("0.6.0") - .booleanConf - .createWithDefault(true) - - private[spark] val SHUFFLE_SPILL_COMPRESS = - ConfigBuilder("spark.shuffle.spill.compress") - .doc("Whether to compress data spilled during shuffles. Compression will use " + - "spark.io.compression.codec.") - .version("0.9.0") - .booleanConf - .createWithDefault(true) - - private[spark] val MAP_STATUS_COMPRESSION_CODEC = - ConfigBuilder("spark.shuffle.mapStatus.compression.codec") - .internal() - .doc("The codec used to compress MapStatus, which is generated by ShuffleMapTask. " + - "By default, Spark provides four codecs: lz4, lzf, snappy, and zstd. You can also " + - "use fully qualified class names to specify the codec.") - .version("3.0.0") - .stringConf - .createWithDefault("zstd") - - private[spark] val SHUFFLE_SPILL_INITIAL_MEM_THRESHOLD = - ConfigBuilder("spark.shuffle.spill.initialMemoryThreshold") - .internal() - .doc("Initial threshold for the size of a collection before we start tracking its " + - "memory usage.") - .version("1.1.1") - .bytesConf(ByteUnit.BYTE) - .createWithDefault(5 * 1024 * 1024) - - private[spark] val SHUFFLE_SPILL_BATCH_SIZE = - ConfigBuilder("spark.shuffle.spill.batchSize") - .internal() - .doc("Size of object batches when reading/writing from serializers.") - .version("0.9.0") - .longConf - .createWithDefault(10000) - - private[spark] val SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD = - ConfigBuilder("spark.shuffle.sort.bypassMergeThreshold") - .doc("In the sort-based shuffle manager, avoid merge-sorting data if there is no " + - "map-side aggregation and there are at most this many reduce partitions") - .version("1.1.1") - .intConf - .createWithDefault(200) - - private[spark] val SHUFFLE_MANAGER = - ConfigBuilder("spark.shuffle.manager") - .version("1.1.0") - .stringConf - .createWithDefault("sort") - - private[spark] val SHUFFLE_REDUCE_LOCALITY_ENABLE = - ConfigBuilder("spark.shuffle.reduceLocality.enabled") - .doc("Whether to compute locality preferences for reduce tasks") - .version("1.5.0") - .booleanConf - .createWithDefault(true) - - private[spark] val SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST = - ConfigBuilder("spark.shuffle.mapOutput.minSizeForBroadcast") - .doc("The size at which we use Broadcast to send the map output statuses to the executors.") - .version("2.0.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("512k") - - private[spark] val SHUFFLE_MAPOUTPUT_DISPATCHER_NUM_THREADS = - ConfigBuilder("spark.shuffle.mapOutput.dispatcher.numThreads") - .version("2.0.0") - .intConf - .createWithDefault(8) - - private[spark] val SHUFFLE_DETECT_CORRUPT = - ConfigBuilder("spark.shuffle.detectCorrupt") - .doc("Whether to detect any corruption in fetched blocks.") - .version("2.2.0") - .booleanConf - .createWithDefault(true) - - private[spark] val SHUFFLE_DETECT_CORRUPT_MEMORY = - ConfigBuilder("spark.shuffle.detectCorrupt.useExtraMemory") - .doc("If enabled, part of a compressed/encrypted stream will be de-compressed/de-crypted " + - "by using extra memory to detect early corruption. Any IOException thrown will cause " + - "the task to be retried once and if it fails again with same exception, then " + - "FetchFailedException will be thrown to retry previous stage") - .version("3.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val SHUFFLE_SYNC = - ConfigBuilder("spark.shuffle.sync") - .doc("Whether to force outstanding writes to disk.") - .version("0.8.0") - .booleanConf - .createWithDefault(false) - - private[spark] val SHUFFLE_UNSAFE_FAST_MERGE_ENABLE = - ConfigBuilder("spark.shuffle.unsafe.fastMergeEnabled") - .doc("Whether to perform a fast spill merge.") - .version("1.4.0") - .booleanConf - .createWithDefault(true) - - private[spark] val SHUFFLE_SORT_USE_RADIXSORT = - ConfigBuilder("spark.shuffle.sort.useRadixSort") - .doc("Whether to use radix sort for sorting in-memory partition ids. Radix sort is much " + - "faster, but requires additional memory to be reserved memory as pointers are added.") - .version("2.0.0") - .booleanConf - .createWithDefault(true) - - private[spark] val SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS = - ConfigBuilder("spark.shuffle.minNumPartitionsToHighlyCompress") - .internal() - .doc("Number of partitions to determine if MapStatus should use HighlyCompressedMapStatus") - .version("2.4.0") - .intConf - .checkValue(v => v > 0, "The value should be a positive integer.") - .createWithDefault(2000) - - private[spark] val SHUFFLE_USE_OLD_FETCH_PROTOCOL = - ConfigBuilder("spark.shuffle.useOldFetchProtocol") - .doc("Whether to use the old protocol while doing the shuffle block fetching. " + - "It is only enabled while we need the compatibility in the scenario of new Spark " + - "version job fetching shuffle blocks from old version external shuffle service.") - .version("3.0.0") - .booleanConf - .createWithDefault(false) - - private[spark] val SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED = - ConfigBuilder("spark.shuffle.readHostLocalDisk") - .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and external " + - s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled), shuffle " + - "blocks requested from those block managers which are running on the same host are read " + - "from the disk directly instead of being fetched as remote blocks over the network.") - .version("3.0.0") - .booleanConf - .createWithDefault(true) - - private[spark] val STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE = - ConfigBuilder("spark.storage.localDiskByExecutors.cacheSize") - .doc("The max number of executors for which the local dirs are stored. This size is " + - "both applied for the driver and both for the executors side to avoid having an " + - "unbounded store. This cache will be used to avoid the network in case of fetching disk " + - s"persisted RDD blocks or shuffle blocks " + - s"(when `${SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED.key}` is set) from the same host.") - .version("3.0.0") - .intConf - .createWithDefault(1000) - - private[spark] val MEMORY_MAP_LIMIT_FOR_TESTS = - ConfigBuilder("spark.storage.memoryMapLimitForTests") - .internal() - .doc("For testing only, controls the size of chunks when memory mapping a file") - .version("2.3.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) - - private[spark] val BARRIER_SYNC_TIMEOUT = - ConfigBuilder("spark.barrier.sync.timeout") - .doc("The timeout in seconds for each barrier() call from a barrier task. If the " + - "coordinator didn't receive all the sync messages from barrier tasks within the " + - "configured time, throw a SparkException to fail all the tasks. The default value is set " + - "to 31536000(3600 * 24 * 365) so the barrier() call shall wait for one year.") - .version("2.4.0") - .timeConf(TimeUnit.SECONDS) - .checkValue(v => v > 0, "The value should be a positive time value.") - .createWithDefaultString("365d") - - private[spark] val UNSCHEDULABLE_TASKSET_TIMEOUT = - ConfigBuilder("spark.scheduler.blacklist.unschedulableTaskSetTimeout") - .doc("The timeout in seconds to wait to acquire a new executor and schedule a task " + - "before aborting a TaskSet which is unschedulable because of being completely blacklisted.") - .version("2.4.1") - .timeConf(TimeUnit.SECONDS) - .checkValue(v => v >= 0, "The value should be a non negative time value.") - .createWithDefault(120) - - private[spark] val BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL = - ConfigBuilder("spark.scheduler.barrier.maxConcurrentTasksCheck.interval") - .doc("Time in seconds to wait between a max concurrent tasks check failure and the next " + - "check. A max concurrent tasks check ensures the cluster can launch more concurrent " + - "tasks than required by a barrier stage on job submitted. The check can fail in case " + - "a cluster has just started and not enough executors have registered, so we wait for a " + - "little while and try to perform the check again. If the check fails more than a " + - "configured max failure times for a job then fail current job submission. Note this " + - "config only applies to jobs that contain one or more barrier stages, we won't perform " + - "the check on non-barrier jobs.") - .version("2.4.0") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("15s") - - private[spark] val BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES = - ConfigBuilder("spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures") - .doc("Number of max concurrent tasks check failures allowed before fail a job submission. " + - "A max concurrent tasks check ensures the cluster can launch more concurrent tasks than " + - "required by a barrier stage on job submitted. The check can fail in case a cluster " + - "has just started and not enough executors have registered, so we wait for a little " + - "while and try to perform the check again. If the check fails more than a configured " + - "max failure times for a job then fail current job submission. Note this config only " + - "applies to jobs that contain one or more barrier stages, we won't perform the check on " + - "non-barrier jobs.") - .version("2.4.0") - .intConf - .checkValue(v => v > 0, "The max failures should be a positive value.") - .createWithDefault(40) - - private[spark] val UNSAFE_EXCEPTION_ON_MEMORY_LEAK = - ConfigBuilder("spark.unsafe.exceptionOnMemoryLeak") - .internal() - .version("1.4.0") - .booleanConf - .createWithDefault(false) - - private[spark] val UNSAFE_SORTER_SPILL_READ_AHEAD_ENABLED = - ConfigBuilder("spark.unsafe.sorter.spill.read.ahead.enabled") - .internal() - .version("2.3.0") - .booleanConf - .createWithDefault(true) - - private[spark] val UNSAFE_SORTER_SPILL_READER_BUFFER_SIZE = - ConfigBuilder("spark.unsafe.sorter.spill.reader.buffer.size") - .internal() - .version("2.1.0") - .bytesConf(ByteUnit.BYTE) - .checkValue(v => 1024 * 1024 <= v && v <= MAX_BUFFER_SIZE_BYTES, - s"The value must be in allowed range [1,048,576, ${MAX_BUFFER_SIZE_BYTES}].") - .createWithDefault(1024 * 1024) - - private[spark] val DEFAULT_PLUGINS_LIST = "spark.plugins.defaultList" - - private[spark] val PLUGINS = - ConfigBuilder("spark.plugins") - .withPrepended(DEFAULT_PLUGINS_LIST, separator = ",") - .doc("Comma-separated list of class names implementing " + - "org.apache.spark.api.plugin.SparkPlugin to load into the application.") - .version("3.0.0") - .stringConf - .toSequence - .createWithDefault(Nil) - - private[spark] val CLEANER_PERIODIC_GC_INTERVAL = - ConfigBuilder("spark.cleaner.periodicGC.interval") - .version("1.6.0") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("30min") - - private[spark] val CLEANER_REFERENCE_TRACKING = - ConfigBuilder("spark.cleaner.referenceTracking") - .version("1.0.0") - .booleanConf - .createWithDefault(true) - - private[spark] val CLEANER_REFERENCE_TRACKING_BLOCKING = - ConfigBuilder("spark.cleaner.referenceTracking.blocking") - .version("1.0.0") - .booleanConf - .createWithDefault(true) - - private[spark] val CLEANER_REFERENCE_TRACKING_BLOCKING_SHUFFLE = - ConfigBuilder("spark.cleaner.referenceTracking.blocking.shuffle") - .version("1.1.1") - .booleanConf - .createWithDefault(false) - - private[spark] val CLEANER_REFERENCE_TRACKING_CLEAN_CHECKPOINTS = - ConfigBuilder("spark.cleaner.referenceTracking.cleanCheckpoints") - .version("1.4.0") - .booleanConf - .createWithDefault(false) - - private[spark] val EXECUTOR_LOGS_ROLLING_STRATEGY = - ConfigBuilder("spark.executor.logs.rolling.strategy") - .version("1.1.0") - .stringConf - .createWithDefault("") - - private[spark] val EXECUTOR_LOGS_ROLLING_TIME_INTERVAL = - ConfigBuilder("spark.executor.logs.rolling.time.interval") - .version("1.1.0") - .stringConf - .createWithDefault("daily") - - private[spark] val EXECUTOR_LOGS_ROLLING_MAX_SIZE = - ConfigBuilder("spark.executor.logs.rolling.maxSize") - .version("1.4.0") - .stringConf - .createWithDefault((1024 * 1024).toString) - - private[spark] val EXECUTOR_LOGS_ROLLING_MAX_RETAINED_FILES = - ConfigBuilder("spark.executor.logs.rolling.maxRetainedFiles") - .version("1.1.0") - .intConf - .createWithDefault(-1) - - private[spark] val EXECUTOR_LOGS_ROLLING_ENABLE_COMPRESSION = - ConfigBuilder("spark.executor.logs.rolling.enableCompression") - .version("2.0.2") - .booleanConf - .createWithDefault(false) - - private[spark] val MASTER_REST_SERVER_ENABLED = ConfigBuilder("spark.master.rest.enabled") - .version("1.3.0") - .booleanConf - .createWithDefault(false) - - private[spark] val MASTER_REST_SERVER_PORT = ConfigBuilder("spark.master.rest.port") - .version("1.3.0") - .intConf - .createWithDefault(6066) - - private[spark] val MASTER_UI_PORT = ConfigBuilder("spark.master.ui.port") - .version("1.1.0") - .intConf - .createWithDefault(8080) - - private[spark] val IO_COMPRESSION_SNAPPY_BLOCKSIZE = - ConfigBuilder("spark.io.compression.snappy.blockSize") - .doc("Block size in bytes used in Snappy compression, in the case when " + - "Snappy compression codec is used. Lowering this block size " + - "will also lower shuffle memory usage when Snappy is used") - .version("1.4.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("32k") - - private[spark] val IO_COMPRESSION_LZ4_BLOCKSIZE = - ConfigBuilder("spark.io.compression.lz4.blockSize") - .doc("Block size in bytes used in LZ4 compression, in the case when LZ4 compression" + - "codec is used. Lowering this block size will also lower shuffle memory " + - "usage when LZ4 is used.") - .version("1.4.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("32k") - - private[spark] val IO_COMPRESSION_CODEC = - ConfigBuilder("spark.io.compression.codec") - .doc("The codec used to compress internal data such as RDD partitions, event log, " + - "broadcast variables and shuffle outputs. By default, Spark provides four codecs: " + - "lz4, lzf, snappy, and zstd. You can also use fully qualified class names to specify " + - "the codec") - .version("0.8.0") - .stringConf - .createWithDefaultString("lz4") - - private[spark] val IO_COMPRESSION_ZSTD_BUFFERSIZE = - ConfigBuilder("spark.io.compression.zstd.bufferSize") - .doc("Buffer size in bytes used in Zstd compression, in the case when Zstd " + - "compression codec is used. Lowering this size will lower the shuffle " + - "memory usage when Zstd is used, but it might increase the compression " + - "cost because of excessive JNI call overhead") - .version("2.3.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("32k") - - private[spark] val IO_COMPRESSION_ZSTD_LEVEL = - ConfigBuilder("spark.io.compression.zstd.level") - .doc("Compression level for Zstd compression codec. Increasing the compression " + - "level will result in better compression at the expense of more CPU and memory") - .version("2.3.0") - .intConf - .createWithDefault(1) - - private[spark] val IO_WARNING_LARGEFILETHRESHOLD = - ConfigBuilder("spark.io.warning.largeFileThreshold") - .internal() - .doc("If the size in bytes of a file loaded by Spark exceeds this threshold, " + - "a warning is logged with the possible reasons.") - .version("3.0.0") - .bytesConf(ByteUnit.BYTE) - .createWithDefault(1024 * 1024 * 1024) - - private[spark] val EVENT_LOG_COMPRESSION_CODEC = - ConfigBuilder("spark.eventLog.compression.codec") - .doc("The codec used to compress event log. By default, Spark provides four codecs: " + - "lz4, lzf, snappy, and zstd. You can also use fully qualified class names to specify " + - "the codec. If this is not given, spark.io.compression.codec will be used.") - .version("3.0.0") - .fallbackConf(IO_COMPRESSION_CODEC) - - private[spark] val BUFFER_SIZE = - ConfigBuilder("spark.buffer.size") - .version("0.5.0") - .intConf - .checkValue(_ >= 0, "The buffer size must not be negative") - .createWithDefault(65536) - - private[spark] val LOCALITY_WAIT_PROCESS = ConfigBuilder("spark.locality.wait.process") - .version("0.8.0") - .fallbackConf(LOCALITY_WAIT) - - private[spark] val LOCALITY_WAIT_NODE = ConfigBuilder("spark.locality.wait.node") - .version("0.8.0") - .fallbackConf(LOCALITY_WAIT) - - private[spark] val LOCALITY_WAIT_RACK = ConfigBuilder("spark.locality.wait.rack") - .version("0.8.0") - .fallbackConf(LOCALITY_WAIT) - - private[spark] val REDUCER_MAX_SIZE_IN_FLIGHT = ConfigBuilder("spark.reducer.maxSizeInFlight") - .doc("Maximum size of map outputs to fetch simultaneously from each reduce task, " + - "in MiB unless otherwise specified. Since each output requires us to create a " + - "buffer to receive it, this represents a fixed memory overhead per reduce task, " + - "so keep it small unless you have a large amount of memory") - .version("1.4.0") - .bytesConf(ByteUnit.MiB) - .createWithDefaultString("48m") - - private[spark] val REDUCER_MAX_REQS_IN_FLIGHT = ConfigBuilder("spark.reducer.maxReqsInFlight") - .doc("This configuration limits the number of remote requests to fetch blocks at " + - "any given point. When the number of hosts in the cluster increase, " + - "it might lead to very large number of inbound connections to one or more nodes, " + - "causing the workers to fail under load. By allowing it to limit the number of " + - "fetch requests, this scenario can be mitigated") - .version("2.0.0") - .intConf - .createWithDefault(Int.MaxValue) - - private[spark] val BROADCAST_COMPRESS = ConfigBuilder("spark.broadcast.compress") - .doc("Whether to compress broadcast variables before sending them. " + - "Generally a good idea. Compression will use spark.io.compression.codec") - .version("0.6.0") - .booleanConf.createWithDefault(true) - - private[spark] val BROADCAST_BLOCKSIZE = ConfigBuilder("spark.broadcast.blockSize") - .doc("Size of each piece of a block for TorrentBroadcastFactory, in " + - "KiB unless otherwise specified. Too large a value decreases " + - "parallelism during broadcast (makes it slower); however, " + - "if it is too small, BlockManager might take a performance hit") - .version("0.5.0") - .bytesConf(ByteUnit.KiB) - .createWithDefaultString("4m") - - private[spark] val BROADCAST_CHECKSUM = ConfigBuilder("spark.broadcast.checksum") - .doc("Whether to enable checksum for broadcast. If enabled, " + - "broadcasts will include a checksum, which can help detect " + - "corrupted blocks, at the cost of computing and sending a little " + - "more data. It's possible to disable it if the network has other " + - "mechanisms to guarantee data won't be corrupted during broadcast") - .version("2.1.1") - .booleanConf - .createWithDefault(true) - - private[spark] val BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD = - ConfigBuilder("spark.broadcast.UDFCompressionThreshold") - .doc("The threshold at which user-defined functions (UDFs) and Python RDD commands " + - "are compressed by broadcast in bytes unless otherwise specified") - .version("3.0.0") - .bytesConf(ByteUnit.BYTE) - .checkValue(v => v >= 0, "The threshold should be non-negative.") - .createWithDefault(1L * 1024 * 1024) - - private[spark] val RDD_COMPRESS = ConfigBuilder("spark.rdd.compress") - .doc("Whether to compress serialized RDD partitions " + - "(e.g. for StorageLevel.MEMORY_ONLY_SER in Scala " + - "or StorageLevel.MEMORY_ONLY in Python). Can save substantial " + - "space at the cost of some extra CPU time. " + - "Compression will use spark.io.compression.codec") - .version("0.6.0") - .booleanConf - .createWithDefault(false) - - private[spark] val RDD_PARALLEL_LISTING_THRESHOLD = - ConfigBuilder("spark.rdd.parallelListingThreshold") - .version("2.0.0") - .intConf - .createWithDefault(10) - - private[spark] val RDD_LIMIT_SCALE_UP_FACTOR = - ConfigBuilder("spark.rdd.limit.scaleUpFactor") - .version("2.1.0") - .intConf - .createWithDefault(4) - - private[spark] val SERIALIZER = ConfigBuilder("spark.serializer") - .version("0.5.0") - .stringConf - .createWithDefault("org.apache.spark.serializer.JavaSerializer") - - private[spark] val SERIALIZER_OBJECT_STREAM_RESET = - ConfigBuilder("spark.serializer.objectStreamReset") - .version("1.0.0") - .intConf - .createWithDefault(100) - - private[spark] val SERIALIZER_EXTRA_DEBUG_INFO = ConfigBuilder("spark.serializer.extraDebugInfo") - .version("1.3.0") - .booleanConf - .createWithDefault(true) - - private[spark] val JARS = ConfigBuilder("spark.jars") - .version("0.9.0") - .stringConf - .toSequence - .createWithDefault(Nil) - - private[spark] val FILES = ConfigBuilder("spark.files") - .version("1.0.0") - .stringConf - .toSequence - .createWithDefault(Nil) - - private[spark] val SUBMIT_DEPLOY_MODE = ConfigBuilder("spark.submit.deployMode") - .version("1.5.0") - .stringConf - .createWithDefault("client") - - private[spark] val SUBMIT_PYTHON_FILES = ConfigBuilder("spark.submit.pyFiles") - .version("1.0.1") - .stringConf - .toSequence - .createWithDefault(Nil) - - private[spark] val SCHEDULER_ALLOCATION_FILE = - ConfigBuilder("spark.scheduler.allocation.file") - .version("0.8.1") - .stringConf - .createOptional - - private[spark] val SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO = - ConfigBuilder("spark.scheduler.minRegisteredResourcesRatio") - .version("1.1.1") - .doubleConf - .createOptional - - private[spark] val SCHEDULER_MAX_REGISTERED_RESOURCE_WAITING_TIME = - ConfigBuilder("spark.scheduler.maxRegisteredResourcesWaitingTime") - .version("1.1.1") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString("30s") - - private[spark] val SCHEDULER_MODE = - ConfigBuilder("spark.scheduler.mode") - .version("0.8.0") - .stringConf - .createWithDefault(SchedulingMode.FIFO.toString) - - private[spark] val SCHEDULER_REVIVE_INTERVAL = - ConfigBuilder("spark.scheduler.revive.interval") - .version("0.8.1") - .timeConf(TimeUnit.MILLISECONDS) - .createOptional - - private[spark] val SPECULATION_ENABLED = - ConfigBuilder("spark.speculation") - .version("0.6.0") - .booleanConf - .createWithDefault(false) - - private[spark] val SPECULATION_INTERVAL = - ConfigBuilder("spark.speculation.interval") - .version("0.6.0") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefault(100) - - private[spark] val SPECULATION_MULTIPLIER = - ConfigBuilder("spark.speculation.multiplier") - .version("0.6.0") - .doubleConf - .createWithDefault(1.5) - - private[spark] val SPECULATION_QUANTILE = - ConfigBuilder("spark.speculation.quantile") - .version("0.6.0") - .doubleConf - .createWithDefault(0.75) - - private[spark] val SPECULATION_TASK_DURATION_THRESHOLD = - ConfigBuilder("spark.speculation.task.duration.threshold") - .doc("Task duration after which scheduler would try to speculative run the task. If " + - "provided, tasks would be speculatively run if current stage contains less tasks " + - "than or equal to the number of slots on a single executor and the task is taking " + - "longer time than the threshold. This config helps speculate stage with very few " + - "tasks. Regular speculation configs may also apply if the executor slots are " + - "large enough. E.g. tasks might be re-launched if there are enough successful runs " + - "even though the threshold hasn't been reached. The number of slots is computed based " + - "on the conf values of spark.executor.cores and spark.task.cpus minimum 1.") - .version("3.0.0") - .timeConf(TimeUnit.MILLISECONDS) - .createOptional - - private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir") - .doc("Staging directory used while submitting applications.") - .version("2.0.0") - .stringConf - .createOptional - - private[spark] val BUFFER_PAGESIZE = ConfigBuilder("spark.buffer.pageSize") - .doc("The amount of memory used per page in bytes") - .version("1.5.0") - .bytesConf(ByteUnit.BYTE) - .createOptional - -} diff --git a/src/main/scala/org/apache/spark/memory/ExtendedMemoryPool.scala b/src/main/scala/org/apache/spark/memory/ExtendedMemoryPool.scala deleted file mode 100644 index 50dc05b8..00000000 --- a/src/main/scala/org/apache/spark/memory/ExtendedMemoryPool.scala +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.memory - -import javax.annotation.concurrent.GuardedBy - -import scala.collection.mutable - -import org.apache.spark.internal.Logging - -private[memory] class ExtendedMemoryPool(lock: Object) extends MemoryPool(lock) with Logging { - - private[this] val poolName: String = "extended memory" - - /** - * Map from taskAttemptId -> memory consumption in bytes - */ - @GuardedBy("lock") - private val extendedMemoryForTask = new mutable.HashMap[Long, Long]() - - @GuardedBy("lock") - private[this] var _memoryUsed: Long = 0L - - override def memoryUsed: Long = lock.synchronized { - _memoryUsed - } - /** - * Returns the memory consumption, in bytes, for the given task. - */ - def getMemoryUsageForTask(taskAttemptId: Long): Long = lock.synchronized { - extendedMemoryForTask.getOrElse(taskAttemptId, 0L) - } - - /** - * Try to acquire up to `numBytes` of extended memory for the given task and return the number - * of bytes obtained, or 0 if none can be allocated. - * - * @param numBytes number of bytes to acquire - * @param taskAttemptId the task attempt acquiring memory - * @return the number of bytes granted to the task. - */ - private[memory] def acquireMemory( - numBytes: Long, - taskAttemptId: Long): Long = lock.synchronized { - assert(numBytes > 0, s"invalid number of bytes requested: $numBytes") - - // Add this task to the taskMemory map just so we can keep an accurate count of the number - // of active tasks, to let other tasks ramp down their memory in calls to `acquireMemory` - if (!extendedMemoryForTask.contains(taskAttemptId)) { - extendedMemoryForTask(taskAttemptId) = 0L - // This will later cause waiting tasks to wake up and check numTasks again - lock.notifyAll() - } - - if (memoryFree >= numBytes) { - _memoryUsed += numBytes; - extendedMemoryForTask(taskAttemptId) += numBytes - return numBytes - } - 0L // Never reached - } - - /** - * Release `numBytes` of extended memory acquired by the given task. - */ - def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = lock.synchronized { - val curMem = extendedMemoryForTask.getOrElse(taskAttemptId, 0L) - val memoryToFree = if (curMem < numBytes) { - logWarning( - s"Internal error: release called on $numBytes bytes but task only has $curMem bytes " + - s"of memory from the $poolName pool") - curMem - } else { - numBytes - } - if (extendedMemoryForTask.contains(taskAttemptId)) { - extendedMemoryForTask(taskAttemptId) -= memoryToFree - if (extendedMemoryForTask(taskAttemptId) <= 0) { - extendedMemoryForTask.remove(taskAttemptId) - } - } - _memoryUsed -= memoryToFree - lock.notifyAll() // Notify waiters in acquireMemory() that memory has been freed - } - - /** - * Release all memory for the given task and mark it as inactive (e.g. when a task ends). - * - * @return the number of bytes freed. - */ - def releaseAllMemoryForTask(taskAttemptId: Long): Long = lock.synchronized { - val numBytesToFree = getMemoryUsageForTask(taskAttemptId) - releaseMemory(numBytesToFree, taskAttemptId) - numBytesToFree - } -} - diff --git a/src/main/scala/org/apache/spark/storage/PMemBlockObjectWriter.scala b/src/main/scala/org/apache/spark/storage/PMemBlockObjectWriter.scala deleted file mode 100644 index 051ceea5..00000000 --- a/src/main/scala/org/apache/spark/storage/PMemBlockObjectWriter.scala +++ /dev/null @@ -1,295 +0,0 @@ -/* - * 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.storage - -import java.io.{BufferedOutputStream, File, OutputStream} - -import com.intel.oap.common.storage.stream.{ChunkOutputStream, DataStore} - -import org.apache.spark.internal.Logging -import org.apache.spark.memory.PMemManagerInitializer -import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} -import org.apache.spark.shuffle.ShuffleWriteMetricsReporter -import org.apache.spark.util.Utils - -/** - * A class for writing JVM objects directly to a file on disk. This class allows data to be appended - * to an existing block. For efficiency, it retains the underlying file channel across - * multiple commits. This channel is kept open until close() is called. In case of faults, - * callers should instead close with revertPartialWritesAndClose() to atomically revert the - * uncommitted partial writes. - * - * This class does not support concurrent writes. Also, once the writer has been opened it cannot be - * reopened again. - */ -private[spark] class PMemBlockObjectWriter( - file: File, - serializerManager: SerializerManager, - serializerInstance: SerializerInstance, - bufferSize: Int, - syncWrites: Boolean, - // These write metrics concurrently shared with other active DiskBlockObjectWriters who - // are themselves performing writes. All updates must be relative. - writeMetrics: ShuffleWriteMetricsReporter, - blockId: BlockId = null) - extends DiskBlockObjectWriter(file, serializerManager, - serializerInstance, bufferSize, syncWrites, writeMetrics, blockId) - with Logging { - - /** - * Guards against close calls, e.g. from a wrapping stream. - * Call manualClose to close the stream that was extended by this trait. - * Commit uses this trait to close object streams without paying the - * cost of closing and opening the underlying file. - */ - private trait ManualCloseOutputStream extends OutputStream { - abstract override def close(): Unit = { - flush() - } - - def manualClose(): Unit = { - super.close() - } - } - - // private var dataStore: DataStore = dataStore - /** The file channel, used for repositioning / truncating the file. */ - // private var channel: FileChannel = null - private var mcs: ManualCloseOutputStream = null - private var bs: OutputStream = null - private var fos: ChunkOutputStream = null - private var ts: TimeTrackingOutputStream = null - private var objOut: SerializationStream = null - private var initialized = false - private var streamOpen = false - private var hasBeenClosed = false - private var dataStore: DataStore = null - - def getFOS(): ChunkOutputStream = { - fos - } - - // for tmp usage in UT - def getDataStore(): DataStore = { - dataStore - } - - /** - * Cursors used to represent positions in the file. - * - * xxxxxxxxxx|----------| - * ^ ^ - * | | - * | reportedPosition - * committedPosition - * - * reportedPosition: Position at the time of the last update to the write metrics. - * same as chunkoutputstream.position() - * committedPosition: Offset after last committed write. - * -----: Current writes to the underlying file. - * xxxxx: Committed contents of the file. - */ - // private var committedPosition = file.length() - private var committedPosition = 0; - private var reportedPosition = committedPosition - - /** - * Keep track of number of records written and also use this to periodically - * output bytes written since the latter is expensive to do for each record. - * And we reset it after every commitAndGet called. - */ - private var numRecordsWritten = 0 - - private def initialize(): Unit = { - dataStore = new DataStore(PMemManagerInitializer.getPMemManager(), - PMemManagerInitializer.getProperties()); - fos = ChunkOutputStream.getChunkOutputStreamInstance(file.toString, dataStore) - committedPosition = fos.position().toInt - reportedPosition = committedPosition - ts = new TimeTrackingOutputStream(writeMetrics, fos) - class ManualCloseBufferedOutputStream - extends BufferedOutputStream(ts, bufferSize) with ManualCloseOutputStream - mcs = new ManualCloseBufferedOutputStream - } - - override def open(): PMemBlockObjectWriter = { - if (hasBeenClosed) { - throw new IllegalStateException("Writer already closed. Cannot be reopened.") - } - if (!initialized) { - initialize() - initialized = true - } - - bs = serializerManager.wrapStream(blockId, mcs) - objOut = serializerInstance.serializeStream(bs) - streamOpen = true - this - } - - /** - * Close and cleanup all resources. - * Should call after committing or reverting partial writes. - */ - private def closeResources(): Unit = { - if (initialized) Utils.tryWithSafeFinally { - mcs.manualClose() - } { - mcs = null - bs = null - fos = null - ts = null - objOut = null - initialized = false - streamOpen = false - hasBeenClosed = true - } - } - - /** - * Commits any remaining partial writes and closes resources. - */ - override def close() { - if (initialized) { - Utils.tryWithSafeFinally { - commitAndGet() - } { - closeResources() - } - } - } - - /** - * Flush the partial writes and commit them as a single atomic block. - * A commit may write additional bytes to frame the atomic block. - * - * @return file segment with previous offset and length committed on this call. - */ - override def commitAndGet(): FileSegment = { - if (streamOpen) { - // NOTE: Because Kryo doesn't flush the underlying stream we explicitly flush both the - // serializer stream and the lower level stream. - objOut.flush() - bs.flush() - objOut.close() - streamOpen = false - - if (syncWrites) { - // Force outstanding writes to disk and track how long it takes - val start = System.nanoTime() - fos.getFD.sync() - writeMetrics.incWriteTime(System.nanoTime() - start) - } - val pos = fos.position().toInt - val previousCommitedPosition = committedPosition - val length = pos - committedPosition - committedPosition = pos - // In certain compression codecs, more bytes are written after streams are closed - writeMetrics.incBytesWritten(committedPosition - reportedPosition) - reportedPosition = committedPosition - numRecordsWritten = 0 - new FileSegment(file, previousCommitedPosition, length) - } else { - new FileSegment(file, committedPosition, 0) - } - } - - - /** - * Reverts writes that haven't been committed yet. Callers should invoke this function - * when there are runtime exceptions. This method will not throw, though it may be - * unsuccessful in truncating written data. - * - * @return the file that this DiskBlockObjectWriter wrote to. - */ - override def revertPartialWritesAndClose(): File = { - var cos: ChunkOutputStream = null - // Discard current writes. We do this by flushing the outstanding writes and then - // truncating the file to its initial position. - Utils.tryWithSafeFinally { - if (initialized) { - writeMetrics.decBytesWritten(reportedPosition - committedPosition) - writeMetrics.decRecordsWritten(numRecordsWritten) - streamOpen = false - closeResources() - } - } { - cos = ChunkOutputStream.getChunkOutputStreamInstance(file.toString, dataStore) - cos.truncate(committedPosition) - } - new File(file.toString) - } - - - /** - * Writes a key-value pair. - */ - override def write(key: Any, value: Any): Unit = { - if (!streamOpen) { - open() - } - - objOut.writeKey(key) - objOut.writeValue(value) - recordWritten() - } - - override def write(b: Int): Unit = throw new UnsupportedOperationException() - - override def write(kvBytes: Array[Byte], offs: Int, len: Int): Unit = { - if (!streamOpen) { - open() - } - - bs.write(kvBytes, offs, len) - } - - /** - * Notify the writer that a record worth of bytes has been written with OutputStream#write. - */ - override def recordWritten(): Unit = { - numRecordsWritten += 1 - writeMetrics.incRecordsWritten(1) - - if (numRecordsWritten % 16384 == 0) { - updateBytesWritten() - } - } - - // For testing - private[spark] override def flush(): Unit = { - objOut.flush() - bs.flush() - } - - /** - * Report the number of bytes written in this writer's shuffle write metrics. - * Note that this is only valid before the underlying streams are closed. - */ - private def updateBytesWritten() { - // val pos = channel.position() - // in high level, sometimes 16384 records written so this function called - // but the data is not actually wrote to PMem because writeBuffer isn't full - // and spill not finished - if (fos != null) { - val pos = fos.position().toInt - writeMetrics.incBytesWritten(pos - reportedPosition) - reportedPosition = pos - } - } -}