[ https://issues.apache.org/jira/browse/HDFS-8897?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
John Zhuge updated HDFS-8897: ----------------------------- Status: Patch Available (was: In Progress) > Balancer should handle fs.defaultFS trailing slash in HA > -------------------------------------------------------- > > Key: HDFS-8897 > URL: https://issues.apache.org/jira/browse/HDFS-8897 > Project: Hadoop HDFS > Issue Type: Bug > Components: balancer & mover > Affects Versions: 2.7.1 > Environment: Centos 6.6 > Reporter: LINTE > Assignee: John Zhuge > Attachments: HDFS-8897.001.patch, HDFS-8897.002.patch, > HDFS-8897.003.patch > > > When balancer is launched, it should test if there is already a > /system/balancer.id file in HDFS. > When the file doesn't exist, the balancer don't want to run : > 15/08/14 16:35:12 INFO balancer.Balancer: namenodes = [hdfs://sandbox/, > hdfs://sandbox] > 15/08/14 16:35:12 INFO balancer.Balancer: parameters = > Balancer.Parameters[BalancingPolicy.Node, threshold=10.0, max idle iteration > = 5, number of nodes to be excluded = 0, number of nodes to be included = 0] > Time Stamp Iteration# Bytes Already Moved Bytes Left To Move > Bytes Being Moved > 15/08/14 16:35:14 INFO balancer.KeyManager: Block token params received from > NN: update interval=10hrs, 0sec, token lifetime=10hrs, 0sec > 15/08/14 16:35:14 INFO block.BlockTokenSecretManager: Setting block keys > 15/08/14 16:35:14 INFO balancer.KeyManager: Update block keys every 2hrs, > 30mins, 0sec > 15/08/14 16:35:14 INFO block.BlockTokenSecretManager: Setting block keys > 15/08/14 16:35:14 INFO balancer.KeyManager: Block token params received from > NN: update interval=10hrs, 0sec, token lifetime=10hrs, 0sec > 15/08/14 16:35:14 INFO block.BlockTokenSecretManager: Setting block keys > 15/08/14 16:35:14 INFO balancer.KeyManager: Update block keys every 2hrs, > 30mins, 0sec > java.io.IOException: Another Balancer is running.. Exiting ... > Aug 14, 2015 4:35:14 PM Balancing took 2.408 seconds > Looking at the audit log file when trying to run the balancer, the balancer > create the /system/balancer.id and then delete it on exiting ... > 2015-08-14 16:37:45,844 INFO FSNamesystem.audit: allowed=true > ugi=hdfs@SANDBOX.HADOOP (auth:KERBEROS) ip=/x.x.x.x cmd=getfileinfo > src=/system/balancer.id dst=null perm=null proto=rpc > 2015-08-14 16:37:45,900 INFO FSNamesystem.audit: allowed=true > ugi=hdfs@SANDBOX.HADOOP (auth:KERBEROS) ip=/x.x.x.x cmd=create > src=/system/balancer.id dst=null perm=hdfs:hadoop:rw-r----- > proto=rpc > 2015-08-14 16:37:45,919 INFO FSNamesystem.audit: allowed=true > ugi=hdfs@SANDBOX.HADOOP (auth:KERBEROS) ip=/x.x.x.x cmd=getfileinfo > src=/system/balancer.id dst=null perm=null proto=rpc > 2015-08-14 16:37:46,090 INFO FSNamesystem.audit: allowed=true > ugi=hdfs@SANDBOX.HADOOP (auth:KERBEROS) ip=/x.x.x.x cmd=getfileinfo > src=/system/balancer.id dst=null perm=null proto=rpc > 2015-08-14 16:37:46,112 INFO FSNamesystem.audit: allowed=true > ugi=hdfs@SANDBOX.HADOOP (auth:KERBEROS) ip=/x.x.x.x cmd=getfileinfo > src=/system/balancer.id dst=null perm=null proto=rpc > 2015-08-14 16:37:46,117 INFO FSNamesystem.audit: allowed=true > ugi=hdfs@SANDBOX.HADOOP (auth:KERBEROS) ip=/x.x.x.x cmd=delete > src=/system/balancer.id dst=null perm=null proto=rpc > The error seems to be located in > org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java > The function checkAndMarkRunning return null even if the /system/balancer.id > doesn't exist before entering this function; if it exists, then it is deleted > and the balancer exit with the same error. > ---- > private OutputStream checkAndMarkRunning() throws IOException { > try { > if (fs.exists(idPath)) { > // try appending to it so that it will fail fast if another balancer > is > // running. > IOUtils.closeStream(fs.append(idPath)); > fs.delete(idPath, true); > } > final FSDataOutputStream fsout = fs.create(idPath, false); > // mark balancer idPath to be deleted during filesystem closure > fs.deleteOnExit(idPath); > if (write2IdFile) { > fsout.writeBytes(InetAddress.getLocalHost().getHostName()); > fsout.hflush(); > } > return fsout; > } catch(RemoteException e) { > > if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){ > return null; > } else { > throw e; > } > } > } > ---- > Regards -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org