public Response get()

in hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java [252:599]


  public Response get(@PathParam("path") String path,
                      @Context UriInfo uriInfo,
                      @QueryParam(OperationParam.NAME) OperationParam op,
                      @Context HttpServletRequest request)
    throws IOException, FileSystemAccessException {
    // Restrict access to only GETFILESTATUS and LISTSTATUS in write-only mode
    if((op.value() != HttpFSFileSystem.Operation.GETFILESTATUS) &&
            (op.value() != HttpFSFileSystem.Operation.LISTSTATUS) &&
            accessMode == AccessMode.WRITEONLY) {
      return Response.status(Response.Status.FORBIDDEN).build();
    }
    UserGroupInformation user = HttpUserGroupInformation.get();
    Response response;
    path = makeAbsolute(path);
    final Parameters params = getParams(request);
    MDC.put(HttpFSFileSystem.OP_PARAM, op.value().name());
    MDC.put("hostname", request.getRemoteAddr());
    switch (op.value()) {
    case OPEN: {
      Boolean noRedirect = params.get(
          NoRedirectParam.NAME, NoRedirectParam.class);
      if (noRedirect) {
        URI redirectURL = createOpenRedirectionURL(uriInfo);
        final String js = JsonUtil.toJsonString("Location", redirectURL);
        response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      } else {
        //Invoking the command directly using an unmanaged FileSystem that is
        // released by the FileSystemReleaseFilter
        final FSOperations.FSOpen command = new FSOperations.FSOpen(path);
        final FileSystem fs = createFileSystem(user);
        InputStream is = null;
        UserGroupInformation ugi = UserGroupInformation
            .createProxyUser(user.getShortUserName(),
                UserGroupInformation.getLoginUser());
        try {
          is = ugi.doAs(new PrivilegedExceptionAction<InputStream>() {
            @Override
            public InputStream run() throws Exception {
              return command.execute(fs);
            }
          });
        } catch (InterruptedException ie) {
          LOG.warn("Open interrupted.", ie);
          Thread.currentThread().interrupt();
        }
        Long offset = params.get(OffsetParam.NAME, OffsetParam.class);
        Long len = params.get(LenParam.NAME, LenParam.class);
        AUDIT_LOG.info("[{}] offset [{}] len [{}]",
            new Object[] { path, offset, len });
        InputStreamEntity entity = new InputStreamEntity(is, offset, len);
        response = Response.ok(entity).type(MediaType.APPLICATION_OCTET_STREAM)
            .build();
      }
      break;
    }
    case GETFILESTATUS: {
      FSOperations.FSFileStatus command = new FSOperations.FSFileStatus(path);
      Map json = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", path);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case LISTSTATUS: {
      String filter = params.get(FilterParam.NAME, FilterParam.class);
      FSOperations.FSListStatus command =
          new FSOperations.FSListStatus(path, filter);
      Map json = fsExecute(user, command);
      AUDIT_LOG.info("[{}] filter [{}]", path, (filter != null) ? filter : "-");
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETHOMEDIRECTORY: {
      enforceRootPath(op.value(), path);
      FSOperations.FSHomeDir command = new FSOperations.FSHomeDir();
      JSONObject json = fsExecute(user, command);
      AUDIT_LOG.info("Home Directory for [{}]", user);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case INSTRUMENTATION: {
      enforceRootPath(op.value(), path);
      Groups groups = HttpFSServerWebApp.get().get(Groups.class);
      Set<String> userGroups = groups.getGroupsSet(user.getShortUserName());
      if (!userGroups.contains(HttpFSServerWebApp.get().getAdminGroup())) {
        throw new AccessControlException(
            "User not in HttpFSServer admin group");
      }
      Instrumentation instrumentation =
          HttpFSServerWebApp.get().get(Instrumentation.class);
      Map snapshot = instrumentation.getSnapshot();
      response = Response.ok(snapshot).build();
      break;
    }
    case GETCONTENTSUMMARY: {
      FSOperations.FSContentSummary command =
          new FSOperations.FSContentSummary(path);
      Map json = fsExecute(user, command);
      AUDIT_LOG.info("Content summary for [{}]", path);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETQUOTAUSAGE: {
      FSOperations.FSQuotaUsage command =
          new FSOperations.FSQuotaUsage(path);
      Map json = fsExecute(user, command);
      AUDIT_LOG.info("Quota Usage for [{}]", path);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETFILECHECKSUM: {
      FSOperations.FSFileChecksum command =
          new FSOperations.FSFileChecksum(path);

      Boolean noRedirect = params.get(
          NoRedirectParam.NAME, NoRedirectParam.class);
      AUDIT_LOG.info("[{}]", path);
      if (noRedirect) {
        URI redirectURL = createOpenRedirectionURL(uriInfo);
        final String js = JsonUtil.toJsonString("Location", redirectURL);
        response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      } else {
        Map json = fsExecute(user, command);
        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      }
      break;
    }
    case GETFILEBLOCKLOCATIONS: {
      long offset = 0;
      long len = Long.MAX_VALUE;
      Long offsetParam = params.get(OffsetParam.NAME, OffsetParam.class);
      Long lenParam = params.get(LenParam.NAME, LenParam.class);
      AUDIT_LOG.info("[{}] offset [{}] len [{}]", path, offsetParam, lenParam);
      if (offsetParam != null && offsetParam > 0) {
        offset = offsetParam;
      }
      if (lenParam != null && lenParam > 0) {
        len = lenParam;
      }
      FSOperations.FSFileBlockLocations command =
          new FSOperations.FSFileBlockLocations(path, offset, len);
      @SuppressWarnings("rawtypes")
      Map locations = fsExecute(user, command);
      final String json = JsonUtil.toJsonString("BlockLocations", locations);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETACLSTATUS: {
      FSOperations.FSAclStatus command = new FSOperations.FSAclStatus(path);
      Map json = fsExecute(user, command);
      AUDIT_LOG.info("ACL status for [{}]", path);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETXATTRS: {
      List<String> xattrNames =
          params.getValues(XAttrNameParam.NAME, XAttrNameParam.class);
      XAttrCodec encoding =
          params.get(XAttrEncodingParam.NAME, XAttrEncodingParam.class);
      FSOperations.FSGetXAttrs command =
          new FSOperations.FSGetXAttrs(path, xattrNames, encoding);
      @SuppressWarnings("rawtypes") Map json = fsExecute(user, command);
      AUDIT_LOG.info("XAttrs for [{}]", path);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case LISTXATTRS: {
      FSOperations.FSListXAttrs command = new FSOperations.FSListXAttrs(path);
      @SuppressWarnings("rawtypes") Map json = fsExecute(user, command);
      AUDIT_LOG.info("XAttr names for [{}]", path);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case LISTSTATUS_BATCH: {
      String startAfter = params.get(
          HttpFSParametersProvider.StartAfterParam.NAME,
          HttpFSParametersProvider.StartAfterParam.class);
      byte[] token = HttpFSUtils.EMPTY_BYTES;
      if (startAfter != null) {
        token = startAfter.getBytes(StandardCharsets.UTF_8);
      }
      FSOperations.FSListStatusBatch command = new FSOperations
          .FSListStatusBatch(path, token);
      @SuppressWarnings("rawtypes") Map json = fsExecute(user, command);
      AUDIT_LOG.info("[{}] token [{}]", path, token);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETTRASHROOT: {
      FSOperations.FSTrashRoot command = new FSOperations.FSTrashRoot(path);
      JSONObject json = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", path);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETALLSTORAGEPOLICY: {
      FSOperations.FSGetAllStoragePolicies command =
          new FSOperations.FSGetAllStoragePolicies();
      JSONObject json = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", path);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETSTORAGEPOLICY: {
      FSOperations.FSGetStoragePolicy command =
          new FSOperations.FSGetStoragePolicy(path);
      JSONObject json = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", path);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETSNAPSHOTDIFF: {
      String oldSnapshotName = params.get(OldSnapshotNameParam.NAME,
          OldSnapshotNameParam.class);
      String snapshotName = params.get(SnapshotNameParam.NAME,
          SnapshotNameParam.class);
      FSOperations.FSGetSnapshotDiff command =
          new FSOperations.FSGetSnapshotDiff(path, oldSnapshotName,
              snapshotName);
      String js = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", path);
      response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETSNAPSHOTDIFFLISTING: {
      String oldSnapshotName = params.get(OldSnapshotNameParam.NAME,
          OldSnapshotNameParam.class);
      String snapshotName = params.get(SnapshotNameParam.NAME,
          SnapshotNameParam.class);
      String snapshotDiffStartPath = params
          .get(HttpFSParametersProvider.SnapshotDiffStartPathParam.NAME,
              HttpFSParametersProvider.SnapshotDiffStartPathParam.class);
      Integer snapshotDiffIndex = params.get(HttpFSParametersProvider.SnapshotDiffIndexParam.NAME,
          HttpFSParametersProvider.SnapshotDiffIndexParam.class);
      FSOperations.FSGetSnapshotDiffListing command =
          new FSOperations.FSGetSnapshotDiffListing(path, oldSnapshotName,
              snapshotName, snapshotDiffStartPath, snapshotDiffIndex);
      String js = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", path);
      response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETSNAPSHOTTABLEDIRECTORYLIST: {
      FSOperations.FSGetSnapshottableDirListing command =
          new FSOperations.FSGetSnapshottableDirListing();
      String js = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", "/");
      response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETSNAPSHOTLIST: {
      FSOperations.FSGetSnapshotListing command =
          new FSOperations.FSGetSnapshotListing(path);
      String js = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", "/");
      response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETSERVERDEFAULTS: {
      FSOperations.FSGetServerDefaults command =
          new FSOperations.FSGetServerDefaults();
      String js = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", "/");
      response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case CHECKACCESS: {
      String mode = params.get(FsActionParam.NAME, FsActionParam.class);
      FsActionParam fsparam = new FsActionParam(mode);
      FSOperations.FSAccess command = new FSOperations.FSAccess(path,
          FsAction.getFsAction(fsparam.value()));
      fsExecute(user, command);
      AUDIT_LOG.info("[{}]", "/");
      response = Response.ok().build();
      break;
    }
    case GETECPOLICY: {
      FSOperations.FSGetErasureCodingPolicy command =
          new FSOperations.FSGetErasureCodingPolicy(path);
      String js = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", path);
      response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETECPOLICIES: {
      FSOperations.FSGetErasureCodingPolicies command =
          new FSOperations.FSGetErasureCodingPolicies();
      String js = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", path);
      response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETECCODECS: {
      FSOperations.FSGetErasureCodingCodecs command =
          new FSOperations.FSGetErasureCodingCodecs();
      Map json = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", path);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GET_BLOCK_LOCATIONS: {
      long offset = 0;
      long len = Long.MAX_VALUE;
      Long offsetParam = params.get(OffsetParam.NAME, OffsetParam.class);
      Long lenParam = params.get(LenParam.NAME, LenParam.class);
      AUDIT_LOG.info("[{}] offset [{}] len [{}]", path, offsetParam, lenParam);
      if (offsetParam != null && offsetParam > 0) {
        offset = offsetParam;
      }
      if (lenParam != null && lenParam > 0) {
        len = lenParam;
      }
      FSOperations.FSFileBlockLocationsLegacy command =
          new FSOperations.FSFileBlockLocationsLegacy(path, offset, len);
      @SuppressWarnings("rawtypes")
      Map locations = fsExecute(user, command);
      final String json = JsonUtil.toJsonString("LocatedBlocks", locations);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETFILELINKSTATUS: {
      FSOperations.FSFileLinkStatus command =
          new FSOperations.FSFileLinkStatus(path);
      @SuppressWarnings("rawtypes") Map js = fsExecute(user, command);
      AUDIT_LOG.info("[{}]", path);
      response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETSTATUS: {
      FSOperations.FSStatus command = new FSOperations.FSStatus(path);
      @SuppressWarnings("rawtypes") Map js = fsExecute(user, command);
      response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    case GETTRASHROOTS: {
      Boolean allUsers = params.get(AllUsersParam.NAME, AllUsersParam.class);
      FSOperations.FSGetTrashRoots command = new FSOperations.FSGetTrashRoots(allUsers);
      Map json = fsExecute(user, command);
      AUDIT_LOG.info("allUsers [{}]", allUsers);
      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
      break;
    }
    default: {
      throw new IOException(
          MessageFormat.format("Invalid HTTP GET operation [{0}]", op.value()));
    }
    }
    return response;
  }