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