protected Response get()

in hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java [1338:1617]


  protected Response get(
      final UserGroupInformation ugi,
      final DelegationParam delegation,
      final UserParam username,
      final DoAsParam doAsUser,
      final String fullpath,
      final GetOpParam op,
      final OffsetParam offset,
      final LengthParam length,
      final RenewerParam renewer,
      final BufferSizeParam bufferSize,
      final List<XAttrNameParam> xattrNames,
      final XAttrEncodingParam xattrEncoding,
      final ExcludeDatanodesParam excludeDatanodes,
      final FsActionParam fsAction,
      final SnapshotNameParam snapshotName,
      final OldSnapshotNameParam oldSnapshotName,
      final SnapshotDiffStartPathParam snapshotDiffStartPath,
      final SnapshotDiffIndexParam snapshotDiffIndex,
      final TokenKindParam tokenKind,
      final TokenServiceParam tokenService,
      final NoRedirectParam noredirectParam,
      final StartAfterParam startAfter,
      final AllUsersParam allUsers
      ) throws IOException, URISyntaxException {
    final Configuration conf = (Configuration) context
        .getAttribute(JspHelper.CURRENT_CONF);
    final ClientProtocol cp = getRpcClientProtocol();

    switch(op.getValue()) {
    case OPEN:
    {
      final NameNode namenode = (NameNode)context.getAttribute("name.node");
      ResponseBuilder rb = Response.noContent();
      final URI uri = redirectURI(rb, namenode, ugi, delegation, username,
          doAsUser, fullpath, op.getValue(), offset.getValue(), -1L,
          excludeDatanodes.getValue(), offset, length, bufferSize);
      if(!noredirectParam.getValue()) {
        return rb.status(Status.TEMPORARY_REDIRECT).location(uri)
            .type(MediaType.APPLICATION_OCTET_STREAM).build();
      } else {
        final String js = JsonUtil.toJsonString("Location", uri);
        return rb.status(Status.OK).entity(js).type(MediaType.APPLICATION_JSON)
            .build();
      }
    }
    case GETFILEBLOCKLOCATIONS:
    {
      final long offsetValue = offset.getValue();
      final Long lengthValue = length.getValue();
      LocatedBlocks locatedBlocks = getRpcClientProtocol()
          .getBlockLocations(fullpath, offsetValue, lengthValue != null ?
              lengthValue : Long.MAX_VALUE);
      BlockLocation[] locations =
          DFSUtilClient.locatedBlocks2Locations(locatedBlocks);
      final String js = JsonUtil.toJsonString(locations);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GET_BLOCK_LOCATIONS:
    {
      final long offsetValue = offset.getValue();
      final Long lengthValue = length.getValue();
      final LocatedBlocks locatedblocks = cp.getBlockLocations(fullpath,
          offsetValue, lengthValue != null? lengthValue: Long.MAX_VALUE);
      final String js = JsonUtil.toJsonString(locatedblocks);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETFILESTATUS:
    {
      final HdfsFileStatus status = cp.getFileInfo(fullpath);
      if (status == null) {
        throw new FileNotFoundException("File does not exist: " + fullpath);
      }

      final String js = JsonUtil.toJsonString(status, true);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case LISTSTATUS:
    {
      final StreamingOutput streaming = getListingStream(cp, fullpath);
      return Response.ok(streaming).type(MediaType.APPLICATION_JSON).build();
    }
    case GETCONTENTSUMMARY:
    {
      final ContentSummary contentsummary = cp.getContentSummary(fullpath);
      final String js = JsonUtil.toJsonString(contentsummary);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETQUOTAUSAGE:
    {
      final QuotaUsage quotaUsage = cp.getQuotaUsage(fullpath);
      final String js = JsonUtil.toJsonString(quotaUsage);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETFILECHECKSUM:
    {
      final NameNode namenode = (NameNode)context.getAttribute("name.node");
      final URI uri = redirectURI(null, namenode, ugi, delegation, username,
          doAsUser, fullpath, op.getValue(), -1L, -1L, null);
      if(!noredirectParam.getValue()) {
        return Response.temporaryRedirect(uri)
          .type(MediaType.APPLICATION_OCTET_STREAM).build();
      } else {
        final String js = JsonUtil.toJsonString("Location", uri);
        return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
      }
    }
    case GETDELEGATIONTOKEN:
    {
      if (delegation.getValue() != null) {
        throw new IllegalArgumentException(delegation.getName()
            + " parameter is not null.");
      }
      final Token<? extends TokenIdentifier> token = generateDelegationToken(
          ugi, renewer.getValue());

      final String setServiceName = tokenService.getValue();
      final String setKind = tokenKind.getValue();
      if (setServiceName != null) {
        token.setService(new Text(setServiceName));
      }
      if (setKind != null) {
        token.setKind(new Text(setKind));
      }
      final String js = JsonUtil.toJsonString(token);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETHOMEDIRECTORY: {
      String userHome = DFSUtilClient.getHomeDirectory(conf, ugi);
      final String js = JsonUtil.toJsonString("Path", userHome);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETACLSTATUS: {
      AclStatus status = cp.getAclStatus(fullpath);
      if (status == null) {
        throw new FileNotFoundException("File does not exist: " + fullpath);
      }

      final String js = JsonUtil.toJsonString(status);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETXATTRS: {
      validateOpParams(op, xattrEncoding);
      List<String> names = null;
      if (xattrNames != null) {
        names = Lists.newArrayListWithCapacity(xattrNames.size());
        for (XAttrNameParam xattrName : xattrNames) {
          if (xattrName.getXAttrName() != null) {
            names.add(xattrName.getXAttrName());
          }
        }
      }
      List<XAttr> xAttrs = cp.getXAttrs(fullpath, (names != null &&
          !names.isEmpty()) ? XAttrHelper.buildXAttrs(names) : null);
      final String js = JsonUtil.toJsonString(xAttrs,
          xattrEncoding.getEncoding());
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case LISTXATTRS: {
      final List<XAttr> xAttrs = cp.listXAttrs(fullpath);
      final String js = JsonUtil.toJsonString(xAttrs);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case CHECKACCESS: {
      validateOpParams(op, fsAction);
      cp.checkAccess(fullpath, FsAction.getFsAction(fsAction.getValue()));
      return Response.ok().build();
    }
    case GETTRASHROOT: {
      final String trashPath = getTrashRoot(conf, fullpath);
      final String jsonStr = JsonUtil.toJsonString("Path", trashPath);
      return Response.ok(jsonStr).type(MediaType.APPLICATION_JSON).build();
    }
    case GETTRASHROOTS: {
      Boolean value = allUsers.getValue();
      final Collection<FileStatus> trashPaths = getTrashRoots(conf, value);
      final String js = JsonUtil.toJsonString(trashPaths);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case LISTSTATUS_BATCH:
    {
      byte[] start = HdfsFileStatus.EMPTY_NAME;
      if (startAfter != null && startAfter.getValue() != null) {
        start = startAfter.getValue().getBytes(StandardCharsets.UTF_8);
      }
      final DirectoryListing listing = getDirectoryListing(cp, fullpath, start);
      final String js = JsonUtil.toJsonString(listing);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETALLSTORAGEPOLICY: {
      BlockStoragePolicy[] storagePolicies = cp.getStoragePolicies();
      final String js = JsonUtil.toJsonString(storagePolicies);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETSTORAGEPOLICY: {
      BlockStoragePolicy storagePolicy = cp.getStoragePolicy(fullpath);
      final String js = JsonUtil.toJsonString(storagePolicy);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETECPOLICY: {
      ErasureCodingPolicy ecpolicy = cp.getErasureCodingPolicy(fullpath);
      final String js = JsonUtil.toJsonString(ecpolicy);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETSERVERDEFAULTS: {
      // Since none of the server defaults values are hot reloaded, we can
      // cache the output of serverDefaults.
      String serverDefaultsResponse =
          (String) context.getAttribute("serverDefaults");
      if (serverDefaultsResponse == null) {
        FsServerDefaults serverDefaults = cp.getServerDefaults();
        serverDefaultsResponse = JsonUtil.toJsonString(serverDefaults);
        context.setAttribute("serverDefaults", serverDefaultsResponse);
      }
      return Response.ok(serverDefaultsResponse)
          .type(MediaType.APPLICATION_JSON).build();
    }
    case GETSNAPSHOTDIFF: {
      SnapshotDiffReport diffReport =
          cp.getSnapshotDiffReport(fullpath, oldSnapshotName.getValue(),
              snapshotName.getValue());
      final String js = JsonUtil.toJsonString(diffReport);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETSNAPSHOTDIFFLISTING: {
      SnapshotDiffReportListing diffReport = cp.getSnapshotDiffReportListing(
          fullpath, oldSnapshotName.getValue(), snapshotName.getValue(),
          DFSUtilClient.string2Bytes(snapshotDiffStartPath.getValue()),
          snapshotDiffIndex.getValue());
      final String js = JsonUtil.toJsonString(diffReport);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETSNAPSHOTTABLEDIRECTORYLIST: {
      SnapshottableDirectoryStatus[] snapshottableDirectoryList =
          cp.getSnapshottableDirListing();
      final String js = JsonUtil.toJsonString(snapshottableDirectoryList);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETSNAPSHOTLIST: {
      SnapshotStatus[] snapshotList =
          cp.getSnapshotListing(fullpath);
      final String js = JsonUtil.toJsonString(snapshotList);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETLINKTARGET: {
      String target = cp.getLinkTarget(fullpath);
      final String js = JsonUtil.toJsonString("Path", target);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETFILELINKSTATUS: {
      HdfsFileStatus status = cp.getFileLinkInfo(fullpath);
      if (status == null) {
        throw new FileNotFoundException("File does not exist: " + fullpath);
      }
      final String js = JsonUtil.toJsonString(status, true);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETSTATUS: {
      long[] states = cp.getStats();
      FsStatus status = new FsStatus(
          DFSClient.getStateAtIndex(states, 0),
          DFSClient.getStateAtIndex(states, 1),
          DFSClient.getStateAtIndex(states, 2));
      final String js = JsonUtil.toJsonString(status);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETECPOLICIES: {
      ErasureCodingPolicyInfo[] ecPolicyInfos = cp.getErasureCodingPolicies();
      final String js = JsonUtil.toJsonString(ecPolicyInfos);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    case GETECCODECS: {
      Map<String, String> ecCodecs = cp.getErasureCodingCodecs();
      final String js = JsonUtil.toJsonString("ErasureCodingCodecs", ecCodecs);
      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
    }
    default:
      throw new UnsupportedOperationException(op + " is not supported");
    }
  }