HBASE-2075 Master requires HDFS superuser privileges due to waitOnSafeMode
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@894525 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
42411929ff
commit
6802181855
|
@ -141,6 +141,7 @@ Release 0.21.0 - Unreleased
|
||||||
at the same time
|
at the same time
|
||||||
HBASE-2026 NPE in StoreScanner on compaction
|
HBASE-2026 NPE in StoreScanner on compaction
|
||||||
HBASE-2072 fs.automatic.close isn't passed to FileSystem
|
HBASE-2072 fs.automatic.close isn't passed to FileSystem
|
||||||
|
HBASE-2075 Master requires HDFS superuser privileges due to waitOnSafeMode
|
||||||
|
|
||||||
IMPROVEMENTS
|
IMPROVEMENTS
|
||||||
HBASE-1760 Cleanup TODOs in HTable
|
HBASE-1760 Cleanup TODOs in HTable
|
||||||
|
|
|
@ -251,13 +251,18 @@ public class FSUtils {
|
||||||
// Are there any data nodes up yet?
|
// Are there any data nodes up yet?
|
||||||
// Currently the safe mode check falls through if the namenode is up but no
|
// Currently the safe mode check falls through if the namenode is up but no
|
||||||
// datanodes have reported in yet.
|
// datanodes have reported in yet.
|
||||||
while (dfs.getDataNodeStats().length == 0) {
|
try {
|
||||||
LOG.info("Waiting for dfs to come up...");
|
while (dfs.getDataNodeStats().length == 0) {
|
||||||
try {
|
LOG.info("Waiting for dfs to come up...");
|
||||||
Thread.sleep(wait);
|
try {
|
||||||
} catch (InterruptedException e) {
|
Thread.sleep(wait);
|
||||||
//continue
|
} catch (InterruptedException e) {
|
||||||
|
//continue
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
// getDataNodeStats can fail if superuser privilege is required to run
|
||||||
|
// the datanode report, just ignore it
|
||||||
}
|
}
|
||||||
// Make sure dfs is not in safe mode
|
// Make sure dfs is not in safe mode
|
||||||
while (dfs.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_GET)) {
|
while (dfs.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_GET)) {
|
||||||
|
|
Loading…
Reference in New Issue