hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-4095) Hlog may not be rolled in a long time if checkLowReplication's request of LogRoll is blocked
Date Thu, 18 Aug 2011 20:25:27 GMT

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

stack commented on HBASE-4095:
------------------------------

Thank you for your persistence Jieshan on trying to get this patch in.  I'm basically +1 on
v9.  Below is a small question.

{code}
-        this.logRollRequested = false;
+        this.logRollRunning = false;
{code}

Should the setting of this.logRollRunning be set instead in a finally block in here in rollWriter?
 If an exception thrown after we set the logRollRunning, it looks like logRollRunning could
stay set.   My guess is that not doing this would probably not be noticed in that we probably
crash out the regionserver if a rollWriter fails but having the flag stuck set might make
for some unexpected state?

So, it looks like we'll roll 5 times by default before we'll turn off the low replication
log rolling facility -- which is better than a log per sync, right?

You seem to lose some 'liveness' regards file replication setting.  In code before this patch,
when rollwriter ran, it'd ask the FS what the replication on the new writer is and that going
forward would be the replication to use.  See here:

{code}
-      int nextInitialReplication = fs.getFileStatus(newPath).getReplication();
{code}

Instead you set the replication once on instantiation of HLog.  See here:

{code}
+    this.minTolerableReplication = conf.getInt(
+        "hbase.regionserver.hlog.tolerable.lowreplication",
+        this.fs.getDefaultReplication());
{code}

.. and rather than ask the files replication you use the filesystem default value.

Do you have a good reason for changing this behavior?

Thanks Jieshan.






> Hlog may not be rolled in a long time if checkLowReplication's request of LogRoll is
blocked
> --------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4095
>                 URL: https://issues.apache.org/jira/browse/HBASE-4095
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver
>    Affects Versions: 0.90.3
>            Reporter: Jieshan Bean
>            Assignee: Jieshan Bean
>             Fix For: 0.90.5
>
>         Attachments: HBASE-4095-90-v2.patch, HBASE-4095-90.patch, HBASE-4095-trunk-v2.patch,
HBASE-4095-trunk.patch, HBase-4095-V4-Branch.patch, HBase-4095-V5-Branch.patch, HBase-4095-V5-trunk.patch,
HBase-4095-V6-branch.patch, HBase-4095-V6-trunk.patch, HBase-4095-V7-branch.patch, HBase-4095-V7-trunk.patch,
HBase-4095-V8-branch.patch, HBase-4095-V8-trunk.patch, HBase-4095-V9-branch.patch, HBase-4095-V9-trunk.patch,
HlogFileIsVeryLarge.gif, LatestLLTResults-20110810.rar, RelatedLogs2011-07-28.txt, TestResultForPatch-V4.rar,
flowChart-IntroductionToThePatch.gif, hbase-root-regionserver-193-195-5-111.rar, surefire-report-V5-trunk.html,
surefire-report-branch.html
>
>
> Some large Hlog files(Larger than 10G) appeared in our environment, and I got the reason
why they got so huge:
> 1. The replicas is less than the expect number. So the method of checkLowReplication
will be called each sync.
> 2. The method checkLowReplication request log-roll first, and set logRollRequested as
true: 
> {noformat}
> private void checkLowReplication() {
> // if the number of replicas in HDFS has fallen below the initial
> // value, then roll logs.
> try {
>   int numCurrentReplicas = getLogReplication();
>   if (numCurrentReplicas != 0 &&
> 	  numCurrentReplicas < this.initialReplication) {
> 	LOG.warn("HDFS pipeline error detected. " +
> 		"Found " + numCurrentReplicas + " replicas but expecting " +
> 		this.initialReplication + " replicas. " +
> 		" Requesting close of hlog.");
> 	requestLogRoll();
> 	logRollRequested = true;
>   }
> } catch (Exception e) {
>   LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e +
> 	  " still proceeding ahead...");
> }
> }
> {noformat}
> 3.requestLogRoll() just commit the roll request. It may not execute in time, for it must
got the un-fair lock of cacheFlushLock.
> But the lock may be carried by the cacheflush threads.
> 4.logRollRequested was true until the log-roll executed. So during the time, each request
of log-roll in sync() was skipped.
> Here's the logs while the problem happened(Please notice the file size of hlog "193-195-5-111%3A20020.1309937386639"
in the last row):
> 2011-07-06 15:28:59,284 WARN org.apache.hadoop.hbase.regionserver.wal.HLog: HDFS pipeline
error detected. Found 2 replicas but expecting 3 replicas.  Requesting close of hlog.
> 2011-07-06 15:29:46,714 INFO org.apache.hadoop.hbase.regionserver.wal.HLog: Roll /hbase/.logs/193-195-5-111,20020,1309922880081/193-195-5-111%3A20020.1309937339119,
entries=32434, filesize=239589754. New hlog /hbase/.logs/193-195-5-111,20020,1309922880081/193-195-5-111%3A20020.1309937386639
> 2011-07-06 15:29:56,929 WARN org.apache.hadoop.hbase.regionserver.wal.HLog: HDFS pipeline
error detected. Found 2 replicas but expecting 3 replicas.  Requesting close of hlog.
> 2011-07-06 15:29:56,933 INFO org.apache.hadoop.hbase.regionserver.Store: Renaming flushed
file at hdfs://193.195.5.112:9000/hbase/Htable_UFDR_034/a3780cf0c909d8cf8f8ed618b290cc95/.tmp/4656903854447026847
to hdfs://193.195.5.112:9000/hbase/Htable_UFDR_034/a3780cf0c909d8cf8f8ed618b290cc95/value/8603005630220380983
> 2011-07-06 15:29:57,391 INFO org.apache.hadoop.hbase.regionserver.Store: Added hdfs://193.195.5.112:9000/hbase/Htable_UFDR_034/a3780cf0c909d8cf8f8ed618b290cc95/value/8603005630220380983,
entries=445880, sequenceid=248900, memsize=207.5m, filesize=130.1m
> 2011-07-06 15:29:57,478 INFO org.apache.hadoop.hbase.regionserver.HRegion: Finished memstore
flush of ~207.5m for region Htable_UFDR_034,07664,1309936974158.a3780cf0c909d8cf8f8ed618b290cc95.
in 10839ms, sequenceid=248900, compaction requested=false
> 2011-07-06 15:28:59,236 INFO org.apache.hadoop.hbase.regionserver.wal.HLog: Roll /hbase/.logs/193-195-5-111,20020,1309922880081/193-195-5-111%3A20020.1309926531955,
entries=216459, filesize=2370387468. New hlog /hbase/.logs/193-195-5-111,20020,1309922880081/193-195-5-111%3A20020.1309937339119
> 2011-07-06 15:29:46,714 INFO org.apache.hadoop.hbase.regionserver.wal.HLog: Roll /hbase/.logs/193-195-5-111,20020,1309922880081/193-195-5-111%3A20020.1309937339119,
entries=32434, filesize=239589754. New hlog /hbase/.logs/193-195-5-111,20020,1309922880081/193-195-5-111%3A20020.1309937386639
> 2011-07-06 16:29:58,775 DEBUG org.apache.hadoop.hbase.regionserver.LogRoller: Hlog roll
period 3600000ms elapsed
> 2011-07-06 16:29:58,775 DEBUG org.apache.hadoop.hbase.regionserver.LogRoller: Hlog roll
period 3600000ms elapsed
> 2011-07-06 16:30:01,978 INFO org.apache.hadoop.hbase.regionserver.wal.HLog: Roll /hbase/.logs/193-195-5-111,20020,1309922880081/193-195-5-111%3A20020.1309937386639,
entries=1135576, filesize=19220372830. New hlog /hbase/.logs/193-195-5-111,20020,1309922880081/193-195-5-111%3A20020.1309940998890

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Mime
View raw message