[
https://issues.apache.org/jira/browse/HDDS-10547?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Kirill Sizov updated HDDS-10547:
--------------------------------
Description:
If {{hdds.datanode.chunk.data.validation.check}} is set to {{true}}, any
attempt to put a file into Ozone fails.
{noformat}
2024-03-18 10:42:45 2024-03-18 07:42:45 DEBUG FilePerBlockStrategy:155 -
Writing chunk ChunkInfo{chunkName='113750153625600001_chunk_1, offset=0,
len=174} (overwrite: false) in stage COMBINED to file
/data/hdds/hdds/CID-690724fb-e793-4016-ad36-427c2721634a/current/containerDir0/1/chunks/113750153625600001.block
2024-03-18 10:42:45 2024-03-18 07:42:45 ERROR ChunkUtils:424 - Unexpected
buffer size. expected: 174, actual: 0
2024-03-18 10:42:45 2024-03-18 07:42:45 WARN KeyValueHandler:97 - Operation:
WriteChunk , Trace ID: , Message: Unexpected buffer size. expected: 174,
actual: 0 , Result: INVALID_WRITE_SIZE , StorageContainerException Occurred.
2024-03-18 10:42:45
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException:
Unexpected buffer size. expected: 174, actual: 0
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.checkSize(ChunkUtils.java:425)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.validateBufferSize(ChunkUtils.java:415)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.writeData(ChunkUtils.java:125)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.writeData(ChunkUtils.java:117)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.impl.FilePerBlockStrategy.writeChunk(FilePerBlockStrategy.java:165)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerDispatcher.writeChunk(ChunkManagerDispatcher.java:75)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.handleWriteChunk(KeyValueHandler.java:822)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.dispatchRequest(KeyValueHandler.java:263)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.handle(KeyValueHandler.java:222)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatchRequest(HddsDispatcher.java:329)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.lambda$dispatch$0(HddsDispatcher.java:179)
2024-03-18 10:42:45 at
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:89)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatch(HddsDispatcher.java:178)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.common.transport.server.GrpcXceiverService$1.onNext(GrpcXceiverService.java:112)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.common.transport.server.GrpcXceiverService$1.onNext(GrpcXceiverService.java:105)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.ForwardingServerCallListener.onMessage(ForwardingServerCallListener.java:33)
2024-03-18 10:42:45 at
org.apache.hadoop.hdds.tracing.GrpcServerInterceptor$1.onMessage(GrpcServerInterceptor.java:49)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:329)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:314)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:833)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
2024-03-18 10:42:45 at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
2024-03-18 10:42:45 at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
2024-03-18 10:42:45 at java.base/java.lang.Thread.run(Thread.java:829)
{noformat}
This issue was caused by HDDS-9611.
The fix is to restore the buffer posiiton after computing the checksum as the
code is further reading from the same buffer but apparently the remaining
length is 0.
was:
If {{hdds.datanode.chunk.data.validation.check}} is set to {{true}}, any
attempt to put a file into Ozone fails.
{noformat}
2024-03-18 10:42:45 2024-03-18 07:42:45 DEBUG FilePerBlockStrategy:155 -
Writing chunk ChunkInfo{chunkName='113750153625600001_chunk_1, offset=0,
len=174} (overwrite: false) in stage COMBINED to file
/data/hdds/hdds/CID-690724fb-e793-4016-ad36-427c2721634a/current/containerDir0/1/chunks/113750153625600001.block
2024-03-18 10:42:45 2024-03-18 07:42:45 ERROR ChunkUtils:424 - Unexpected
buffer size. expected: 174, actual: 0
2024-03-18 10:42:45 2024-03-18 07:42:45 WARN KeyValueHandler:97 - Operation:
WriteChunk , Trace ID: , Message: Unexpected buffer size. expected: 174,
actual: 0 , Result: INVALID_WRITE_SIZE , StorageContainerException Occurred.
2024-03-18 10:42:45
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException:
Unexpected buffer size. expected: 174, actual: 0
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.checkSize(ChunkUtils.java:425)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.validateBufferSize(ChunkUtils.java:415)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.writeData(ChunkUtils.java:125)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.writeData(ChunkUtils.java:117)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.impl.FilePerBlockStrategy.writeChunk(FilePerBlockStrategy.java:165)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerDispatcher.writeChunk(ChunkManagerDispatcher.java:75)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.handleWriteChunk(KeyValueHandler.java:822)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.dispatchRequest(KeyValueHandler.java:263)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.handle(KeyValueHandler.java:222)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatchRequest(HddsDispatcher.java:329)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.lambda$dispatch$0(HddsDispatcher.java:179)
2024-03-18 10:42:45 at
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:89)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatch(HddsDispatcher.java:178)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.common.transport.server.GrpcXceiverService$1.onNext(GrpcXceiverService.java:112)
2024-03-18 10:42:45 at
org.apache.hadoop.ozone.container.common.transport.server.GrpcXceiverService$1.onNext(GrpcXceiverService.java:105)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.ForwardingServerCallListener.onMessage(ForwardingServerCallListener.java:33)
2024-03-18 10:42:45 at
org.apache.hadoop.hdds.tracing.GrpcServerInterceptor$1.onMessage(GrpcServerInterceptor.java:49)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:329)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:314)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:833)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
2024-03-18 10:42:45 at
org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
2024-03-18 10:42:45 at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
2024-03-18 10:42:45 at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
2024-03-18 10:42:45 at java.base/java.lang.Thread.run(Thread.java:829)
{noformat}
This issue was caused by HDDS-9611.
The fix is to do a {{ChunkBuffer.rewind()}} after computing the checksum as the
code is further reading from the same buffer but apparently the remaining
length is 0.
> Failure to write a file if checksum validation is on
> ----------------------------------------------------
>
> Key: HDDS-10547
> URL: https://issues.apache.org/jira/browse/HDDS-10547
> Project: Apache Ozone
> Issue Type: Bug
> Reporter: Kirill Sizov
> Assignee: Kirill Sizov
> Priority: Critical
>
> If {{hdds.datanode.chunk.data.validation.check}} is set to {{true}}, any
> attempt to put a file into Ozone fails.
> {noformat}
> 2024-03-18 10:42:45 2024-03-18 07:42:45 DEBUG FilePerBlockStrategy:155 -
> Writing chunk ChunkInfo{chunkName='113750153625600001_chunk_1, offset=0,
> len=174} (overwrite: false) in stage COMBINED to file
> /data/hdds/hdds/CID-690724fb-e793-4016-ad36-427c2721634a/current/containerDir0/1/chunks/113750153625600001.block
> 2024-03-18 10:42:45 2024-03-18 07:42:45 ERROR ChunkUtils:424 - Unexpected
> buffer size. expected: 174, actual: 0
> 2024-03-18 10:42:45 2024-03-18 07:42:45 WARN KeyValueHandler:97 - Operation:
> WriteChunk , Trace ID: , Message: Unexpected buffer size. expected: 174,
> actual: 0 , Result: INVALID_WRITE_SIZE , StorageContainerException Occurred.
> 2024-03-18 10:42:45
> org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException:
> Unexpected buffer size. expected: 174, actual: 0
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.checkSize(ChunkUtils.java:425)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.validateBufferSize(ChunkUtils.java:415)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.writeData(ChunkUtils.java:125)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils.writeData(ChunkUtils.java:117)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.keyvalue.impl.FilePerBlockStrategy.writeChunk(FilePerBlockStrategy.java:165)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerDispatcher.writeChunk(ChunkManagerDispatcher.java:75)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.handleWriteChunk(KeyValueHandler.java:822)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.dispatchRequest(KeyValueHandler.java:263)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.handle(KeyValueHandler.java:222)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatchRequest(HddsDispatcher.java:329)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.lambda$dispatch$0(HddsDispatcher.java:179)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:89)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatch(HddsDispatcher.java:178)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.common.transport.server.GrpcXceiverService$1.onNext(GrpcXceiverService.java:112)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.ozone.container.common.transport.server.GrpcXceiverService$1.onNext(GrpcXceiverService.java:105)
> 2024-03-18 10:42:45 at
> org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262)
> 2024-03-18 10:42:45 at
> org.apache.ratis.thirdparty.io.grpc.ForwardingServerCallListener.onMessage(ForwardingServerCallListener.java:33)
> 2024-03-18 10:42:45 at
> org.apache.hadoop.hdds.tracing.GrpcServerInterceptor$1.onMessage(GrpcServerInterceptor.java:49)
> 2024-03-18 10:42:45 at
> org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:329)
> 2024-03-18 10:42:45 at
> org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:314)
> 2024-03-18 10:42:45 at
> org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:833)
> 2024-03-18 10:42:45 at
> org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
> 2024-03-18 10:42:45 at
> org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
> 2024-03-18 10:42:45 at
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
> 2024-03-18 10:42:45 at
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
> 2024-03-18 10:42:45 at java.base/java.lang.Thread.run(Thread.java:829)
> {noformat}
> This issue was caused by HDDS-9611.
> The fix is to restore the buffer posiiton after computing the checksum as the
> code is further reading from the same buffer but apparently the remaining
> length is 0.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]