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    package org.apache.hadoop.fs.http.server;
019    
020    import org.apache.hadoop.classification.InterfaceAudience;
021    import org.apache.hadoop.fs.ContentSummary;
022    import org.apache.hadoop.fs.FileChecksum;
023    import org.apache.hadoop.fs.FileStatus;
024    import org.apache.hadoop.fs.FileSystem;
025    import org.apache.hadoop.fs.GlobFilter;
026    import org.apache.hadoop.fs.Path;
027    import org.apache.hadoop.fs.PathFilter;
028    import org.apache.hadoop.fs.XAttrCodec;
029    import org.apache.hadoop.fs.XAttrSetFlag;
030    import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
031    import org.apache.hadoop.fs.permission.AclEntry;
032    import org.apache.hadoop.fs.permission.AclStatus;
033    import org.apache.hadoop.fs.permission.FsPermission;
034    import org.apache.hadoop.hdfs.protocol.AclException;
035    import org.apache.hadoop.io.IOUtils;
036    import org.apache.hadoop.lib.service.FileSystemAccess;
037    import org.json.simple.JSONArray;
038    import org.json.simple.JSONObject;
039    
040    import java.io.IOException;
041    import java.io.InputStream;
042    import java.io.OutputStream;
043    import java.util.EnumSet;
044    import java.util.LinkedHashMap;
045    import java.util.List;
046    import java.util.Map;
047    import java.util.Map.Entry;
048    
049    /**
050     * FileSystem operation executors used by {@link HttpFSServer}.
051     */
052    @InterfaceAudience.Private
053    public class FSOperations {
054    
055      /**
056       * This class is used to group a FileStatus and an AclStatus together.
057       * It's needed for the GETFILESTATUS and LISTSTATUS calls, which take
058       * most info from the FileStatus and a wee bit from the AclStatus.
059       */
060      private static class StatusPair {
061        private FileStatus fileStatus;
062        private AclStatus aclStatus;
063    
064        /**
065         * Simple constructor
066         * @param fileStatus Existing FileStatus object
067         * @param aclStatus Existing AclStatus object
068         */
069        public StatusPair(FileStatus fileStatus, AclStatus aclStatus) {
070          this.fileStatus = fileStatus;
071          this.aclStatus = aclStatus;
072        }
073    
074        /**
075         * Create one StatusPair by performing the underlying calls to
076         * fs.getFileStatus and fs.getAclStatus
077         * @param fs The FileSystem where 'path' lives
078         * @param path The file/directory to query
079         * @throws IOException
080         */
081        public StatusPair(FileSystem fs, Path path) throws IOException {
082          fileStatus = fs.getFileStatus(path);
083          aclStatus = null;
084          try {
085            aclStatus = fs.getAclStatus(path);
086          } catch (AclException e) {
087            /*
088             * The cause is almost certainly an "ACLS aren't enabled"
089             * exception, so leave aclStatus at null and carry on.
090             */
091          } catch (UnsupportedOperationException e) {
092            /* Ditto above - this is the case for a local file system */
093          }
094        }
095    
096        /**
097         * Return a Map suitable for conversion into JSON format
098         * @return The JSONish Map
099         */
100        public Map<String,Object> toJson() {
101          Map<String,Object> json = new LinkedHashMap<String,Object>();
102          json.put(HttpFSFileSystem.FILE_STATUS_JSON, toJsonInner(true));
103          return json;
104        }
105    
106        /**
107         * Return in inner part of the JSON for the status - used by both the
108         * GETFILESTATUS and LISTSTATUS calls.
109         * @param emptyPathSuffix Whether or not to include PATH_SUFFIX_JSON
110         * @return The JSONish Map
111         */
112        public Map<String,Object> toJsonInner(boolean emptyPathSuffix) {
113          Map<String,Object> json = new LinkedHashMap<String,Object>();
114          json.put(HttpFSFileSystem.PATH_SUFFIX_JSON,
115                  (emptyPathSuffix) ? "" : fileStatus.getPath().getName());
116          json.put(HttpFSFileSystem.TYPE_JSON,
117                  HttpFSFileSystem.FILE_TYPE.getType(fileStatus).toString());
118          json.put(HttpFSFileSystem.LENGTH_JSON, fileStatus.getLen());
119          json.put(HttpFSFileSystem.OWNER_JSON, fileStatus.getOwner());
120          json.put(HttpFSFileSystem.GROUP_JSON, fileStatus.getGroup());
121          json.put(HttpFSFileSystem.PERMISSION_JSON,
122                  HttpFSFileSystem.permissionToString(fileStatus.getPermission()));
123          json.put(HttpFSFileSystem.ACCESS_TIME_JSON, fileStatus.getAccessTime());
124          json.put(HttpFSFileSystem.MODIFICATION_TIME_JSON,
125                  fileStatus.getModificationTime());
126          json.put(HttpFSFileSystem.BLOCK_SIZE_JSON, fileStatus.getBlockSize());
127          json.put(HttpFSFileSystem.REPLICATION_JSON, fileStatus.getReplication());
128          if ( (aclStatus != null) && !(aclStatus.getEntries().isEmpty()) ) {
129            json.put(HttpFSFileSystem.ACL_BIT_JSON,true);
130          }
131          return json;
132        }
133      }
134    
135      /**
136       * Simple class used to contain and operate upon a list of StatusPair
137       * objects.  Used by LISTSTATUS.
138       */
139      private static class StatusPairs {
140        private StatusPair[] statusPairs;
141    
142        /**
143         * Construct a list of StatusPair objects
144         * @param fs The FileSystem where 'path' lives
145         * @param path The directory to query
146         * @param filter A possible filter for entries in the directory
147         * @throws IOException
148         */
149        public StatusPairs(FileSystem fs, Path path, PathFilter filter)
150                throws IOException {
151          /* Grab all the file statuses at once in an array */
152          FileStatus[] fileStatuses = fs.listStatus(path, filter);
153    
154          /* We'll have an array of StatusPairs of the same length */
155          AclStatus aclStatus = null;
156          statusPairs = new StatusPair[fileStatuses.length];
157    
158          /*
159           * For each FileStatus, attempt to acquire an AclStatus.  If the
160           * getAclStatus throws an exception, we assume that ACLs are turned
161           * off entirely and abandon the attempt.
162           */
163          boolean useAcls = true;   // Assume ACLs work until proven otherwise
164          for (int i = 0; i < fileStatuses.length; i++) {
165            if (useAcls) {
166              try {
167                aclStatus = fs.getAclStatus(fileStatuses[i].getPath());
168              } catch (AclException e) {
169                /* Almost certainly due to an "ACLs not enabled" exception */
170                aclStatus = null;
171                useAcls = false;
172              } catch (UnsupportedOperationException e) {
173                /* Ditto above - this is the case for a local file system */
174                aclStatus = null;
175                useAcls = false;
176              }
177            }
178            statusPairs[i] = new StatusPair(fileStatuses[i], aclStatus);
179          }
180        }
181    
182        /**
183         * Return a Map suitable for conversion into JSON.
184         * @return A JSONish Map
185         */
186        @SuppressWarnings({"unchecked"})
187        public Map<String,Object> toJson() {
188          Map<String,Object> json = new LinkedHashMap<String,Object>();
189          Map<String,Object> inner = new LinkedHashMap<String,Object>();
190          JSONArray statuses = new JSONArray();
191          for (StatusPair s : statusPairs) {
192            statuses.add(s.toJsonInner(false));
193          }
194          inner.put(HttpFSFileSystem.FILE_STATUS_JSON, statuses);
195          json.put(HttpFSFileSystem.FILE_STATUSES_JSON, inner);
196          return json;
197        }
198      }
199    
200      /** Converts an <code>AclStatus</code> object into a JSON object.
201       *
202       * @param aclStatus AclStatus object
203       *
204       * @return The JSON representation of the ACLs for the file
205       */
206      @SuppressWarnings({"unchecked"})
207      private static Map<String,Object> aclStatusToJSON(AclStatus aclStatus) {
208        Map<String,Object> json = new LinkedHashMap<String,Object>();
209        Map<String,Object> inner = new LinkedHashMap<String,Object>();
210        JSONArray entriesArray = new JSONArray();
211        inner.put(HttpFSFileSystem.OWNER_JSON, aclStatus.getOwner());
212        inner.put(HttpFSFileSystem.GROUP_JSON, aclStatus.getGroup());
213        inner.put(HttpFSFileSystem.ACL_STICKY_BIT_JSON, aclStatus.isStickyBit());
214        for ( AclEntry e : aclStatus.getEntries() ) {
215          entriesArray.add(e.toString());
216        }
217        inner.put(HttpFSFileSystem.ACL_ENTRIES_JSON, entriesArray);
218        json.put(HttpFSFileSystem.ACL_STATUS_JSON, inner);
219        return json;
220      }
221    
222      /**
223       * Converts a <code>FileChecksum</code> object into a JSON array
224       * object.
225       *
226       * @param checksum file checksum.
227       *
228       * @return The JSON representation of the file checksum.
229       */
230      @SuppressWarnings({"unchecked"})
231      private static Map fileChecksumToJSON(FileChecksum checksum) {
232        Map json = new LinkedHashMap();
233        json.put(HttpFSFileSystem.CHECKSUM_ALGORITHM_JSON, checksum.getAlgorithmName());
234        json.put(HttpFSFileSystem.CHECKSUM_BYTES_JSON,
235                 org.apache.hadoop.util.StringUtils.byteToHexString(checksum.getBytes()));
236        json.put(HttpFSFileSystem.CHECKSUM_LENGTH_JSON, checksum.getLength());
237        Map response = new LinkedHashMap();
238        response.put(HttpFSFileSystem.FILE_CHECKSUM_JSON, json);
239        return response;
240      }
241    
242      /**
243       * Converts xAttrs to a JSON object.
244       *
245       * @param xAttrs file xAttrs.
246       * @param encoding format of xattr values.
247       *
248       * @return The JSON representation of the xAttrs.
249       * @throws IOException 
250       */
251      @SuppressWarnings({"unchecked", "rawtypes"})
252      private static Map xAttrsToJSON(Map<String, byte[]> xAttrs, 
253          XAttrCodec encoding) throws IOException {
254        Map jsonMap = new LinkedHashMap();
255        JSONArray jsonArray = new JSONArray();
256        if (xAttrs != null) {
257          for (Entry<String, byte[]> e : xAttrs.entrySet()) {
258            Map json = new LinkedHashMap();
259            json.put(HttpFSFileSystem.XATTR_NAME_JSON, e.getKey());
260            if (e.getValue() != null) {
261              json.put(HttpFSFileSystem.XATTR_VALUE_JSON, 
262                  XAttrCodec.encodeValue(e.getValue(), encoding));
263            }
264            jsonArray.add(json);
265          }
266        }
267        jsonMap.put(HttpFSFileSystem.XATTRS_JSON, jsonArray);
268        return jsonMap;
269      }
270    
271      /**
272       * Converts xAttr names to a JSON object.
273       *
274       * @param names file xAttr names.
275       *
276       * @return The JSON representation of the xAttr names.
277       * @throws IOException 
278       */
279      @SuppressWarnings({"unchecked", "rawtypes"})
280      private static Map xAttrNamesToJSON(List<String> names) throws IOException {
281        Map jsonMap = new LinkedHashMap();
282        jsonMap.put(HttpFSFileSystem.XATTRNAMES_JSON, JSONArray.toJSONString(names));
283        return jsonMap;
284      }
285    
286      /**
287       * Converts a <code>ContentSummary</code> object into a JSON array
288       * object.
289       *
290       * @param contentSummary the content summary
291       *
292       * @return The JSON representation of the content summary.
293       */
294      @SuppressWarnings({"unchecked"})
295      private static Map contentSummaryToJSON(ContentSummary contentSummary) {
296        Map json = new LinkedHashMap();
297        json.put(HttpFSFileSystem.CONTENT_SUMMARY_DIRECTORY_COUNT_JSON, contentSummary.getDirectoryCount());
298        json.put(HttpFSFileSystem.CONTENT_SUMMARY_FILE_COUNT_JSON, contentSummary.getFileCount());
299        json.put(HttpFSFileSystem.CONTENT_SUMMARY_LENGTH_JSON, contentSummary.getLength());
300        json.put(HttpFSFileSystem.CONTENT_SUMMARY_QUOTA_JSON, contentSummary.getQuota());
301        json.put(HttpFSFileSystem.CONTENT_SUMMARY_SPACE_CONSUMED_JSON, contentSummary.getSpaceConsumed());
302        json.put(HttpFSFileSystem.CONTENT_SUMMARY_SPACE_QUOTA_JSON, contentSummary.getSpaceQuota());
303        Map response = new LinkedHashMap();
304        response.put(HttpFSFileSystem.CONTENT_SUMMARY_JSON, json);
305        return response;
306      }
307    
308      /**
309       * Converts an object into a Json Map with with one key-value entry.
310       * <p/>
311       * It assumes the given value is either a JSON primitive type or a
312       * <code>JsonAware</code> instance.
313       *
314       * @param name name for the key of the entry.
315       * @param value for the value of the entry.
316       *
317       * @return the JSON representation of the key-value pair.
318       */
319      @SuppressWarnings("unchecked")
320      private static JSONObject toJSON(String name, Object value) {
321        JSONObject json = new JSONObject();
322        json.put(name, value);
323        return json;
324      }
325    
326      /**
327       * Executor that performs an append FileSystemAccess files system operation.
328       */
329      @InterfaceAudience.Private
330      public static class FSAppend implements FileSystemAccess.FileSystemExecutor<Void> {
331        private InputStream is;
332        private Path path;
333    
334        /**
335         * Creates an Append executor.
336         *
337         * @param is input stream to append.
338         * @param path path of the file to append.
339         */
340        public FSAppend(InputStream is, String path) {
341          this.is = is;
342          this.path = new Path(path);
343        }
344    
345        /**
346         * Executes the filesystem operation.
347         *
348         * @param fs filesystem instance to use.
349         *
350         * @return void.
351         *
352         * @throws IOException thrown if an IO error occured.
353         */
354        @Override
355        public Void execute(FileSystem fs) throws IOException {
356          int bufferSize = fs.getConf().getInt("httpfs.buffer.size", 4096);
357          OutputStream os = fs.append(path, bufferSize);
358          IOUtils.copyBytes(is, os, bufferSize, true);
359          os.close();
360          return null;
361        }
362    
363      }
364    
365      /**
366       * Executor that performs an append FileSystemAccess files system operation.
367       */
368      @InterfaceAudience.Private
369      public static class FSConcat implements FileSystemAccess.FileSystemExecutor<Void> {
370        private Path path;
371        private Path[] sources;
372    
373        /**
374         * Creates a Concat executor.
375         *
376         * @param path target path to concat to.
377         * @param sources comma seperated absolute paths to use as sources.
378         */
379        public FSConcat(String path, String[] sources) {
380          this.sources = new Path[sources.length];
381    
382          for(int i = 0; i < sources.length; i++) {
383            this.sources[i] = new Path(sources[i]);
384          }
385    
386          this.path = new Path(path);
387        }
388    
389        /**
390         * Executes the filesystem operation.
391         *
392         * @param fs filesystem instance to use.
393         *
394         * @return void.
395         *
396         * @throws IOException thrown if an IO error occured.
397         */
398        @Override
399        public Void execute(FileSystem fs) throws IOException {
400          fs.concat(path, sources);
401          return null;
402        }
403    
404      }
405    
406      /**
407       * Executor that performs a content-summary FileSystemAccess files system operation.
408       */
409      @InterfaceAudience.Private
410      public static class FSContentSummary implements FileSystemAccess.FileSystemExecutor<Map> {
411        private Path path;
412    
413        /**
414         * Creates a content-summary executor.
415         *
416         * @param path the path to retrieve the content-summary.
417         */
418        public FSContentSummary(String path) {
419          this.path = new Path(path);
420        }
421    
422        /**
423         * Executes the filesystem operation.
424         *
425         * @param fs filesystem instance to use.
426         *
427         * @return a Map object (JSON friendly) with the content-summary.
428         *
429         * @throws IOException thrown if an IO error occured.
430         */
431        @Override
432        public Map execute(FileSystem fs) throws IOException {
433          ContentSummary contentSummary = fs.getContentSummary(path);
434          return contentSummaryToJSON(contentSummary);
435        }
436    
437      }
438    
439      /**
440       * Executor that performs a create FileSystemAccess files system operation.
441       */
442      @InterfaceAudience.Private
443      public static class FSCreate implements FileSystemAccess.FileSystemExecutor<Void> {
444        private InputStream is;
445        private Path path;
446        private short permission;
447        private boolean override;
448        private short replication;
449        private long blockSize;
450    
451        /**
452         * Creates a Create executor.
453         *
454         * @param is input stream to for the file to create.
455         * @param path path of the file to create.
456         * @param perm permission for the file.
457         * @param override if the file should be overriden if it already exist.
458         * @param repl the replication factor for the file.
459         * @param blockSize the block size for the file.
460         */
461        public FSCreate(InputStream is, String path, short perm, boolean override,
462                        short repl, long blockSize) {
463          this.is = is;
464          this.path = new Path(path);
465          this.permission = perm;
466          this.override = override;
467          this.replication = repl;
468          this.blockSize = blockSize;
469        }
470    
471        /**
472         * Executes the filesystem operation.
473         *
474         * @param fs filesystem instance to use.
475         *
476         * @return The URI of the created file.
477         *
478         * @throws IOException thrown if an IO error occured.
479         */
480        @Override
481        public Void execute(FileSystem fs) throws IOException {
482          if (replication == -1) {
483            replication = fs.getDefaultReplication(path);
484          }
485          if (blockSize == -1) {
486            blockSize = fs.getDefaultBlockSize(path);
487          }
488          FsPermission fsPermission = new FsPermission(permission);
489          int bufferSize = fs.getConf().getInt("httpfs.buffer.size", 4096);
490          OutputStream os = fs.create(path, fsPermission, override, bufferSize, replication, blockSize, null);
491          IOUtils.copyBytes(is, os, bufferSize, true);
492          os.close();
493          return null;
494        }
495    
496      }
497    
498      /**
499       * Executor that performs a delete FileSystemAccess files system operation.
500       */
501      @InterfaceAudience.Private
502      public static class FSDelete implements FileSystemAccess.FileSystemExecutor<JSONObject> {
503        private Path path;
504        private boolean recursive;
505    
506        /**
507         * Creates a Delete executor.
508         *
509         * @param path path to delete.
510         * @param recursive if the delete should be recursive or not.
511         */
512        public FSDelete(String path, boolean recursive) {
513          this.path = new Path(path);
514          this.recursive = recursive;
515        }
516    
517        /**
518         * Executes the filesystem operation.
519         *
520         * @param fs filesystem instance to use.
521         *
522         * @return <code>true</code> if the delete operation was successful,
523         *         <code>false</code> otherwise.
524         *
525         * @throws IOException thrown if an IO error occured.
526         */
527        @Override
528        public JSONObject execute(FileSystem fs) throws IOException {
529          boolean deleted = fs.delete(path, recursive);
530          return toJSON(HttpFSFileSystem.DELETE_JSON.toLowerCase(), deleted);
531        }
532    
533      }
534    
535      /**
536       * Executor that performs a file-checksum FileSystemAccess files system operation.
537       */
538      @InterfaceAudience.Private
539      public static class FSFileChecksum implements FileSystemAccess.FileSystemExecutor<Map> {
540        private Path path;
541    
542        /**
543         * Creates a file-checksum executor.
544         *
545         * @param path the path to retrieve the checksum.
546         */
547        public FSFileChecksum(String path) {
548          this.path = new Path(path);
549        }
550    
551        /**
552         * Executes the filesystem operation.
553         *
554         * @param fs filesystem instance to use.
555         *
556         * @return a Map object (JSON friendly) with the file checksum.
557         *
558         * @throws IOException thrown if an IO error occured.
559         */
560        @Override
561        public Map execute(FileSystem fs) throws IOException {
562          FileChecksum checksum = fs.getFileChecksum(path);
563          return fileChecksumToJSON(checksum);
564        }
565    
566      }
567    
568      /**
569       * Executor that performs a file-status FileSystemAccess files system operation.
570       */
571      @InterfaceAudience.Private
572      public static class FSFileStatus implements FileSystemAccess.FileSystemExecutor<Map> {
573        private Path path;
574    
575        /**
576         * Creates a file-status executor.
577         *
578         * @param path the path to retrieve the status.
579         */
580        public FSFileStatus(String path) {
581          this.path = new Path(path);
582        }
583    
584        /**
585         * Executes the filesystem getFileStatus operation and returns the
586         * result in a JSONish Map.
587         *
588         * @param fs filesystem instance to use.
589         *
590         * @return a Map object (JSON friendly) with the file status.
591         *
592         * @throws IOException thrown if an IO error occurred.
593         */
594        @Override
595        public Map execute(FileSystem fs) throws IOException {
596          StatusPair sp = new StatusPair(fs, path);
597          return sp.toJson();
598        }
599    
600      }
601    
602      /**
603       * Executor that performs a home-dir FileSystemAccess files system operation.
604       */
605      @InterfaceAudience.Private
606      public static class FSHomeDir implements FileSystemAccess.FileSystemExecutor<JSONObject> {
607    
608        /**
609         * Executes the filesystem operation.
610         *
611         * @param fs filesystem instance to use.
612         *
613         * @return a JSON object with the user home directory.
614         *
615         * @throws IOException thrown if an IO error occured.
616         */
617        @Override
618        @SuppressWarnings("unchecked")
619        public JSONObject execute(FileSystem fs) throws IOException {
620          Path homeDir = fs.getHomeDirectory();
621          JSONObject json = new JSONObject();
622          json.put(HttpFSFileSystem.HOME_DIR_JSON, homeDir.toUri().getPath());
623          return json;
624        }
625    
626      }
627    
628      /**
629       * Executor that performs a list-status FileSystemAccess files system operation.
630       */
631      @InterfaceAudience.Private
632      public static class FSListStatus implements FileSystemAccess.FileSystemExecutor<Map>, PathFilter {
633        private Path path;
634        private PathFilter filter;
635    
636        /**
637         * Creates a list-status executor.
638         *
639         * @param path the directory to retrieve the status of its contents.
640         * @param filter glob filter to use.
641         *
642         * @throws IOException thrown if the filter expression is incorrect.
643         */
644        public FSListStatus(String path, String filter) throws IOException {
645          this.path = new Path(path);
646          this.filter = (filter == null) ? this : new GlobFilter(filter);
647        }
648    
649        /**
650         * Returns data for a JSON Map containing the information for
651         * the set of files in 'path' that match 'filter'.
652         *
653         * @param fs filesystem instance to use.
654         *
655         * @return a Map with the file status of the directory
656         *         contents that match the filter
657         *
658         * @throws IOException thrown if an IO error occurred.
659         */
660        @Override
661        public Map execute(FileSystem fs) throws IOException {
662          StatusPairs sp = new StatusPairs(fs, path, filter);
663          return sp.toJson();
664        }
665    
666        @Override
667        public boolean accept(Path path) {
668          return true;
669        }
670    
671      }
672    
673      /**
674       * Executor that performs a mkdirs FileSystemAccess files system operation.
675       */
676      @InterfaceAudience.Private
677      public static class FSMkdirs implements FileSystemAccess.FileSystemExecutor<JSONObject> {
678    
679        private Path path;
680        private short permission;
681    
682        /**
683         * Creates a mkdirs executor.
684         *
685         * @param path directory path to create.
686         * @param permission permission to use.
687         */
688        public FSMkdirs(String path, short permission) {
689          this.path = new Path(path);
690          this.permission = permission;
691        }
692    
693        /**
694         * Executes the filesystem operation.
695         *
696         * @param fs filesystem instance to use.
697         *
698         * @return <code>true</code> if the mkdirs operation was successful,
699         *         <code>false</code> otherwise.
700         *
701         * @throws IOException thrown if an IO error occured.
702         */
703        @Override
704        public JSONObject execute(FileSystem fs) throws IOException {
705          FsPermission fsPermission = new FsPermission(permission);
706          boolean mkdirs = fs.mkdirs(path, fsPermission);
707          return toJSON(HttpFSFileSystem.MKDIRS_JSON, mkdirs);
708        }
709    
710      }
711    
712      /**
713       * Executor that performs a open FileSystemAccess files system operation.
714       */
715      @InterfaceAudience.Private
716      public static class FSOpen implements FileSystemAccess.FileSystemExecutor<InputStream> {
717        private Path path;
718    
719        /**
720         * Creates a open executor.
721         *
722         * @param path file to open.
723         */
724        public FSOpen(String path) {
725          this.path = new Path(path);
726        }
727    
728        /**
729         * Executes the filesystem operation.
730         *
731         * @param fs filesystem instance to use.
732         *
733         * @return The inputstream of the file.
734         *
735         * @throws IOException thrown if an IO error occured.
736         */
737        @Override
738        public InputStream execute(FileSystem fs) throws IOException {
739          int bufferSize = HttpFSServerWebApp.get().getConfig().getInt("httpfs.buffer.size", 4096);
740          return fs.open(path, bufferSize);
741        }
742    
743      }
744    
745      /**
746       * Executor that performs a rename FileSystemAccess files system operation.
747       */
748      @InterfaceAudience.Private
749      public static class FSRename implements FileSystemAccess.FileSystemExecutor<JSONObject> {
750        private Path path;
751        private Path toPath;
752    
753        /**
754         * Creates a rename executor.
755         *
756         * @param path path to rename.
757         * @param toPath new name.
758         */
759        public FSRename(String path, String toPath) {
760          this.path = new Path(path);
761          this.toPath = new Path(toPath);
762        }
763    
764        /**
765         * Executes the filesystem operation.
766         *
767         * @param fs filesystem instance to use.
768         *
769         * @return <code>true</code> if the rename operation was successful,
770         *         <code>false</code> otherwise.
771         *
772         * @throws IOException thrown if an IO error occured.
773         */
774        @Override
775        public JSONObject execute(FileSystem fs) throws IOException {
776          boolean renamed = fs.rename(path, toPath);
777          return toJSON(HttpFSFileSystem.RENAME_JSON, renamed);
778        }
779    
780      }
781    
782      /**
783       * Executor that performs a set-owner FileSystemAccess files system operation.
784       */
785      @InterfaceAudience.Private
786      public static class FSSetOwner implements FileSystemAccess.FileSystemExecutor<Void> {
787        private Path path;
788        private String owner;
789        private String group;
790    
791        /**
792         * Creates a set-owner executor.
793         *
794         * @param path the path to set the owner.
795         * @param owner owner to set.
796         * @param group group to set.
797         */
798        public FSSetOwner(String path, String owner, String group) {
799          this.path = new Path(path);
800          this.owner = owner;
801          this.group = group;
802        }
803    
804        /**
805         * Executes the filesystem operation.
806         *
807         * @param fs filesystem instance to use.
808         *
809         * @return void.
810         *
811         * @throws IOException thrown if an IO error occured.
812         */
813        @Override
814        public Void execute(FileSystem fs) throws IOException {
815          fs.setOwner(path, owner, group);
816          return null;
817        }
818    
819      }
820    
821      /**
822       * Executor that performs a set-permission FileSystemAccess files system operation.
823       */
824      @InterfaceAudience.Private
825      public static class FSSetPermission implements FileSystemAccess.FileSystemExecutor<Void> {
826    
827        private Path path;
828        private short permission;
829    
830        /**
831         * Creates a set-permission executor.
832         *
833         * @param path path to set the permission.
834         * @param permission permission to set.
835         */
836        public FSSetPermission(String path, short permission) {
837          this.path = new Path(path);
838          this.permission = permission;
839        }
840    
841        /**
842         * Executes the filesystem operation.
843         *
844         * @param fs filesystem instance to use.
845         *
846         * @return void.
847         *
848         * @throws IOException thrown if an IO error occured.
849         */
850        @Override
851        public Void execute(FileSystem fs) throws IOException {
852          FsPermission fsPermission = new FsPermission(permission);
853          fs.setPermission(path, fsPermission);
854          return null;
855        }
856    
857      }
858    
859      /**
860       * Executor that sets the acl for a file in a FileSystem
861       */
862      @InterfaceAudience.Private
863      public static class FSSetAcl implements FileSystemAccess.FileSystemExecutor<Void> {
864    
865        private Path path;
866        private List<AclEntry> aclEntries;
867    
868        /**
869         * Creates a set-acl executor.
870         *
871         * @param path path to set the acl.
872         * @param aclSpec acl to set.
873         */
874        public FSSetAcl(String path, String aclSpec) {
875          this.path = new Path(path);
876          this.aclEntries = AclEntry.parseAclSpec(aclSpec, true);
877        }
878    
879        /**
880         * Executes the filesystem operation.
881         *
882         * @param fs filesystem instance to use.
883         *
884         * @return void.
885         *
886         * @throws IOException thrown if an IO error occurred.
887         */
888        @Override
889        public Void execute(FileSystem fs) throws IOException {
890          fs.setAcl(path, aclEntries);
891          return null;
892        }
893    
894      }
895    
896      /**
897       * Executor that removes all acls from a file in a FileSystem
898       */
899      @InterfaceAudience.Private
900      public static class FSRemoveAcl implements FileSystemAccess.FileSystemExecutor<Void> {
901    
902        private Path path;
903    
904        /**
905         * Creates a remove-acl executor.
906         *
907         * @param path path from which to remove the acl.
908         */
909        public FSRemoveAcl(String path) {
910          this.path = new Path(path);
911        }
912    
913        /**
914         * Executes the filesystem operation.
915         *
916         * @param fs filesystem instance to use.
917         *
918         * @return void.
919         *
920         * @throws IOException thrown if an IO error occurred.
921         */
922        @Override
923        public Void execute(FileSystem fs) throws IOException {
924          fs.removeAcl(path);
925          return null;
926        }
927    
928      }
929    
930      /**
931       * Executor that modifies acl entries for a file in a FileSystem
932       */
933      @InterfaceAudience.Private
934      public static class FSModifyAclEntries implements FileSystemAccess.FileSystemExecutor<Void> {
935    
936        private Path path;
937        private List<AclEntry> aclEntries;
938    
939        /**
940         * Creates a modify-acl executor.
941         *
942         * @param path path to set the acl.
943         * @param aclSpec acl to set.
944         */
945        public FSModifyAclEntries(String path, String aclSpec) {
946          this.path = new Path(path);
947          this.aclEntries = AclEntry.parseAclSpec(aclSpec, true);
948        }
949    
950        /**
951         * Executes the filesystem operation.
952         *
953         * @param fs filesystem instance to use.
954         *
955         * @return void.
956         *
957         * @throws IOException thrown if an IO error occurred.
958         */
959        @Override
960        public Void execute(FileSystem fs) throws IOException {
961          fs.modifyAclEntries(path, aclEntries);
962          return null;
963        }
964    
965      }
966    
967      /**
968       * Executor that removes acl entries from a file in a FileSystem
969       */
970      @InterfaceAudience.Private
971      public static class FSRemoveAclEntries implements FileSystemAccess.FileSystemExecutor<Void> {
972    
973        private Path path;
974        private List<AclEntry> aclEntries;
975    
976        /**
977         * Creates a remove acl entry executor.
978         *
979         * @param path path to set the acl.
980         * @param aclSpec acl parts to remove.
981         */
982        public FSRemoveAclEntries(String path, String aclSpec) {
983          this.path = new Path(path);
984          this.aclEntries = AclEntry.parseAclSpec(aclSpec, true);
985        }
986    
987        /**
988         * Executes the filesystem operation.
989         *
990         * @param fs filesystem instance to use.
991         *
992         * @return void.
993         *
994         * @throws IOException thrown if an IO error occurred.
995         */
996        @Override
997        public Void execute(FileSystem fs) throws IOException {
998          fs.removeAclEntries(path, aclEntries);
999          return null;
1000        }
1001    
1002      }
1003    
1004      /**
1005       * Executor that removes the default acl from a directory in a FileSystem
1006       */
1007      @InterfaceAudience.Private
1008      public static class FSRemoveDefaultAcl implements FileSystemAccess.FileSystemExecutor<Void> {
1009    
1010        private Path path;
1011    
1012        /**
1013         * Creates an executor for removing the default acl.
1014         *
1015         * @param path path to set the acl.
1016         */
1017        public FSRemoveDefaultAcl(String path) {
1018          this.path = new Path(path);
1019        }
1020    
1021        /**
1022         * Executes the filesystem operation.
1023         *
1024         * @param fs filesystem instance to use.
1025         *
1026         * @return void.
1027         *
1028         * @throws IOException thrown if an IO error occurred.
1029         */
1030        @Override
1031        public Void execute(FileSystem fs) throws IOException {
1032          fs.removeDefaultAcl(path);
1033          return null;
1034        }
1035    
1036      }
1037    
1038      /**
1039       * Executor that gets the ACL information for a given file.
1040       */
1041      @InterfaceAudience.Private
1042      public static class FSAclStatus implements FileSystemAccess.FileSystemExecutor<Map> {
1043        private Path path;
1044    
1045        /**
1046         * Creates an executor for getting the ACLs for a file.
1047         *
1048         * @param path the path to retrieve the ACLs.
1049         */
1050        public FSAclStatus(String path) {
1051          this.path = new Path(path);
1052        }
1053    
1054        /**
1055         * Executes the filesystem operation.
1056         *
1057         * @param fs filesystem instance to use.
1058         *
1059         * @return a Map object (JSON friendly) with the file status.
1060         *
1061         * @throws IOException thrown if an IO error occurred.
1062         */
1063        @Override
1064        public Map execute(FileSystem fs) throws IOException {
1065          AclStatus status = fs.getAclStatus(path);
1066          return aclStatusToJSON(status);
1067        }
1068    
1069      }
1070    
1071      /**
1072       * Executor that performs a set-replication FileSystemAccess files system operation.
1073       */
1074      @InterfaceAudience.Private
1075      public static class FSSetReplication implements FileSystemAccess.FileSystemExecutor<JSONObject> {
1076        private Path path;
1077        private short replication;
1078    
1079        /**
1080         * Creates a set-replication executor.
1081         *
1082         * @param path path to set the replication factor.
1083         * @param replication replication factor to set.
1084         */
1085        public FSSetReplication(String path, short replication) {
1086          this.path = new Path(path);
1087          this.replication = replication;
1088        }
1089    
1090        /**
1091         * Executes the filesystem operation.
1092         *
1093         * @param fs filesystem instance to use.
1094         *
1095         * @return <code>true</code> if the replication value was set,
1096         *         <code>false</code> otherwise.
1097         *
1098         * @throws IOException thrown if an IO error occured.
1099         */
1100        @Override
1101        @SuppressWarnings("unchecked")
1102        public JSONObject execute(FileSystem fs) throws IOException {
1103          boolean ret = fs.setReplication(path, replication);
1104          JSONObject json = new JSONObject();
1105          json.put(HttpFSFileSystem.SET_REPLICATION_JSON, ret);
1106          return json;
1107        }
1108    
1109      }
1110    
1111      /**
1112       * Executor that performs a set-times FileSystemAccess files system operation.
1113       */
1114      @InterfaceAudience.Private
1115      public static class FSSetTimes implements FileSystemAccess.FileSystemExecutor<Void> {
1116        private Path path;
1117        private long mTime;
1118        private long aTime;
1119    
1120        /**
1121         * Creates a set-times executor.
1122         *
1123         * @param path path to set the times.
1124         * @param mTime modified time to set.
1125         * @param aTime access time to set.
1126         */
1127        public FSSetTimes(String path, long mTime, long aTime) {
1128          this.path = new Path(path);
1129          this.mTime = mTime;
1130          this.aTime = aTime;
1131        }
1132    
1133        /**
1134         * Executes the filesystem operation.
1135         *
1136         * @param fs filesystem instance to use.
1137         *
1138         * @return void.
1139         *
1140         * @throws IOException thrown if an IO error occured.
1141         */
1142        @Override
1143        public Void execute(FileSystem fs) throws IOException {
1144          fs.setTimes(path, mTime, aTime);
1145          return null;
1146        }
1147    
1148      }
1149    
1150      /**
1151       * Executor that performs a setxattr FileSystemAccess files system operation.
1152       */
1153      @InterfaceAudience.Private
1154      public static class FSSetXAttr implements 
1155          FileSystemAccess.FileSystemExecutor<Void> {
1156    
1157        private Path path;
1158        private String name;
1159        private byte[] value;
1160        private EnumSet<XAttrSetFlag> flag;
1161    
1162        public FSSetXAttr(String path, String name, String encodedValue, 
1163            EnumSet<XAttrSetFlag> flag) throws IOException {
1164          this.path = new Path(path);
1165          this.name = name;
1166          this.value = XAttrCodec.decodeValue(encodedValue);
1167          this.flag = flag;
1168        }
1169    
1170        @Override
1171        public Void execute(FileSystem fs) throws IOException {
1172          fs.setXAttr(path, name, value, flag);
1173          return null;
1174        }
1175      }
1176    
1177      /**
1178       * Executor that performs a removexattr FileSystemAccess files system 
1179       * operation.
1180       */
1181      @InterfaceAudience.Private
1182      public static class FSRemoveXAttr implements 
1183          FileSystemAccess.FileSystemExecutor<Void> {
1184    
1185        private Path path;
1186        private String name;
1187    
1188        public FSRemoveXAttr(String path, String name) {
1189          this.path = new Path(path);
1190          this.name = name;
1191        }
1192    
1193        @Override
1194        public Void execute(FileSystem fs) throws IOException {
1195          fs.removeXAttr(path, name);
1196          return null;
1197        }
1198      }
1199    
1200      /**
1201       * Executor that performs listing xattrs FileSystemAccess files system 
1202       * operation.
1203       */
1204      @SuppressWarnings("rawtypes")
1205      @InterfaceAudience.Private
1206      public static class FSListXAttrs implements 
1207          FileSystemAccess.FileSystemExecutor<Map> {
1208        private Path path;
1209    
1210        /**
1211         * Creates listing xattrs executor.
1212         *
1213         * @param path the path to retrieve the xattrs.
1214         */
1215        public FSListXAttrs(String path) {
1216          this.path = new Path(path);
1217        }
1218    
1219        /**
1220         * Executes the filesystem operation.
1221         *
1222         * @param fs filesystem instance to use.
1223         *
1224         * @return Map a map object (JSON friendly) with the xattr names.
1225         *
1226         * @throws IOException thrown if an IO error occured.
1227         */
1228        @Override
1229        public Map execute(FileSystem fs) throws IOException {
1230          List<String> names = fs.listXAttrs(path);
1231          return xAttrNamesToJSON(names);
1232        }
1233      }
1234    
1235      /**
1236       * Executor that performs getting xattrs FileSystemAccess files system 
1237       * operation.
1238       */
1239      @SuppressWarnings("rawtypes")
1240      @InterfaceAudience.Private
1241      public static class FSGetXAttrs implements 
1242          FileSystemAccess.FileSystemExecutor<Map> {
1243        private Path path;
1244        private List<String> names;
1245        private XAttrCodec encoding;
1246    
1247        /**
1248         * Creates getting xattrs executor.
1249         *
1250         * @param path the path to retrieve the xattrs.
1251         */
1252        public FSGetXAttrs(String path, List<String> names, XAttrCodec encoding) {
1253          this.path = new Path(path);
1254          this.names = names;
1255          this.encoding = encoding;
1256        }
1257    
1258        /**
1259         * Executes the filesystem operation.
1260         *
1261         * @param fs filesystem instance to use.
1262         *
1263         * @return Map a map object (JSON friendly) with the xattrs.
1264         *
1265         * @throws IOException thrown if an IO error occured.
1266         */
1267        @Override
1268        public Map execute(FileSystem fs) throws IOException {
1269          Map<String, byte[]> xattrs = null;
1270          if (names != null && !names.isEmpty()) {
1271            xattrs = fs.getXAttrs(path, names);
1272          } else {
1273            xattrs = fs.getXAttrs(path);
1274          }
1275          return xAttrsToJSON(xattrs, encoding);
1276        }
1277      }
1278    }