[ 
https://issues.apache.org/jira/browse/HDDS-11193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17868549#comment-17868549
 ] 

Pratyush Bhatt commented on HDDS-11193:
---------------------------------------

Hi [~duongnguyen] , I was using this fix, but I am still hitting the above 
java.lang.IllegalArgumentException issue, and all the RS went down with below 
issue:
{code:java}
2024-07-23 05:32:26,367 ERROR 
org.apache.hadoop.hbase.regionserver.CompactSplit: Compaction failed 
region=11loadtesttabtyui22aaqw23,a22221f2,1721725742844.9bcd68da6a7f7c0150854b20b2340a41.,
 storeName=9bcd68da6a7f7c0150854b20b2340a41/ch8, priority=13, 
startTime=1721737880870
java.io.IOException: java.lang.IllegalArgumentException
        at 
org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.ensureIOException(AbstractFSWAL.java:858)
        at 
org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.blockOnSync(AbstractFSWAL.java:853)
        at 
org.apache.hadoop.hbase.regionserver.wal.FSHLog.publishSyncThenBlockOnCompletion(FSHLog.java:809)
        at org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:858)
        at org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:847)
        at 
org.apache.hadoop.hbase.regionserver.wal.WALUtil.doFullMarkerAppendTransaction(WALUtil.java:163)
        at 
org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeMarker(WALUtil.java:140)
        at 
org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeCompactionMarker(WALUtil.java:74)
        at 
org.apache.hadoop.hbase.regionserver.HStore.writeCompactionWalRecord(HStore.java:1262)
        at 
org.apache.hadoop.hbase.regionserver.HStore.lambda$replaceStoreFiles$3(HStore.java:1272)
        at 
org.apache.hadoop.hbase.regionserver.StoreEngine.replaceStoreFiles(StoreEngine.java:483)
        at 
org.apache.hadoop.hbase.regionserver.HStore.replaceStoreFiles(HStore.java:1270)
        at 
org.apache.hadoop.hbase.regionserver.HStore.doCompaction(HStore.java:1206)
        at org.apache.hadoop.hbase.regionserver.HStore.compact(HStore.java:1188)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.compact(HRegion.java:2261)
        at 
org.apache.hadoop.hbase.regionserver.CompactSplit$CompactionRunner.doCompaction(CompactSplit.java:625)
        at 
org.apache.hadoop.hbase.regionserver.CompactSplit$CompactionRunner.run(CompactSplit.java:673)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.IllegalArgumentException
        at 
org.apache.hadoop.ozone.shaded.com.google.common.base.Preconditions.checkArgument(Preconditions.java:129)
        at 
org.apache.hadoop.ozone.client.io.KeyOutputStream.handleException(KeyOutputStream.java:318)
        at 
org.apache.hadoop.ozone.client.io.KeyOutputStream.handleFlushOrClose(KeyOutputStream.java:499)
        at 
org.apache.hadoop.ozone.client.io.KeyOutputStream.hsync(KeyOutputStream.java:463)
        at 
org.apache.hadoop.ozone.client.io.OzoneOutputStream.hsync(OzoneOutputStream.java:118)
        at 
org.apache.hadoop.hdds.tracing.TracingUtil.executeInSpan(TracingUtil.java:184)
        at 
org.apache.hadoop.hdds.tracing.TracingUtil.executeInNewSpan(TracingUtil.java:149)
        at 
org.apache.hadoop.fs.ozone.OzoneFSOutputStream.hsync(OzoneFSOutputStream.java:80)
        at 
org.apache.hadoop.fs.ozone.OzoneFSOutputStream.hflush(OzoneFSOutputStream.java:75)
        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-07-23 05:32:26,371 ERROR 
org.apache.hadoop.hbase.regionserver.HRegionServer: ***** ABORTING region 
server vc0134.xyz,22101,1721724988754: Failed log close in log roller 
*****{code}
cc: [~ashishkr] [~weichiu] 

> KeyOutputStream flakiness when running write and hsync concurrently
> -------------------------------------------------------------------
>
>                 Key: HDDS-11193
>                 URL: https://issues.apache.org/jira/browse/HDDS-11193
>             Project: Apache Ozone
>          Issue Type: Sub-task
>            Reporter: Duong
>            Assignee: Duong
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: HDDS-7593
>
>
> Testing KeyOutputStream with TestHSync#testConcurrentWriteHSync sometimes 
> fails. Bellow are the known error stacktraces.
> Repeated test run CI: 
> [https://github.com/apache/ozone/actions/runs/9961420355]
> #1
> {code:java}
> java.io.IOException: java.lang.IndexOutOfBoundsException
>     at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.handleWrite(KeyOutputStream.java:243)
>     at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.write(KeyOutputStream.java:212)
>     at 
> org.apache.hadoop.ozone.client.io.OzoneOutputStream.write(OzoneOutputStream.java:94)
>     at 
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.lambda$write$1(OzoneFSOutputStream.java:58)
>     at 
> org.apache.hadoop.hdds.tracing.TracingUtil.executeInSpan(TracingUtil.java:184)
>     at 
> org.apache.hadoop.hdds.tracing.TracingUtil.executeInNewSpan(TracingUtil.java:149)
>     at 
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.write(OzoneFSOutputStream.java:54)
>     at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:62)
>     at java.base/java.io.DataOutputStream.write(DataOutputStream.java:112)
>     at java.base/java.io.FilterOutputStream.write(FilterOutputStream.java:108)
>     at 
> org.apache.hadoop.fs.ozone.TestHSync.lambda$runConcurrentWriteHSync$8(TestHSync.java:683)
>     at java.base/java.lang.Thread.run(Thread.java:840)
> Caused by: java.lang.IndexOutOfBoundsException
>     at 
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.write(BlockOutputStream.java:328)
>     at 
> org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry.write(BlockOutputStreamEntry.java:138)
>     at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.writeToOutputStream(KeyOutputStream.java:255)
>     at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.handleWrite(KeyOutputStream.java:233)
>  {code}
> #2
> {code:java}
> java.io.IOException: java.lang.IllegalArgumentException
>     at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.handleWrite(KeyOutputStream.java:243)
>     at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.write(KeyOutputStream.java:212)
>     at 
> org.apache.hadoop.ozone.client.io.OzoneOutputStream.write(OzoneOutputStream.java:94)
>     at 
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.lambda$write$1(OzoneFSOutputStream.java:58)
>     at 
> org.apache.hadoop.hdds.tracing.TracingUtil.executeInSpan(TracingUtil.java:184)
>     at 
> org.apache.hadoop.hdds.tracing.TracingUtil.executeInNewSpan(TracingUtil.java:149)
>     at 
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.write(OzoneFSOutputStream.java:54)
>     at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:62)
>     at java.io.DataOutputStream.write(DataOutputStream.java:107)
>     at java.io.FilterOutputStream.write(FilterOutputStream.java:97)
>     at 
> org.apache.hadoop.fs.ozone.TestHSync.lambda$runConcurrentWriteHSync$8(TestHSync.java:683)
>     at java.lang.Thread.run(Thread.java:750)
> Caused by: java.lang.IllegalArgumentException
>     at 
> com.google.common.base.Preconditions.checkArgument(Preconditions.java:129)
>     at 
> org.apache.hadoop.ozone.client.io.BlockOutputStreamEntryPool.allocateBlockIfNeeded(BlockOutputStreamEntryPool.java:398)
>     at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.handleWrite(KeyOutputStream.java:222)
>  {code}
> #3
> {code:java}
> Error:  org.apache.hadoop.fs.ozone.TestHSync.testConcurrentWriteHSync(int, 
> int, boolean)[3]  Time elapsed: 12.476 s  <<< ERROR!
> java.lang.IllegalArgumentException: Expected offset: 6134992 expected len: 
> 6143720
>       at 
> com.google.common.base.Preconditions.checkArgument(Preconditions.java:143)
>       at 
> org.apache.hadoop.ozone.client.io.BlockOutputStreamEntryPool.commitKey(BlockOutputStreamEntryPool.java:321)
>       at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.close(KeyOutputStream.java:563)
>       at 
> org.apache.hadoop.ozone.client.io.OzoneOutputStream.close(OzoneOutputStream.java:105)
>       at 
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.close(OzoneFSOutputStream.java:70)
>       at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:77)
>       at 
> org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
>       at 
> org.apache.hadoop.fs.ozone.TestHSync.testConcurrentWriteHSync(TestHSync.java:757)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>  {code}
>  
> #4
> {code:java}
> java.io.IOException: Inconsistent read for blockID=conID: 4 locID: 
> 113750153625600158 bcsId: 3233 replicaIndex: null length=196608 
> position=181568 numBytesToRead=8 numBytesRead=-1
>     at 
> org.apache.hadoop.ozone.client.io.KeyInputStream.checkPartBytesRead(KeyInputStream.java:195)
>     at 
> org.apache.hadoop.hdds.scm.storage.MultipartInputStream.readWithStrategy(MultipartInputStream.java:97)
>     at 
> org.apache.hadoop.hdds.scm.storage.ExtendedInputStream.read(ExtendedInputStream.java:56)
>     at 
> org.apache.hadoop.fs.ozone.OzoneFSInputStream.read(OzoneFSInputStream.java:81)
>     at java.io.DataInputStream.read(DataInputStream.java:100)
>     at 
> org.apache.hadoop.fs.ozone.TestHSync.validateWrittenFile(TestHSync.java:767)
>     at 
> org.apache.hadoop.fs.ozone.TestHSync.testConcurrentWriteHSync(TestHSync.java:758)
>     at java.lang.reflect.Method.invoke(Method.java:498)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org
For additional commands, e-mail: issues-h...@ozone.apache.org

Reply via email to