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 019 package org.apache.hadoop.fs; 020 021 import com.google.common.annotations.VisibleForTesting; 022 023 import java.io.IOException; 024 import java.lang.ref.WeakReference; 025 import java.util.concurrent.DelayQueue; 026 import java.util.concurrent.Delayed; 027 import java.util.concurrent.TimeUnit; 028 029 import org.apache.commons.logging.Log; 030 import org.apache.commons.logging.LogFactory; 031 import org.apache.hadoop.classification.InterfaceAudience; 032 import org.apache.hadoop.security.token.Token; 033 import org.apache.hadoop.security.token.TokenIdentifier; 034 import org.apache.hadoop.util.Time; 035 036 /** 037 * A daemon thread that waits for the next file system to renew. 038 */ 039 @InterfaceAudience.Private 040 public class DelegationTokenRenewer 041 extends Thread { 042 private static final Log LOG = LogFactory 043 .getLog(DelegationTokenRenewer.class); 044 045 /** The renewable interface used by the renewer. */ 046 public interface Renewable { 047 /** @return the renew token. */ 048 public Token<?> getRenewToken(); 049 050 /** Set delegation token. */ 051 public <T extends TokenIdentifier> void setDelegationToken(Token<T> token); 052 } 053 054 /** 055 * An action that will renew and replace the file system's delegation 056 * tokens automatically. 057 */ 058 private static class RenewAction<T extends FileSystem & Renewable> 059 implements Delayed { 060 /** when should the renew happen */ 061 private long renewalTime; 062 /** a weak reference to the file system so that it can be garbage collected */ 063 private final WeakReference<T> weakFs; 064 065 private RenewAction(final T fs) { 066 this.weakFs = new WeakReference<T>(fs); 067 updateRenewalTime(); 068 } 069 070 /** Get the delay until this event should happen. */ 071 @Override 072 public long getDelay(final TimeUnit unit) { 073 final long millisLeft = renewalTime - Time.now(); 074 return unit.convert(millisLeft, TimeUnit.MILLISECONDS); 075 } 076 077 @Override 078 public int compareTo(final Delayed delayed) { 079 final RenewAction<?> that = (RenewAction<?>)delayed; 080 return this.renewalTime < that.renewalTime? -1 081 : this.renewalTime == that.renewalTime? 0: 1; 082 } 083 084 @Override 085 public int hashCode() { 086 return (int)renewalTime ^ (int)(renewalTime >>> 32); 087 } 088 089 @Override 090 public boolean equals(final Object that) { 091 if (that == null || !(that instanceof RenewAction)) { 092 return false; 093 } 094 return compareTo((Delayed)that) == 0; 095 } 096 097 /** 098 * Set a new time for the renewal. 099 * It can only be called when the action is not in the queue. 100 * @param newTime the new time 101 */ 102 private void updateRenewalTime() { 103 renewalTime = renewCycle + Time.now(); 104 } 105 106 /** 107 * Renew or replace the delegation token for this file system. 108 * @return 109 * @throws IOException 110 */ 111 private boolean renew() throws IOException, InterruptedException { 112 final T fs = weakFs.get(); 113 final boolean b = fs != null; 114 if (b) { 115 synchronized(fs) { 116 try { 117 fs.getRenewToken().renew(fs.getConf()); 118 } catch (IOException ie) { 119 try { 120 Token<?>[] tokens = fs.addDelegationTokens(null, null); 121 if (tokens.length == 0) { 122 throw new IOException("addDelegationTokens returned no tokens"); 123 } 124 fs.setDelegationToken(tokens[0]); 125 } catch (IOException ie2) { 126 throw new IOException("Can't renew or get new delegation token ", ie); 127 } 128 } 129 } 130 } 131 return b; 132 } 133 134 @Override 135 public String toString() { 136 Renewable fs = weakFs.get(); 137 return fs == null? "evaporated token renew" 138 : "The token will be renewed in " + getDelay(TimeUnit.SECONDS) 139 + " secs, renewToken=" + fs.getRenewToken(); 140 } 141 } 142 143 /** Wait for 95% of a day between renewals */ 144 private static final int RENEW_CYCLE = 24 * 60 * 60 * 950; 145 146 @InterfaceAudience.Private 147 protected static int renewCycle = RENEW_CYCLE; 148 149 /** Queue to maintain the RenewActions to be processed by the {@link #run()} */ 150 private volatile DelayQueue<RenewAction<?>> queue = new DelayQueue<RenewAction<?>>(); 151 152 /** For testing purposes */ 153 @VisibleForTesting 154 protected int getRenewQueueLength() { 155 return queue.size(); 156 } 157 158 /** 159 * Create the singleton instance. However, the thread can be started lazily in 160 * {@link #addRenewAction(FileSystem)} 161 */ 162 private static DelegationTokenRenewer INSTANCE = null; 163 164 private DelegationTokenRenewer(final Class<? extends FileSystem> clazz) { 165 super(clazz.getSimpleName() + "-" + DelegationTokenRenewer.class.getSimpleName()); 166 setDaemon(true); 167 } 168 169 public static synchronized DelegationTokenRenewer getInstance() { 170 if (INSTANCE == null) { 171 INSTANCE = new DelegationTokenRenewer(FileSystem.class); 172 } 173 return INSTANCE; 174 } 175 176 /** Add a renew action to the queue. */ 177 public synchronized <T extends FileSystem & Renewable> void addRenewAction(final T fs) { 178 queue.add(new RenewAction<T>(fs)); 179 if (!isAlive()) { 180 start(); 181 } 182 } 183 184 /** 185 * Remove the associated renew action from the queue 186 * 187 * @throws IOException 188 */ 189 public synchronized <T extends FileSystem & Renewable> void removeRenewAction( 190 final T fs) throws IOException { 191 for (RenewAction<?> action : queue) { 192 if (action.weakFs.get() == fs) { 193 try { 194 fs.getRenewToken().cancel(fs.getConf()); 195 } catch (InterruptedException ie) { 196 LOG.error("Interrupted while canceling token for " + fs.getUri() 197 + "filesystem"); 198 if (LOG.isDebugEnabled()) { 199 LOG.debug(ie.getStackTrace()); 200 } 201 } 202 queue.remove(action); 203 return; 204 } 205 } 206 } 207 208 @SuppressWarnings("static-access") 209 @Override 210 public void run() { 211 for(;;) { 212 RenewAction<?> action = null; 213 try { 214 synchronized (this) { 215 action = queue.take(); 216 if (action.renew()) { 217 action.updateRenewalTime(); 218 queue.add(action); 219 } 220 } 221 } catch (InterruptedException ie) { 222 return; 223 } catch (Exception ie) { 224 action.weakFs.get().LOG.warn("Failed to renew token, action=" + action, 225 ie); 226 } 227 } 228 } 229 }