Index: src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java (revision 1157722) +++ src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java (working copy) @@ -129,7 +129,7 @@ private final long blocksize; private final String prefix; private final Path oldLogDir; - private boolean logRollRequested; + private boolean logRollRunning; private static Class logWriterClass; @@ -140,7 +140,10 @@ } private OutputStream hdfs_out; // OutputStream associated with the current SequenceFile.writer - private int initialReplication; // initial replication factor of SequenceFile.writer + + // Minimum tolerable replicas, if the actual value is lower than it, + // rollWriter will be triggered + private int minTolerableReplication; private Method getNumCurrentReplicas; // refers to DFSOutputStream.getNumCurrentReplicas final static Object [] NO_ARGS = new Object []{}; @@ -189,9 +192,19 @@ // The timestamp (in ms) when the log file was created. private volatile long filenum = -1; - //number of transactions in the current Hlog. + // number of transactions in the current Hlog. private final AtomicInteger numEntries = new AtomicInteger(0); + // If live datanode count is lower than the default replicas value, + // RollWriter will be triggered in each sync(So the RollWriter will be + // triggered one by one in a short time). Using it as a workaround to slow + // down the roll frequency triggered by checkLowReplication(). + private volatile int consecutiveLogRolls = 0; + private final int fewerReplicasRollLimit; + // If consecutiveLogRolls is larger than fewerReplicasRollLimit, + // then disable the rolling in checkLowReplication(). + // Enable it if the replications recover. + private volatile boolean fewerReplicasRollEnabled = true; // If > than this size, roll the log. This is typically 0.95 times the size // of the default Hdfs block size. private final long logrollsize; @@ -358,6 +371,11 @@ } } this.maxLogs = conf.getInt("hbase.regionserver.maxlogs", 32); + this.minTolerableReplication = + conf.getInt("hbase.regionserver.hlog.mintolerablereplicas", + this.fs.getDefaultReplication()); + this.fewerReplicasRollLimit = + conf.getInt("hbase.regionserver.hlog.fewerreplicasrolllimit", 5); this.enabled = conf.getBoolean("hbase.regionserver.hlog.enabled", true); LOG.info("HLog configuration: blocksize=" + StringUtils.byteDesc(this.blocksize) + @@ -474,13 +492,13 @@ if (closed) { return regionsToFlush; } + this.logRollRunning = true; // Do all the preparation outside of the updateLock to block // as less as possible the incoming writes long currentFilenum = this.filenum; this.filenum = System.currentTimeMillis(); Path newPath = computeFilename(); HLog.Writer nextWriter = this.createWriterInstance(fs, newPath, conf); - int nextInitialReplication = fs.getFileStatus(newPath).getReplication(); // Can we get at the dfsclient outputstream? If an instance of // SFLW, it'll have done the necessary reflection to get at the // protected field name. @@ -500,7 +518,6 @@ // Clean up current writer. Path oldFile = cleanupCurrentWriter(currentFilenum); this.writer = nextWriter; - this.initialReplication = nextInitialReplication; this.hdfs_out = nextHdfsOut; LOG.info((oldFile != null? @@ -510,7 +527,7 @@ this.fs.getFileStatus(oldFile).getLen() + ". ": "") + "New hlog " + FSUtils.getPath(newPath)); this.numEntries.set(0); - this.logRollRequested = false; + this.logRollRunning = false; } // Can we delete any of the old log files? if (this.outputfiles.size() > 0) { @@ -982,7 +999,7 @@ synchronized (this.updateLock) { syncTime += System.currentTimeMillis() - now; syncOps++; - if (!logRollRequested) { + if (!logRollRunning) { checkLowReplication(); if (this.writer.getLength() > this.logrollsize) { requestLogRoll(); @@ -998,18 +1015,42 @@ } private void checkLowReplication() { - // if the number of replicas in HDFS has fallen below the initial + // if the number of replicas in HDFS has fallen below the configured // 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; + numCurrentReplicas < this.minTolerableReplication) { + if (this.fewerReplicasRollEnabled) { + if (this.consecutiveLogRolls < this.fewerReplicasRollLimit) { + LOG.warn("HDFS pipeline error detected. " + + "Found " + numCurrentReplicas + " replicas but expecting no less than " + + this.minTolerableReplication + " replicas. " + + " Requesting close of hlog."); + requestLogRoll(); + // If rollWriter is requested, increase consecutiveLogRolls. Once it is + // larger than fewerReplicasRollLimit, disable the FewerReplicas-Roller + this.consecutiveLogRolls++; + } else { + LOG.warn("Too many consecutive RollWriter requests, it's a sign of " + + "the live datanodes count is less than the minimum tolerable replicas."); + this.consecutiveLogRolls = 0; + this.fewerReplicasRollEnabled = false; + } + } + } else if (numCurrentReplicas >= this.minTolerableReplication) { + if (!this.fewerReplicasRollEnabled) { + // The new writer's log replicas is always the default value. + // So we should not enable FewerReplicas-Roller. If numEntries + // is lower than or equals 1, we consider it as a new writer. + if (this.numEntries.get() <= 1) { + return; + } + // Once the live datanode number and the replicas return to normal, + // enable the FewerReplicas-Roller. + this.fewerReplicasRollEnabled = true; + LOG.info("FewerReplicas-Roller was enabled."); + } } } catch (Exception e) { LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e + @@ -1300,6 +1341,15 @@ dirName.append(serverName); return dirName.toString(); } + + /** + * Get fewerReplicas-Roller status + * @return fewerReplicasRollEnabled + */ + public boolean isFewerReplicasRollEnabled() + { + return fewerReplicasRollEnabled; + } /** * Get the directory we are making logs in. Index: src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java (revision 1157722) +++ src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java (working copy) @@ -137,6 +137,10 @@ // the namenode might still try to choose the recently-dead datanode // for a pipeline, so try to a new pipeline multiple times TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 30); + TEST_UTIL.getConfiguration() + .setInt("hbase.regionserver.hlog.mintolerablereplicas", 2); + TEST_UTIL.getConfiguration() + .setInt("hbase.regionserver.hlog.fewerreplicasrolllimit", 3); TEST_UTIL.startMiniCluster(2); cluster = TEST_UTIL.getHBaseCluster(); @@ -224,6 +228,20 @@ // continue } } + + void batchWriteData(HTable table, int start) throws IOException { + for (int i = 0; i < 5; i++) { + Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", (start + i)))); + put.add(HConstants.CATALOG_FAMILY, null, value); + table.put(put); + // sleep to let the log roller run (if it needs to) + try { + Thread.sleep(500); + } catch (InterruptedException e) { + // continue + } + } + } /** * Give me the HDFS pipeline for this log file @@ -316,10 +334,22 @@ assertTrue("Missing datanode should've triggered a log roll", newFilenum > oldFilenum && newFilenum > curTime); - - // write some more log data (this should use a new hdfs_out) - writeData(table, 3); - assertTrue("The log should not roll again.", log.getFilenum() == newFilenum); + // kill another datanode in the pipeline, so the replicas will be lower than + // the configured value 2. + assertTrue(dfsCluster.stopDataNode(pipeline[1].getName()) != null); + Thread.sleep(10000); + batchWriteData(table, 3); + assertTrue("LowerReplicas Roller should've been disabled", + !log.isFewerReplicasRollEnabled()); + dfsCluster + .startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null); + dfsCluster.waitActive(); + // Force roll writer. The new log file will have the defalt replication, + // and the lowerReplcas roller will be enabled. + log.rollWriter(); + batchWriteData(table, 8); + assertTrue("LowerReplicas Roller should've been enabled", + log.isFewerReplicasRollEnabled()); assertTrue("New log file should have the default replication", log .getLogReplication() == fs.getDefaultReplication()); }