FMX commented on code in PR #2088: URL: https://github.com/apache/incubator-celeborn/pull/2088#discussion_r1390690488
########## common/src/main/java/org/apache/celeborn/common/util/MemCacheManager.java: ########## @@ -0,0 +1,85 @@ +/* + * 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.celeborn.common.util; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; + +import io.netty.buffer.ByteBuf; + +import org.apache.celeborn.common.CelebornConf; + +public class MemCacheManager { + private static MemCacheManager memCacheManager; + private ConcurrentMap<String, ByteBuf> caches = new ConcurrentHashMap<>(); + private long cacheCapacity; + private boolean cacheEnable; + private AtomicLong currentCacheSize = new AtomicLong(0); + + public MemCacheManager(CelebornConf conf) { + cacheCapacity = conf.workerMemoryCacheCapacity(); + cacheEnable = conf.workerMemoryCacheEnabled() && !conf.hasHDFSStorage(); + } + + public void putCache(String key, ByteBuf cache) { + int cacheSize = cache.readableBytes(); + caches.put(key, cache); + currentCacheSize.getAndAdd(cacheSize); + } + + public boolean canCache(int cacheSize) { Review Comment: Here is a concurrent issue. Multiple threads might try to flush different file writers. You should invoke `canCache` and `putCache` atomically. ########## worker/src/main/java/org/apache/celeborn/service/deploy/worker/storage/PartitionFilesSorter.java: ########## @@ -637,6 +639,12 @@ private void initializeFiles() throws IOException { hdfsSortedOutput = StorageManager.hadoopFs().create(new Path(sortedFilePath), true, 256 * 1024); } else { + if (memCacheManager.contains(originFilePath)) { + FileChannel channel = FileChannelUtils.createWritableFileChannel(originFilePath); + channel.write(memCacheManager.getCache(originFilePath).nioBuffer()); Review Comment: The shuffle file is in memory. I think you can sort the file in memory and return the sorted shuffle data. ########## worker/src/main/java/org/apache/celeborn/service/deploy/worker/storage/FileWriter.java: ########## @@ -159,14 +162,24 @@ protected void flush(boolean finalFlush) throws IOException { int numBytes = flushBuffer.readableBytes(); if (numBytes != 0) { notifier.checkException(); - notifier.numPendingFlushes.incrementAndGet(); - FlushTask task = null; - if (channel != null) { - task = new LocalFlushTask(flushBuffer, channel, notifier); - } else if (fileInfo.isHdfs()) { - task = new HdfsFlushTask(flushBuffer, fileInfo.getHdfsPath(), notifier); + if (finalFlush + && channel != null + && fileInfo.getFileLength() == 0 + && cacheManager.canCache(numBytes)) { + cacheManager.putCache(fileInfo.getFilePath(), flushBuffer.copy()); Review Comment: Actually, this copy is unnecessary. You can just use this buffer and return it when cache is released. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
