RexXiong commented on code in PR #3065: URL: https://github.com/apache/celeborn/pull/3065#discussion_r1919589039
########## worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/TierWriter.scala: ########## @@ -0,0 +1,339 @@ +/* + * 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.service.deploy.worker.storage + +import java.io.IOException +import java.nio.ByteBuffer +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicInteger + +import io.netty.buffer.{ByteBuf, ByteBufAllocator, CompositeByteBuf} + +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.exception.AlreadyClosedException +import org.apache.celeborn.common.internal.Logging +import org.apache.celeborn.common.meta.{FileInfo, MemoryFileInfo} +import org.apache.celeborn.common.metrics.source.AbstractSource +import org.apache.celeborn.common.protocol.StorageInfo +import org.apache.celeborn.common.unsafe.Platform +import org.apache.celeborn.service.deploy.worker.WorkerSource +import org.apache.celeborn.service.deploy.worker.memory.MemoryManager + +abstract class TierWriterBase( + val conf: CelebornConf, + val metaHandler: PartitionMetaHandler, + val numPendingWrites: AtomicInteger, + val notifier: FlushNotifier, + val fileInfo: FileInfo, + val source: AbstractSource, + val storageType: StorageInfo.Type, + val filename: String, + val shuffleKey: String, + val storageManager: StorageManager) extends Logging { + val metricsCollectCriticalEnabled: Boolean = conf.metricsCollectCriticalEnabled + val flushLock = new AnyRef + val WAIT_INTERVAL_MS = 5 + + var flushBuffer: CompositeByteBuf = _ + var writerCloseTimeoutMs: Long = conf.workerWriterCloseTimeoutMs + var flusherBufferSize = 0L + var chunkSize: Long = conf.shuffleChunkSize + + @volatile var closed = false + @volatile var destroyed = false + + val memoryFileAllocator: ByteBufAllocator = storageManager.storageBufferAllocator + var memoryFileStorageMaxFileSize: Long = conf.workerMemoryFileStorageMaxFileSize + + takeBuffer() + + def write(buf: ByteBuf): Unit = { + ensureNotClosed() + if (notifier.hasException) return + + flushLock.synchronized { + metaHandler.beforeWrite(buf) + ensureNotClosed() + writerInternal(buf) + updateMemoryMetric(buf.readableBytes) + } + + metaHandler.afterWrite(buf.readableBytes()) + numPendingWrites.decrementAndGet() + } + + protected def writerInternal(buf: ByteBuf): Unit + + def updateMemoryMetric(numBytes: Int): Unit + + def needEvict(): Boolean + + def evict(file: TierWriterBase): Unit + + def swapFlushBuffer(inputBuffer: CompositeByteBuf): Unit = { + if (flushBuffer != null) { + returnBuffer(false) + } + flushBuffer = inputBuffer + } + + def close(evict: Boolean = false): Long = { + try { + ensureNotClosed() + waitOnNoPending(numPendingWrites) + closed = true + finalFlush() + + waitOnNoPending(notifier.numPendingFlushes) + metaHandler.afterClose() + } finally { + returnBuffer(false) + try { + closeStreams() + } catch { + case e: IOException => + logWarning(s"close file writer ${this} failed", e) + } + } + if (!evict) { + notifyFileCommitted() + } + + fileInfo.getFileLength + } + + def ensureNotClosed(): Unit = { + if (closed) { + val msg = getFileAlreadyClosedMsg + logWarning(msg) + throw new AlreadyClosedException(msg) + } + } + + def getFileAlreadyClosedMsg: String = { + s"PartitionDataWriter has already closed! File name:${filename}" + } + + // this method is not used in memory tier writer + def notifyFileCommitted(): Unit = {} + + // this method is not used in memory tier writer + def finalFlush(): Unit = {} + + @throws[IOException] + protected def waitOnNoPending(counter: AtomicInteger): Unit = { + var waitTime = writerCloseTimeoutMs + while (counter.get > 0 && waitTime > 0) { + try { + notifier.checkException() + TimeUnit.MILLISECONDS.sleep(WAIT_INTERVAL_MS) + } catch { + case e: InterruptedException => + val ioe = new IOException(e) + notifier.setException(ioe) + throw ioe + } + waitTime -= WAIT_INTERVAL_MS + } + if (counter.get > 0) { Review Comment: numPendingWrites can bypass `counter.get>0` ########## worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/TierWriter.scala: ########## @@ -0,0 +1,339 @@ +/* + * 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.service.deploy.worker.storage + +import java.io.IOException +import java.nio.ByteBuffer +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicInteger + +import io.netty.buffer.{ByteBuf, ByteBufAllocator, CompositeByteBuf} + +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.exception.AlreadyClosedException +import org.apache.celeborn.common.internal.Logging +import org.apache.celeborn.common.meta.{FileInfo, MemoryFileInfo} +import org.apache.celeborn.common.metrics.source.AbstractSource +import org.apache.celeborn.common.protocol.StorageInfo +import org.apache.celeborn.common.unsafe.Platform +import org.apache.celeborn.service.deploy.worker.WorkerSource +import org.apache.celeborn.service.deploy.worker.memory.MemoryManager + +abstract class TierWriterBase( + val conf: CelebornConf, + val metaHandler: PartitionMetaHandler, + val numPendingWrites: AtomicInteger, + val notifier: FlushNotifier, + val fileInfo: FileInfo, + val source: AbstractSource, + val storageType: StorageInfo.Type, + val filename: String, + val shuffleKey: String, + val storageManager: StorageManager) extends Logging { + val metricsCollectCriticalEnabled: Boolean = conf.metricsCollectCriticalEnabled + val flushLock = new AnyRef + val WAIT_INTERVAL_MS = 5 + + var flushBuffer: CompositeByteBuf = _ + var writerCloseTimeoutMs: Long = conf.workerWriterCloseTimeoutMs + var flusherBufferSize = 0L + var chunkSize: Long = conf.shuffleChunkSize + + @volatile var closed = false + @volatile var destroyed = false + + val memoryFileAllocator: ByteBufAllocator = storageManager.storageBufferAllocator + var memoryFileStorageMaxFileSize: Long = conf.workerMemoryFileStorageMaxFileSize + + takeBuffer() + + def write(buf: ByteBuf): Unit = { + ensureNotClosed() + if (notifier.hasException) return + + flushLock.synchronized { + metaHandler.beforeWrite(buf) + ensureNotClosed() + writerInternal(buf) + updateMemoryMetric(buf.readableBytes) + } + + metaHandler.afterWrite(buf.readableBytes()) + numPendingWrites.decrementAndGet() + } + + protected def writerInternal(buf: ByteBuf): Unit + + def updateMemoryMetric(numBytes: Int): Unit + + def needEvict(): Boolean + + def evict(file: TierWriterBase): Unit + + def swapFlushBuffer(inputBuffer: CompositeByteBuf): Unit = { + if (flushBuffer != null) { + returnBuffer(false) + } + flushBuffer = inputBuffer + } + + def close(evict: Boolean = false): Long = { Review Comment: should we synchronized when close? ########## worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/TierWriter.scala: ########## @@ -0,0 +1,339 @@ +/* + * 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.service.deploy.worker.storage + +import java.io.IOException +import java.nio.ByteBuffer +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicInteger + +import io.netty.buffer.{ByteBuf, ByteBufAllocator, CompositeByteBuf} + +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.exception.AlreadyClosedException +import org.apache.celeborn.common.internal.Logging +import org.apache.celeborn.common.meta.{FileInfo, MemoryFileInfo} +import org.apache.celeborn.common.metrics.source.AbstractSource +import org.apache.celeborn.common.protocol.StorageInfo +import org.apache.celeborn.common.unsafe.Platform +import org.apache.celeborn.service.deploy.worker.WorkerSource +import org.apache.celeborn.service.deploy.worker.memory.MemoryManager + +abstract class TierWriterBase( + val conf: CelebornConf, + val metaHandler: PartitionMetaHandler, + val numPendingWrites: AtomicInteger, + val notifier: FlushNotifier, + val fileInfo: FileInfo, + val source: AbstractSource, + val storageType: StorageInfo.Type, + val filename: String, + val shuffleKey: String, + val storageManager: StorageManager) extends Logging { + val metricsCollectCriticalEnabled: Boolean = conf.metricsCollectCriticalEnabled + val flushLock = new AnyRef + val WAIT_INTERVAL_MS = 5 + + var flushBuffer: CompositeByteBuf = _ + var writerCloseTimeoutMs: Long = conf.workerWriterCloseTimeoutMs + var flusherBufferSize = 0L + var chunkSize: Long = conf.shuffleChunkSize + + @volatile var closed = false + @volatile var destroyed = false + + val memoryFileAllocator: ByteBufAllocator = storageManager.storageBufferAllocator + var memoryFileStorageMaxFileSize: Long = conf.workerMemoryFileStorageMaxFileSize + + takeBuffer() + + def write(buf: ByteBuf): Unit = { + ensureNotClosed() + if (notifier.hasException) return + + flushLock.synchronized { + metaHandler.beforeWrite(buf) + ensureNotClosed() + writerInternal(buf) + updateMemoryMetric(buf.readableBytes) + } + + metaHandler.afterWrite(buf.readableBytes()) + numPendingWrites.decrementAndGet() + } + + protected def writerInternal(buf: ByteBuf): Unit + + def updateMemoryMetric(numBytes: Int): Unit + + def needEvict(): Boolean + + def evict(file: TierWriterBase): Unit + + def swapFlushBuffer(inputBuffer: CompositeByteBuf): Unit = { + if (flushBuffer != null) { + returnBuffer(false) + } + flushBuffer = inputBuffer + } + + def close(evict: Boolean = false): Long = { + try { + ensureNotClosed() + waitOnNoPending(numPendingWrites) + closed = true + finalFlush() + + waitOnNoPending(notifier.numPendingFlushes) + metaHandler.afterClose() + } finally { + returnBuffer(false) + try { + closeStreams() + } catch { + case e: IOException => + logWarning(s"close file writer ${this} failed", e) + } + } + if (!evict) { + notifyFileCommitted() + } + + fileInfo.getFileLength + } + + def ensureNotClosed(): Unit = { + if (closed) { + val msg = getFileAlreadyClosedMsg + logWarning(msg) + throw new AlreadyClosedException(msg) + } + } + + def getFileAlreadyClosedMsg: String = { + s"PartitionDataWriter has already closed! File name:${filename}" + } + + // this method is not used in memory tier writer + def notifyFileCommitted(): Unit = {} + + // this method is not used in memory tier writer + def finalFlush(): Unit = {} + + @throws[IOException] + protected def waitOnNoPending(counter: AtomicInteger): Unit = { + var waitTime = writerCloseTimeoutMs + while (counter.get > 0 && waitTime > 0) { + try { + notifier.checkException() + TimeUnit.MILLISECONDS.sleep(WAIT_INTERVAL_MS) + } catch { + case e: InterruptedException => + val ioe = new IOException(e) + notifier.setException(ioe) + throw ioe + } + waitTime -= WAIT_INTERVAL_MS + } + if (counter.get > 0) { + val ioe = new IOException("Wait pending actions timeout.") + notifier.setException(ioe) + throw ioe + } + notifier.checkException() + } + + def closeStreams(): Unit + + def genFlushTask(finalFlush: Boolean, keepBuffer: Boolean): FlushTask + + def flush(finalFlush: Boolean, fromEvict: Boolean = false): Unit = { + if (flushBuffer != null) { + val numBytes = flushBuffer.readableBytes() + var flushTask: FlushTask = null + if (numBytes != 0) { + if (fromEvict) { + // notifier.numPendingFlushes.incrementAndGet() + val dupBuf = flushBuffer.retainedDuplicate() + // this flusher buffer is from memory tier writer, so that we can not keep the buffer + flushTask = genFlushTask(finalFlush, false) + MemoryManager.instance.releaseMemoryFileStorage(numBytes) Review Comment: move to MemoryTierWriter, releaseMemoryFileStorage and incrementDiskBuffer after evict finished ########## worker/src/main/scala/org/apache/celeborn/service/deploy/worker/storage/TierWriter.scala: ########## @@ -0,0 +1,339 @@ +/* + * 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.service.deploy.worker.storage + +import java.io.IOException +import java.nio.ByteBuffer +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicInteger + +import io.netty.buffer.{ByteBuf, ByteBufAllocator, CompositeByteBuf} + +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.exception.AlreadyClosedException +import org.apache.celeborn.common.internal.Logging +import org.apache.celeborn.common.meta.{FileInfo, MemoryFileInfo} +import org.apache.celeborn.common.metrics.source.AbstractSource +import org.apache.celeborn.common.protocol.StorageInfo +import org.apache.celeborn.common.unsafe.Platform +import org.apache.celeborn.service.deploy.worker.WorkerSource +import org.apache.celeborn.service.deploy.worker.memory.MemoryManager + +abstract class TierWriterBase( + val conf: CelebornConf, + val metaHandler: PartitionMetaHandler, + val numPendingWrites: AtomicInteger, + val notifier: FlushNotifier, + val fileInfo: FileInfo, + val source: AbstractSource, + val storageType: StorageInfo.Type, + val filename: String, + val shuffleKey: String, + val storageManager: StorageManager) extends Logging { + val metricsCollectCriticalEnabled: Boolean = conf.metricsCollectCriticalEnabled + val flushLock = new AnyRef + val WAIT_INTERVAL_MS = 5 + + var flushBuffer: CompositeByteBuf = _ + var writerCloseTimeoutMs: Long = conf.workerWriterCloseTimeoutMs + var flusherBufferSize = 0L + var chunkSize: Long = conf.shuffleChunkSize + + @volatile var closed = false + @volatile var destroyed = false + + val memoryFileAllocator: ByteBufAllocator = storageManager.storageBufferAllocator + var memoryFileStorageMaxFileSize: Long = conf.workerMemoryFileStorageMaxFileSize + + takeBuffer() + + def write(buf: ByteBuf): Unit = { + ensureNotClosed() + if (notifier.hasException) return + + flushLock.synchronized { + metaHandler.beforeWrite(buf) + ensureNotClosed() + writerInternal(buf) + updateMemoryMetric(buf.readableBytes) Review Comment: move updateMemoryMetric to MemoryTierWriter -- 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]
