HDFS-4015. Safemode should count and report orphaned blocks. (Contributed by Anu Engineer)

This commit is contained in:
Arpit Agarwal 2015-10-23 17:27:45 -07:00
parent 5679e46b7f
commit 86c92227fc
16 changed files with 408 additions and 17 deletions

View File

@ -2005,6 +2005,16 @@ public long getCorruptBlocksCount() throws IOException {
return callGetStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
}
/**
* Returns number of bytes that reside in Blocks with future generation
* stamps.
* @return Bytes in Blocks with future generation stamps.
* @throws IOException
*/
public long getBytesInFutureBlocks() throws IOException {
return callGetStats()[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX];
}
/**
* @return a list in which each entry describes a corrupt file/block
* @throws IOException

View File

@ -536,6 +536,17 @@ public Collection<BlockStoragePolicy> getAllStoragePolicies()
return Arrays.asList(dfs.getStoragePolicies());
}
/**
* Returns number of bytes within blocks with future generation stamp. These
* are bytes that will be potentially deleted if we forceExit from safe mode.
*
* @return number of bytes.
*/
public long getBytesWithFutureGenerationStamps() throws IOException {
statistics.incrementReadOps(1);
return dfs.getBytesInFutureBlocks();
}
/**
* Deprecated. Prefer {@link FileSystem#getAllStoragePolicies()}
* @throws IOException

View File

@ -712,10 +712,12 @@ SnapshottableDirectoryStatus[] getSnapshottableDirListing()
int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
int GET_STATS_MISSING_BLOCKS_IDX = 5;
int GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX = 6;
int GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX = 7;
int STATS_ARRAY_LENGTH = 8;
/**
* Get a set of statistics about the filesystem.
* Right now, only seven values are returned.
* Right now, only eight values are returned.
* <ul>
* <li> [0] contains the total storage capacity of the system, in bytes.</li>
* <li> [1] contains the total used space of the system, in bytes.</li>
@ -725,6 +727,7 @@ SnapshottableDirectoryStatus[] getSnapshottableDirListing()
* <li> [5] contains number of blocks without any good replicas left. </li>
* <li> [6] contains number of blocks which have replication factor
* 1 and have lost the only replica. </li>
* <li> [7] contains number of bytes that are at risk for deletion. </li>
* </ul>
* Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of
* actual numbers to index into the array.

View File

@ -118,7 +118,7 @@ public final class HdfsConstants {
// SafeMode actions
public enum SafeModeAction {
SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET
SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET, SAFEMODE_FORCE_EXIT
}
public enum RollingUpgradeAction {

View File

@ -1508,6 +1508,8 @@ public static SafeModeActionProto convert(SafeModeAction a) {
return SafeModeActionProto.SAFEMODE_ENTER;
case SAFEMODE_GET:
return SafeModeActionProto.SAFEMODE_GET;
case SAFEMODE_FORCE_EXIT:
return SafeModeActionProto.SAFEMODE_FORCE_EXIT;
default:
throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
}
@ -1522,7 +1524,7 @@ public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
}
public static long[] convert(GetFsStatsResponseProto res) {
long[] result = new long[7];
long[] result = new long[ClientProtocol.STATS_ARRAY_LENGTH];
result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
@ -1534,6 +1536,8 @@ public static long[] convert(GetFsStatsResponseProto res) {
res.getMissingBlocks();
result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
res.getMissingReplOneBlocks();
result[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX] =
res.hasBlocksInFuture() ? res.getBlocksInFuture() : 0;
return result;
}
@ -1897,6 +1901,12 @@ public static GetFsStatsResponseProto convert(long[] fsStats) {
ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
result.setMissingReplOneBlocks(
fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
if (fsStats.length >=
ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX + 1) {
result.setBlocksInFuture(
fsStats[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX]);
}
return result.build();
}
@ -1920,6 +1930,8 @@ public static SafeModeAction convert(
return SafeModeAction.SAFEMODE_ENTER;
case SAFEMODE_GET:
return SafeModeAction.SAFEMODE_GET;
case SAFEMODE_FORCE_EXIT:
return SafeModeAction.SAFEMODE_FORCE_EXIT;
default:
throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
}

View File

@ -306,6 +306,7 @@ message GetFsStatsResponseProto {
required uint64 corrupt_blocks = 5;
required uint64 missing_blocks = 6;
optional uint64 missing_repl_one_blocks = 7;
optional uint64 blocks_in_future = 8;
}
enum DatanodeReportTypeProto { // type of the datanode report
@ -348,6 +349,7 @@ enum SafeModeActionProto {
SAFEMODE_LEAVE = 1;
SAFEMODE_ENTER = 2;
SAFEMODE_GET = 3;
SAFEMODE_FORCE_EXIT = 4;
}
message SetSafeModeRequestProto {

View File

@ -1566,6 +1566,9 @@ Release 2.8.0 - UNRELEASED
HDFS-8808. dfs.image.transfer.bandwidthPerSec should not apply to
-bootstrapStandby (zhz)
HDFS-4015. Safemode should count and report orphaned blocks.
(Anu Engineer via Arpit Agarwal)
OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

View File

@ -68,6 +68,7 @@
import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
@ -292,6 +293,12 @@ public int getPendingDataNodeMessageCount() {
/** Check whether there are any non-EC blocks using StripedID */
private boolean hasNonEcBlockUsingStripedID = false;
/** Keeps track of how many bytes are in Future Generation blocks. */
private AtomicLong numberOfBytesInFutureBlocks;
/** Reports if Name node was started with Rollback option. */
private boolean inRollBack = false;
public BlockManager(final Namesystem namesystem, final Configuration conf)
throws IOException {
this.namesystem = namesystem;
@ -370,6 +377,8 @@ public BlockManager(final Namesystem namesystem, final Configuration conf)
DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT,
DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT_DEFAULT);
this.blockReportLeaseManager = new BlockReportLeaseManager(conf);
this.numberOfBytesInFutureBlocks = new AtomicLong();
this.inRollBack = isInRollBackMode(NameNode.getStartupOption(conf));
LOG.info("defaultReplication = " + defaultReplication);
LOG.info("maxReplication = " + maxReplication);
@ -2274,8 +2283,18 @@ private void processFirstBlockReport(
}
BlockInfo storedBlock = getStoredBlock(iblk);
// If block does not belong to any file, we are done.
if (storedBlock == null) continue;
// If block does not belong to any file, we check if it violates
// an integrity assumption of Name node
if (storedBlock == null) {
if (namesystem.isInStartupSafeMode()
&& !shouldPostponeBlocksFromFuture
&& !inRollBack
&& namesystem.isGenStampInFuture(iblk)) {
numberOfBytesInFutureBlocks.addAndGet(iblk.getBytesOnDisk());
}
continue;
}
// If block is corrupt, mark it and continue to next block.
BlockUCState ucState = storedBlock.getBlockUCState();
@ -4154,4 +4173,40 @@ public boolean shouldPopulateReplQueues() {
return false;
return haContext.getState().shouldPopulateReplQueues();
}
/**
* Returns the number of bytes that reside in blocks with Generation Stamps
* greater than generation stamp known to Namenode.
*
* @return Bytes in future
*/
public long getBytesInFuture() {
return numberOfBytesInFutureBlocks.get();
}
/**
* Clears the bytes in future counter.
*/
public void clearBytesInFuture() {
numberOfBytesInFutureBlocks.set(0);
}
/**
* Returns true if Namenode was started with a RollBack option.
*
* @param option - StartupOption
* @return boolean
*/
private boolean isInRollBackMode(HdfsServerConstants.StartupOption option) {
if (option == HdfsServerConstants.StartupOption.ROLLBACK) {
return true;
}
if ((option == HdfsServerConstants.StartupOption.ROLLINGUPGRADE) &&
(option.getRollingUpgradeStartupOption() ==
HdfsServerConstants.RollingUpgradeStartupOption.ROLLBACK)) {
return true;
}
return false;
}
}

View File

@ -181,6 +181,7 @@ public synchronized long[] getStats() {
-1L,
-1L,
-1L,
-1L,
-1L};
}

View File

@ -3903,6 +3903,8 @@ long[] getStats() {
stats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = getMissingBlocksCount();
stats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
getMissingReplOneBlocksCount();
stats[ClientProtocol.GET_STATS_BYTES_IN_FUTURE_BLOCKS_IDX] =
blockManager.getBytesInFuture();
return stats;
}
@ -4299,13 +4301,25 @@ private void enter() {
* Leave safe mode.
* <p>
* Check for invalid, under- & over-replicated blocks in the end of startup.
* @param force - true to force exit
*/
private synchronized void leave() {
private synchronized void leave(boolean force) {
// if not done yet, initialize replication queues.
// In the standby, do not populate repl queues
if (!blockManager.isPopulatingReplQueues() && blockManager.shouldPopulateReplQueues()) {
blockManager.initializeReplQueues();
}
if (!force && (blockManager.getBytesInFuture() > 0)) {
LOG.error("Refusing to leave safe mode without a force flag. " +
"Exiting safe mode will cause a deletion of " + blockManager
.getBytesInFuture() + " byte(s). Please use " +
"-forceExit flag to exit safe mode forcefully if data loss is " +
"acceptable.");
return;
}
long timeInSafemode = now() - startTime;
NameNode.stateChangeLog.info("STATE* Leaving safe mode after "
+ timeInSafemode/1000 + " secs");
@ -4403,7 +4417,7 @@ private void checkMode() {
// the threshold is reached or was reached before
if (!isOn() || // safe mode is off
extension <= 0 || threshold <= 0) { // don't need to wait
this.leave(); // leave safe mode
this.leave(false); // leave safe mode
return;
}
if (reached > 0) { // threshold has already been reached before
@ -4560,6 +4574,21 @@ String getTurnOffTip() {
+ "the minimum number %d. ",
numLive, datanodeThreshold);
}
if(blockManager.getBytesInFuture() > 0) {
msg += "Name node detected blocks with generation stamps " +
"in future. This means that Name node metadata is inconsistent." +
"This can happen if Name node metadata files have been manually " +
"replaced. Exiting safe mode will cause loss of " + blockManager
.getBytesInFuture() + " byte(s). Please restart name node with " +
"right metadata or use \"hdfs dfsadmin -safemode forceExit" +
"if you are certain that the NameNode was started with the" +
"correct FsImage and edit logs. If you encountered this during" +
"a rollback, it is safe to exit with -safemode forceExit.";
return msg;
}
msg += (reached > 0) ? "In safe mode extension. " : "";
msg += "Safe mode will be turned off automatically ";
@ -4661,7 +4690,7 @@ public void run() {
}
if (safeMode.canLeave()) {
// Leave safe mode.
safeMode.leave();
safeMode.leave(false);
smmthread = null;
break;
}
@ -4686,11 +4715,31 @@ boolean setSafeMode(SafeModeAction action) throws IOException {
checkSuperuserPrivilege();
switch(action) {
case SAFEMODE_LEAVE: // leave safe mode
if (blockManager.getBytesInFuture() > 0) {
LOG.error("Refusing to leave safe mode without a force flag. " +
"Exiting safe mode will cause a deletion of " + blockManager
.getBytesInFuture() + " byte(s). Please use " +
"-forceExit flag to exit safe mode forcefully and data loss is " +
"acceptable.");
return isInSafeMode();
}
leaveSafeMode();
break;
case SAFEMODE_ENTER: // enter safe mode
enterSafeMode(false);
break;
case SAFEMODE_FORCE_EXIT:
if (blockManager.getBytesInFuture() > 0) {
LOG.warn("Leaving safe mode due to forceExit. This will cause a data "
+ "loss of " + blockManager.getBytesInFuture() + " byte(s).");
safeMode.leave(true);
blockManager.clearBytesInFuture();
} else {
LOG.warn("forceExit used when normal exist would suffice. Treating " +
"force exit as normal safe mode exit.");
}
leaveSafeMode();
break;
default:
LOG.error("Unexpected safe mode action");
}
@ -4869,7 +4918,7 @@ void leaveSafeMode() {
NameNode.stateChangeLog.info("STATE* Safe mode is already OFF");
return;
}
safeMode.leave();
safeMode.leave(false);
} finally {
writeUnlock();
}
@ -7445,5 +7494,21 @@ public ErasureCodingPolicy getErasureCodingPolicyForPath(String src)
throws IOException {
return FSDirErasureCodingOp.getErasureCodingPolicy(this, src);
}
/**
* Gets number of bytes in the blocks in future generation stamps.
*
* @return number of bytes that can be deleted if exited from safe mode.
*/
public long getBytesInFuture() {
return blockManager.getBytesInFuture();
}
@VisibleForTesting
synchronized void enableSafeModeForTesting(Configuration conf) {
SafeModeInfo newSafemode = new SafeModeInfo(conf);
newSafemode.enter();
this.safeMode = newSafemode;
}
}

View File

@ -1373,7 +1373,7 @@ private static void setStartupOption(Configuration conf, StartupOption opt) {
conf.set(DFS_NAMENODE_STARTUP_KEY, opt.name());
}
static StartupOption getStartupOption(Configuration conf) {
public static StartupOption getStartupOption(Configuration conf) {
return StartupOption.valueOf(conf.get(DFS_NAMENODE_STARTUP_KEY,
StartupOption.REGULAR.toString()));
}
@ -1688,6 +1688,11 @@ public long getLastHATransitionTime() {
return state.getLastHATransitionTime();
}
@Override //NameNodeStatusMXBean
public long getBytesWithFutureGenerationStamps() {
return getNamesystem().getBytesInFuture();
}
/**
* Shutdown the NN immediately in an ungraceful way. Used when it would be
* unsafe for the NN to continue operating, e.g. during a failed HA state

View File

@ -61,4 +61,10 @@ public interface NameNodeStatusMXBean {
* @return the most recent HA transition time in milliseconds from the epoch.
*/
public long getLastHATransitionTime();
/**
* Gets number of bytes in blocks with future generation stamps.
* @return number of bytes that can be deleted if exited from safe mode.
*/
long getBytesWithFutureGenerationStamps();
}

View File

@ -462,10 +462,21 @@ public void report(String[] argv, int i) throws IOException {
long capacity = ds.getCapacity();
long used = ds.getUsed();
long remaining = ds.getRemaining();
long bytesInFuture = dfs.getBytesWithFutureGenerationStamps();
long presentCapacity = used + remaining;
boolean mode = dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET);
if (mode) {
System.out.println("Safe mode is ON");
if (bytesInFuture > 0) {
System.out.println("\nWARNING: ");
System.out.println("Name node has detected blocks with generation " +
"stamps in future.");
System.out.println("Forcing exit from safemode will cause " +
bytesInFuture + " byte(s) to be deleted.");
System.out.println("If you are sure that the NameNode was started with"
+ " the correct metadata files then you may proceed with " +
"'-safemode forceExit'\n");
}
}
System.out.println("Configured Capacity: " + capacity
+ " (" + StringUtils.byteDesc(capacity) + ")");
@ -574,6 +585,8 @@ public void setSafeMode(String[] argv, int idx) throws IOException {
} else if ("wait".equalsIgnoreCase(argv[idx])) {
action = HdfsConstants.SafeModeAction.SAFEMODE_GET;
waitExitSafe = true;
} else if ("forceExit".equalsIgnoreCase(argv[idx])){
action = HdfsConstants.SafeModeAction.SAFEMODE_FORCE_EXIT;
} else {
printUsage("-safemode");
return;
@ -949,7 +962,8 @@ private void printHelp(String cmd) {
"\tand etc. on all the DNs.\n" +
"\tOptional flags may be used to filter the list of displayed DNs.\n";
String safemode = "-safemode <enter|leave|get|wait>: Safe mode maintenance command.\n" +
String safemode = "-safemode <enter|leave|get|wait|forceExit>: Safe mode " +
"maintenance command.\n" +
"\t\tSafe mode is a Namenode state in which it\n" +
"\t\t\t1. does not accept changes to the name space (read-only)\n" +
"\t\t\t2. does not replicate or delete blocks.\n" +
@ -1643,7 +1657,7 @@ private static void printUsage(String cmd) {
+ " [-report] [-live] [-dead] [-decommissioning]");
} else if ("-safemode".equals(cmd)) {
System.err.println("Usage: hdfs dfsadmin"
+ " [-safemode enter | leave | get | wait]");
+ " [-safemode enter | leave | get | wait | forceExit]");
} else if ("-allowSnapshot".equalsIgnoreCase(cmd)) {
System.err.println("Usage: hdfs dfsadmin"
+ " [-allowSnapshot <snapshotDir>]");

View File

@ -1,4 +1,4 @@
<!---
<!---
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
@ -317,7 +317,7 @@ Usage:
hdfs dfsadmin [GENERIC_OPTIONS]
[-report [-live] [-dead] [-decommissioning]]
[-safemode enter | leave | get | wait]
[-safemode enter | leave | get | wait | forceExit]
[-saveNamespace]
[-rollEdits]
[-restoreFailedStorage true |false |check]
@ -351,7 +351,7 @@ Usage:
| COMMAND\_OPTION | Description |
|:---- |:---- |
| `-report` `[-live]` `[-dead]` `[-decommissioning]` | Reports basic filesystem information and statistics, The dfs usage can be different from "du" usage, because it measures raw space used by replication, checksums, snapshots and etc. on all the DNs. Optional flags may be used to filter the list of displayed DataNodes. |
| `-safemode` enter\|leave\|get\|wait | Safe mode maintenance command. Safe mode is a Namenode state in which it <br/>1. does not accept changes to the name space (read-only) <br/>2. does not replicate or delete blocks. <br/>Safe mode is entered automatically at Namenode startup, and leaves safe mode automatically when the configured minimum percentage of blocks satisfies the minimum replication condition. Safe mode can also be entered manually, but then it can only be turned off manually as well. |
| `-safemode` enter\|leave\|get\|wait\|forceExit | Safe mode maintenance command. Safe mode is a Namenode state in which it <br/>1. does not accept changes to the name space (read-only) <br/>2. does not replicate or delete blocks. <br/>Safe mode is entered automatically at Namenode startup, and leaves safe mode automatically when the configured minimum percentage of blocks satisfies the minimum replication condition. If Namenode detects any anomaly then it will linger in safe mode till that issue is resolved. If that anomaly is the consequence of a deliberate action, then administrator can use -safemode forceExit to exit safe mode. The cases where forceExit may be required are<br/> 1. Namenode metadata is not consistent. If Namenode detects that metadata has been modified out of band and can cause data loss, then Namenode will enter forceExit state. At that point user can either restart Namenode with correct metadata files or forceExit (if data loss is acceptable).<br/>2. Rollback causes metadata to be replaced and rarely it can trigger safe mode forceExit state in Namenode. In that case you may proceed by issuing -safemode forceExit.<br/> Safe mode can also be entered manually, but then it can only be turned off manually as well. |
| `-saveNamespace` | Save current namespace into storage directories and reset edits log. Requires safe mode. |
| `-rollEdits` | Rolls the edit log on the active NameNode. |
| `-restoreFailedStorage` true\|false\|check | This option will turn on/off automatic attempt to restore failed storage replicas. If a failed storage becomes available again the system will attempt to restore edits and/or fsimage during checkpoint. 'check' option will return current setting. |

View File

@ -0,0 +1,186 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.hamcrest.CoreMatchers;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.internal.util.reflection.Whitebox;
import java.io.IOException;
import java.io.OutputStream;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.mockito.Mockito.*;
public class TestNameNodeMetadataConsistency {
private static final Path filePath1 = new Path("/testdata1.txt");
private static final Path filePath2 = new Path("/testdata2.txt");
private static final int SCAN_INTERVAL = 1;
private static final int SCAN_WAIT = 3;
MiniDFSCluster cluster;
HdfsConfiguration conf;
@Before
public void InitTest() throws IOException {
conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY,
SCAN_INTERVAL);
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(1)
.build();
}
@After
public void cleanup() {
if (cluster != null) {
cluster.shutdown();
}
}
/**
* This test creates a file and modifies the block generation stamp to number
* that name node has not seen yet. It then asserts that name node moves into
* safe mode while it is in startup mode.
*/
@Test
public void testGenerationStampInFuture() throws
IOException, InterruptedException {
String testData = " This is test data";
int datalen = testData.length();
cluster.waitActive();
FileSystem fs = cluster.getFileSystem();
OutputStream ostream = fs.create(filePath1);
ostream.write(testData.getBytes());
ostream.close();
ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, filePath1);
long genStamp = block.getGenerationStamp();
// Re-write the Generation Stamp to a Generation Stamp in future.
cluster.changeGenStampOfBlock(0, block, genStamp + 1);
MiniDFSCluster.DataNodeProperties dnProps = cluster.stopDataNode(0);
// Simulate Namenode forgetting a Block
cluster.restartNameNode(true);
BlockInfo bInfo = cluster.getNameNode().getNamesystem().getBlockManager
().getStoredBlock(block.getLocalBlock());
cluster.getNameNode().getNamesystem().writeLock();
cluster.getNameNode().getNamesystem().getBlockManager()
.removeBlock(bInfo);
cluster.getNameNode().getNamesystem().writeUnlock();
// we also need to tell block manager that we are in the startup path
FSNamesystem spyNameSystem = spy(cluster.getNameNode().getNamesystem());
spyNameSystem.enableSafeModeForTesting(conf);
Whitebox.setInternalState(cluster.getNameNode()
.getNamesystem().getBlockManager(),
"namesystem", spyNameSystem);
Whitebox.setInternalState(cluster.getNameNode(),
"namesystem", spyNameSystem);
Mockito.doReturn(true).when(spyNameSystem).isInStartupSafeMode();
// Since Data Node is already shutdown we didn't remove blocks
cluster.restartDataNode(dnProps);
waitTil(TimeUnit.SECONDS.toMillis(SCAN_WAIT));
cluster.triggerBlockReports();
// Give some buffer to process the block reports
waitTil(TimeUnit.SECONDS.toMillis(SCAN_WAIT));
// Make sure that we find all written bytes in future block
assertEquals(datalen, cluster.getNameNode().getBytesWithFutureGenerationStamps());
// Assert safemode reason
String safeModeMessage = cluster.getNameNode().getNamesystem()
.getSafeModeTip();
assertThat(safeModeMessage, CoreMatchers.containsString("Name node " +
"detected blocks with generation stamps in future"));
}
/**
* Pretty much the same tests as above but does not setup safeMode == true,
* hence we should not have positive count of Blocks in future.
*/
@Test
public void testEnsureGenStampsIsStartupOnly() throws
IOException, InterruptedException {
String testData = " This is test data";
cluster.restartDataNodes();
cluster.restartNameNodes();
cluster.waitActive();
FileSystem fs = cluster.getFileSystem();
OutputStream ostream = fs.create(filePath2);
ostream.write(testData.getBytes());
ostream.close();
ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, filePath2);
long genStamp = block.getGenerationStamp();
// Re-write the Generation Stamp to a Generation Stamp in future.
cluster.changeGenStampOfBlock(0, block, genStamp + 1);
MiniDFSCluster.DataNodeProperties dnProps = cluster.stopDataNode(0);
// Simulate Namenode forgetting a Block
cluster.restartNameNode(true);
BlockInfo bInfo = cluster.getNameNode().getNamesystem().getBlockManager
().getStoredBlock(block.getLocalBlock());
cluster.getNameNode().getNamesystem().writeLock();
cluster.getNameNode().getNamesystem().getBlockManager()
.removeBlock(bInfo);
cluster.getNameNode().getNamesystem().writeUnlock();
cluster.restartDataNode(dnProps);
waitTil(TimeUnit.SECONDS.toMillis(SCAN_WAIT));
cluster.triggerBlockReports();
waitTil(TimeUnit.SECONDS.toMillis(SCAN_WAIT));
// Make sure that there are no bytes in future since isInStartupSafe
// mode is not true.
assertEquals(0, cluster.getNameNode().getBytesWithFutureGenerationStamps());
}
private void waitTil(long waitPeriod) {
try {
Thread.sleep(waitPeriod);
} catch (InterruptedException e) {
e.printStackTrace();
}
}
}

View File

@ -15684,7 +15684,7 @@
<comparators>
<comparator>
<type>RegexpComparator</type>
<expected-output>^-safemode &lt;enter\|leave\|get\|wait&gt;:( |\t)*Safe mode maintenance command.( )*</expected-output>
<expected-output>^-safemode &lt;enter\|leave\|get\|wait\|forceExit&gt;:( |\t)*Safe mode maintenance command.( )*</expected-output>
</comparator>
<comparator>
<type>RegexpComparator</type>
@ -16229,6 +16229,24 @@
</comparators>
</test>
<test> <!-- TESTED -->
<description>safemode: Test for foceExit </description>
<test-commands>
<dfs-admin-command>-fs NAMENODE -safemode enter</dfs-admin-command>
<dfs-admin-command>-fs NAMENODE -safemode forceExit</dfs-admin-command>
</test-commands>
<cleanup-commands>
<!-- No cleanup -->
</cleanup-commands>
<comparators>
<comparator>
<type>TokenComparator</type>
<expected-output>Safe mode is OFF</expected-output>
</comparator>
</comparators>
</test>
<test> <!-- TESTED -->
<description>safemode: Test for leave - Namenode is already in safemode</description>
<test-commands>