Skip to content

[CELEBORN-1791] All NettyMemoryMetrics should register to source #3016

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

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.slf4j.LoggerFactory;

import org.apache.celeborn.common.CelebornConf;
import org.apache.celeborn.common.metrics.source.AbstractSource;
import org.apache.celeborn.common.protocol.TransportModuleConstants;
import org.apache.celeborn.common.util.ThreadUtils;
import org.apache.celeborn.common.util.Utils;
Expand Down Expand Up @@ -95,12 +96,13 @@ public class MemoryManager {

@VisibleForTesting
public static MemoryManager initialize(CelebornConf conf) {
return initialize(conf, null);
return initialize(conf, null, null);
}

public static MemoryManager initialize(CelebornConf conf, StorageManager storageManager) {
public static MemoryManager initialize(
CelebornConf conf, StorageManager storageManager, AbstractSource source) {
if (_INSTANCE == null) {
_INSTANCE = new MemoryManager(conf, storageManager);
_INSTANCE = new MemoryManager(conf, storageManager, source);
}
return _INSTANCE;
}
Expand All @@ -115,7 +117,7 @@ public static MemoryManager instance() {
return _INSTANCE;
}

private MemoryManager(CelebornConf conf, StorageManager storageManager) {
private MemoryManager(CelebornConf conf, StorageManager storageManager, AbstractSource source) {
double pausePushDataRatio = conf.workerDirectMemoryRatioToPauseReceive();
double pauseReplicateRatio = conf.workerDirectMemoryRatioToPauseReplicate();
this.resumeRatio = conf.workerDirectMemoryRatioToResume();
Expand Down Expand Up @@ -190,7 +192,7 @@ private MemoryManager(CelebornConf conf, StorageManager storageManager) {

if (readBufferThreshold > 0) {
// if read buffer threshold is zero means that there will be no map data partitions
readBufferDispatcher = new ReadBufferDispatcher(this, conf);
readBufferDispatcher = new ReadBufferDispatcher(this, conf, source);
readBufferTargetUpdateService.scheduleWithFixedDelay(
() -> {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.slf4j.LoggerFactory;

import org.apache.celeborn.common.CelebornConf;
import org.apache.celeborn.common.metrics.source.AbstractSource;
import org.apache.celeborn.common.network.util.NettyUtils;
import org.apache.celeborn.common.network.util.TransportConf;
import org.apache.celeborn.common.util.ThreadUtils;
Expand All @@ -47,12 +48,13 @@ public class ReadBufferDispatcher {
@VisibleForTesting public volatile boolean stopFlag = false;
@VisibleForTesting public final AtomicReference<Thread> dispatcherThread;

public ReadBufferDispatcher(MemoryManager memoryManager, CelebornConf conf) {
public ReadBufferDispatcher(
MemoryManager memoryManager, CelebornConf conf, AbstractSource source) {
readBufferAllocationWait = conf.readBufferAllocationWait();
long checkThreadInterval = conf.readBufferDispatcherCheckThreadInterval();
// readBuffer is not a module name, it's a placeholder.
readBufferAllocator =
NettyUtils.getByteBufAllocator(new TransportConf("readBuffer", conf), null, true);
NettyUtils.getByteBufAllocator(new TransportConf("readBuffer", conf), source, true);
this.memoryManager = memoryManager;
dispatcherThread =
new AtomicReference<>(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ private[celeborn] class Worker(

val storageManager = new StorageManager(conf, workerSource)

val memoryManager: MemoryManager = MemoryManager.initialize(conf, storageManager)
val memoryManager: MemoryManager = MemoryManager.initialize(conf, storageManager, workerSource)
memoryManager.registerMemoryListener(storageManager)

val partitionsSorter = new PartitionFilesSorter(memoryManager, conf, workerSource)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ final private[worker] class StorageManager(conf: CelebornConf, workerSource: Abs
DeviceMonitor.createDeviceMonitor(conf, this, deviceInfos, tmpDiskInfos, workerSource)

val storageBufferAllocator: ByteBufAllocator =
NettyUtils.getByteBufAllocator(new TransportConf("StorageManager", conf), null, true)
NettyUtils.getByteBufAllocator(new TransportConf("StorageManager", conf), workerSource, true)

// (mountPoint -> LocalFlusher)
private val (
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ public long[] prepare(int mapCount) {

StorageManager storageManager = Mockito.mock(StorageManager.class);
Mockito.when(storageManager.storageBufferAllocator()).thenAnswer(a -> allocator);
MemoryManager.initialize(conf, storageManager);
MemoryManager.initialize(conf, storageManager, null);
partitionDataWriter = Mockito.mock(PartitionDataWriter.class);
when(partitionDataWriter.getMemoryFileInfo()).thenAnswer(i -> fileInfo);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ public static void beforeAll() {
Mockito.when(storageManager.evictedFileCount()).thenAnswer(a -> evictCount);
Mockito.when(storageManager.localOrDfsStorageAvailable()).thenAnswer(a -> true);
Mockito.when(storageManager.storageBufferAllocator()).thenAnswer(a -> allocator);
MemoryManager.initialize(conf, storageManager);
MemoryManager.initialize(conf, storageManager, null);
}

public static void setupChunkServer(FileInfo info) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ class ReadBufferDispactherSuite extends CelebornFunSuite {
})

val conf = new CelebornConf()
val readBufferDispatcher = new ReadBufferDispatcher(mockedMemoryManager, conf)
val readBufferDispatcher = new ReadBufferDispatcher(mockedMemoryManager, conf, null)
val requestFuture = new CompletableFuture[Void]()

val request = new ReadBufferRequest(
Expand All @@ -67,7 +67,7 @@ class ReadBufferDispactherSuite extends CelebornFunSuite {
val mockedMemoryManager = mock(classOf[MemoryManager])
val conf = new CelebornConf()
conf.set("celeborn.worker.readBufferDispatcherThreadWatchdog.checkInterval", "100ms")
val readBufferDispatcher = new ReadBufferDispatcher(mockedMemoryManager, conf)
val readBufferDispatcher = new ReadBufferDispatcher(mockedMemoryManager, conf, null)
val threadId1 = readBufferDispatcher.dispatcherThread.get().getId
readBufferDispatcher.stopFlag = true
Thread.sleep(1500)
Expand Down
Loading