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

Jimmy Xiang commented on HBASE-6318:
------------------------------------

Wrong patch name.

The map is a Collections.synchronzedMap.  So it protects race against 
getOutputCounts(), and all other methods like put() and get() on this map.

The attached patch uses synchronizd on a strong consistent iterator.  There are 
not many threads here.
So performance wise, it should be fine.

However, there is still possible leakage.  For example, some thread created a 
writer, right before putting it to the map, it is blocked
because closeLogWriters() has the lock.  This writer will not be closed.

Probably it is better to fix the t.join() interruption, or the leakage is rare, 
and not a big deal?

                
> SplitLogWorker exited due to ConcurrentModificationException
> ------------------------------------------------------------
>
>                 Key: HBASE-6318
>                 URL: https://issues.apache.org/jira/browse/HBASE-6318
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver
>    Affects Versions: 0.96.0
>            Reporter: Jimmy Xiang
>         Attachments: 6138.patch, 6318-suggest.txt, 6318.log
>
>
> In playing with 0.96 code on a live cluster, found this issue:
> 2012-07-03 12:13:32,572 ERROR 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker: unexpected error
> java.util.ConcurrentModificationException
>         at java.util.TreeMap$PrivateEntryIterator.nextEntry(TreeMap.java:1100)
>         at java.util.TreeMap$ValueIterator.next(TreeMap.java:1145)
>         at 
> org.apache.hadoop.hbase.regionserver.wal.HLogSplitter$OutputSink.closeLogWriters(HLogSplitter.java:1330)
>         at 
> org.apache.hadoop.hbase.regionserver.wal.HLogSplitter$OutputSink.finishWritingAndClose(HLogSplitter.java:1221)
>         at 
> org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:441)
>         at 
> org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:369)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker$1.exec(SplitLogWorker.java:113)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.grabTask(SplitLogWorker.java:276)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.taskLoop(SplitLogWorker.java:197)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.run(SplitLogWorker.java:164)
>         at java.lang.Thread.run(Thread.java:662)
> 2012-07-03 12:13:32,575 INFO 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker: SplitLogWorker 
> ****.cloudera.com,57020,1341335300238 exiting

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to