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.hdfs.web;
20 
21 import java.io.FileNotFoundException;
22 import java.io.IOException;
23 import java.io.InputStream;
24 import java.net.ConnectException;
25 import java.net.HttpURLConnection;
26 import java.net.InetSocketAddress;
27 import java.net.URI;
28 import java.net.URISyntaxException;
29 import java.net.URL;
30 import java.security.PrivilegedExceptionAction;
31 import java.text.ParseException;
32 import java.text.SimpleDateFormat;
33 import java.util.ArrayList;
34 import java.util.TimeZone;
35 
36 import org.apache.hadoop.classification.InterfaceAudience;
37 import org.apache.hadoop.classification.InterfaceStability;
38 import org.apache.hadoop.conf.Configuration;
39 import org.apache.hadoop.fs.ContentSummary;
40 import org.apache.hadoop.fs.DelegationTokenRenewer;
41 import org.apache.hadoop.fs.FSDataInputStream;
42 import org.apache.hadoop.fs.FSDataOutputStream;
43 import org.apache.hadoop.fs.FileChecksum;
44 import org.apache.hadoop.fs.FileStatus;
45 import org.apache.hadoop.fs.FileSystem;
46 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
47 import org.apache.hadoop.fs.Path;
48 import org.apache.hadoop.fs.permission.FsPermission;
49 import org.apache.hadoop.hdfs.DFSConfigKeys;
50 import org.apache.hadoop.hdfs.DFSUtil;
51 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
52 import org.apache.hadoop.hdfs.server.common.JspHelper;
53 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
54 import org.apache.hadoop.io.Text;
55 import org.apache.hadoop.ipc.RemoteException;
56 import org.apache.hadoop.net.NetUtils;
57 import org.apache.hadoop.security.Credentials;
58 import org.apache.hadoop.security.SecurityUtil;
59 import org.apache.hadoop.security.UserGroupInformation;
60 import org.apache.hadoop.security.token.Token;
61 import org.apache.hadoop.security.token.TokenIdentifier;
62 import org.apache.hadoop.util.Progressable;
63 import org.apache.hadoop.util.ServletUtil;
64 import org.xml.sax.Attributes;
65 import org.xml.sax.InputSource;
66 import org.xml.sax.SAXException;
67 import org.xml.sax.XMLReader;
68 import org.xml.sax.helpers.DefaultHandler;
69 import org.xml.sax.helpers.XMLReaderFactory;
70 
71 /**
72  * An implementation of a protocol for accessing filesystems over HTTP.
73  * The following implementation provides a limited, read-only interface
74  * to a filesystem over HTTP.
75  * @see org.apache.hadoop.hdfs.server.namenode.ListPathsServlet
76  * @see org.apache.hadoop.hdfs.server.namenode.FileDataServlet
77  */
78 @InterfaceAudience.Private
79 @InterfaceStability.Evolving
80 public class HftpFileSystem extends FileSystem
81     implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
82   public static final String SCHEME = "hftp";
83 
84   static {
85     HttpURLConnection.setFollowRedirects(true);
86   }
87 
88   URLConnectionFactory connectionFactory;
89 
90   public static final Text TOKEN_KIND = new Text("HFTP delegation");
91 
92   protected UserGroupInformation ugi;
93   private URI hftpURI;
94 
95   protected URI nnUri;
96 
97   public static final String HFTP_TIMEZONE = "UTC";
98   public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
99 
100   protected TokenAspect<? extends HftpFileSystem> tokenAspect;
101   private Token<?> delegationToken;
102   private Token<?> renewToken;
103   protected Text tokenServiceName;
104 
105   @Override
getCanonicalUri()106   public URI getCanonicalUri() {
107     return super.getCanonicalUri();
108   }
109 
getDateFormat()110   public static final SimpleDateFormat getDateFormat() {
111     final SimpleDateFormat df = new SimpleDateFormat(HFTP_DATE_FORMAT);
112     df.setTimeZone(TimeZone.getTimeZone(HFTP_TIMEZONE));
113     return df;
114   }
115 
116   protected static final ThreadLocal<SimpleDateFormat> df =
117     new ThreadLocal<SimpleDateFormat>() {
118     @Override
119     protected SimpleDateFormat initialValue() {
120       return getDateFormat();
121     }
122   };
123 
124   @Override
getDefaultPort()125   protected int getDefaultPort() {
126     return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY,
127         DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT);
128   }
129 
130   /**
131    *  We generate the address with one of the following ports, in
132    *  order of preference.
133    *  1. Port from the hftp URI e.g. hftp://namenode:4000/ will return 4000.
134    *  2. Port configured via DFS_NAMENODE_HTTP_PORT_KEY
135    *  3. DFS_NAMENODE_HTTP_PORT_DEFAULT i.e. 50070.
136    *
137    * @param uri
138    */
getNamenodeAddr(URI uri)139   protected InetSocketAddress getNamenodeAddr(URI uri) {
140     // use authority so user supplied uri can override port
141     return NetUtils.createSocketAddr(uri.getAuthority(), getDefaultPort());
142   }
143 
getNamenodeUri(URI uri)144   protected URI getNamenodeUri(URI uri) {
145     return DFSUtil.createUri(getUnderlyingProtocol(), getNamenodeAddr(uri));
146   }
147 
148   /**
149    * See the documentation of {@Link #getNamenodeAddr(URI)} for the logic
150    * behind selecting the canonical service name.
151    * @return
152    */
153   @Override
getCanonicalServiceName()154   public String getCanonicalServiceName() {
155     return SecurityUtil.buildTokenService(nnUri).toString();
156   }
157 
158   @Override
canonicalizeUri(URI uri)159   protected URI canonicalizeUri(URI uri) {
160     return NetUtils.getCanonicalUri(uri, getDefaultPort());
161   }
162 
163   /**
164    * Return the protocol scheme for the FileSystem.
165    * <p/>
166    *
167    * @return <code>hftp</code>
168    */
169   @Override
getScheme()170   public String getScheme() {
171     return SCHEME;
172   }
173 
174   /**
175    * Initialize connectionFactory and tokenAspect. This function is intended to
176    * be overridden by HsFtpFileSystem.
177    */
initTokenAspect()178   protected void initTokenAspect() {
179     tokenAspect = new TokenAspect<HftpFileSystem>(this, tokenServiceName, TOKEN_KIND);
180   }
181 
182   @Override
initialize(final URI name, final Configuration conf)183   public void initialize(final URI name, final Configuration conf)
184   throws IOException {
185     super.initialize(name, conf);
186     setConf(conf);
187     this.connectionFactory = URLConnectionFactory
188         .newDefaultURLConnectionFactory(conf);
189     this.ugi = UserGroupInformation.getCurrentUser();
190     this.nnUri = getNamenodeUri(name);
191     this.tokenServiceName = SecurityUtil.buildTokenService(nnUri);
192 
193     try {
194       this.hftpURI = new URI(name.getScheme(), name.getAuthority(),
195                              null, null, null);
196     } catch (URISyntaxException e) {
197       throw new IllegalArgumentException(e);
198     }
199 
200     initTokenAspect();
201     if (UserGroupInformation.isSecurityEnabled()) {
202       tokenAspect.initDelegationToken(ugi);
203     }
204   }
205 
206   @Override
getRenewToken()207   public Token<?> getRenewToken() {
208     return renewToken;
209   }
210 
211   /**
212    * Return the underlying protocol that is used to talk to the namenode.
213    */
getUnderlyingProtocol()214   protected String getUnderlyingProtocol() {
215     return "http";
216   }
217 
218   @Override
setDelegationToken(Token<T> token)219   public synchronized <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
220     /**
221      * XXX The kind of the token has been changed by DelegationTokenFetcher. We
222      * use the token for renewal, since the reflection utilities needs the value
223      * of the kind field to correctly renew the token.
224      *
225      * For other operations, however, the client has to send a
226      * HDFS_DELEGATION_KIND token over the wire so that it can talk to Hadoop
227      * 0.20.203 clusters. Later releases fix this problem. See HDFS-5440 for
228      * more details.
229      */
230     renewToken = token;
231     delegationToken = new Token<T>(token);
232     delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
233   }
234 
235   @Override
getDelegationToken(final String renewer)236   public synchronized Token<?> getDelegationToken(final String renewer)
237       throws IOException {
238     try {
239       // Renew TGT if needed
240       UserGroupInformation connectUgi = ugi.getRealUser();
241       final String proxyUser = connectUgi == null ? null : ugi
242           .getShortUserName();
243       if (connectUgi == null) {
244         connectUgi = ugi;
245       }
246       return connectUgi.doAs(new PrivilegedExceptionAction<Token<?>>() {
247         @Override
248         public Token<?> run() throws IOException {
249           Credentials c;
250           try {
251             c = DelegationTokenFetcher.getDTfromRemote(connectionFactory,
252                 nnUri, renewer, proxyUser);
253           } catch (IOException e) {
254             if (e.getCause() instanceof ConnectException) {
255               LOG.warn("Couldn't connect to " + nnUri +
256                   ", assuming security is disabled");
257               return null;
258             }
259             if (LOG.isDebugEnabled()) {
260               LOG.debug("Exception getting delegation token", e);
261             }
262             throw e;
263           }
264           for (Token<? extends TokenIdentifier> t : c.getAllTokens()) {
265             if(LOG.isDebugEnabled()) {
266               LOG.debug("Got dt for " + getUri() + ";t.service="
267                   +t.getService());
268             }
269             return t;
270           }
271           return null;
272         }
273       });
274     } catch (InterruptedException e) {
275       throw new RuntimeException(e);
276     }
277   }
278 
279   @Override
280   public URI getUri() {
281     return hftpURI;
282   }
283 
284   /**
285    * Return a URL pointing to given path on the namenode.
286    *
287    * @param path to obtain the URL for
288    * @param query string to append to the path
289    * @return namenode URL referring to the given path
290    * @throws IOException on error constructing the URL
291    */
292   protected URL getNamenodeURL(String path, String query) throws IOException {
293     final URL url = new URL(getUnderlyingProtocol(), nnUri.getHost(),
294           nnUri.getPort(), path + '?' + query);
295     if (LOG.isTraceEnabled()) {
296       LOG.trace("url=" + url);
297     }
298     return url;
299   }
300 
301   /**
302    * Get encoded UGI parameter string for a URL.
303    *
304    * @return user_shortname,group1,group2...
305    */
306   private String getEncodedUgiParameter() {
307     StringBuilder ugiParameter = new StringBuilder(
308         ServletUtil.encodeQueryValue(ugi.getShortUserName()));
309     for(String g: ugi.getGroupNames()) {
310       ugiParameter.append(",");
311       ugiParameter.append(ServletUtil.encodeQueryValue(g));
312     }
313     return ugiParameter.toString();
314   }
315 
316   /**
317    * Open an HTTP connection to the namenode to read file data and metadata.
318    * @param path The path component of the URL
319    * @param query The query component of the URL
320    */
321   protected HttpURLConnection openConnection(String path, String query)
322       throws IOException {
323     query = addDelegationTokenParam(query);
324     final URL url = getNamenodeURL(path, query);
325     final HttpURLConnection connection;
326     connection = (HttpURLConnection)connectionFactory.openConnection(url);
327     connection.setRequestMethod("GET");
328     connection.connect();
329     return connection;
330   }
331 
332   protected String addDelegationTokenParam(String query) throws IOException {
333     String tokenString = null;
334     if (UserGroupInformation.isSecurityEnabled()) {
335       synchronized (this) {
336         tokenAspect.ensureTokenInitialized();
337         if (delegationToken != null) {
338           tokenString = delegationToken.encodeToUrlString();
339           return (query + JspHelper.getDelegationTokenUrlParam(tokenString));
340         }
341       }
342     }
343     return query;
344   }
345 
346   static class RangeHeaderUrlOpener extends ByteRangeInputStream.URLOpener {
347     private final URLConnectionFactory connFactory;
348 
349     RangeHeaderUrlOpener(URLConnectionFactory connFactory, final URL url) {
350       super(url);
351       this.connFactory = connFactory;
352     }
353 
354     protected HttpURLConnection openConnection() throws IOException {
355       return (HttpURLConnection)connFactory.openConnection(url);
356     }
357 
358     /** Use HTTP Range header for specifying offset. */
359     @Override
360     protected HttpURLConnection connect(final long offset,
361         final boolean resolved) throws IOException {
362       final HttpURLConnection conn = openConnection();
363       conn.setRequestMethod("GET");
364       if (offset != 0L) {
365         conn.setRequestProperty("Range", "bytes=" + offset + "-");
366       }
367       conn.connect();
368 
369       //Expects HTTP_OK or HTTP_PARTIAL response codes.
370       final int code = conn.getResponseCode();
371       if (offset != 0L && code != HttpURLConnection.HTTP_PARTIAL) {
372         throw new IOException("HTTP_PARTIAL expected, received " + code);
373       } else if (offset == 0L && code != HttpURLConnection.HTTP_OK) {
374         throw new IOException("HTTP_OK expected, received " + code);
375       }
376       return conn;
377     }
378   }
379 
380   static class RangeHeaderInputStream extends ByteRangeInputStream {
381     RangeHeaderInputStream(RangeHeaderUrlOpener o, RangeHeaderUrlOpener r)
382     throws IOException {
383       super(o, r);
384     }
385 
386     RangeHeaderInputStream(URLConnectionFactory connFactory, final URL url)
387     throws IOException {
388       this(new RangeHeaderUrlOpener(connFactory, url),
389           new RangeHeaderUrlOpener(connFactory, null));
390     }
391 
392     @Override
393     protected URL getResolvedUrl(final HttpURLConnection connection) {
394       return connection.getURL();
395     }
396   }
397 
398   @Override
399   public FSDataInputStream open(Path f, int buffersize) throws IOException {
400     f = f.makeQualified(getUri(), getWorkingDirectory());
401     String path = "/data" + ServletUtil.encodePath(f.toUri().getPath());
402     String query = addDelegationTokenParam("ugi=" + getEncodedUgiParameter());
403     URL u = getNamenodeURL(path, query);
404     return new FSDataInputStream(new RangeHeaderInputStream(connectionFactory, u));
405   }
406 
407   @Override
408   public void close() throws IOException {
409     super.close();
410     tokenAspect.removeRenewAction();
411   }
412 
413   /** Class to parse and store a listing reply from the server. */
414   class LsParser extends DefaultHandler {
415 
416     final ArrayList<FileStatus> fslist = new ArrayList<FileStatus>();
417 
418     @Override
419     public void startElement(String ns, String localname, String qname,
420                 Attributes attrs) throws SAXException {
421       if ("listing".equals(qname)) return;
422       if (!"file".equals(qname) && !"directory".equals(qname)) {
423         if (RemoteException.class.getSimpleName().equals(qname)) {
424           throw new SAXException(RemoteException.valueOf(attrs));
425         }
426         throw new SAXException("Unrecognized entry: " + qname);
427       }
428       long modif;
429       long atime = 0;
430       try {
431         final SimpleDateFormat ldf = df.get();
432         modif = ldf.parse(attrs.getValue("modified")).getTime();
433         String astr = attrs.getValue("accesstime");
434         if (astr != null) {
435           atime = ldf.parse(astr).getTime();
436         }
437       } catch (ParseException e) { throw new SAXException(e); }
438       FileStatus fs = "file".equals(qname)
439         ? new FileStatus(
440               Long.parseLong(attrs.getValue("size")), false,
441               Short.valueOf(attrs.getValue("replication")).shortValue(),
442               Long.parseLong(attrs.getValue("blocksize")),
443               modif, atime, FsPermission.valueOf(attrs.getValue("permission")),
444               attrs.getValue("owner"), attrs.getValue("group"),
445               HftpFileSystem.this.makeQualified(
446                   new Path(getUri().toString(), attrs.getValue("path"))))
447         : new FileStatus(0L, true, 0, 0L,
448               modif, atime, FsPermission.valueOf(attrs.getValue("permission")),
449               attrs.getValue("owner"), attrs.getValue("group"),
450               HftpFileSystem.this.makeQualified(
451                   new Path(getUri().toString(), attrs.getValue("path"))));
452       fslist.add(fs);
453     }
454 
455     private void fetchList(String path, boolean recur) throws IOException {
456       try {
457         XMLReader xr = XMLReaderFactory.createXMLReader();
458         xr.setContentHandler(this);
459         HttpURLConnection connection = openConnection(
460             "/listPaths" + ServletUtil.encodePath(path),
461             "ugi=" + getEncodedUgiParameter() + (recur ? "&recursive=yes" : ""));
462         InputStream resp = connection.getInputStream();
463         xr.parse(new InputSource(resp));
464       } catch(SAXException e) {
465         final Exception embedded = e.getException();
466         if (embedded != null && embedded instanceof IOException) {
467           throw (IOException)embedded;
468         }
469         throw new IOException("invalid xml directory content", e);
470       }
471     }
472 
473     public FileStatus getFileStatus(Path f) throws IOException {
474       fetchList(f.toUri().getPath(), false);
475       if (fslist.size() == 0) {
476         throw new FileNotFoundException("File does not exist: " + f);
477       }
478       return fslist.get(0);
479     }
480 
481     public FileStatus[] listStatus(Path f, boolean recur) throws IOException {
482       fetchList(f.toUri().getPath(), recur);
483       if (fslist.size() > 0 && (fslist.size() != 1 || fslist.get(0).isDirectory())) {
484         fslist.remove(0);
485       }
486       return fslist.toArray(new FileStatus[0]);
487     }
488 
489     public FileStatus[] listStatus(Path f) throws IOException {
490       return listStatus(f, false);
491     }
492   }
493 
494   @Override
495   public FileStatus[] listStatus(Path f) throws IOException {
496     LsParser lsparser = new LsParser();
497     return lsparser.listStatus(f);
498   }
499 
500   @Override
501   public FileStatus getFileStatus(Path f) throws IOException {
502     LsParser lsparser = new LsParser();
503     return lsparser.getFileStatus(f);
504   }
505 
506   private class ChecksumParser extends DefaultHandler {
507     private FileChecksum filechecksum;
508 
509     @Override
510     public void startElement(String ns, String localname, String qname,
511                 Attributes attrs) throws SAXException {
512       if (!MD5MD5CRC32FileChecksum.class.getName().equals(qname)) {
513         if (RemoteException.class.getSimpleName().equals(qname)) {
514           throw new SAXException(RemoteException.valueOf(attrs));
515         }
516         throw new SAXException("Unrecognized entry: " + qname);
517       }
518 
519       filechecksum = MD5MD5CRC32FileChecksum.valueOf(attrs);
520     }
521 
522     private FileChecksum getFileChecksum(String f) throws IOException {
523       final HttpURLConnection connection = openConnection(
524           "/fileChecksum" + ServletUtil.encodePath(f),
525           "ugi=" + getEncodedUgiParameter());
526       try {
527         final XMLReader xr = XMLReaderFactory.createXMLReader();
528         xr.setContentHandler(this);
529         xr.parse(new InputSource(connection.getInputStream()));
530       } catch(SAXException e) {
531         final Exception embedded = e.getException();
532         if (embedded != null && embedded instanceof IOException) {
533           throw (IOException)embedded;
534         }
535         throw new IOException("invalid xml directory content", e);
536       } finally {
537         connection.disconnect();
538       }
539       return filechecksum;
540     }
541   }
542 
543   @Override
544   public FileChecksum getFileChecksum(Path f) throws IOException {
545     final String s = makeQualified(f).toUri().getPath();
546     return new ChecksumParser().getFileChecksum(s);
547   }
548 
549   @Override
550   public Path getWorkingDirectory() {
551     return new Path("/").makeQualified(getUri(), null);
552   }
553 
554   @Override
555   public void setWorkingDirectory(Path f) { }
556 
557   /** This optional operation is not yet supported. */
558   @Override
559   public FSDataOutputStream append(Path f, int bufferSize,
560       Progressable progress) throws IOException {
561     throw new IOException("Not supported");
562   }
563 
564   @Override
565   public FSDataOutputStream create(Path f, FsPermission permission,
566       boolean overwrite, int bufferSize, short replication,
567       long blockSize, Progressable progress) throws IOException {
568     throw new IOException("Not supported");
569   }
570 
571   @Override
572   public boolean rename(Path src, Path dst) throws IOException {
573     throw new IOException("Not supported");
574   }
575 
576   @Override
577   public boolean delete(Path f, boolean recursive) throws IOException {
578     throw new IOException("Not supported");
579   }
580 
581   @Override
582   public boolean mkdirs(Path f, FsPermission permission) throws IOException {
583     throw new IOException("Not supported");
584   }
585 
586   /**
587    * A parser for parsing {@link ContentSummary} xml.
588    */
589   private class ContentSummaryParser extends DefaultHandler {
590     private ContentSummary contentsummary;
591 
592     @Override
593     public void startElement(String ns, String localname, String qname,
594                 Attributes attrs) throws SAXException {
595       if (!ContentSummary.class.getName().equals(qname)) {
596         if (RemoteException.class.getSimpleName().equals(qname)) {
597           throw new SAXException(RemoteException.valueOf(attrs));
598         }
599         throw new SAXException("Unrecognized entry: " + qname);
600       }
601 
602       contentsummary = toContentSummary(attrs);
603     }
604 
605     /**
606      * Connect to the name node and get content summary.
607      * @param path The path
608      * @return The content summary for the path.
609      * @throws IOException
610      */
611     private ContentSummary getContentSummary(String path) throws IOException {
612       final HttpURLConnection connection = openConnection(
613           "/contentSummary" + ServletUtil.encodePath(path),
614           "ugi=" + getEncodedUgiParameter());
615       InputStream in = null;
616       try {
617         in = connection.getInputStream();
618 
619         final XMLReader xr = XMLReaderFactory.createXMLReader();
620         xr.setContentHandler(this);
621         xr.parse(new InputSource(in));
622       } catch(FileNotFoundException fnfe) {
623         //the server may not support getContentSummary
624         return null;
625       } catch(SAXException saxe) {
626         final Exception embedded = saxe.getException();
627         if (embedded != null && embedded instanceof IOException) {
628           throw (IOException)embedded;
629         }
630         throw new IOException("Invalid xml format", saxe);
631       } finally {
632         if (in != null) {
633           in.close();
634         }
635         connection.disconnect();
636       }
637       return contentsummary;
638     }
639   }
640 
641   /** Return the object represented in the attributes. */
642   private static ContentSummary toContentSummary(Attributes attrs
643       ) throws SAXException {
644     final String length = attrs.getValue("length");
645     final String fileCount = attrs.getValue("fileCount");
646     final String directoryCount = attrs.getValue("directoryCount");
647     final String quota = attrs.getValue("quota");
648     final String spaceConsumed = attrs.getValue("spaceConsumed");
649     final String spaceQuota = attrs.getValue("spaceQuota");
650 
651     if (length == null
652         || fileCount == null
653         || directoryCount == null
654         || quota == null
655         || spaceConsumed == null
656         || spaceQuota == null) {
657       return null;
658     }
659 
660     try {
661       return new ContentSummary(
662           Long.parseLong(length),
663           Long.parseLong(fileCount),
664           Long.parseLong(directoryCount),
665           Long.parseLong(quota),
666           Long.parseLong(spaceConsumed),
667           Long.parseLong(spaceQuota));
668     } catch(Exception e) {
669       throw new SAXException("Invalid attributes: length=" + length
670           + ", fileCount=" + fileCount
671           + ", directoryCount=" + directoryCount
672           + ", quota=" + quota
673           + ", spaceConsumed=" + spaceConsumed
674           + ", spaceQuota=" + spaceQuota, e);
675     }
676   }
677 
678   @Override
679   public ContentSummary getContentSummary(Path f) throws IOException {
680     final String s = makeQualified(f).toUri().getPath();
681     final ContentSummary cs = new ContentSummaryParser().getContentSummary(s);
682     return cs != null? cs: super.getContentSummary(f);
683   }
684 
685   @SuppressWarnings("unchecked")
686   @Override
687   public long renewDelegationToken(final Token<?> token) throws IOException {
688     // update the kerberos credentials, if they are coming from a keytab
689     UserGroupInformation connectUgi = ugi.getRealUser();
690     if (connectUgi == null) {
691       connectUgi = ugi;
692     }
693     try {
694       return connectUgi.doAs(new PrivilegedExceptionAction<Long>() {
695         @Override
696         public Long run() throws Exception {
697           InetSocketAddress serviceAddr = SecurityUtil
698               .getTokenServiceAddr(token);
699           return DelegationTokenFetcher.renewDelegationToken(connectionFactory,
700               DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr),
701               (Token<DelegationTokenIdentifier>) token);
702         }
703       });
704     } catch (InterruptedException e) {
705       throw new IOException(e);
706     }
707   }
708 
709   @SuppressWarnings("unchecked")
710   @Override
711   public void cancelDelegationToken(final Token<?> token) throws IOException {
712     UserGroupInformation connectUgi = ugi.getRealUser();
713     if (connectUgi == null) {
714       connectUgi = ugi;
715     }
716     try {
717       connectUgi.doAs(new PrivilegedExceptionAction<Void>() {
718         @Override
719         public Void run() throws Exception {
720           InetSocketAddress serviceAddr = SecurityUtil
721               .getTokenServiceAddr(token);
722           DelegationTokenFetcher.cancelDelegationToken(connectionFactory,
723               DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr),
724               (Token<DelegationTokenIdentifier>) token);
725           return null;
726         }
727       });
728     } catch (InterruptedException e) {
729       throw new IOException(e);
730     }
731   }
732 }
733