-
Notifications
You must be signed in to change notification settings - Fork 525
Share sparkey readers across DoFns with memory-aware heap fallback #5903
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
Draft
spkrka
wants to merge
1
commit into
spotify:main
Choose a base branch
from
spkrka:krka/shared-reader
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Draft
Changes from all commits
Commits
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
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
113 changes: 113 additions & 0 deletions
113
scio-extra/src/main/scala/com/spotify/scio/extra/sparkey/HostMemoryTracker.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,113 @@ | ||
| /* | ||
| * Copyright 2026 Spotify AB. | ||
| * | ||
| * Licensed 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 com.spotify.scio.extra.sparkey | ||
|
|
||
| import com.sun.management.OperatingSystemMXBean | ||
| import java.lang.management.ManagementFactory | ||
| import java.util.concurrent.atomic.AtomicLong | ||
| import org.slf4j.LoggerFactory | ||
|
|
||
| /** | ||
| * Tracks memory budgets for sparkey readers on this JVM, covering both off-heap and on-heap memory. | ||
| * On Dataflow, the JVM heap is hardcoded to ~70% of worker memory, leaving only ~30% for off-heap | ||
| * use (page cache, OS, kernel, etc.). | ||
| * | ||
| * Sparkey readers are opened via mmap (off-heap) by default. When the off-heap budget is exhausted, | ||
| * readers can fall back to heap-backed mode. This tracker provides atomic budget claiming for both | ||
| * pools to coordinate across threads. | ||
| * | ||
| * Budget is claimed but never released — readers are cached for the JVM lifetime and never closed. | ||
| * If a reader fails to open partway through (e.g. a sharded sparkey with some shards already | ||
| * claimed), the budget for those shards is leaked. This is acceptable since a reader failure is | ||
| * fatal to the pipeline. | ||
| */ | ||
| private[sparkey] class HostMemoryTracker(offHeapBudget: Long, heapBudget: Long) { | ||
| private val logger = LoggerFactory.getLogger(this.getClass) | ||
|
|
||
| private val remainingOffHeap = new AtomicLong(offHeapBudget) | ||
| private val remainingHeap = new AtomicLong(heapBudget) | ||
|
|
||
| logger.info( | ||
| "Memory budgets — off-heap: {} bytes, heap: {} bytes", | ||
| Array[AnyRef](Long.box(offHeapBudget), Long.box(heapBudget)): _* | ||
| ) | ||
|
|
||
| /** | ||
| * Atomically try to claim `bytes` from the off-heap budget. Returns true if the claim succeeded | ||
| * (enough budget remaining), false otherwise. On success, the budget is reduced by `bytes`. On | ||
| * failure, the budget is unchanged. | ||
| */ | ||
| def tryClaimOffHeap(bytes: Long): Boolean = tryClaim(remainingOffHeap, "off-heap", bytes) | ||
|
|
||
| /** | ||
| * Atomically try to claim `bytes` from the heap budget. Returns true if the claim succeeded | ||
| * (enough budget remaining), false otherwise. | ||
| */ | ||
| def tryClaimHeap(bytes: Long): Boolean = tryClaim(remainingHeap, "heap", bytes) | ||
|
|
||
| private def tryClaim(budget: AtomicLong, name: String, bytes: Long): Boolean = { | ||
| val prev = | ||
| budget.getAndAccumulate(bytes, (current, b) => if (current >= b) current - b else current) | ||
| if (prev >= bytes) { | ||
| logger.info( | ||
| "Claimed {} bytes of {} budget, {} bytes remaining", | ||
| Array[AnyRef](Long.box(bytes), name, Long.box(prev - bytes)): _* | ||
| ) | ||
| true | ||
| } else { | ||
| logger.debug( | ||
| "Cannot claim {} bytes of {} budget, only {} bytes remaining", | ||
| Array[AnyRef](Long.box(bytes), name, Long.box(prev)): _* | ||
| ) | ||
| false | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private[sparkey] object HostMemoryTracker { | ||
| private val logger = LoggerFactory.getLogger(this.getClass) | ||
|
|
||
| // Reserve 2 GB of off-heap memory for OS, kernel structures, JVM class files, Beam shuffle, etc. | ||
| private val OffHeapReserveBytes: Long = 2L * 1024 * 1024 * 1024 | ||
|
|
||
| // Reserve 10% of max heap (min 4 GB) for GC headroom and non-sparkey allocations. | ||
| private val HeapReserveBytes: Long = { | ||
| val maxHeap = Runtime.getRuntime.maxMemory() | ||
| Math.max(4L * 1024 * 1024 * 1024, (maxHeap * 0.1).toLong) | ||
| } | ||
|
|
||
| private val offHeapBudget: Long = { | ||
| val totalPhysical = ManagementFactory.getOperatingSystemMXBean | ||
| .asInstanceOf[OperatingSystemMXBean] | ||
| .getTotalPhysicalMemorySize | ||
| val maxHeap = Runtime.getRuntime.maxMemory() | ||
| Math.max(0, totalPhysical - maxHeap - OffHeapReserveBytes) | ||
| } | ||
|
|
||
| private val heapBudget: Long = { | ||
| val maxHeap = Runtime.getRuntime.maxMemory() | ||
| Math.max(0, maxHeap - HeapReserveBytes) | ||
| } | ||
|
|
||
| logger.info( | ||
| "Host memory — off-heap reserve: {}, heap reserve: {}", | ||
| Array[AnyRef](Long.box(OffHeapReserveBytes), Long.box(HeapReserveBytes)): _* | ||
| ) | ||
|
|
||
| val instance: HostMemoryTracker = new HostMemoryTracker(offHeapBudget, heapBudget) | ||
| } | ||
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
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
75 changes: 75 additions & 0 deletions
75
scio-extra/src/test/scala/com/spotify/scio/extra/sparkey/HostMemoryTrackerTest.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,75 @@ | ||
| /* | ||
| * Copyright 2026 Spotify AB. | ||
| * | ||
| * Licensed 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 com.spotify.scio.extra.sparkey | ||
|
|
||
| import org.scalatest.flatspec.AnyFlatSpec | ||
| import org.scalatest.matchers.should.Matchers | ||
|
|
||
| class HostMemoryTrackerTest extends AnyFlatSpec with Matchers { | ||
|
|
||
| "HostMemoryTracker" should "claim off-heap budget when sufficient" in { | ||
| val tracker = new HostMemoryTracker(offHeapBudget = 100, heapBudget = 50) | ||
| tracker.tryClaimOffHeap(60) shouldBe true | ||
| tracker.tryClaimOffHeap(40) shouldBe true | ||
| } | ||
|
|
||
| it should "reject off-heap claim when insufficient" in { | ||
| val tracker = new HostMemoryTracker(offHeapBudget = 100, heapBudget = 50) | ||
| tracker.tryClaimOffHeap(60) shouldBe true | ||
| tracker.tryClaimOffHeap(50) shouldBe false | ||
| } | ||
|
|
||
| it should "claim heap budget when sufficient" in { | ||
| val tracker = new HostMemoryTracker(offHeapBudget = 0, heapBudget = 100) | ||
| tracker.tryClaimHeap(60) shouldBe true | ||
| tracker.tryClaimHeap(40) shouldBe true | ||
| } | ||
|
|
||
| it should "reject heap claim when insufficient" in { | ||
| val tracker = new HostMemoryTracker(offHeapBudget = 0, heapBudget = 100) | ||
| tracker.tryClaimHeap(60) shouldBe true | ||
| tracker.tryClaimHeap(50) shouldBe false | ||
| } | ||
|
|
||
| it should "handle zero budgets" in { | ||
| val tracker = new HostMemoryTracker(offHeapBudget = 0, heapBudget = 0) | ||
| tracker.tryClaimOffHeap(1) shouldBe false | ||
| tracker.tryClaimHeap(1) shouldBe false | ||
| } | ||
|
|
||
| it should "handle exact budget claims" in { | ||
| val tracker = new HostMemoryTracker(offHeapBudget = 100, heapBudget = 50) | ||
| tracker.tryClaimOffHeap(100) shouldBe true | ||
| tracker.tryClaimOffHeap(1) shouldBe false | ||
| } | ||
|
|
||
| it should "track off-heap and heap budgets independently" in { | ||
| val tracker = new HostMemoryTracker(offHeapBudget = 100, heapBudget = 100) | ||
| tracker.tryClaimOffHeap(100) shouldBe true | ||
| tracker.tryClaimOffHeap(1) shouldBe false | ||
| // heap should still be available | ||
| tracker.tryClaimHeap(100) shouldBe true | ||
| tracker.tryClaimHeap(1) shouldBe false | ||
| } | ||
|
|
||
| "HostMemoryTracker.instance" should "exist as a singleton" in { | ||
| HostMemoryTracker.instance should not be null | ||
| // same reference | ||
| HostMemoryTracker.instance shouldBe theSameInstanceAs(HostMemoryTracker.instance) | ||
| } | ||
| } |
Oops, something went wrong.
Oops, something went wrong.
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.
I guess this is technically not purely a sparkey thing, could also be used for other resources/sideinput types