waitinfuture commented on code in PR #2081:
URL:
https://github.com/apache/incubator-celeborn/pull/2081#discussion_r1389187870
##########
common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala:
##########
@@ -2057,6 +2060,22 @@ object CelebornConf extends Logging {
.bytesConf(ByteUnit.BYTE)
.createWithDefaultString("2g")
+ val FILE_CACHE_MAX_SIZE: ConfigEntry[Long] =
+ buildConf("celeborn.shuffle.cache.max")
+ .categories("worker")
+ .doc("Max size for shuffle file cache to memory")
+ .version("0.3.2")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefaultString("256m")
+
+ val FILE_CACHE_ENABLE: ConfigEntry[Boolean] =
+ buildConf("celeborn.shuffle.cache.enable")
Review Comment:
celeborn.worker.memoryCache.enabled
##########
common/src/main/java/org/apache/celeborn/common/meta/MemoryBuffers.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.meta;
+
+import java.util.List;
+
+import io.netty.buffer.ByteBuf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.network.buffer.ManagedBuffer;
+import org.apache.celeborn.common.network.buffer.NettyManagedBuffer;
+import org.apache.celeborn.common.network.util.TransportConf;
+import org.apache.celeborn.common.util.MemCacheManager;
+
+public class MemoryBuffers implements Buffers {
+ private static final Logger logger =
LoggerFactory.getLogger(FileManagedBuffers.class);
Review Comment:
`MemoryBuffers.class`
##########
worker/src/main/java/org/apache/celeborn/service/deploy/worker/storage/FileWriter.java:
##########
@@ -159,14 +162,21 @@ 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 && fileInfo.getFileLength() == 0 &&
cacheManager.canCache(numBytes)) {
Review Comment:
Need to check whether it's local disk here `channel != null`
##########
common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala:
##########
@@ -2057,6 +2060,22 @@ object CelebornConf extends Logging {
.bytesConf(ByteUnit.BYTE)
.createWithDefaultString("2g")
+ val FILE_CACHE_MAX_SIZE: ConfigEntry[Long] =
+ buildConf("celeborn.shuffle.cache.max")
Review Comment:
celeborn.worker.memoryCache.capacity
##########
worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/StorageManager.scala:
##########
@@ -558,6 +560,10 @@ final private[worker] class StorageManager(conf:
CelebornConf, workerSource: Abs
if (removedFileInfos != null) {
removedFileInfos.asScala.foreach {
case (_, fileInfo) =>
+ if (memCacheManager.contains(fileInfo.getFilePath)) {
Review Comment:
Multiple threads can simultaneously call `cleanupExpiredShuffleKey`. IMO we
can change `removeCache` as follows:
```
public void removeCache(String key) {
ByteBuf cache = caches.remove(key);
if (cache != null) {
currentCacheSize.getAndAdd(-1 * cache.readableBytes());
cache.release();
}
}
```
Here we can just call `removeCache` without checking contains.
##########
common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala:
##########
@@ -2057,6 +2060,22 @@ object CelebornConf extends Logging {
.bytesConf(ByteUnit.BYTE)
.createWithDefaultString("2g")
+ val FILE_CACHE_MAX_SIZE: ConfigEntry[Long] =
Review Comment:
WORKER_MEMORYCACHE_CAPACITY
##########
common/src/main/java/org/apache/celeborn/common/meta/MemoryBuffers.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.meta;
+
+import java.util.List;
+
+import io.netty.buffer.ByteBuf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.network.buffer.ManagedBuffer;
+import org.apache.celeborn.common.network.buffer.NettyManagedBuffer;
+import org.apache.celeborn.common.network.util.TransportConf;
+import org.apache.celeborn.common.util.MemCacheManager;
+
+public class MemoryBuffers implements Buffers {
+ private static final Logger logger =
LoggerFactory.getLogger(FileManagedBuffers.class);
+
+ private final long[] offsets;
+ private final int numChunks;
+ private final String filePath;
+
+ private final TransportConf conf;
Review Comment:
`conf` is not used
##########
common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala:
##########
@@ -913,6 +913,9 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable
with Logging with Se
def partitionSplitMinimumSize: Long = get(WORKER_PARTITION_SPLIT_MIN_SIZE)
def partitionSplitMaximumSize: Long = get(WORKER_PARTITION_SPLIT_MAX_SIZE)
+ def fileCacheMaxSize: Long = get(FILE_CACHE_MAX_SIZE)
+ def fileCacheEnable: Boolean = get(FILE_CACHE_ENABLE)
Review Comment:
workerMemoryCacheEnabled
##########
common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala:
##########
@@ -913,6 +913,9 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable
with Logging with Se
def partitionSplitMinimumSize: Long = get(WORKER_PARTITION_SPLIT_MIN_SIZE)
def partitionSplitMaximumSize: Long = get(WORKER_PARTITION_SPLIT_MAX_SIZE)
+ def fileCacheMaxSize: Long = get(FILE_CACHE_MAX_SIZE)
Review Comment:
workerMemoryCacheCapacity
##########
worker/src/main/java/org/apache/celeborn/service/deploy/worker/storage/PartitionFilesSorter.java:
##########
@@ -611,10 +614,23 @@ public void sort() throws InterruptedException {
private void initializeFiles() throws IOException {
if (isHdfs) {
+ if (memCacheManager.contains(originFilePath)) {
Review Comment:
Since this PR only affects local disk, we can remove this code block
##########
common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala:
##########
@@ -2057,6 +2060,22 @@ object CelebornConf extends Logging {
.bytesConf(ByteUnit.BYTE)
.createWithDefaultString("2g")
+ val FILE_CACHE_MAX_SIZE: ConfigEntry[Long] =
+ buildConf("celeborn.shuffle.cache.max")
+ .categories("worker")
+ .doc("Max size for shuffle file cache to memory")
+ .version("0.3.2")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefaultString("256m")
+
+ val FILE_CACHE_ENABLE: ConfigEntry[Boolean] =
Review Comment:
WORKER_MEMORYCACHE_ENABLED
--
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]