[
https://issues.apache.org/jira/browse/HDDS-10599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17834040#comment-17834040
]
Wei-Chiu Chuang commented on HDDS-10599:
----------------------------------------
ok the problem is that lastChunkBuffer is a byte buffer with length defined by
ozone.client.stream.buffer.size (default 4MB)
whereas the current buffer is a linked list of byte buffers, each one of length
defined by ozone.client.stream.buffer.increment (default is 0 which makes one
big byte buffer length defined by ozone.client.stream.buffer.size)
If ozone.client.stream.buffer.increment is not zero, the allocated byte buffer
is not going to match the length of lastChunkBuffer, therefore goes out of
bound.
Solution:
(1) do not set ozone.client.stream.buffer.increment for now.
(2) fix lastChunkBuffer so that it is either removed, or use ChunkBuffer to
account for the byte buffers.
> [Hbase Ozone] Regionserver Crash with "BlockOutputStream: Failed to flush
> Java.lang.IllegalArgumentException"
> -------------------------------------------------------------------------------------------------------------
>
> Key: HDDS-10599
> URL: https://issues.apache.org/jira/browse/HDDS-10599
> Project: Apache Ozone
> Issue Type: Bug
> Components: SCM
> Reporter: Pratyush Bhatt
> Assignee: Wei-Chiu Chuang
> Priority: Major
>
> Simply Triggering YCSB is crashing the Regionservers with below error:
> {code:java}
> 2024-03-27 07:16:44,910 ERROR
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream: Failed to flush. error:
> null
> java.lang.IllegalArgumentException
> at java.nio.Buffer.limit(Buffer.java:275)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.appendLastChunkBuffer(BlockOutputStream.java:859)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.updateBlockDataForWriteChunk(BlockOutputStream.java:819)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.writeChunkToContainer(BlockOutputStream.java:768)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.writeChunk(BlockOutputStream.java:565)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlushInternal(BlockOutputStream.java:598)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlush(BlockOutputStream.java:573)
> at
> org.apache.hadoop.hdds.scm.storage.RatisBlockOutputStream.hsync(RatisBlockOutputStream.java:139)
> at
> org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry.hsync(BlockOutputStreamEntry.java:163)
> at
> org.apache.hadoop.ozone.client.io.KeyOutputStream.handleStreamAction(KeyOutputStream.java:524)
> at
> org.apache.hadoop.ozone.client.io.KeyOutputStream.handleFlushOrClose(KeyOutputStream.java:487)
> at
> org.apache.hadoop.ozone.client.io.KeyOutputStream.hsync(KeyOutputStream.java:457)
> at
> org.apache.hadoop.ozone.client.io.OzoneOutputStream.hsync(OzoneOutputStream.java:118)
> at
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.hsync(OzoneFSOutputStream.java:70)
> at
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.hflush(OzoneFSOutputStream.java:65)
> at
> org.apache.hadoop.fs.FSDataOutputStream.hflush(FSDataOutputStream.java:136)
> at
> org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter.sync(ProtobufLogWriter.java:84)
> at
> org.apache.hadoop.hbase.regionserver.wal.FSHLog$SyncRunner.run(FSHLog.java:669)
> 2024-03-27 07:16:44,912 WARN org.apache.hadoop.hbase.regionserver.wal.FSHLog:
> UNEXPECTED
> java.lang.IllegalArgumentException
> at java.nio.Buffer.limit(Buffer.java:275)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.appendLastChunkBuffer(BlockOutputStream.java:859)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.updateBlockDataForWriteChunk(BlockOutputStream.java:819)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.writeChunkToContainer(BlockOutputStream.java:768)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.writeChunk(BlockOutputStream.java:565)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlushInternal(BlockOutputStream.java:598)
> at
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlush(BlockOutputStream.java:573)
> at
> org.apache.hadoop.hdds.scm.storage.RatisBlockOutputStream.hsync(RatisBlockOutputStream.java:139)
> at
> org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry.hsync(BlockOutputStreamEntry.java:163)
> at
> org.apache.hadoop.ozone.client.io.KeyOutputStream.handleStreamAction(KeyOutputStream.java:524)
> at
> org.apache.hadoop.ozone.client.io.KeyOutputStream.handleFlushOrClose(KeyOutputStream.java:487)
> at
> org.apache.hadoop.ozone.client.io.KeyOutputStream.hsync(KeyOutputStream.java:457)
> at
> org.apache.hadoop.ozone.client.io.OzoneOutputStream.hsync(OzoneOutputStream.java:118)
> at
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.hsync(OzoneFSOutputStream.java:70)
> at
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.hflush(OzoneFSOutputStream.java:65)
> at
> org.apache.hadoop.fs.FSDataOutputStream.hflush(FSDataOutputStream.java:136)
> at
> org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter.sync(ProtobufLogWriter.java:84)
> at
> org.apache.hadoop.hbase.regionserver.wal.FSHLog$SyncRunner.run(FSHLog.java:669)
> 2024-03-27 07:16:44,914 ERROR
> org.apache.hadoop.hbase.regionserver.wal.FSHLog: Error syncing, request close
> of WAL {code}
> I think this is because of setting :
> {code:java}
> "ozone.client.stream.buffer.increment": "16KB",
> "ozone.client.bytes.per.checksum": "16KB" {code}
> When we remove these properties, then YCSB works fine.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]