001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018 019package org.apache.hadoop.hbase.util; 020 021import java.io.FileNotFoundException; 022import java.io.IOException; 023import java.io.InterruptedIOException; 024import java.lang.reflect.Method; 025import java.net.InetSocketAddress; 026import java.net.URI; 027import java.util.Collection; 028import java.util.HashSet; 029import java.util.Map; 030import java.util.Set; 031 032import org.apache.hadoop.conf.Configuration; 033import org.apache.hadoop.fs.FileSystem; 034import org.apache.hadoop.fs.Path; 035import org.apache.hadoop.hdfs.DistributedFileSystem; 036import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException; 037import org.apache.yetus.audience.InterfaceAudience; 038import org.apache.yetus.audience.InterfaceStability; 039import org.slf4j.Logger; 040import org.slf4j.LoggerFactory; 041 042import org.apache.hbase.thirdparty.com.google.common.collect.Sets; 043 044/** 045 * Implementation for hdfs 046 */ 047@InterfaceAudience.Private 048@InterfaceStability.Evolving 049public class FSHDFSUtils extends FSUtils { 050 private static final Logger LOG = LoggerFactory.getLogger(FSHDFSUtils.class); 051 private static Class dfsUtilClazz; 052 private static Method getNNAddressesMethod; 053 054 /** 055 * @param fs 056 * @param conf 057 * @return A set containing all namenode addresses of fs 058 */ 059 private static Set<InetSocketAddress> getNNAddresses(DistributedFileSystem fs, 060 Configuration conf) { 061 Set<InetSocketAddress> addresses = new HashSet<>(); 062 String serviceName = fs.getCanonicalServiceName(); 063 064 if (serviceName.startsWith("ha-hdfs")) { 065 try { 066 if (dfsUtilClazz == null) { 067 dfsUtilClazz = Class.forName("org.apache.hadoop.hdfs.DFSUtil"); 068 } 069 if (getNNAddressesMethod == null) { 070 try { 071 // getNNServiceRpcAddressesForCluster is available only in version 072 // equal to or later than Hadoop 2.6 073 getNNAddressesMethod = 074 dfsUtilClazz.getMethod("getNNServiceRpcAddressesForCluster", Configuration.class); 075 } catch (NoSuchMethodException e) { 076 // If hadoop version is older than hadoop 2.6 077 getNNAddressesMethod = 078 dfsUtilClazz.getMethod("getNNServiceRpcAddresses", Configuration.class); 079 } 080 081 } 082 083 Map<String, Map<String, InetSocketAddress>> addressMap = 084 (Map<String, Map<String, InetSocketAddress>>) getNNAddressesMethod 085 .invoke(null, conf); 086 String nameService = serviceName.substring(serviceName.indexOf(":") + 1); 087 if (addressMap.containsKey(nameService)) { 088 Map<String, InetSocketAddress> nnMap = addressMap.get(nameService); 089 for (Map.Entry<String, InetSocketAddress> e2 : nnMap.entrySet()) { 090 InetSocketAddress addr = e2.getValue(); 091 addresses.add(addr); 092 } 093 } 094 } catch (Exception e) { 095 LOG.warn("DFSUtil.getNNServiceRpcAddresses failed. serviceName=" + serviceName, e); 096 } 097 } else { 098 URI uri = fs.getUri(); 099 int port = uri.getPort(); 100 if (port < 0) { 101 int idx = serviceName.indexOf(':'); 102 port = Integer.parseInt(serviceName.substring(idx+1)); 103 } 104 InetSocketAddress addr = new InetSocketAddress(uri.getHost(), port); 105 addresses.add(addr); 106 } 107 108 return addresses; 109 } 110 111 /** 112 * @param conf the Configuration of HBase 113 * @param srcFs 114 * @param desFs 115 * @return Whether srcFs and desFs are on same hdfs or not 116 */ 117 public static boolean isSameHdfs(Configuration conf, FileSystem srcFs, FileSystem desFs) { 118 // By getCanonicalServiceName, we could make sure both srcFs and desFs 119 // show a unified format which contains scheme, host and port. 120 String srcServiceName = srcFs.getCanonicalServiceName(); 121 String desServiceName = desFs.getCanonicalServiceName(); 122 123 if (srcServiceName == null || desServiceName == null) { 124 return false; 125 } 126 if (srcServiceName.equals(desServiceName)) { 127 return true; 128 } 129 if (srcServiceName.startsWith("ha-hdfs") && desServiceName.startsWith("ha-hdfs")) { 130 Collection<String> internalNameServices = 131 conf.getTrimmedStringCollection("dfs.internal.nameservices"); 132 if (!internalNameServices.isEmpty()) { 133 if (internalNameServices.contains(srcServiceName.split(":")[1])) { 134 return true; 135 } else { 136 return false; 137 } 138 } 139 } 140 if (srcFs instanceof DistributedFileSystem && desFs instanceof DistributedFileSystem) { 141 //If one serviceName is an HA format while the other is a non-HA format, 142 // maybe they refer to the same FileSystem. 143 //For example, srcFs is "ha-hdfs://nameservices" and desFs is "hdfs://activeNamenode:port" 144 Set<InetSocketAddress> srcAddrs = getNNAddresses((DistributedFileSystem) srcFs, conf); 145 Set<InetSocketAddress> desAddrs = getNNAddresses((DistributedFileSystem) desFs, conf); 146 if (Sets.intersection(srcAddrs, desAddrs).size() > 0) { 147 return true; 148 } 149 } 150 151 return false; 152 } 153 154 /** 155 * Recover the lease from HDFS, retrying multiple times. 156 */ 157 @Override 158 public void recoverFileLease(final FileSystem fs, final Path p, 159 Configuration conf, CancelableProgressable reporter) 160 throws IOException { 161 // lease recovery not needed for local file system case. 162 if (!(fs instanceof DistributedFileSystem)) return; 163 recoverDFSFileLease((DistributedFileSystem)fs, p, conf, reporter); 164 } 165 166 /* 167 * Run the dfs recover lease. recoverLease is asynchronous. It returns: 168 * -false when it starts the lease recovery (i.e. lease recovery not *yet* done) 169 * - true when the lease recovery has succeeded or the file is closed. 170 * But, we have to be careful. Each time we call recoverLease, it starts the recover lease 171 * process over from the beginning. We could put ourselves in a situation where we are 172 * doing nothing but starting a recovery, interrupting it to start again, and so on. 173 * The findings over in HBASE-8354 have it that the namenode will try to recover the lease 174 * on the file's primary node. If all is well, it should return near immediately. But, 175 * as is common, it is the very primary node that has crashed and so the namenode will be 176 * stuck waiting on a socket timeout before it will ask another datanode to start the 177 * recovery. It does not help if we call recoverLease in the meantime and in particular, 178 * subsequent to the socket timeout, a recoverLease invocation will cause us to start 179 * over from square one (possibly waiting on socket timeout against primary node). So, 180 * in the below, we do the following: 181 * 1. Call recoverLease. 182 * 2. If it returns true, break. 183 * 3. If it returns false, wait a few seconds and then call it again. 184 * 4. If it returns true, break. 185 * 5. If it returns false, wait for what we think the datanode socket timeout is 186 * (configurable) and then try again. 187 * 6. If it returns true, break. 188 * 7. If it returns false, repeat starting at step 5. above. 189 * 190 * If HDFS-4525 is available, call it every second and we might be able to exit early. 191 */ 192 boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p, 193 final Configuration conf, final CancelableProgressable reporter) 194 throws IOException { 195 LOG.info("Recover lease on dfs file " + p); 196 long startWaiting = EnvironmentEdgeManager.currentTime(); 197 // Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS 198 // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves 199 // beyond that limit 'to be safe'. 200 long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting; 201 // This setting should be a little bit above what the cluster dfs heartbeat is set to. 202 long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000); 203 // This should be set to how long it'll take for us to timeout against primary datanode if it 204 // is dead. We set it to 64 seconds, 4 second than the default READ_TIMEOUT in HDFS, the 205 // default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY. If recovery is still failing after this 206 // timeout, then further recovery will take liner backoff with this base, to avoid endless 207 // preemptions when this value is not properly configured. 208 long subsequentPauseBase = conf.getLong("hbase.lease.recovery.dfs.timeout", 64 * 1000); 209 210 Method isFileClosedMeth = null; 211 // whether we need to look for isFileClosed method 212 boolean findIsFileClosedMeth = true; 213 boolean recovered = false; 214 // We break the loop if we succeed the lease recovery, timeout, or we throw an exception. 215 for (int nbAttempt = 0; !recovered; nbAttempt++) { 216 recovered = recoverLease(dfs, nbAttempt, p, startWaiting); 217 if (recovered) break; 218 checkIfCancelled(reporter); 219 if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) break; 220 try { 221 // On the first time through wait the short 'firstPause'. 222 if (nbAttempt == 0) { 223 Thread.sleep(firstPause); 224 } else { 225 // Cycle here until (subsequentPause * nbAttempt) elapses. While spinning, check 226 // isFileClosed if available (should be in hadoop 2.0.5... not in hadoop 1 though. 227 long localStartWaiting = EnvironmentEdgeManager.currentTime(); 228 while ((EnvironmentEdgeManager.currentTime() - localStartWaiting) < 229 subsequentPauseBase * nbAttempt) { 230 Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000)); 231 if (findIsFileClosedMeth) { 232 try { 233 isFileClosedMeth = dfs.getClass().getMethod("isFileClosed", 234 new Class[]{ Path.class }); 235 } catch (NoSuchMethodException nsme) { 236 LOG.debug("isFileClosed not available"); 237 } finally { 238 findIsFileClosedMeth = false; 239 } 240 } 241 if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) { 242 recovered = true; 243 break; 244 } 245 checkIfCancelled(reporter); 246 } 247 } 248 } catch (InterruptedException ie) { 249 InterruptedIOException iioe = new InterruptedIOException(); 250 iioe.initCause(ie); 251 throw iioe; 252 } 253 } 254 return recovered; 255 } 256 257 boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout, 258 final int nbAttempt, final Path p, final long startWaiting) { 259 if (recoveryTimeout < EnvironmentEdgeManager.currentTime()) { 260 LOG.warn("Cannot recoverLease after trying for " + 261 conf.getInt("hbase.lease.recovery.timeout", 900000) + 262 "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " + 263 getLogMessageDetail(nbAttempt, p, startWaiting)); 264 return true; 265 } 266 return false; 267 } 268 269 /** 270 * Try to recover the lease. 271 * @param dfs 272 * @param nbAttempt 273 * @param p 274 * @param startWaiting 275 * @return True if dfs#recoverLease came by true. 276 * @throws FileNotFoundException 277 */ 278 boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt, final Path p, 279 final long startWaiting) 280 throws FileNotFoundException { 281 boolean recovered = false; 282 try { 283 recovered = dfs.recoverLease(p); 284 LOG.info((recovered? "Recovered lease, ": "Failed to recover lease, ") + 285 getLogMessageDetail(nbAttempt, p, startWaiting)); 286 } catch (IOException e) { 287 if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) { 288 // This exception comes out instead of FNFE, fix it 289 throw new FileNotFoundException("The given WAL wasn't found at " + p); 290 } else if (e instanceof FileNotFoundException) { 291 throw (FileNotFoundException)e; 292 } 293 LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e); 294 } 295 return recovered; 296 } 297 298 /** 299 * @param nbAttempt 300 * @param p 301 * @param startWaiting 302 * @return Detail to append to any log message around lease recovering. 303 */ 304 private String getLogMessageDetail(final int nbAttempt, final Path p, final long startWaiting) { 305 return "attempt=" + nbAttempt + " on file=" + p + " after " + 306 (EnvironmentEdgeManager.currentTime() - startWaiting) + "ms"; 307 } 308 309 /** 310 * Call HDFS-4525 isFileClosed if it is available. 311 * @param dfs 312 * @param m 313 * @param p 314 * @return True if file is closed. 315 */ 316 private boolean isFileClosed(final DistributedFileSystem dfs, final Method m, final Path p) { 317 try { 318 return (Boolean) m.invoke(dfs, p); 319 } catch (SecurityException e) { 320 LOG.warn("No access", e); 321 } catch (Exception e) { 322 LOG.warn("Failed invocation for " + p.toString(), e); 323 } 324 return false; 325 } 326 327 void checkIfCancelled(final CancelableProgressable reporter) 328 throws InterruptedIOException { 329 if (reporter == null) return; 330 if (!reporter.progress()) throw new InterruptedIOException("Operation cancelled"); 331 } 332}