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
019package org.apache.hadoop.fs.http.server;
020
021import org.apache.hadoop.conf.Configuration;
022import org.apache.hadoop.fs.FileSystem;
023import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
024import org.apache.hadoop.fs.http.server.HttpFSParams.AccessTimeParam;
025import org.apache.hadoop.fs.http.server.HttpFSParams.BlockSizeParam;
026import org.apache.hadoop.fs.http.server.HttpFSParams.DataParam;
027import org.apache.hadoop.fs.http.server.HttpFSParams.DeleteOpParam;
028import org.apache.hadoop.fs.http.server.HttpFSParams.DeleteRecursiveParam;
029import org.apache.hadoop.fs.http.server.HttpFSParams.DoAsParam;
030import org.apache.hadoop.fs.http.server.HttpFSParams.FilterParam;
031import org.apache.hadoop.fs.http.server.HttpFSParams.FsPathParam;
032import org.apache.hadoop.fs.http.server.HttpFSParams.GetOpParam;
033import org.apache.hadoop.fs.http.server.HttpFSParams.GroupParam;
034import org.apache.hadoop.fs.http.server.HttpFSParams.LenParam;
035import org.apache.hadoop.fs.http.server.HttpFSParams.ModifiedTimeParam;
036import org.apache.hadoop.fs.http.server.HttpFSParams.OffsetParam;
037import org.apache.hadoop.fs.http.server.HttpFSParams.OverwriteParam;
038import org.apache.hadoop.fs.http.server.HttpFSParams.OwnerParam;
039import org.apache.hadoop.fs.http.server.HttpFSParams.PermissionParam;
040import org.apache.hadoop.fs.http.server.HttpFSParams.PostOpParam;
041import org.apache.hadoop.fs.http.server.HttpFSParams.PutOpParam;
042import org.apache.hadoop.fs.http.server.HttpFSParams.ReplicationParam;
043import org.apache.hadoop.fs.http.server.HttpFSParams.ToPathParam;
044import org.apache.hadoop.lib.service.FileSystemAccess;
045import org.apache.hadoop.lib.service.FileSystemAccessException;
046import org.apache.hadoop.lib.service.Groups;
047import org.apache.hadoop.lib.service.Instrumentation;
048import org.apache.hadoop.lib.service.ProxyUser;
049import org.apache.hadoop.lib.servlet.FileSystemReleaseFilter;
050import org.apache.hadoop.lib.servlet.HostnameFilter;
051import org.apache.hadoop.lib.wsrs.InputStreamEntity;
052import org.json.simple.JSONObject;
053import org.slf4j.Logger;
054import org.slf4j.LoggerFactory;
055import org.slf4j.MDC;
056
057import javax.ws.rs.Consumes;
058import javax.ws.rs.DELETE;
059import javax.ws.rs.DefaultValue;
060import javax.ws.rs.GET;
061import javax.ws.rs.POST;
062import javax.ws.rs.PUT;
063import javax.ws.rs.Path;
064import javax.ws.rs.PathParam;
065import javax.ws.rs.Produces;
066import javax.ws.rs.QueryParam;
067import javax.ws.rs.core.Context;
068import javax.ws.rs.core.MediaType;
069import javax.ws.rs.core.Response;
070import javax.ws.rs.core.UriBuilder;
071import javax.ws.rs.core.UriInfo;
072import java.io.IOException;
073import java.io.InputStream;
074import java.net.URI;
075import java.security.AccessControlException;
076import java.security.Principal;
077import java.text.MessageFormat;
078import java.util.List;
079import java.util.Map;
080
081/**
082 * Main class of HttpFSServer server.
083 * <p/>
084 * The <code>HttpFSServer</code> class uses Jersey JAX-RS to binds HTTP requests to the
085 * different operations.
086 */
087@Path(HttpFSFileSystem.SERVICE_VERSION)
088public class HttpFSServer {
089  private static Logger AUDIT_LOG = LoggerFactory.getLogger("httpfsaudit");
090
091  /**
092   * Special binding for '/' as it is not handled by the wildcard binding.
093   *
094   * @param user principal making the request.
095   * @param op GET operation, default value is @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.GetOpValues#OPEN}.
096   * @param filter Glob filter, default value is none. Used only if the
097   * operation is @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.GetOpValues#LISTSTATUS}
098   * @param doAs user being impersonated, defualt value is none. It can be used
099   * only if the current user is a HttpFSServer proxyuser.
100   *
101   * @return the request response
102   *
103   * @throws IOException thrown if an IO error occurred. Thrown exceptions are
104   * handled by {@link HttpFSExceptionProvider}.
105   * @throws FileSystemAccessException thrown if a FileSystemAccess releated error occurred. Thrown
106   * exceptions are handled by {@link HttpFSExceptionProvider}.
107   */
108  @GET
109  @Path("/")
110  @Produces(MediaType.APPLICATION_JSON)
111  public Response root(@Context Principal user,
112                       @QueryParam(GetOpParam.NAME) GetOpParam op,
113                       @QueryParam(FilterParam.NAME) @DefaultValue(FilterParam.DEFAULT) FilterParam filter,
114                       @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) DoAsParam doAs)
115    throws IOException, FileSystemAccessException {
116    return get(user, new FsPathParam(""), op, new OffsetParam(OffsetParam.DEFAULT),
117               new LenParam(LenParam.DEFAULT), filter, doAs,
118               new OverwriteParam(OverwriteParam.DEFAULT),
119               new BlockSizeParam(BlockSizeParam.DEFAULT),
120               new PermissionParam(PermissionParam.DEFAULT),
121               new ReplicationParam(ReplicationParam.DEFAULT));
122  }
123
124  /**
125   * Resolves the effective user that will be used to request a FileSystemAccess filesystem.
126   * <p/>
127   * If the doAs-user is NULL or the same as the user, it returns the user.
128   * <p/>
129   * Otherwise it uses proxyuser rules (see {@link ProxyUser} to determine if the
130   * current user can impersonate the doAs-user.
131   * <p/>
132   * If the current user cannot impersonate the doAs-user an
133   * <code>AccessControlException</code> will be thrown.
134   *
135   * @param user principal for whom the filesystem instance is.
136   * @param doAs do-as user, if any.
137   *
138   * @return the effective user.
139   *
140   * @throws IOException thrown if an IO error occurrs.
141   * @throws AccessControlException thrown if the current user cannot impersonate
142   * the doAs-user.
143   */
144  private String getEffectiveUser(Principal user, String doAs) throws IOException {
145    String effectiveUser = user.getName();
146    if (doAs != null && !doAs.equals(user.getName())) {
147      ProxyUser proxyUser = HttpFSServerWebApp.get().get(ProxyUser.class);
148      proxyUser.validate(user.getName(), HostnameFilter.get(), doAs);
149      effectiveUser = doAs;
150      AUDIT_LOG.info("Proxy user [{}] DoAs user [{}]", user.getName(), doAs);
151    }
152    return effectiveUser;
153  }
154
155  /**
156   * Executes a {@link FileSystemAccess.FileSystemExecutor} using a filesystem for the effective
157   * user.
158   *
159   * @param user principal making the request.
160   * @param doAs do-as user, if any.
161   * @param executor FileSystemExecutor to execute.
162   *
163   * @return FileSystemExecutor response
164   *
165   * @throws IOException thrown if an IO error occurrs.
166   * @throws FileSystemAccessException thrown if a FileSystemAccess releated error occurred. Thrown
167   * exceptions are handled by {@link HttpFSExceptionProvider}.
168   */
169  private <T> T fsExecute(Principal user, String doAs, FileSystemAccess.FileSystemExecutor<T> executor)
170    throws IOException, FileSystemAccessException {
171    String hadoopUser = getEffectiveUser(user, doAs);
172    FileSystemAccess fsAccess = HttpFSServerWebApp.get().get(FileSystemAccess.class);
173    Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getDefaultConfiguration();
174    return fsAccess.execute(hadoopUser, conf, executor);
175  }
176
177  /**
178   * Returns a filesystem instance. The fileystem instance is wired for release at the completion of
179   * the current Servlet request via the {@link FileSystemReleaseFilter}.
180   * <p/>
181   * If a do-as user is specified, the current user must be a valid proxyuser, otherwise an
182   * <code>AccessControlException</code> will be thrown.
183   *
184   * @param user principal for whom the filesystem instance is.
185   * @param doAs do-as user, if any.
186   *
187   * @return a filesystem for the specified user or do-as user.
188   *
189   * @throws IOException thrown if an IO error occurred. Thrown exceptions are
190   * handled by {@link HttpFSExceptionProvider}.
191   * @throws FileSystemAccessException thrown if a FileSystemAccess releated error occurred. Thrown
192   * exceptions are handled by {@link HttpFSExceptionProvider}.
193   */
194  private FileSystem createFileSystem(Principal user, String doAs) throws IOException, FileSystemAccessException {
195    String hadoopUser = getEffectiveUser(user, doAs);
196    FileSystemAccess fsAccess = HttpFSServerWebApp.get().get(FileSystemAccess.class);
197    Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getDefaultConfiguration();
198    FileSystem fs = fsAccess.createFileSystem(hadoopUser, conf);
199    FileSystemReleaseFilter.setFileSystem(fs);
200    return fs;
201  }
202
203  /**
204   * Binding to handle all GET requests, supported operations are
205   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.GetOpValues}.
206   * <p/>
207   * The @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.GetOpValues#INSTRUMENTATION} operation is available only
208   * to users that are in HttpFSServer's admin group (see {@link HttpFSServer}. It returns
209   * HttpFSServer instrumentation data. The specified path must be '/'.
210   *
211   * @param user principal making the request.
212   * @param path path for the GET request.
213   * @param op GET operation, default value is @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.GetOpValues#OPEN}.
214   * @param offset of the  file being fetch, used only with
215   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.GetOpValues#OPEN} operations.
216   * @param len amounts of bytes, used only with @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.GetOpValues#OPEN}
217   * operations.
218   * @param filter Glob filter, default value is none. Used only if the
219   * operation is @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.GetOpValues#LISTSTATUS}
220   * @param doAs user being impersonated, defualt value is none. It can be used
221   * only if the current user is a HttpFSServer proxyuser.
222   * @param override default is true. Used only for
223   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#CREATE} operations.
224   * @param blockSize block size to set, used only by
225   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#CREATE} operations.
226   * @param permission permission to set, used only by
227   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#SETPERMISSION}.
228   * @param replication replication factor to set, used only by
229   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#SETREPLICATION}.
230   *
231   * @return the request response.
232   *
233   * @throws IOException thrown if an IO error occurred. Thrown exceptions are
234   * handled by {@link HttpFSExceptionProvider}.
235   * @throws FileSystemAccessException thrown if a FileSystemAccess releated error occurred. Thrown
236   * exceptions are handled by {@link HttpFSExceptionProvider}.
237   */
238  @GET
239  @Path("{path:.*}")
240  @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
241  public Response get(@Context Principal user,
242                      @PathParam("path") @DefaultValue("") FsPathParam path,
243                      @QueryParam(GetOpParam.NAME) GetOpParam op,
244                      @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT) OffsetParam offset,
245                      @QueryParam(LenParam.NAME) @DefaultValue(LenParam.DEFAULT) LenParam len,
246                      @QueryParam(FilterParam.NAME) @DefaultValue(FilterParam.DEFAULT) FilterParam filter,
247                      @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) DoAsParam doAs,
248
249                      //these params are only for createHandle operation acceptance purposes
250                      @QueryParam(OverwriteParam.NAME) @DefaultValue(OverwriteParam.DEFAULT) OverwriteParam override,
251                      @QueryParam(BlockSizeParam.NAME) @DefaultValue(BlockSizeParam.DEFAULT) BlockSizeParam blockSize,
252                      @QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT)
253                      PermissionParam permission,
254                      @QueryParam(ReplicationParam.NAME) @DefaultValue(ReplicationParam.DEFAULT)
255                      ReplicationParam replication
256  )
257    throws IOException, FileSystemAccessException {
258    Response response = null;
259    if (op == null) {
260      throw new UnsupportedOperationException(MessageFormat.format("Missing [{0}] parameter", GetOpParam.NAME));
261    } else {
262      path.makeAbsolute();
263      MDC.put(HttpFSFileSystem.OP_PARAM, op.value().name());
264      switch (op.value()) {
265        case OPEN: {
266          //Invoking the command directly using an unmanaged FileSystem that is released by the
267          //FileSystemReleaseFilter
268          FSOperations.FSOpen command = new FSOperations.FSOpen(path.value());
269          FileSystem fs = createFileSystem(user, doAs.value());
270          InputStream is = command.execute(fs);
271          AUDIT_LOG.info("[{}] offset [{}] len [{}]", new Object[]{path, offset, len});
272          InputStreamEntity entity = new InputStreamEntity(is, offset.value(), len.value());
273          response = Response.ok(entity).type(MediaType.APPLICATION_OCTET_STREAM).build();
274          break;
275        }
276        case GETFILESTATUS: {
277          FSOperations.FSFileStatus command = new FSOperations.FSFileStatus(path.value());
278          Map json = fsExecute(user, doAs.value(), command);
279          AUDIT_LOG.info("[{}]", path);
280          response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
281          break;
282        }
283        case LISTSTATUS: {
284          FSOperations.FSListStatus command = new FSOperations.FSListStatus(path.value(), filter.value());
285          Map json = fsExecute(user, doAs.value(), command);
286          if (filter.value() == null) {
287            AUDIT_LOG.info("[{}]", path);
288          } else {
289            AUDIT_LOG.info("[{}] filter [{}]", path, filter.value());
290          }
291          response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
292          break;
293        }
294        case GETHOMEDIR: {
295          FSOperations.FSHomeDir command = new FSOperations.FSHomeDir();
296          JSONObject json = fsExecute(user, doAs.value(), command);
297          AUDIT_LOG.info("");
298          response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
299          break;
300        }
301        case INSTRUMENTATION: {
302          if (!path.value().equals("/")) {
303            throw new UnsupportedOperationException(
304              MessageFormat.format("Invalid path for {0}={1}, must be '/'",
305                                   GetOpParam.NAME, HttpFSFileSystem.GetOpValues.INSTRUMENTATION));
306          }
307          Groups groups = HttpFSServerWebApp.get().get(Groups.class);
308          List<String> userGroups = groups.getGroups(user.getName());
309          if (!userGroups.contains(HttpFSServerWebApp.get().getAdminGroup())) {
310            throw new AccessControlException("User not in HttpFSServer admin group");
311          }
312          Instrumentation instrumentation = HttpFSServerWebApp.get().get(Instrumentation.class);
313          Map snapshot = instrumentation.getSnapshot();
314          response = Response.ok(snapshot).build();
315          break;
316        }
317        case GETCONTENTSUMMARY: {
318          FSOperations.FSContentSummary command = new FSOperations.FSContentSummary(path.value());
319          Map json = fsExecute(user, doAs.value(), command);
320          AUDIT_LOG.info("[{}]", path);
321          response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
322          break;
323        }
324        case GETFILECHECKSUM: {
325          FSOperations.FSFileChecksum command = new FSOperations.FSFileChecksum(path.value());
326          Map json = fsExecute(user, doAs.value(), command);
327          AUDIT_LOG.info("[{}]", path);
328          response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
329          break;
330        }
331        case GETDELEGATIONTOKEN: {
332          response = Response.status(Response.Status.BAD_REQUEST).build();
333          break;
334        }
335        case GETFILEBLOCKLOCATIONS: {
336          response = Response.status(Response.Status.BAD_REQUEST).build();
337          break;
338        }
339      }
340      return response;
341    }
342  }
343
344  /**
345   * Creates the URL for an upload operation (create or append).
346   *
347   * @param uriInfo uri info of the request.
348   * @param uploadOperation operation for the upload URL.
349   *
350   * @return the URI for uploading data.
351   */
352  protected URI createUploadRedirectionURL(UriInfo uriInfo, Enum<?> uploadOperation) {
353    UriBuilder uriBuilder = uriInfo.getRequestUriBuilder();
354    uriBuilder = uriBuilder.replaceQueryParam(PutOpParam.NAME, uploadOperation).
355      queryParam(DataParam.NAME, Boolean.TRUE);
356    return uriBuilder.build(null);
357  }
358
359  /**
360   * Binding to handle all DELETE requests.
361   *
362   * @param user principal making the request.
363   * @param path path for the DELETE request.
364   * @param op DELETE operation, default value is @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.DeleteOpValues#DELETE}.
365   * @param recursive indicates if the delete is recursive, default is <code>false</code>
366   * @param doAs user being impersonated, defualt value is none. It can be used
367   * only if the current user is a HttpFSServer proxyuser.
368   *
369   * @return the request response.
370   *
371   * @throws IOException thrown if an IO error occurred. Thrown exceptions are
372   * handled by {@link HttpFSExceptionProvider}.
373   * @throws FileSystemAccessException thrown if a FileSystemAccess releated error occurred. Thrown
374   * exceptions are handled by {@link HttpFSExceptionProvider}.
375   */
376  @DELETE
377  @Path("{path:.*}")
378  @Produces(MediaType.APPLICATION_JSON)
379  public Response delete(@Context Principal user,
380                         @PathParam("path") FsPathParam path,
381                         @QueryParam(DeleteOpParam.NAME) DeleteOpParam op,
382                         @QueryParam(DeleteRecursiveParam.NAME) @DefaultValue(DeleteRecursiveParam.DEFAULT)
383                         DeleteRecursiveParam recursive,
384                         @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) DoAsParam doAs)
385    throws IOException, FileSystemAccessException {
386    Response response = null;
387    if (op == null) {
388      throw new UnsupportedOperationException(MessageFormat.format("Missing [{0}] parameter", DeleteOpParam.NAME));
389    }
390    switch (op.value()) {
391      case DELETE: {
392        path.makeAbsolute();
393        MDC.put(HttpFSFileSystem.OP_PARAM, "DELETE");
394        AUDIT_LOG.info("[{}] recursive [{}]", path, recursive);
395        FSOperations.FSDelete command = new FSOperations.FSDelete(path.value(), recursive.value());
396        JSONObject json = fsExecute(user, doAs.value(), command);
397        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
398        break;
399      }
400    }
401    return response;
402  }
403
404
405  /**
406   * Binding to handle all PUT requests, supported operations are
407   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues}.
408   *
409   * @param is request input stream, used only for
410   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PostOpValues#APPEND} operations.
411   * @param user principal making the request.
412   * @param uriInfo the request uriInfo.
413   * @param path path for the PUT request.
414   * @param op PUT operation, no default value.
415   * @param toPath new path, used only for
416   * {@link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#RENAME} operations.
417   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#SETTIMES}.
418   * @param owner owner to set, used only for
419   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#SETOWNER} operations.
420   * @param group group to set, used only for
421   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#SETOWNER} operations.
422   * @param override default is true. Used only for
423   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#CREATE} operations.
424   * @param blockSize block size to set, used only by
425   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#CREATE} operations.
426   * @param permission permission to set, used only by
427   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#SETPERMISSION}.
428   * @param replication replication factor to set, used only by
429   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#SETREPLICATION}.
430   * @param modifiedTime modified time, in seconds since EPOC, used only by
431   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#SETTIMES}.
432   * @param accessTime accessed time, in seconds since EPOC, used only by
433   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PutOpValues#SETTIMES}.
434   * @param hasData indicates if the append request is uploading data or not
435   * (just getting the handle).
436   * @param doAs user being impersonated, defualt value is none. It can be used
437   * only if the current user is a HttpFSServer proxyuser.
438   *
439   * @return the request response.
440   *
441   * @throws IOException thrown if an IO error occurred. Thrown exceptions are
442   * handled by {@link HttpFSExceptionProvider}.
443   * @throws FileSystemAccessException thrown if a FileSystemAccess releated error occurred. Thrown
444   * exceptions are handled by {@link HttpFSExceptionProvider}.
445   */
446  @PUT
447  @Path("{path:.*}")
448  @Consumes({"*/*"})
449  @Produces({MediaType.APPLICATION_JSON})
450  public Response put(InputStream is,
451                      @Context Principal user,
452                      @Context UriInfo uriInfo,
453                      @PathParam("path") FsPathParam path,
454                      @QueryParam(PutOpParam.NAME) PutOpParam op,
455                      @QueryParam(ToPathParam.NAME) @DefaultValue(ToPathParam.DEFAULT) ToPathParam toPath,
456                      @QueryParam(OwnerParam.NAME) @DefaultValue(OwnerParam.DEFAULT) OwnerParam owner,
457                      @QueryParam(GroupParam.NAME) @DefaultValue(GroupParam.DEFAULT) GroupParam group,
458                      @QueryParam(OverwriteParam.NAME) @DefaultValue(OverwriteParam.DEFAULT) OverwriteParam override,
459                      @QueryParam(BlockSizeParam.NAME) @DefaultValue(BlockSizeParam.DEFAULT) BlockSizeParam blockSize,
460                      @QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT)
461                      PermissionParam permission,
462                      @QueryParam(ReplicationParam.NAME) @DefaultValue(ReplicationParam.DEFAULT)
463                      ReplicationParam replication,
464                      @QueryParam(ModifiedTimeParam.NAME) @DefaultValue(ModifiedTimeParam.DEFAULT)
465                      ModifiedTimeParam modifiedTime,
466                      @QueryParam(AccessTimeParam.NAME) @DefaultValue(AccessTimeParam.DEFAULT)
467                      AccessTimeParam accessTime,
468                      @QueryParam(DataParam.NAME) @DefaultValue(DataParam.DEFAULT) DataParam hasData,
469                      @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) DoAsParam doAs)
470    throws IOException, FileSystemAccessException {
471    Response response = null;
472    if (op == null) {
473      throw new UnsupportedOperationException(MessageFormat.format("Missing [{0}] parameter", PutOpParam.NAME));
474    }
475    path.makeAbsolute();
476    MDC.put(HttpFSFileSystem.OP_PARAM, op.value().name());
477    switch (op.value()) {
478      case CREATE: {
479        if (!hasData.value()) {
480          response = Response.temporaryRedirect(
481            createUploadRedirectionURL(uriInfo, HttpFSFileSystem.PutOpValues.CREATE)).build();
482        } else {
483          FSOperations.FSCreate
484            command = new FSOperations.FSCreate(is, path.value(), permission.value(), override.value(),
485                                                replication.value(), blockSize.value());
486          fsExecute(user, doAs.value(), command);
487          AUDIT_LOG.info("[{}] permission [{}] override [{}] replication [{}] blockSize [{}]",
488                         new Object[]{path, permission, override, replication, blockSize});
489          response = Response.status(Response.Status.CREATED).build();
490        }
491        break;
492      }
493      case MKDIRS: {
494        FSOperations.FSMkdirs command = new FSOperations.FSMkdirs(path.value(), permission.value());
495        JSONObject json = fsExecute(user, doAs.value(), command);
496        AUDIT_LOG.info("[{}] permission [{}]", path, permission.value());
497        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
498        break;
499      }
500      case RENAME: {
501        FSOperations.FSRename command = new FSOperations.FSRename(path.value(), toPath.value());
502        JSONObject json = fsExecute(user, doAs.value(), command);
503        AUDIT_LOG.info("[{}] to [{}]", path, toPath);
504        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
505        break;
506      }
507      case SETOWNER: {
508        FSOperations.FSSetOwner command = new FSOperations.FSSetOwner(path.value(), owner.value(), group.value());
509        fsExecute(user, doAs.value(), command);
510        AUDIT_LOG.info("[{}] to (O/G)[{}]", path, owner.value() + ":" + group.value());
511        response = Response.ok().build();
512        break;
513      }
514      case SETPERMISSION: {
515        FSOperations.FSSetPermission command = new FSOperations.FSSetPermission(path.value(), permission.value());
516        fsExecute(user, doAs.value(), command);
517        AUDIT_LOG.info("[{}] to [{}]", path, permission.value());
518        response = Response.ok().build();
519        break;
520      }
521      case SETREPLICATION: {
522        FSOperations.FSSetReplication command = new FSOperations.FSSetReplication(path.value(), replication.value());
523        JSONObject json = fsExecute(user, doAs.value(), command);
524        AUDIT_LOG.info("[{}] to [{}]", path, replication.value());
525        response = Response.ok(json).build();
526        break;
527      }
528      case SETTIMES: {
529        FSOperations.FSSetTimes
530          command = new FSOperations.FSSetTimes(path.value(), modifiedTime.value(), accessTime.value());
531        fsExecute(user, doAs.value(), command);
532        AUDIT_LOG.info("[{}] to (M/A)[{}]", path, modifiedTime.value() + ":" + accessTime.value());
533        response = Response.ok().build();
534        break;
535      }
536      case RENEWDELEGATIONTOKEN: {
537        response = Response.status(Response.Status.BAD_REQUEST).build();
538        break;
539      }
540      case CANCELDELEGATIONTOKEN: {
541        response = Response.status(Response.Status.BAD_REQUEST).build();
542        break;
543      }
544    }
545    return response;
546  }
547
548  /**
549   * Binding to handle all OPST requests, supported operations are
550   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PostOpValues}.
551   *
552   * @param is request input stream, used only for
553   * @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PostOpValues#APPEND} operations.
554   * @param user principal making the request.
555   * @param uriInfo the request uriInfo.
556   * @param path path for the POST request.
557   * @param op POST operation, default is @link org.apache.hadoop.fs.http.client.HttpFSFileSystem.PostOpValues#APPEND}.
558   * @param hasData indicates if the append request is uploading data or not (just getting the handle).
559   * @param doAs user being impersonated, defualt value is none. It can be used
560   * only if the current user is a HttpFSServer proxyuser.
561   *
562   * @return the request response.
563   *
564   * @throws IOException thrown if an IO error occurred. Thrown exceptions are
565   * handled by {@link HttpFSExceptionProvider}.
566   * @throws FileSystemAccessException thrown if a FileSystemAccess releated error occurred. Thrown
567   * exceptions are handled by {@link HttpFSExceptionProvider}.
568   */
569  @POST
570  @Path("{path:.*}")
571  @Consumes({"*/*"})
572  @Produces({MediaType.APPLICATION_JSON})
573  public Response post(InputStream is,
574                       @Context Principal user,
575                       @Context UriInfo uriInfo,
576                       @PathParam("path") FsPathParam path,
577                       @QueryParam(PostOpParam.NAME) PostOpParam op,
578                       @QueryParam(DataParam.NAME) @DefaultValue(DataParam.DEFAULT) DataParam hasData,
579                       @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) DoAsParam doAs)
580    throws IOException, FileSystemAccessException {
581    Response response = null;
582    if (op == null) {
583      throw new UnsupportedOperationException(MessageFormat.format("Missing [{0}] parameter", PostOpParam.NAME));
584    }
585    path.makeAbsolute();
586    MDC.put(HttpFSFileSystem.OP_PARAM, op.value().name());
587    switch (op.value()) {
588      case APPEND: {
589        if (!hasData.value()) {
590          response = Response.temporaryRedirect(
591            createUploadRedirectionURL(uriInfo, HttpFSFileSystem.PostOpValues.APPEND)).build();
592        } else {
593          FSOperations.FSAppend command = new FSOperations.FSAppend(is, path.value());
594          fsExecute(user, doAs.value(), command);
595          AUDIT_LOG.info("[{}]", path);
596          response = Response.ok().type(MediaType.APPLICATION_JSON).build();
597        }
598        break;
599      }
600    }
601    return response;
602  }
603
604}