-
Notifications
You must be signed in to change notification settings - Fork 199
Wip umm snappy commons #3
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
12 commits
Select commit
Hold shift + click to select a range
5044816
Added SnappyShuffleMemoryManager, SnappyBlockManager, SnappyMemorySto…
b5cc9df
Added a dunit test
d0c8697
Minor changes
f991f82
Resolving conflicts in rebase
54ce8e5
Added a check for eviction up event
f2337b2
Incorporate review comments
c9a3744
Test change
de37af0
Test change
ddfa144
Renamed a test file
0f05355
Test change
3a246b2
Using Utils.classForName
0d50b90
Incorporate review comments
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
126 changes: 126 additions & 0 deletions
126
snappy-dunits/src/test/scala/io/snappydata/dunit/cluster/SnappyResourceEventsDUnitTest.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,126 @@ | ||
| package io.snappydata.dunit.cluster | ||
|
|
||
| import scala.Predef._ | ||
|
|
||
| import com.gemstone.gemfire.internal.cache.GemFireCacheImpl | ||
| import com.gemstone.gemfire.internal.cache.control.{HeapMemoryMonitor, InternalResourceManager} | ||
| import com.pivotal.gemfirexd.internal.engine.Misc | ||
| import com.pivotal.gemfirexd.internal.engine.distributed.utils.GemFireXDUtils | ||
| import dunit.DistributedTestBase | ||
| import io.snappydata.ServiceManager | ||
|
|
||
| import org.apache.spark.SparkEnv | ||
| import org.apache.spark.storage.{RDDInfo, StorageLevel} | ||
|
|
||
| /** | ||
| * Created by shirishd on 19/10/15. | ||
| */ | ||
| class SnappyResourceEventsDUnitTest (s: String) extends ClusterManagerTestBase(s) { | ||
|
|
||
| override def tearDown2(): Unit = { | ||
| Array(vm3, vm2, vm1, vm0).foreach(_.invoke(this.getClass, "resetGFResourceManager")) | ||
| super.tearDown2() | ||
| } | ||
|
|
||
| def testCriticalUp(): Unit = { | ||
| vm1.invoke(this.getClass, "startSnappyServer", startArgs) | ||
| vm0.invoke(this.getClass, "startSnappyLead", startArgs) | ||
|
|
||
| // Execute the job | ||
| vm0.invoke(this.getClass, "runSparkJob") | ||
| vm1.invoke(this.getClass, "raiseCriticalUpMemoryEvent") | ||
| vm0.invoke(this.getClass, "runSparkJobAfterThresholdBreach") | ||
|
|
||
| vm1.invoke(this.getClass, "assertShuffleMemoryManagerBehavior") | ||
| } | ||
|
|
||
| def testEvictionUp(): Unit = { | ||
| vm1.invoke(this.getClass, "startSnappyServer", startArgs) | ||
| vm0.invoke(this.getClass, "startSnappyLead", startArgs) | ||
|
|
||
| // Execute the job | ||
| vm0.invoke(this.getClass, "runSparkJob") | ||
| vm1.invoke(this.getClass, "raiseEvictionUpMemoryEvent") | ||
| vm0.invoke(this.getClass, "runSparkJobAfterThresholdBreach") | ||
|
|
||
| } | ||
| } | ||
|
|
||
| object SnappyResourceEventsDUnitTest extends ClusterManagerTestUtils { | ||
|
|
||
| def runSparkJob(): Unit = { | ||
| val rdd1 = sc.makeRDD(Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)).cache() | ||
| println(rdd1.count) | ||
| assert(!sc.getRDDStorageInfo.isEmpty) | ||
| } | ||
|
|
||
| def assertShuffleMemoryManagerBehavior(): Unit = { | ||
| assert(SparkEnv.get.shuffleMemoryManager.tryToAcquire(1000) == 0) | ||
| } | ||
|
|
||
| def getInMemorySizeForCachedRDDs: Long = { | ||
| val rddInfo: Array[RDDInfo] = sc.getRDDStorageInfo | ||
| var sum = 0L; | ||
| for (i <- 0 to rddInfo.length - 1) { | ||
| sum = sum + rddInfo(i).memSize | ||
| } | ||
| sum | ||
| } | ||
|
|
||
| def runSparkJobAfterThresholdBreach(): Unit = { | ||
| val sum1: Long = getInMemorySizeForCachedRDDs | ||
| println("1. cached rdd mem size before caching rdd when critical or eviction up = " + sum1) | ||
|
|
||
| val rdd2 = sc.makeRDD(Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)).cache() | ||
| println(rdd2.count) | ||
| val sum2: Long = getInMemorySizeForCachedRDDs | ||
| println("2. cached rdd mem size after caching first rdd when critical or eviction up = " + sum2) | ||
| // make sure that after eviction up new rdd being cached does not result in | ||
| // increased memory usage | ||
| assert(!(sum2 > sum1)) | ||
|
|
||
| val rdd3 = sc.makeRDD(Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)).cache() | ||
| println(rdd3.count) | ||
| val sum3: Long = getInMemorySizeForCachedRDDs | ||
| println("3. cached rdd mem size after caching second rdd when critical or eviction up = " + sum3) | ||
| // make sure that after eviction up new rdd being cached does not result in | ||
| // increased memory usage | ||
| assert(!(sum3 > sum2)) | ||
| } | ||
|
|
||
| def raiseCriticalUpMemoryEvent(): Unit = { | ||
| println("About to raise CRITICAL UP event") | ||
| val gfCache: GemFireCacheImpl = Misc.getGemFireCache | ||
| val resMgr: InternalResourceManager = gfCache.getResourceManager | ||
| HeapMemoryMonitor.setTestDisableMemoryUpdates(true) | ||
| resMgr.getHeapMonitor.setTestMaxMemoryBytes(100) | ||
| HeapMemoryMonitor.setTestBytesUsedForThresholdSet(92) | ||
| resMgr.setCriticalHeapPercentage(90F) | ||
|
|
||
| resMgr.getHeapMonitor().updateStateAndSendEvent(92); | ||
| println("CRITICAL UP event sent") | ||
| } | ||
|
|
||
| def raiseEvictionUpMemoryEvent(): Unit = { | ||
| println("About to raise EVICTION UP event") | ||
| val gfCache: GemFireCacheImpl = Misc.getGemFireCache | ||
| val resMgr: InternalResourceManager = gfCache.getResourceManager | ||
| HeapMemoryMonitor.setTestDisableMemoryUpdates(true) | ||
| resMgr.getHeapMonitor.setTestMaxMemoryBytes(100) | ||
| HeapMemoryMonitor.setTestBytesUsedForThresholdSet(90) | ||
| resMgr.setEvictionHeapPercentage(40F) | ||
| resMgr.getHeapMonitor().updateStateAndSendEvent(85); | ||
| println("EVICTION UP event sent") | ||
|
|
||
| } | ||
|
|
||
| def resetGFResourceManager(): Unit = { | ||
| val service = ServiceManager.currentFabricServiceInstance | ||
| if (service != null) { | ||
| val gfCache: GemFireCacheImpl = Misc.getGemFireCache | ||
| val resMgr: InternalResourceManager = gfCache.getResourceManager | ||
| resMgr.getHeapMonitor.setTestMaxMemoryBytes(0) | ||
| resMgr.getHeapMonitor.updateStateAndSendEvent(10) | ||
| } | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
31 changes: 31 additions & 0 deletions
31
snappy-tools/src/main/scala/org/apache/spark/shuffle/SnappyShuffleMemoryManager.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,31 @@ | ||
| package org.apache.spark.shuffle | ||
|
|
||
| import org.apache.spark.storage.SnappyMemoryUtils | ||
|
|
||
| /** | ||
| * Created by shirishd on 15/10/15. | ||
| */ | ||
|
|
||
| private[spark] class SnappyShuffleMemoryManager protected(override val maxMemory: Long, | ||
| override val pageSizeBytes: Long) extends ShuffleMemoryManager(maxMemory, pageSizeBytes) { | ||
|
|
||
| override def tryToAcquire(numBytes: Long): Long = synchronized { | ||
| val taskAttemptId = currentTaskAttemptId() | ||
| assert(numBytes > 0, "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 tryToAcquire | ||
| if (!taskMemory.contains(taskAttemptId)) { | ||
| taskMemory(taskAttemptId) = 0L | ||
| notifyAll() // Will later cause waiting tasks to wake up and check numThreads again | ||
| } | ||
|
|
||
| if (SnappyMemoryUtils.isCriticalUp) { | ||
| logInfo(s"Will not store $numBytes bytes as CRITICAL UP event is detected") | ||
| 0 | ||
| } else { | ||
| super.tryToAcquire(numBytes) | ||
| } | ||
| } | ||
| } | ||
|
|
29 changes: 29 additions & 0 deletions
29
snappy-tools/src/main/scala/org/apache/spark/storage/SnappyBlockManager.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,29 @@ | ||
| package org.apache.spark.storage | ||
|
|
||
| import org.apache.spark.network.BlockTransferService | ||
| import org.apache.spark.rpc.RpcEnv | ||
| import org.apache.spark.serializer.Serializer | ||
| import org.apache.spark.shuffle.ShuffleManager | ||
| import org.apache.spark.{MapOutputTracker, SecurityManager, SparkConf} | ||
|
|
||
| /** | ||
| * Created by shirishd on 12/10/15. | ||
| */ | ||
|
|
||
| private[spark] class SnappyBlockManager( | ||
| executorId: String, | ||
| rpcEnv: RpcEnv, | ||
| override val master: BlockManagerMaster, | ||
| defaultSerializer: Serializer, | ||
| override val conf: SparkConf, | ||
| mapOutputTracker: MapOutputTracker, | ||
| shuffleManager: ShuffleManager, | ||
| blockTransferService: BlockTransferService, | ||
| securityManager: SecurityManager, | ||
| numUsableCores: Int) | ||
| extends BlockManager(executorId, rpcEnv, master, defaultSerializer, conf, mapOutputTracker, | ||
| shuffleManager, blockTransferService, securityManager, numUsableCores) { | ||
|
|
||
| override private[spark] val memoryStore = new SnappyMemoryStore(this, BlockManager.getMaxMemory(conf)) | ||
| } | ||
|
|
35 changes: 35 additions & 0 deletions
35
snappy-tools/src/main/scala/org/apache/spark/storage/SnappyMemoryStore.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,35 @@ | ||
| package org.apache.spark.storage | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
|
|
||
| /** | ||
| * Created by shirishd on 9/10/15. | ||
| */ | ||
| private[spark] class SnappyMemoryStore(blockManager: BlockManager, maxMemory: Long) | ||
| extends MemoryStore(blockManager, maxMemory) { | ||
|
|
||
| override def freeMemory: Long = { | ||
| if(SnappyMemoryUtils.isEvictionUp) { | ||
| logInfo(s"Snappy-store EVICTION UP event detected") | ||
| 0 | ||
| } else { | ||
| super.freeMemory | ||
| } | ||
| } | ||
|
|
||
| override def ensureFreeSpace( | ||
| blockIdToAdd: BlockId, | ||
| space: Long): ResultWithDroppedBlocks = { | ||
|
|
||
| if (SnappyMemoryUtils.isCriticalUp) { | ||
| logInfo(s"Will not store $blockIdToAdd as CRITICAL UP event is detected") | ||
| val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] | ||
| return ResultWithDroppedBlocks(success = false, droppedBlocks) | ||
| } | ||
| super.ensureFreeSpace(blockIdToAdd, space) | ||
| } | ||
| } | ||
|
|
||
|
|
||
|
|
24 changes: 24 additions & 0 deletions
24
snappy-tools/src/main/scala/org/apache/spark/storage/SnappyMemoryUtils.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,24 @@ | ||
| package org.apache.spark.storage | ||
|
|
||
| import com.pivotal.gemfirexd.internal.engine.store.GemFireStore | ||
|
|
||
| /** | ||
| * Created by shirishd on 29/10/15. | ||
| */ | ||
| object SnappyMemoryUtils { | ||
| /** | ||
| * Checks whether GemFire critical threshold is breached | ||
| * @return | ||
| */ | ||
| def isCriticalUp: Boolean = { | ||
| Option(GemFireStore.getBootingInstance).exists(g => g.thresholdListener.isCritical) | ||
| } | ||
|
|
||
| /** | ||
| * Checks whether GemFire eviction threshold is breached | ||
| * @return | ||
| */ | ||
| def isEvictionUp: Boolean = { | ||
| Option(GemFireStore.getBootingInstance).exists(g => g.thresholdListener.isEviction) | ||
| } | ||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One issue with this approach going forward could be that it will work only with SnappyCM and not with Yarn (when we add full support for that). Perhaps a better approach might be to introduce a callback somewhere in base ShuffleMemoryManager.tryAcquire that can be plugged in whenever driver joins the GemXD cluster. Not critical to correct for 0.2 milestone but I think should track in Jira.