1 /**
2  * Licensed to the Apache Software Foundation (ASF) under one
3  * or more contributor license agreements.  See the NOTICE file
4  * distributed with this work for additional information
5  * regarding copyright ownership.  The ASF licenses this file
6  * to you under the Apache License, Version 2.0 (the
7  * "License"); you may not use this file except in compliance
8  * with the License.  You may obtain a copy of the License at
9  *
10  *     http://www.apache.org/licenses/LICENSE-2.0
11  *
12  * Unless required by applicable law or agreed to in writing, software
13  * distributed under the License is distributed on an "AS IS" BASIS,
14  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15  * See the License for the specific language governing permissions and
16  * limitations under the License.
17  */
18 
19 package org.apache.hadoop.security.token.delegation;
20 
21 import java.io.ByteArrayInputStream;
22 import java.io.DataInputStream;
23 import java.io.IOException;
24 import java.util.Arrays;
25 import java.util.HashMap;
26 import java.util.HashSet;
27 import java.util.Iterator;
28 import java.util.Map;
29 import java.util.Set;
30 
31 import javax.crypto.SecretKey;
32 
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35 import org.apache.hadoop.classification.InterfaceAudience;
36 import org.apache.hadoop.classification.InterfaceStability;
37 import org.apache.hadoop.io.Text;
38 import org.apache.hadoop.security.AccessControlException;
39 import org.apache.hadoop.security.HadoopKerberosName;
40 import org.apache.hadoop.security.token.SecretManager;
41 import org.apache.hadoop.security.token.Token;
42 import org.apache.hadoop.util.Daemon;
43 import org.apache.hadoop.util.Time;
44 
45 import com.google.common.base.Preconditions;
46 
47 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "Hive"})
48 @InterfaceStability.Evolving
49 public abstract
50 class AbstractDelegationTokenSecretManager<TokenIdent
51 extends AbstractDelegationTokenIdentifier>
52    extends SecretManager<TokenIdent> {
53   private static final Log LOG = LogFactory
54       .getLog(AbstractDelegationTokenSecretManager.class);
55 
56   /**
57    * Cache of currently valid tokens, mapping from DelegationTokenIdentifier
58    * to DelegationTokenInformation. Protected by this object lock.
59    */
60   protected final Map<TokenIdent, DelegationTokenInformation> currentTokens
61       = new HashMap<TokenIdent, DelegationTokenInformation>();
62 
63   /**
64    * Sequence number to create DelegationTokenIdentifier.
65    * Protected by this object lock.
66    */
67   protected int delegationTokenSequenceNumber = 0;
68 
69   /**
70    * Access to allKeys is protected by this object lock
71    */
72   protected final Map<Integer, DelegationKey> allKeys
73       = new HashMap<Integer, DelegationKey>();
74 
75   /**
76    * Access to currentId is protected by this object lock.
77    */
78   protected int currentId = 0;
79   /**
80    * Access to currentKey is protected by this object lock
81    */
82   private DelegationKey currentKey;
83 
84   private long keyUpdateInterval;
85   private long tokenMaxLifetime;
86   private long tokenRemoverScanInterval;
87   private long tokenRenewInterval;
88   /**
89    * Whether to store a token's tracking ID in its TokenInformation.
90    * Can be overridden by a subclass.
91    */
92   protected boolean storeTokenTrackingId;
93   private Thread tokenRemoverThread;
94   protected volatile boolean running;
95 
96   /**
97    * If the delegation token update thread holds this lock, it will
98    * not get interrupted.
99    */
100   protected Object noInterruptsLock = new Object();
101 
AbstractDelegationTokenSecretManager(long delegationKeyUpdateInterval, long delegationTokenMaxLifetime, long delegationTokenRenewInterval, long delegationTokenRemoverScanInterval)102   public AbstractDelegationTokenSecretManager(long delegationKeyUpdateInterval,
103       long delegationTokenMaxLifetime, long delegationTokenRenewInterval,
104       long delegationTokenRemoverScanInterval) {
105     this.keyUpdateInterval = delegationKeyUpdateInterval;
106     this.tokenMaxLifetime = delegationTokenMaxLifetime;
107     this.tokenRenewInterval = delegationTokenRenewInterval;
108     this.tokenRemoverScanInterval = delegationTokenRemoverScanInterval;
109     this.storeTokenTrackingId = false;
110   }
111 
112   /** should be called before this object is used */
startThreads()113   public void startThreads() throws IOException {
114     Preconditions.checkState(!running);
115     updateCurrentKey();
116     synchronized (this) {
117       running = true;
118       tokenRemoverThread = new Daemon(new ExpiredTokenRemover());
119       tokenRemoverThread.start();
120     }
121   }
122 
123   /**
124    * Reset all data structures and mutable state.
125    */
reset()126   public synchronized void reset() {
127     setCurrentKeyId(0);
128     allKeys.clear();
129     setDelegationTokenSeqNum(0);
130     currentTokens.clear();
131   }
132 
133   /**
134    * Add a previously used master key to cache (when NN restarts),
135    * should be called before activate().
136    * */
addKey(DelegationKey key)137   public synchronized void addKey(DelegationKey key) throws IOException {
138     if (running) // a safety check
139       throw new IOException("Can't add delegation key to a running SecretManager.");
140     if (key.getKeyId() > getCurrentKeyId()) {
141       setCurrentKeyId(key.getKeyId());
142     }
143     allKeys.put(key.getKeyId(), key);
144   }
145 
getAllKeys()146   public synchronized DelegationKey[] getAllKeys() {
147     return allKeys.values().toArray(new DelegationKey[0]);
148   }
149 
150   // HDFS
logUpdateMasterKey(DelegationKey key)151   protected void logUpdateMasterKey(DelegationKey key) throws IOException {
152     return;
153   }
154 
155   // HDFS
logExpireToken(TokenIdent ident)156   protected void logExpireToken(TokenIdent ident) throws IOException {
157     return;
158   }
159 
160   // RM
storeNewMasterKey(DelegationKey key)161   protected void storeNewMasterKey(DelegationKey key) throws IOException {
162     return;
163   }
164 
165   // RM
removeStoredMasterKey(DelegationKey key)166   protected void removeStoredMasterKey(DelegationKey key) {
167     return;
168   }
169 
170   // RM
storeNewToken(TokenIdent ident, long renewDate)171   protected void storeNewToken(TokenIdent ident, long renewDate) throws IOException{
172     return;
173   }
174 
175   // RM
removeStoredToken(TokenIdent ident)176   protected void removeStoredToken(TokenIdent ident) throws IOException {
177 
178   }
179   // RM
updateStoredToken(TokenIdent ident, long renewDate)180   protected void updateStoredToken(TokenIdent ident, long renewDate) throws IOException {
181     return;
182   }
183 
184   /**
185    * For subclasses externalizing the storage, for example Zookeeper
186    * based implementations
187    */
getCurrentKeyId()188   protected synchronized int getCurrentKeyId() {
189     return currentId;
190   }
191 
192   /**
193    * For subclasses externalizing the storage, for example Zookeeper
194    * based implementations
195    */
incrementCurrentKeyId()196   protected synchronized int incrementCurrentKeyId() {
197     return ++currentId;
198   }
199 
200   /**
201    * For subclasses externalizing the storage, for example Zookeeper
202    * based implementations
203    */
setCurrentKeyId(int keyId)204   protected synchronized void setCurrentKeyId(int keyId) {
205     currentId = keyId;
206   }
207 
208   /**
209    * For subclasses externalizing the storage, for example Zookeeper
210    * based implementations
211    */
getDelegationTokenSeqNum()212   protected synchronized int getDelegationTokenSeqNum() {
213     return delegationTokenSequenceNumber;
214   }
215 
216   /**
217    * For subclasses externalizing the storage, for example Zookeeper
218    * based implementations
219    */
incrementDelegationTokenSeqNum()220   protected synchronized int incrementDelegationTokenSeqNum() {
221     return ++delegationTokenSequenceNumber;
222   }
223 
224   /**
225    * For subclasses externalizing the storage, for example Zookeeper
226    * based implementations
227    */
setDelegationTokenSeqNum(int seqNum)228   protected synchronized void setDelegationTokenSeqNum(int seqNum) {
229     delegationTokenSequenceNumber = seqNum;
230   }
231 
232   /**
233    * For subclasses externalizing the storage, for example Zookeeper
234    * based implementations
235    */
getDelegationKey(int keyId)236   protected DelegationKey getDelegationKey(int keyId) {
237     return allKeys.get(keyId);
238   }
239 
240   /**
241    * For subclasses externalizing the storage, for example Zookeeper
242    * based implementations
243    */
storeDelegationKey(DelegationKey key)244   protected void storeDelegationKey(DelegationKey key) throws IOException {
245     allKeys.put(key.getKeyId(), key);
246     storeNewMasterKey(key);
247   }
248 
249   /**
250    * For subclasses externalizing the storage, for example Zookeeper
251    * based implementations
252    */
updateDelegationKey(DelegationKey key)253   protected void updateDelegationKey(DelegationKey key) throws IOException {
254     allKeys.put(key.getKeyId(), key);
255   }
256 
257   /**
258    * For subclasses externalizing the storage, for example Zookeeper
259    * based implementations
260    */
getTokenInfo(TokenIdent ident)261   protected DelegationTokenInformation getTokenInfo(TokenIdent ident) {
262     return currentTokens.get(ident);
263   }
264 
265   /**
266    * For subclasses externalizing the storage, for example Zookeeper
267    * based implementations
268    */
storeToken(TokenIdent ident, DelegationTokenInformation tokenInfo)269   protected void storeToken(TokenIdent ident,
270       DelegationTokenInformation tokenInfo) throws IOException {
271     currentTokens.put(ident, tokenInfo);
272     storeNewToken(ident, tokenInfo.getRenewDate());
273   }
274 
275   /**
276    * For subclasses externalizing the storage, for example Zookeeper
277    * based implementations
278    */
updateToken(TokenIdent ident, DelegationTokenInformation tokenInfo)279   protected void updateToken(TokenIdent ident,
280       DelegationTokenInformation tokenInfo) throws IOException {
281     currentTokens.put(ident, tokenInfo);
282     updateStoredToken(ident, tokenInfo.getRenewDate());
283   }
284 
285   /**
286    * This method is intended to be used for recovering persisted delegation
287    * tokens
288    * This method must be called before this secret manager is activated (before
289    * startThreads() is called)
290    * @param identifier identifier read from persistent storage
291    * @param renewDate token renew time
292    * @throws IOException
293    */
addPersistedDelegationToken( TokenIdent identifier, long renewDate)294   public synchronized void addPersistedDelegationToken(
295       TokenIdent identifier, long renewDate) throws IOException {
296     if (running) {
297       // a safety check
298       throw new IOException(
299           "Can't add persisted delegation token to a running SecretManager.");
300     }
301     int keyId = identifier.getMasterKeyId();
302     DelegationKey dKey = allKeys.get(keyId);
303     if (dKey == null) {
304       LOG.warn("No KEY found for persisted identifier " + identifier.toString());
305       return;
306     }
307     byte[] password = createPassword(identifier.getBytes(), dKey.getKey());
308     if (identifier.getSequenceNumber() > getDelegationTokenSeqNum()) {
309       setDelegationTokenSeqNum(identifier.getSequenceNumber());
310     }
311     if (getTokenInfo(identifier) == null) {
312       currentTokens.put(identifier, new DelegationTokenInformation(renewDate,
313           password, getTrackingIdIfEnabled(identifier)));
314     } else {
315       throw new IOException("Same delegation token being added twice.");
316     }
317   }
318 
319   /**
320    * Update the current master key
321    * This is called once by startThreads before tokenRemoverThread is created,
322    * and only by tokenRemoverThread afterwards.
323    */
updateCurrentKey()324   private void updateCurrentKey() throws IOException {
325     LOG.info("Updating the current master key for generating delegation tokens");
326     /* Create a new currentKey with an estimated expiry date. */
327     int newCurrentId;
328     synchronized (this) {
329       newCurrentId = incrementCurrentKeyId();
330     }
331     DelegationKey newKey = new DelegationKey(newCurrentId, System
332         .currentTimeMillis()
333         + keyUpdateInterval + tokenMaxLifetime, generateSecret());
334     //Log must be invoked outside the lock on 'this'
335     logUpdateMasterKey(newKey);
336     synchronized (this) {
337       currentKey = newKey;
338       storeDelegationKey(currentKey);
339     }
340   }
341 
342   /**
343    * Update the current master key for generating delegation tokens
344    * It should be called only by tokenRemoverThread.
345    */
rollMasterKey()346   void rollMasterKey() throws IOException {
347     synchronized (this) {
348       removeExpiredKeys();
349       /* set final expiry date for retiring currentKey */
350       currentKey.setExpiryDate(Time.now() + tokenMaxLifetime);
351       /*
352        * currentKey might have been removed by removeExpiredKeys(), if
353        * updateMasterKey() isn't called at expected interval. Add it back to
354        * allKeys just in case.
355        */
356       updateDelegationKey(currentKey);
357     }
358     updateCurrentKey();
359   }
360 
removeExpiredKeys()361   private synchronized void removeExpiredKeys() {
362     long now = Time.now();
363     for (Iterator<Map.Entry<Integer, DelegationKey>> it = allKeys.entrySet()
364         .iterator(); it.hasNext();) {
365       Map.Entry<Integer, DelegationKey> e = it.next();
366       if (e.getValue().getExpiryDate() < now) {
367         it.remove();
368         // ensure the tokens generated by this current key can be recovered
369         // with this current key after this current key is rolled
370         if(!e.getValue().equals(currentKey))
371           removeStoredMasterKey(e.getValue());
372       }
373     }
374   }
375 
376   @Override
createPassword(TokenIdent identifier)377   protected synchronized byte[] createPassword(TokenIdent identifier) {
378     int sequenceNum;
379     long now = Time.now();
380     sequenceNum = incrementDelegationTokenSeqNum();
381     identifier.setIssueDate(now);
382     identifier.setMaxDate(now + tokenMaxLifetime);
383     identifier.setMasterKeyId(currentKey.getKeyId());
384     identifier.setSequenceNumber(sequenceNum);
385     LOG.info("Creating password for identifier: " + identifier
386         + ", currentKey: " + currentKey.getKeyId());
387     byte[] password = createPassword(identifier.getBytes(), currentKey.getKey());
388     DelegationTokenInformation tokenInfo = new DelegationTokenInformation(now
389         + tokenRenewInterval, password, getTrackingIdIfEnabled(identifier));
390     try {
391       storeToken(identifier, tokenInfo);
392     } catch (IOException ioe) {
393       LOG.error("Could not store token !!", ioe);
394     }
395     return password;
396   }
397 
398 
399 
400   /**
401    * Find the DelegationTokenInformation for the given token id, and verify that
402    * if the token is expired. Note that this method should be called with
403    * acquiring the secret manager's monitor.
404    */
checkToken(TokenIdent identifier)405   protected DelegationTokenInformation checkToken(TokenIdent identifier)
406       throws InvalidToken {
407     assert Thread.holdsLock(this);
408     DelegationTokenInformation info = getTokenInfo(identifier);
409     if (info == null) {
410       throw new InvalidToken("token (" + identifier.toString()
411           + ") can't be found in cache");
412     }
413     if (info.getRenewDate() < Time.now()) {
414       throw new InvalidToken("token (" + identifier.toString() + ") is expired");
415     }
416     return info;
417   }
418 
419   @Override
retrievePassword(TokenIdent identifier)420   public synchronized byte[] retrievePassword(TokenIdent identifier)
421       throws InvalidToken {
422     return checkToken(identifier).getPassword();
423   }
424 
getTrackingIdIfEnabled(TokenIdent ident)425   protected String getTrackingIdIfEnabled(TokenIdent ident) {
426     if (storeTokenTrackingId) {
427       return ident.getTrackingId();
428     }
429     return null;
430   }
431 
getTokenTrackingId(TokenIdent identifier)432   public synchronized String getTokenTrackingId(TokenIdent identifier) {
433     DelegationTokenInformation info = getTokenInfo(identifier);
434     if (info == null) {
435       return null;
436     }
437     return info.getTrackingId();
438   }
439 
440   /**
441    * Verifies that the given identifier and password are valid and match.
442    * @param identifier Token identifier.
443    * @param password Password in the token.
444    * @throws InvalidToken
445    */
verifyToken(TokenIdent identifier, byte[] password)446   public synchronized void verifyToken(TokenIdent identifier, byte[] password)
447       throws InvalidToken {
448     byte[] storedPassword = retrievePassword(identifier);
449     if (!Arrays.equals(password, storedPassword)) {
450       throw new InvalidToken("token (" + identifier
451           + ") is invalid, password doesn't match");
452     }
453   }
454 
455   /**
456    * Renew a delegation token.
457    * @param token the token to renew
458    * @param renewer the full principal name of the user doing the renewal
459    * @return the new expiration time
460    * @throws InvalidToken if the token is invalid
461    * @throws AccessControlException if the user can't renew token
462    */
renewToken(Token<TokenIdent> token, String renewer)463   public synchronized long renewToken(Token<TokenIdent> token,
464                          String renewer) throws InvalidToken, IOException {
465     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
466     DataInputStream in = new DataInputStream(buf);
467     TokenIdent id = createIdentifier();
468     id.readFields(in);
469     LOG.info("Token renewal for identifier: " + id + "; total currentTokens "
470         +  currentTokens.size());
471 
472     long now = Time.now();
473     if (id.getMaxDate() < now) {
474       throw new InvalidToken(renewer + " tried to renew an expired token");
475     }
476     if ((id.getRenewer() == null) || (id.getRenewer().toString().isEmpty())) {
477       throw new AccessControlException(renewer +
478           " tried to renew a token without a renewer");
479     }
480     if (!id.getRenewer().toString().equals(renewer)) {
481       throw new AccessControlException(renewer +
482           " tries to renew a token with renewer " + id.getRenewer());
483     }
484     DelegationKey key = getDelegationKey(id.getMasterKeyId());
485     if (key == null) {
486       throw new InvalidToken("Unable to find master key for keyId="
487           + id.getMasterKeyId()
488           + " from cache. Failed to renew an unexpired token"
489           + " with sequenceNumber=" + id.getSequenceNumber());
490     }
491     byte[] password = createPassword(token.getIdentifier(), key.getKey());
492     if (!Arrays.equals(password, token.getPassword())) {
493       throw new AccessControlException(renewer +
494           " is trying to renew a token with wrong password");
495     }
496     long renewTime = Math.min(id.getMaxDate(), now + tokenRenewInterval);
497     String trackingId = getTrackingIdIfEnabled(id);
498     DelegationTokenInformation info = new DelegationTokenInformation(renewTime,
499         password, trackingId);
500 
501     if (getTokenInfo(id) == null) {
502       throw new InvalidToken("Renewal request for unknown token");
503     }
504     updateToken(id, info);
505     return renewTime;
506   }
507 
508   /**
509    * Cancel a token by removing it from cache.
510    * @return Identifier of the canceled token
511    * @throws InvalidToken for invalid token
512    * @throws AccessControlException if the user isn't allowed to cancel
513    */
cancelToken(Token<TokenIdent> token, String canceller)514   public synchronized TokenIdent cancelToken(Token<TokenIdent> token,
515       String canceller) throws IOException {
516     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
517     DataInputStream in = new DataInputStream(buf);
518     TokenIdent id = createIdentifier();
519     id.readFields(in);
520     LOG.info("Token cancelation requested for identifier: "+id);
521 
522     if (id.getUser() == null) {
523       throw new InvalidToken("Token with no owner");
524     }
525     String owner = id.getUser().getUserName();
526     Text renewer = id.getRenewer();
527     HadoopKerberosName cancelerKrbName = new HadoopKerberosName(canceller);
528     String cancelerShortName = cancelerKrbName.getShortName();
529     if (!canceller.equals(owner)
530         && (renewer == null || renewer.toString().isEmpty() || !cancelerShortName
531             .equals(renewer.toString()))) {
532       throw new AccessControlException(canceller
533           + " is not authorized to cancel the token");
534     }
535     DelegationTokenInformation info = currentTokens.remove(id);
536     if (info == null) {
537       throw new InvalidToken("Token not found");
538     }
539     removeStoredToken(id);
540     return id;
541   }
542 
543   /**
544    * Convert the byte[] to a secret key
545    * @param key the byte[] to create the secret key from
546    * @return the secret key
547    */
createSecretKey(byte[] key)548   public static SecretKey createSecretKey(byte[] key) {
549     return SecretManager.createSecretKey(key);
550   }
551 
552   /** Class to encapsulate a token's renew date and password. */
553   @InterfaceStability.Evolving
554   public static class DelegationTokenInformation {
555     long renewDate;
556     byte[] password;
557     String trackingId;
558 
DelegationTokenInformation(long renewDate, byte[] password)559     public DelegationTokenInformation(long renewDate, byte[] password) {
560       this(renewDate, password, null);
561     }
562 
DelegationTokenInformation(long renewDate, byte[] password, String trackingId)563     public DelegationTokenInformation(long renewDate, byte[] password,
564         String trackingId) {
565       this.renewDate = renewDate;
566       this.password = password;
567       this.trackingId = trackingId;
568     }
569     /** returns renew date */
getRenewDate()570     public long getRenewDate() {
571       return renewDate;
572     }
573     /** returns password */
getPassword()574     byte[] getPassword() {
575       return password;
576     }
577     /** returns tracking id */
getTrackingId()578     public String getTrackingId() {
579       return trackingId;
580     }
581   }
582 
583   /** Remove expired delegation tokens from cache */
removeExpiredToken()584   private void removeExpiredToken() throws IOException {
585     long now = Time.now();
586     Set<TokenIdent> expiredTokens = new HashSet<TokenIdent>();
587     synchronized (this) {
588       Iterator<Map.Entry<TokenIdent, DelegationTokenInformation>> i =
589           currentTokens.entrySet().iterator();
590       while (i.hasNext()) {
591         Map.Entry<TokenIdent, DelegationTokenInformation> entry = i.next();
592         long renewDate = entry.getValue().getRenewDate();
593         if (renewDate < now) {
594           expiredTokens.add(entry.getKey());
595           i.remove();
596         }
597       }
598     }
599     // don't hold lock on 'this' to avoid edit log updates blocking token ops
600     for (TokenIdent ident : expiredTokens) {
601       logExpireToken(ident);
602       removeStoredToken(ident);
603     }
604   }
605 
stopThreads()606   public void stopThreads() {
607     if (LOG.isDebugEnabled())
608       LOG.debug("Stopping expired delegation token remover thread");
609     running = false;
610 
611     if (tokenRemoverThread != null) {
612       synchronized (noInterruptsLock) {
613         tokenRemoverThread.interrupt();
614       }
615       try {
616         tokenRemoverThread.join();
617       } catch (InterruptedException e) {
618         throw new RuntimeException(
619             "Unable to join on token removal thread", e);
620       }
621     }
622   }
623 
624   /**
625    * is secretMgr running
626    * @return true if secret mgr is running
627    */
isRunning()628   public synchronized boolean isRunning() {
629     return running;
630   }
631 
632   private class ExpiredTokenRemover extends Thread {
633     private long lastMasterKeyUpdate;
634     private long lastTokenCacheCleanup;
635 
636     @Override
run()637     public void run() {
638       LOG.info("Starting expired delegation token remover thread, "
639           + "tokenRemoverScanInterval=" + tokenRemoverScanInterval
640           / (60 * 1000) + " min(s)");
641       try {
642         while (running) {
643           long now = Time.now();
644           if (lastMasterKeyUpdate + keyUpdateInterval < now) {
645             try {
646               rollMasterKey();
647               lastMasterKeyUpdate = now;
648             } catch (IOException e) {
649               LOG.error("Master key updating failed: ", e);
650             }
651           }
652           if (lastTokenCacheCleanup + tokenRemoverScanInterval < now) {
653             removeExpiredToken();
654             lastTokenCacheCleanup = now;
655           }
656           try {
657             Thread.sleep(Math.min(5000, keyUpdateInterval)); // 5 seconds
658           } catch (InterruptedException ie) {
659             LOG.error("ExpiredTokenRemover received " + ie);
660           }
661         }
662       } catch (Throwable t) {
663         LOG.error("ExpiredTokenRemover thread received unexpected exception", t);
664         Runtime.getRuntime().exit(-1);
665       }
666     }
667   }
668 
669   /**
670    * Decode the token identifier. The subclass can customize the way to decode
671    * the token identifier.
672    *
673    * @param token the token where to extract the identifier
674    * @return the delegation token identifier
675    * @throws IOException
676    */
decodeTokenIdentifier(Token<TokenIdent> token)677   public TokenIdent decodeTokenIdentifier(Token<TokenIdent> token) throws IOException {
678     return token.decodeIdentifier();
679   }
680 
681 }
682