You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/07/14 21:41:09 UTC

[GitHub] [ozone] yuangu002 opened a new pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

yuangu002 opened a new pull request #2417:
URL: https://github.com/apache/ozone/pull/2417


   ## What changes were proposed in this pull request?
   Add new API(s) (likely REST APIs) exposed by Recon to be used by CLI and Web UI later.
   
   Other changes from HDDS-5386:
   1. Added a field `dirName` under `NSSummary` to store a directory's name, which will be used in `NSSummaryEndpoint` where we query the next-level path name for the DU endpoint.
   2. Optimized the `childDir` from `List` to `Set` as subdirectories' order doesn't matter for us.
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-5378
   
   ## How was this patch tested?
   Unit test: org.apache.hadoop.ozone.recon.api.TestNSSummaryEndpoint
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677691965



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();
+      }
+      for (OmBucketInfo bucket: buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setResponseCode(ResponseStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setResponseCode(
+              ResponseStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case ROOT:
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      int[] fileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          fileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(fileSizeDist);
+      break;
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets under volume
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(ResponseStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(ResponseStatus.PATH_NOT_FOUND);

Review comment:
       I think you still need to `setFileSizeDist()`, probably to an empty array for `UNKNOWN` and `KEY` cases. Otherwise you get NPE like this because `fileSizeDist` is `null` by default:
   
   ![image](https://user-images.githubusercontent.com/50227127/127206894-4b856184-3ef1-4bec-98bb-208dd02fc391.png)
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] yuangu002 commented on pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#issuecomment-880231104


   @smengcl @avijayanhwx @vivekratnavel Could you review it?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677675832



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();

Review comment:
       ```suggestion
           final long quota = volume.getQuotaInBytes();
           if (quota < 0) {
             // If one volume has unlimited quota, the "root" quota is unlimited.
             quotaInBytes = -1L;
             break;
           }
           quotaInBytes += quota;
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#issuecomment-887744258


   Also I noticed a peculiar thing that, **before** Recon's initial sync with OM, `/namespace` endpoints will return HTTP 500. But `/containers` endpoint won't.
   
   <img width="500" alt="500" src="https://user-images.githubusercontent.com/50227127/127209797-47f42aed-daf8-47a9-a7ee-945479e8b043.png">
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676863345



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,
+                                      String keyName, long bucketObjectId)
+          throws IOException {
+    omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK, volName,
+            bucketName);

Review comment:
       I would argue the bucket lock here is unnecessary. As we don't ever write to the OM DB snapshot on Recon (except when fetching delta updates from OM but in that case the BUCKET_LOCK here wouldn't help either). @avijayanhwx 

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,
+                                      String keyName, long bucketObjectId)
+          throws IOException {
+    omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK, volName,
+            bucketName);

Review comment:
       I would argue the bucket lock here is unnecessary. As we don't ever write to the OM DB snapshot on Recon (except when fetching delta updates from OM but in that case the BUCKET_LOCK here wouldn't help either). CC @avijayanhwx 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677679368



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DUResponse.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ * HTTP Response wrapped for Disk Usage requests.
+ */
+public class DUResponse {
+  /** Path status. */
+  @JsonProperty("status")
+  private ResponseStatus status;
+
+  /** The number of subpaths under the request path. */
+  @JsonProperty("count")
+  private int count;
+
+  /** Encapsulates a DU instance for a subpath. */
+  @JsonProperty("duData")
+  private List<DiskUsage> duData;
+
+  public DUResponse() {
+    this.status = ResponseStatus.OK;
+  }
+
+  public ResponseStatus getStatus() {
+    return this.status;
+  }
+
+  public void setStatus(ResponseStatus status) {
+    this.status = status;
+  }
+
+  public int getCount() {
+    return count;
+  }
+
+  public void setCount(int count) {
+    this.count = count;
+  }
+
+  public List<DiskUsage> getDuData() {
+    return duData;
+  }
+
+  public void setDuData(List<DiskUsage> duData) {
+    this.duData = duData;
+  }
+
+  /**
+   * DU info for a path (path name, data size).
+   */
+  public static class DiskUsage {
+    /** The subpath name. */
+    @JsonProperty("subpath")

Review comment:
       Let's just call this field "path".
   
   ```suggestion
       @JsonProperty("path")
   ```
   
   Renaming this json property only should suffice for now, along with unit tests that checks this field from json, if any. You can leave the rest of the variable names subpath unchanged as they are hidden implementation details.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r671474115



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/FileSizeDistributionResponse.java
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+
+import java.util.Arrays;
+
+/**
+ * HTTP Response wrapped for a file size distribution request.
+ * 'dist': the array that stores the file size distribution for all keys
+ * under the request path.
+ * 'pathNotFound': invalid path request.
+ * 'typeNA': the path exists, but refers to a namespace type (key) that are not
+ * applicable to file size distribution request.
+ */
+public class FileSizeDistributionResponse {
+
+  @JsonProperty("dist")

Review comment:
       i.e. here. [Example](https://github.com/apache/ozone/blob/5eae0e0ecc476cdb77a5e3a921882681dd9ae81b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/MissingContainersResponse.java#L27-L31)

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/BasicResponse.java
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * HTTP Response wrapped for a 'basic' request.
+ * 'type': the namespace the request path is on.
+ * 'bucket': Total number of buckets for volume, 0 for other types.
+ * 'dir': Total number of directories for a bucket or directory, 0 for others.
+ * 'key': Total number of keys for a bucket or directory, 0 for others.
+ * 'pathNotFound': set to true if request path is valid
+ */
+public class BasicResponse {
+  @JsonProperty("type")
+  private EntityType entityType;
+
+  @JsonProperty("bucket")
+  private int totalBucket;
+
+  @JsonProperty("dir")
+  private int totalDir;
+
+  @JsonProperty("key")
+  private int totalKey;

Review comment:
       `numTotalKey`

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/FileSizeDistributionResponse.java
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+
+import java.util.Arrays;
+
+/**
+ * HTTP Response wrapped for a file size distribution request.
+ * 'dist': the array that stores the file size distribution for all keys
+ * under the request path.
+ * 'pathNotFound': invalid path request.
+ * 'typeNA': the path exists, but refers to a namespace type (key) that are not
+ * applicable to file size distribution request.
+ */
+public class FileSizeDistributionResponse {
+
+  @JsonProperty("dist")
+  private int[] fileSizeDist;

Review comment:
       new line after this for clarity
   ```suggestion
     private int[] fileSizeDist;
   
   ```

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/FileSizeDistributionResponse.java
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+
+import java.util.Arrays;
+
+/**
+ * HTTP Response wrapped for a file size distribution request.
+ * 'dist': the array that stores the file size distribution for all keys
+ * under the request path.
+ * 'pathNotFound': invalid path request.
+ * 'typeNA': the path exists, but refers to a namespace type (key) that are not
+ * applicable to file size distribution request.

Review comment:
       Let's move this chunk of javadoc right ahead their respective `@JsonProperty`s.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DUResponse.java
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ * HTTP Response wrapped for Disk Usage requests.
+ * 'duData' encapsulates an instance for a subpath.
+ * 'count' is the number of subpaths under the request path.
+ * 'pathNotFound' set to true if path is invalid.
+ */

Review comment:
       same suggestion here as in FileSizeDistributionResponse.java

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/BasicResponse.java
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * HTTP Response wrapped for a 'basic' request.
+ * 'type': the namespace the request path is on.
+ * 'bucket': Total number of buckets for volume, 0 for other types.
+ * 'dir': Total number of directories for a bucket or directory, 0 for others.
+ * 'key': Total number of keys for a bucket or directory, 0 for others.
+ * 'pathNotFound': set to true if request path is valid
+ */
+public class BasicResponse {
+  @JsonProperty("type")
+  private EntityType entityType;
+
+  @JsonProperty("bucket")
+  private int totalBucket;
+
+  @JsonProperty("dir")
+  private int totalDir;

Review comment:
       `numTotalDir`

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/EntityType.java
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+/**
+ * Enum class for namespace type.
+ */
+public enum EntityType {
+  VOLUME,
+  BUCKET,
+  DIRECTORY,
+  KEY,
+  INVALID, // if path is invalid

Review comment:
       Does `UNKNOWN` EntityType sound better?
   ```suggestion
     UNKNOWN, // if path is invalid
   ```

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/BasicResponse.java
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * HTTP Response wrapped for a 'basic' request.
+ * 'type': the namespace the request path is on.
+ * 'bucket': Total number of buckets for volume, 0 for other types.
+ * 'dir': Total number of directories for a bucket or directory, 0 for others.
+ * 'key': Total number of keys for a bucket or directory, 0 for others.
+ * 'pathNotFound': set to true if request path is valid
+ */
+public class BasicResponse {
+  @JsonProperty("type")
+  private EntityType entityType;
+
+  @JsonProperty("bucket")
+  private int totalBucket;

Review comment:
       Use `numTotalBucket`

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/FileSizeDistributionResponse.java
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+
+import java.util.Arrays;
+
+/**
+ * HTTP Response wrapped for a file size distribution request.
+ * 'dist': the array that stores the file size distribution for all keys
+ * under the request path.
+ * 'pathNotFound': invalid path request.
+ * 'typeNA': the path exists, but refers to a namespace type (key) that are not
+ * applicable to file size distribution request.
+ */
+public class FileSizeDistributionResponse {
+
+  @JsonProperty("dist")
+  private int[] fileSizeDist;
+  @JsonProperty("pathNotFound")
+  private boolean pathNotFound;

Review comment:
       new line after this as well
   ```suggestion
     private boolean pathNotFound;
   
   ```

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/BasicResponse.java
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * HTTP Response wrapped for a 'basic' request.
+ * 'type': the namespace the request path is on.
+ * 'bucket': Total number of buckets for volume, 0 for other types.
+ * 'dir': Total number of directories for a bucket or directory, 0 for others.
+ * 'key': Total number of keys for a bucket or directory, 0 for others.
+ * 'pathNotFound': set to true if request path is valid
+ */
+public class BasicResponse {
+  @JsonProperty("type")
+  private EntityType entityType;
+
+  @JsonProperty("bucket")
+  private int totalBucket;
+
+  @JsonProperty("dir")
+  private int totalDir;
+
+  @JsonProperty("key")
+  private int totalKey;
+
+  @JsonProperty("pathNotFound")
+  private boolean pathNotFound;

Review comment:
       An arguably better approach is to rename this to `status`, and change the type to an enum, such as `BasicResponseStatusType`.
   
   `PATH_NOT_FOUND` can then be one of the statuses.
   
   Just for reference:
   ```
   public enum BasicResponseStatusType {
     OK,
     PATH_NOT_FOUND
   }
   ```
   
   When reusing `BasicResponseStatusType` for other responses such as the ones below we can use more general name for this enum and add more fields to it.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/QuotaUsageResponse.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * HTTP Response wrapped for a quota usage request.
+ * 'allowed': the quota in bytes that the namespace allows.
+ * 'used': the quota that the namespace has consumed in bytes.
+ * 'pathNotFound': invalid path request.
+ * 'typeNA': the path exists, but refers to a namespace type (directory, key)
+ * that are not applicable to a quota usage request.

Review comment:
       same here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676863345



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,
+                                      String keyName, long bucketObjectId)
+          throws IOException {
+    omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK, volName,
+            bucketName);

Review comment:
       I would argue the bucket lock here is unnecessary. As we don't ever write to the OM DB snapshot on Recon.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677679368



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DUResponse.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ * HTTP Response wrapped for Disk Usage requests.
+ */
+public class DUResponse {
+  /** Path status. */
+  @JsonProperty("status")
+  private ResponseStatus status;
+
+  /** The number of subpaths under the request path. */
+  @JsonProperty("count")
+  private int count;
+
+  /** Encapsulates a DU instance for a subpath. */
+  @JsonProperty("duData")
+  private List<DiskUsage> duData;
+
+  public DUResponse() {
+    this.status = ResponseStatus.OK;
+  }
+
+  public ResponseStatus getStatus() {
+    return this.status;
+  }
+
+  public void setStatus(ResponseStatus status) {
+    this.status = status;
+  }
+
+  public int getCount() {
+    return count;
+  }
+
+  public void setCount(int count) {
+    this.count = count;
+  }
+
+  public List<DiskUsage> getDuData() {
+    return duData;
+  }
+
+  public void setDuData(List<DiskUsage> duData) {
+    this.duData = duData;
+  }
+
+  /**
+   * DU info for a path (path name, data size).
+   */
+  public static class DiskUsage {
+    /** The subpath name. */
+    @JsonProperty("subpath")

Review comment:
       Let's just call this field "path".
   
   ```suggestion
       @JsonProperty("path")
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] yuangu002 commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676886544



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,

Review comment:
       > Can we use `OMFileRequest#getOMKeyInfoIfExists` inside?
   > 
   > Only `volumeTable` and `bucketTable` are [`FULL_CACHE`](https://github.com/yuangu002/ozone/blob/afce21398041c85f3ffe5753336a65b317cf0756/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L444-L455).
   > 
   > `dirTable` and `fileTable` are `PARTIAL_CACHE` (by default), so we can just use `.get()`. Deduplicate the function.
   
   Reasons that I didn't do that:
   1. `OMFileRequest#getOMKeyInfoIfExists` contains a `validateBucket` method, which throws OM exception if bucket doesn't exist (in this case, the bucket won't exist, because `validateBucket` uses `get()` to check bucket existence). Plus, we also don't want to throw any `OMException` on the Recon side. Instead, we only want to set `EntityType` as `Unknown` if the bucket doesn't exist.
   2. There's no need for us to instantiate `OzoneFileStatus` class as `getOMKeyInfoIfExists` did. The only thing we want to do is just to distinguish directory from key.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676863345



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,
+                                      String keyName, long bucketObjectId)
+          throws IOException {
+    omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK, volName,
+            bucketName);

Review comment:
       I would argue the bucket lock here is unnecessary. As we don't ever write to the OM DB snapshot on the Recon side.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] yuangu002 commented on pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#issuecomment-885739645


   Testing on clusters:
   Configs: 1 volume, 1 bucket, 4 directories, and 8 keys.
   ![shell cmd](https://user-images.githubusercontent.com/53324985/126809313-7f3ba511-3989-4e22-aa48-3a6f69edb36f.png)
   
   Results:
   [Volume Level]
   ![volume basic](https://user-images.githubusercontent.com/53324985/126809514-9608ac33-15e8-45c8-beb4-ae1d0e557e2f.png)
   ![volume du](https://user-images.githubusercontent.com/53324985/126809538-96f09467-e45d-42d5-aee9-cdc620756460.png)
   ![volume quota](https://user-images.githubusercontent.com/53324985/126809551-c6988d00-f15c-4163-99e7-bacd0b577564.png)
   ![volume distribution](https://user-images.githubusercontent.com/53324985/126809561-8fc7e7f0-11c1-412d-b478-c6fb473781cd.png)
   
   [Bucket Level]
   ![bucket du](https://user-images.githubusercontent.com/53324985/126809635-f213e600-14b7-48d4-87b7-14d3ca8fc294.png)
   
   [Directory Level]
   ![directory basic](https://user-images.githubusercontent.com/53324985/126809682-522111d5-1101-489b-b267-e4a4532dc28b.png)
   ![directory du](https://user-images.githubusercontent.com/53324985/126809692-521c6871-a130-4127-80fd-d867fcf47dc7.png)
   
   I have more complicated tests but there's just a simple demo.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676848713



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;

Review comment:
       ```suggestion
       final String volDBKey = omMetadataManager.getVolumeKey(volName);
       return omMetadataManager.getVolumeTable().getSkipCache(volDBKey) != null;
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r671460328



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,561 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = omMetadataManager.listBuckets(names[0],
+              null, null, Integer.MAX_VALUE);
+      basicResponse.setTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setTotalDir(totalDir);
+      basicResponse.setTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case INVALID:
+      basicResponse = new BasicResponse(EntityType.INVALID);
+      basicResponse.setPathNotFound(true);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = omMetadataManager.listBuckets(volName,
+              null, null, Integer.MAX_VALUE);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case INVALID:
+      duResponse.setPathNotFound(true);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quora usage endpoint that summarize the quota allowed and quota used in

Review comment:
       ```suggestion
      * Quota usage endpoint that summarize the quota allowed and quota used in
   ```
   typo?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676903922



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;

Review comment:
       Normalize `path` beforehand. Currently: `path=/vol1/bucket2/dir3/` gives:
   
   ![image](https://user-images.githubusercontent.com/50227127/127050872-3d199bbd-b77b-4dd1-90e0-6e4037cc33e3.png)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677675832



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();

Review comment:
       ```suggestion
           final long quota = volume.getQuotaInBytes();
           if (quota == -1L) {
             // If one volume has unlimited quota, the "root" quota is unlimited.
             quotaInBytes = -1L;
             break;
           }
           quotaInBytes += quota;
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677679368



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DUResponse.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ * HTTP Response wrapped for Disk Usage requests.
+ */
+public class DUResponse {
+  /** Path status. */
+  @JsonProperty("status")
+  private ResponseStatus status;
+
+  /** The number of subpaths under the request path. */
+  @JsonProperty("count")
+  private int count;
+
+  /** Encapsulates a DU instance for a subpath. */
+  @JsonProperty("duData")
+  private List<DiskUsage> duData;
+
+  public DUResponse() {
+    this.status = ResponseStatus.OK;
+  }
+
+  public ResponseStatus getStatus() {
+    return this.status;
+  }
+
+  public void setStatus(ResponseStatus status) {
+    this.status = status;
+  }
+
+  public int getCount() {
+    return count;
+  }
+
+  public void setCount(int count) {
+    this.count = count;
+  }
+
+  public List<DiskUsage> getDuData() {
+    return duData;
+  }
+
+  public void setDuData(List<DiskUsage> duData) {
+    this.duData = duData;
+  }
+
+  /**
+   * DU info for a path (path name, data size).
+   */
+  public static class DiskUsage {
+    /** The subpath name. */
+    @JsonProperty("subpath")

Review comment:
       Let's just call this field "path".
   
   ```suggestion
       @JsonProperty("path")
   ```
   
   Renaming this json property **only** should suffice, along with unit tests that checks this field from json, if any. You can leave the rest of the variable names subpath unchanged as they are hidden implementation details.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676831377



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")

Review comment:
       ```suggestion
   @Path("/namespace")
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677685230



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();
+      }
+      for (OmBucketInfo bucket: buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setResponseCode(ResponseStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setResponseCode(
+              ResponseStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    FileSizeDistributionResponse distReponse =

Review comment:
       Typo. `distResponse`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] yuangu002 commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676886544



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,

Review comment:
       > Can we use `OMFileRequest#getOMKeyInfoIfExists` inside?
   > 
   > Only `volumeTable` and `bucketTable` are [`FULL_CACHE`](https://github.com/yuangu002/ozone/blob/afce21398041c85f3ffe5753336a65b317cf0756/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L444-L455).
   > 
   > `dirTable` and `fileTable` are `PARTIAL_CACHE` (by default), so we can just use `.get()`. Deduplicate the function.
   
   Reasons that I didn't do that:
   1. `OMFileRequest#getOMKeyInfoIfExists` contains a `validateBucket` method, which throws OM exception if bucket doesn't exist (in this case, the bucket won't exist, because `validateBucket` uses `get()` to check bucket existence). Plus, we also don't want to throw any `OMException` on the Recon side. Instead, we only want to set `EntityType` as `Unknown` if the bucket doesn't exist.
   2. There's no need for us to instantiate `OzoneFileStatus` class as `getOMKeyInfoIfExists` did. The only thing we want to do is just to distinguish directory from key.
   3. The `getOMKeyInfoIfExists` uses [getKeyTable()](https://github.com/yuangu002/ozone/blob/HDDS-5378-2/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java#L584), which doesn't really switch to FileTable in actual cluster setting. Instead, our `determineKeyPath` is only for FSO, so I used `getFileTable` directly.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676846438



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {

Review comment:
       ```suggestion
     static String getKeyName(String[] names) {
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677781589



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,772 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;

Review comment:
       Let's rename `BasicResponse` to `NamespaceSummaryResponse` ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677691965



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();
+      }
+      for (OmBucketInfo bucket: buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setResponseCode(ResponseStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setResponseCode(
+              ResponseStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case ROOT:
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      int[] fileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          fileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(fileSizeDist);
+      break;
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets under volume
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(ResponseStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(ResponseStatus.PATH_NOT_FOUND);

Review comment:
       I think you still need to setFileSizeDist, probably to an empty array for UNKNOWN and KEY cases. Otherwise you get NPE:
   
   ![image](https://user-images.githubusercontent.com/50227127/127206894-4b856184-3ef1-4bec-98bb-208dd02fc391.png)
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676897411



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OMDBUpdatesHandler.java
##########
@@ -124,10 +124,8 @@ private void processEvent(int cfIndex, byte[] keyBytes, byte[]
       }
 
       OMDBUpdateEvent event = builder.build();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(String.format("Generated OM update Event for table : %s, " +
-            "action = %s", tableName, action));
-      }
+      LOG.info(String.format("Generated OM update Event for table : %s, " +

Review comment:
       Yes. Let's revert the changes in this file. This would be noisy for production environment.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] yuangu002 commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676886544



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,

Review comment:
       > Can we use `OMFileRequest#getOMKeyInfoIfExists` inside?
   > 
   > Only `volumeTable` and `bucketTable` are [`FULL_CACHE`](https://github.com/yuangu002/ozone/blob/afce21398041c85f3ffe5753336a65b317cf0756/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L444-L455).
   > 
   > `dirTable` and `fileTable` are `PARTIAL_CACHE` (by default), so we can just use `.get()`. Deduplicate the function.
   
   Reasons that I didn't do that:
   1. `OMFileRequest#getOMKeyInfoIfExists` contains a `validateBucket` method, which throws OM exception if bucket doesn't exist (in this case, the bucket won't exist, because `validateBucket` uses `get()` to check bucket existence). Plus, we also don't want to throw any `OMException` on the Recon side. Instead, we only want to set `EntityType` as `Unknown`.
   2. There's no need for us to instantiate `OzoneFileStatus` class as `getOMKeyInfoIfExists` did. The only thing we want to do is just to distinguish directory from key.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677906878



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DUResponse.java
##########
@@ -31,13 +31,16 @@
   private ResponseStatus status;
 
   /** The number of subpaths under the request path. */
-  @JsonProperty("count")
+  @JsonProperty("subPathCount")
   private int count;
 
   /** Encapsulates a DU instance for a subpath. */
-  @JsonProperty("data")
+  @JsonProperty("subPaths")
   private List<DiskUsage> duData;
 
+  @JsonProperty("keySize")

Review comment:
       ```suggestion
     @JsonProperty("sizeDirectKey")
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677668548



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**

Review comment:
       nit: new line after between import and class java doc
   
   ```suggestion
   
   /**
   ```

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**

Review comment:
       nit: new line between last import and class java doc
   
   ```suggestion
   
   /**
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676859821



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,

Review comment:
       Can we use `OMFileRequest#getOMKeyInfoIfExists` directly?
   
   Only `volumeTable` and `bucketTable` are [`FULL_CACHE`](https://github.com/yuangu002/ozone/blob/afce21398041c85f3ffe5753336a65b317cf0756/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L444-L455).
   
   `dirTable` and `fileTable` are [`PARTIAL_CACHE`] (by default), so we can just use `.get()`. Remove this method to avoid code duplication.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,

Review comment:
       Can we use `OMFileRequest#getOMKeyInfoIfExists` directly?
   
   Only `volumeTable` and `bucketTable` are [`FULL_CACHE`](https://github.com/yuangu002/ozone/blob/afce21398041c85f3ffe5753336a65b317cf0756/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L444-L455).
   
   `dirTable` and `fileTable` are `PARTIAL_CACHE` (by default), so we can just use `.get()`. Remove this method to avoid code duplication.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676846215



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {

Review comment:
       Make the method package-private by removing `public` modifier, as this is only used outside the class in the test `TestNSSummaryEndpoint`.
   ```suggestion
     static String[] parseRequestPath(String path) {
   ```

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {

Review comment:
       Make the method package-private by removing `public` modifier, as this is only used outside the class in the test `TestNSSummaryEndpoint` in the same package.
   ```suggestion
     static String[] parseRequestPath(String path) {
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677782310



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/BasicResponse.java
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * HTTP Response wrapped for a 'basic' request.
+ */
+public class BasicResponse {
+  /** The namespace the request path is on. */
+  @JsonProperty("type")
+  private EntityType entityType;
+
+  @JsonProperty("vol")
+  private int numVolume;
+
+  /** Total number of buckets for volume, 0 for other types. */
+  @JsonProperty("bucket")
+  private int numBucket;
+
+  /** Total number of directories for a bucket or directory, 0 for others. */
+  @JsonProperty("dir")
+  private int numTotalDir;
+
+  /** Total number of keys for a bucket or directory, 0 for others. */
+  @JsonProperty("key")

Review comment:
       ```suggestion
     @JsonProperty("numKey")
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl merged pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl merged pull request #2417:
URL: https://github.com/apache/ozone/pull/2417


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676859821



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,

Review comment:
       Can we use `OMFileRequest#getOMKeyInfoIfExists` inside?
   
   Only `volumeTable` and `bucketTable` are [`FULL_CACHE`](https://github.com/yuangu002/ozone/blob/afce21398041c85f3ffe5753336a65b317cf0756/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L444-L455).
   
   `dirTable` and `fileTable` are `PARTIAL_CACHE` (by default), so we can just use `.get()`. Deduplicate the function.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676847264



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {

Review comment:
       Rename this to `volumeExists`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#issuecomment-888588642


   Merged. Thanks @yuangu002 for the contribution! And thanks @avijayanhwx for reviewing this as well.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677700619



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();
+      }
+      for (OmBucketInfo bucket: buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setResponseCode(ResponseStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setResponseCode(
+              ResponseStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case ROOT:
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      int[] fileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          fileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(fileSizeDist);
+      break;
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets under volume
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(ResponseStatus.TYPE_NOT_APPLICABLE);

Review comment:
       Same here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] yuangu002 edited a comment on pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
yuangu002 edited a comment on pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#issuecomment-887932414


   Future works:
   1. Notification on non-FSO-enabled system.
   2. Refactoring on `/summary` and `/count`
   3. Add an additional parameter `&files=true` to DU endpoint, which counts key as a subpath.
   4. Server failure error during initialization.
   5. Add `path` and `size` on du's current level.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677675832



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();

Review comment:
       ```suggestion
           final long quota = volume.getQuotaInBytes();
           if (quota < 0) {
             // When one volume has unlimited quota, the "root" quota is unlimited.
             quotaInBytes = -1L;
             break;
           }
           quotaInBytes += quota;
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677691965



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();
+      }
+      for (OmBucketInfo bucket: buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setResponseCode(ResponseStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setResponseCode(
+              ResponseStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case ROOT:
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      int[] fileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          fileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(fileSizeDist);
+      break;
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets under volume
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(ResponseStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(ResponseStatus.PATH_NOT_FOUND);

Review comment:
       I think you still need to `setFileSizeDist()`, probably to an empty array for UNKNOWN and KEY cases. Otherwise you get NPE like this because `fileSizeDist` is `null` by default:
   
   ![image](https://user-images.githubusercontent.com/50227127/127206894-4b856184-3ef1-4bec-98bb-208dd02fc391.png)
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676859821



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,

Review comment:
       Can we use `OMFileRequest#getOMKeyInfoIfExists` inside the logic?
   
   Only `volumeTable` and `bucketTable` are [`FULL_CACHE`](https://github.com/yuangu002/ozone/blob/afce21398041c85f3ffe5753336a65b317cf0756/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L444-L455).
   
   `dirTable` and `fileTable` are `PARTIAL_CACHE` (by default), so we can just use `.get()`. Deduplicate the function.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677691965



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();
+      }
+      for (OmBucketInfo bucket: buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setResponseCode(ResponseStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setResponseCode(
+              ResponseStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case ROOT:
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      int[] fileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          fileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(fileSizeDist);
+      break;
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets under volume
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(ResponseStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(ResponseStatus.PATH_NOT_FOUND);

Review comment:
       I think you still need to `setFileSizeDist()`, probably to an empty array for `UNKNOWN` and `KEY` cases. Otherwise you get NPE like this because `fileSizeDist` is `null` by default:
   
   ![image](https://user-images.githubusercontent.com/50227127/127206894-4b856184-3ef1-4bec-98bb-208dd02fc391.png)
   
   e.g.
   
   ```suggestion
         distReponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
         distReponse.setFileSizeDist(new int[0]);
   ```
   
   `new int[0]` above works but is arguable as this is implicitly depending on `getFileSizeDist()`'s `Arrays.copyOf()` to avoid out-of-bound array access.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676867391



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -189,7 +188,7 @@ public String getTaskName() {
       }
 
       // actually fileTable with FSO

Review comment:
       nit
   ```suggestion
         // Get fileTable used by FSO
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677679368



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DUResponse.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ * HTTP Response wrapped for Disk Usage requests.
+ */
+public class DUResponse {
+  /** Path status. */
+  @JsonProperty("status")
+  private ResponseStatus status;
+
+  /** The number of subpaths under the request path. */
+  @JsonProperty("count")
+  private int count;
+
+  /** Encapsulates a DU instance for a subpath. */
+  @JsonProperty("duData")
+  private List<DiskUsage> duData;
+
+  public DUResponse() {
+    this.status = ResponseStatus.OK;
+  }
+
+  public ResponseStatus getStatus() {
+    return this.status;
+  }
+
+  public void setStatus(ResponseStatus status) {
+    this.status = status;
+  }
+
+  public int getCount() {
+    return count;
+  }
+
+  public void setCount(int count) {
+    this.count = count;
+  }
+
+  public List<DiskUsage> getDuData() {
+    return duData;
+  }
+
+  public void setDuData(List<DiskUsage> duData) {
+    this.duData = duData;
+  }
+
+  /**
+   * DU info for a path (path name, data size).
+   */
+  public static class DiskUsage {
+    /** The subpath name. */
+    @JsonProperty("subpath")

Review comment:
       Let's just call this field "path".
   
   ```suggestion
       @JsonProperty("path")
   ```
   
   Renaming this json property only should suffice for now. You can leave the rest of the variable names subpath unchanged.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677679368



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DUResponse.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ * HTTP Response wrapped for Disk Usage requests.
+ */
+public class DUResponse {
+  /** Path status. */
+  @JsonProperty("status")
+  private ResponseStatus status;
+
+  /** The number of subpaths under the request path. */
+  @JsonProperty("count")
+  private int count;
+
+  /** Encapsulates a DU instance for a subpath. */
+  @JsonProperty("duData")
+  private List<DiskUsage> duData;
+
+  public DUResponse() {
+    this.status = ResponseStatus.OK;
+  }
+
+  public ResponseStatus getStatus() {
+    return this.status;
+  }
+
+  public void setStatus(ResponseStatus status) {
+    this.status = status;
+  }
+
+  public int getCount() {
+    return count;
+  }
+
+  public void setCount(int count) {
+    this.count = count;
+  }
+
+  public List<DiskUsage> getDuData() {
+    return duData;
+  }
+
+  public void setDuData(List<DiskUsage> duData) {
+    this.duData = duData;
+  }
+
+  /**
+   * DU info for a path (path name, data size).
+   */
+  public static class DiskUsage {
+    /** The subpath name. */
+    @JsonProperty("subpath")

Review comment:
       Let's just call this field "path".
   
   ```suggestion
       @JsonProperty("path")
   ```
   
   Renaming this json property only should suffice for now.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677712469



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();
+      }
+      for (OmBucketInfo bucket: buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setResponseCode(ResponseStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setResponseCode(
+              ResponseStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);

Review comment:
       And same here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677712235



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);

Review comment:
       Same here.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);

Review comment:
       Same here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676912597



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,

Review comment:
       interesting. this leaves us with some refactoring work to do with OMFileRequest then.
   
   Ideally we want `determineKeyPath` and `getOMKeyInfoIfExists` calling a common helper function (in OMFileRequest) to walk the FS tree. otherwise, a future bug fix/improvement in `getOMKeyInfoIfExists` will almost certainly be **missed** by Recon.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676846629



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {

Review comment:
       Same
   ```suggestion
     static String reformatString(String path) {
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676849211



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;

Review comment:
       ```suggestion
       final String bucketDBKey = 
           omMetadataManager.getBucketKey(volName, bucketName);
       return omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) != null;
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] yuangu002 commented on pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#issuecomment-887932414


   Future works:
   1. Notification on non-FSO-enabled system.
   2. Refactoring on `/summary` and `/count`
   3. Add an additional parameter `&files=true` to DU endpoint, which counts key as a subpath.
   4. Server failure error during initialization.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676846215



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {

Review comment:
       Make the method package-private by removing 'public` modifier, as this is only used outside the class in the test `TestNSSummaryEndpoint`.
   ```suggestion
     static String[] parseRequestPath(String path) {
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] avijayanhwx commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r675810201



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OMDBUpdatesHandler.java
##########
@@ -124,10 +124,8 @@ private void processEvent(int cfIndex, byte[] keyBytes, byte[]
       }
 
       OMDBUpdateEvent event = builder.build();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(String.format("Generated OM update Event for table : %s, " +
-            "action = %s", tableName, action));
-      }
+      LOG.info(String.format("Generated OM update Event for table : %s, " +

Review comment:
       Better to leave this in DEBUG.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/BasicResponse.java
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * HTTP Response wrapped for a 'basic' request.
+ */
+public class BasicResponse {
+  /** The namespace the request path is on. */
+  @JsonProperty("type")
+  private EntityType entityType;
+
+  /** Total number of buckets for volume, 0 for other types. */
+  @JsonProperty("bucket")
+  private int numTotalBucket;

Review comment:
       Why numTotalBucket? It can just be numBucket. Buckets are not nested anyway.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java
##########
@@ -106,6 +108,9 @@ public String getTaskName() {
   }
 
   public Collection<String> getTaskTables() {
+    if (OzoneManagerRatisUtils.isBucketFSOptimized()) {

Review comment:
       Can we remove this change? It needs to be fixed along with more changes through HDDS-5463.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java
##########
@@ -401,6 +401,8 @@ public void syncDataFromOM() {
         // Get updates from OM and apply to local Recon OM DB.
         getAndApplyDeltaUpdatesFromOM(currentSequenceNumber,
             omdbUpdatesHandler);
+        LOG.info("The number of events generated: {}",

Review comment:
       Can we remove this log line?

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/PathStatus.java
##########
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+/**
+ * Enum class for a path request's status.
+ */
+public enum PathStatus {

Review comment:
       nit. suggest rename to QueryResponseStatus or ResponseStatus, so that it can be shared across different Recon endpoint classes.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")

Review comment:
       Please change the paths as discussed offline.

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconWithOzoneManagerFSO.java
##########
@@ -0,0 +1,325 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.recon;
+
+import static java.net.HttpURLConnection.HTTP_CREATED;
+import static java.net.HttpURLConnection.HTTP_OK;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_CONNECTION_REQUEST_TIMEOUT;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_CONNECTION_REQUEST_TIMEOUT_DEFAULT;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_CONNECTION_TIMEOUT;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_CONNECTION_TIMEOUT_DEFAULT;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SOCKET_TIMEOUT;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SOCKET_TIMEOUT_DEFAULT;
+import static org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl.OmSnapshotTaskName.OmSnapshotRequest;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.TestHelper;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.util.EntityUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.gson.Gson;
+import com.google.gson.internal.LinkedTreeMap;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
+
+/**
+ * Test Ozone Recon.
+ */
+public class TestReconWithOzoneManagerFSO {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+  private static MiniOzoneCluster cluster = null;
+  private static OzoneConfiguration conf;
+  private static OMMetadataManager metadataManager;
+  private static CloseableHttpClient httpClient;
+  private static String nssummaryServiceURL;
+  private static String taskStatusURL;
+  private static ObjectStore store;
+  private static String basicEndpoint;
+
+  @BeforeClass
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    int socketTimeout = (int) conf.getTimeDuration(
+            OZONE_RECON_OM_SOCKET_TIMEOUT,
+            conf.get(
+                    ReconServerConfigKeys.RECON_OM_SOCKET_TIMEOUT,
+                    OZONE_RECON_OM_SOCKET_TIMEOUT_DEFAULT),
+            TimeUnit.MILLISECONDS);
+    int connectionTimeout = (int) conf.getTimeDuration(
+            OZONE_RECON_OM_CONNECTION_TIMEOUT,
+            conf.get(
+                    ReconServerConfigKeys.RECON_OM_CONNECTION_TIMEOUT,
+                    OZONE_RECON_OM_CONNECTION_TIMEOUT_DEFAULT),
+            TimeUnit.MILLISECONDS);
+    int connectionRequestTimeout = (int)conf.getTimeDuration(
+            OZONE_RECON_OM_CONNECTION_REQUEST_TIMEOUT,
+            conf.get(
+                    ReconServerConfigKeys.RECON_OM_CONNECTION_REQUEST_TIMEOUT,
+                    OZONE_RECON_OM_CONNECTION_REQUEST_TIMEOUT_DEFAULT),
+            TimeUnit.MILLISECONDS
+    );
+    conf.setBoolean("ozone.om.enable.filesystem.paths", true);
+    conf.set("ozone.om.metadata.layout", "PREFIX");
+
+    RequestConfig config = RequestConfig.custom()
+            .setConnectTimeout(socketTimeout)
+            .setConnectionRequestTimeout(connectionTimeout)
+            .setSocketTimeout(connectionRequestTimeout).build();
+
+    cluster =
+            MiniOzoneCluster.newBuilder(conf)
+                    .setNumDatanodes(1)
+                    .includeRecon(true)
+                    .build();
+    cluster.waitForClusterToBeReady();
+    metadataManager = cluster.getOzoneManager().getMetadataManager();
+
+    cluster.getStorageContainerManager().exitSafeMode();
+
+    InetSocketAddress address =
+            cluster.getReconServer().getHttpServer().getHttpAddress();
+    String reconHTTPAddress = address.getHostName() + ":" + address.getPort();
+    nssummaryServiceURL = "http://" + reconHTTPAddress +
+            "/api/v1/nssummary";
+    taskStatusURL = "http://" + reconHTTPAddress + "/api/v1/task/status";
+
+    basicEndpoint = "/basic";
+
+    store = cluster.getClient().getObjectStore();
+    // initialize HTTPClient
+    httpClient = HttpClientBuilder
+            .create()
+            .setDefaultRequestConfig(config)
+            .build();
+  }
+
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Returns a {@link CloseableHttpClient} configured by given configuration.
+   * If conf is null, returns a default instance.
+   *
+   * @param url        URL
+   * @return a JSON String Response.
+   */
+  private String makeHttpCall(String url, String pathRequest)
+          throws IOException, URISyntaxException {
+    HttpGet httpGet = new HttpGet(url);
+    if (pathRequest != null) {
+      URI uri = new URIBuilder(httpGet.getURI())
+              .addParameter("path", pathRequest).build();
+      httpGet.setURI(uri);
+    }
+
+    HttpResponse response = httpClient.execute(httpGet);
+    int errorCode = response.getStatusLine().getStatusCode();
+    HttpEntity entity = response.getEntity();
+
+    if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
+      return EntityUtils.toString(entity);
+    }
+
+    if (entity != null) {
+      throw new IOException("Unexpected exception when trying to reach " +
+              "Recon Server, " + EntityUtils.toString(entity));
+    } else {
+      throw new IOException("Unexpected null in http payload," +
+              " while processing request");
+    }
+  }
+
+  private void writeTestData(String volumeName,
+                             String bucketName,
+                             String keyName) throws Exception {
+
+    String keyString = UUID.randomUUID().toString();
+    byte[] data = ContainerTestHelper.getFixedLengthString(
+            keyString, 100).getBytes(UTF_8);
+    OzoneOutputStream keyStream = TestHelper.createKey(
+            keyName, ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
+            100, store, volumeName, bucketName);
+    keyStream.write(data);
+    keyStream.close();
+  }
+
+  private void writeKeys(String vol, String bucket, String key)
+          throws Exception {
+    store.createVolume(vol);
+    OzoneVolume volume = store.getVolume(vol);
+    volume.createBucket(bucket);
+    writeTestData(vol, bucket, key);
+  }
+
+  @Test
+  public void testOmDBSyncing() throws Exception {
+    // add a vol, bucket and key
+    addKeys(0, 1);
+
+    OzoneManagerServiceProviderImpl impl = (OzoneManagerServiceProviderImpl)
+            cluster.getReconServer().getOzoneManagerServiceProvider();
+    impl.syncDataFromOM();
+
+    // check if OM metadata has vol0/bucket0/key0 info
+    String bucketKey = metadataManager.getBucketKey("vol0", "bucket0");
+    long bucketId = metadataManager.getBucketTable()
+            .get(bucketKey).getObjectID();
+
+    String ozoneKey = metadataManager.getOzonePathKey(bucketId, "key0");
+    OmKeyInfo keyInfo1 = metadataManager.getKeyTable().get(ozoneKey);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+            omKeyValueTableIterator = metadataManager.getKeyTable().iterator();
+
+    long omMetadataKeyCount = getTableKeyCount(omKeyValueTableIterator);
+    Assert.assertEquals(1, omMetadataKeyCount);
+
+    // verify if OM has /vol0/bucket0/key0
+    Assert.assertEquals("vol0", keyInfo1.getVolumeName());
+    Assert.assertEquals("bucket0", keyInfo1.getBucketName());
+
+    // HTTP call to /api/nssummary
+    String basicNSSummaryResponse = makeHttpCall(
+            nssummaryServiceURL + basicEndpoint, "/vol0");
+    Map basicNSSummaryMap =
+            getReconNSSummary(basicNSSummaryResponse);
+
+    String entityType = (String) basicNSSummaryMap.get("type");
+    Assert.assertEquals(EntityType.VOLUME.toString(), entityType);
+
+    // HTTP call to /api/task/status
+    long omLatestSeqNumber = ((RDBStore) metadataManager.getStore())
+            .getDb().getLatestSequenceNumber();
+
+    String taskStatusResponse = makeHttpCall(taskStatusURL, null);

Review comment:
       IMO we don't need to test other aspects of the OM-Recon integration here. We can keep it simple, and just test the NSSummary endpoint directly by instantiating the endpoint class using a constructor. Something like this
   
   `public class NSSummaryEndpoint {`
   
     `private ReconNamespaceSummaryManager reconNamespaceSummaryManager;`
   
     `private ReconOMMetadataManager omMetadataManager;`
   
     `@Inject
     public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
                       ReconOMMetadataManager omMetadataManager) {`
       `this.reconNamespaceSummaryManager = namespaceSummaryManager;`
       `this.omMetadataManager = omMetadataManager;`
     `}`

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {

Review comment:
       Any reason we are not supporting these APIs (basic, du, quota) at the root ("/") level? HDFS supports that. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676863345



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,
+                                      String keyName, long bucketObjectId)
+          throws IOException {
+    omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK, volName,
+            bucketName);

Review comment:
       I would argue the bucket lock here is unnecessary. As we don't ever write to the OM DB snapshot on Recon (except when fetching delta updates from OM but in that case the BUCKET_LOCK here wouldn't help either).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676923260



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;

Review comment:
       Same here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676859821



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)
+          throws IOException {
+    String bucketDBKey = omMetadataManager.getBucketKey(volName, bucketName);
+    // Check if bucket exists
+    if (omMetadataManager.getBucketTable().getSkipCache(bucketDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Given an object ID, return total count of keys under this object.
+   * @param objectId the object's ID
+   * @return count of keys
+   * @throws IOException ioEx
+   */
+  private int getTotalKeyCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    int totalCnt = nsSummary.getNumOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalCnt += getTotalKeyCount(childId);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total count of directories under this object.
+   * @param objectId the object's ID
+   * @return count of directories
+   * @throws IOException ioEx
+   */
+  private int getTotalDirCount(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0;
+    }
+    Set<Long> subdirs = nsSummary.getChildDir();
+    int totalCnt = subdirs.size();
+    for (long subdir: subdirs) {
+      totalCnt += getTotalDirCount(subdir);
+    }
+    return totalCnt;
+  }
+
+  /**
+   * Given an object ID, return total data size (no replication)
+   * under this object.
+   * @param objectId the object's ID
+   * @return total used data size in bytes
+   * @throws IOException ioEx
+   */
+  private long getTotalSize(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return 0L;
+    }
+    long totalSize = nsSummary.getSizeOfFiles();
+    for (long childId: nsSummary.getChildDir()) {
+      totalSize += getTotalSize(childId);
+    }
+    return totalSize;
+  }
+
+  /**
+   * Given an object ID, return the file size distribution.
+   * @param objectId the object's ID
+   * @return int array indicating file size distribution
+   * @throws IOException ioEx
+   */
+  private int[] getTotalFileSizeDist(long objectId) throws IOException {
+    NSSummary nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
+    if (nsSummary == null) {
+      return new int[ReconConstants.NUM_OF_BINS];
+    }
+    int[] res = nsSummary.getFileSizeBucket();
+    for (long childId: nsSummary.getChildDir()) {
+      int[] subDirFileSizeDist = getTotalFileSizeDist(childId);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        res[i] += subDirFileSizeDist[i];
+      }
+    }
+    return res;
+  }
+
+  private List<OmBucketInfo> listBucketsUnderVolume(final String volumeName)
+          throws IOException {
+    List<OmBucketInfo> result = new ArrayList<>();
+    if (Strings.isNullOrEmpty(volumeName)
+            || !checkVolumeExistence(volumeName)) {
+      return null;
+    }
+    Table bucketTable = omMetadataManager.getBucketTable();
+    String seekPrefix =
+            omMetadataManager.getVolumeKey(volumeName + OM_KEY_PREFIX);
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmBucketInfo>>
+            iterator = bucketTable.iterator();
+
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmBucketInfo> kv = iterator.next();
+
+      String key = kv.getKey();
+      OmBucketInfo omBucketInfo = kv.getValue();
+
+      if (omBucketInfo != null) {
+        // We should return only the keys, whose keys match with the seek prefix
+        if (key.startsWith(seekPrefix)) {
+          result.add(omBucketInfo);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param volName volume name
+   * @param bucketName bucket name
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  private EntityType determineKeyPath(String volName, String bucketName,

Review comment:
       Can we use `OMFileRequest#getOMKeyInfoIfExists` here directly?
   
   Only `volumeTable` and `bucketTable` are [`FULL_CACHE`](https://github.com/yuangu002/ozone/blob/afce21398041c85f3ffe5753336a65b317cf0756/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L444-L455).
   
   `dirTable` and `fileTable` are [`PARTIAL_CACHE`] (by default), so we can just use `.get()`.  to avoid code duplication.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676903922



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;

Review comment:
       If `path` already has a trailing `OM_KEY_PREFIX`, don't append `OM_KEY_PREFIX` again. Otherwise it ends up with double `OM_KEY_PREFIX` like this:
   
   e.g. `path=/vol1/bucket2/dir3/`
   
   ![image](https://user-images.githubusercontent.com/50227127/127050872-3d199bbd-b77b-4dd1-90e0-6e4037cc33e3.png)
   
   Or just normalize `path` beforehand.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676903922



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;

Review comment:
       If `path` already has a trailing `OM_KEY_PREFIX`, don't append `OM_KEY_PREFIX` again. Otherwise it ends up with double `OM_KEY_PREFIX` like this:
   
   e.g. `path=/vol1/bucket2/dir3/`
   
   ![image](https://user-images.githubusercontent.com/50227127/127050872-3d199bbd-b77b-4dd1-90e0-6e4037cc33e3.png)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677712061



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);

Review comment:
       `getEntityType()` need to take normalized path. otherwise `path=//` would still be identified as UNKNOWN.
   
   ```suggestion
       EntityType type = getEntityType(normalizedPath, names);
   ```
   
   `getEntityType()` therefore would need to treat empty string input as ROOT, since that's the value of normalizedPath when input is root.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r676847386



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.PathStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/nssummary")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/basic")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumTotalBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = path + OM_KEY_PREFIX + dirName;
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = path + OM_KEY_PREFIX + subdirName;
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(reformatString(subpath));
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      keyDU.setSubpath(reformatString(path));
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/qu")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.VOLUME) {
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      String volKey = omMetadataManager.getVolumeKey(names[0]);
+      OmVolumeArgs volumeArgs =
+              omMetadataManager.getVolumeTable().getSkipCache(volKey);
+      long quotaInBytes = volumeArgs.getQuotaInBytes();
+      long quotaUsedInBytes = 0L;
+
+      // Get the total data size used by all buckets
+      for (OmBucketInfo bucketInfo: buckets) {
+        long bucketObjectId = bucketInfo.getObjectID();
+        quotaUsedInBytes += getTotalSize(bucketObjectId);
+      }
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.BUCKET) {
+      String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+      OmBucketInfo bucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketKey);
+      long bucketObjectId = bucketInfo.getObjectID();
+      long quotaInBytes = bucketInfo.getQuotaInBytes();
+      long quotaUsedInBytes = getTotalSize(bucketObjectId);
+      quotaUsageResponse.setQuota(quotaInBytes);
+      quotaUsageResponse.setQuotaUsed(quotaUsedInBytes);
+    } else if (type == EntityType.UNKNOWN) {
+      quotaUsageResponse.setStatus(PathStatus.PATH_NOT_FOUND);
+    } else { // directory and key are not applicable for this request
+      quotaUsageResponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+    }
+    return Response.ok(quotaUsageResponse).build();
+  }
+
+  /**
+   * Endpoint that returns aggregate file size distribution under a path.
+   * @param path request path
+   * @return File size distribution response
+   * @throws IOException
+   */
+  @GET
+  @Path("/dist")
+  public Response getFileSizeDistribution(@QueryParam("path") String path)
+          throws IOException {
+    String[] names = parseRequestPath(path);
+    EntityType type = getEntityType(names);
+    FileSizeDistributionResponse distReponse =
+            new FileSizeDistributionResponse();
+    switch (type) {
+    case VOLUME:
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      int[] volumeFileSizeDist = new int[ReconConstants.NUM_OF_BINS];
+
+      // accumulate file size distribution arrays from all buckets
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+        // add on each bin
+        for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+          volumeFileSizeDist[i] += bucketFileSizeDist[i];
+        }
+      }
+      distReponse.setFileSizeDist(volumeFileSizeDist);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      int[] bucketFileSizeDist = getTotalFileSizeDist(bucketObjectId);
+      distReponse.setFileSizeDist(bucketFileSizeDist);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      int[] dirFileSizeDist = getTotalFileSizeDist(dirObjectId);
+      distReponse.setFileSizeDist(dirFileSizeDist);
+      break;
+    case KEY:
+      // key itself doesn't have file size distribution
+      distReponse.setStatus(PathStatus.TYPE_NOT_APPLICABLE);
+      break;
+    case UNKNOWN:
+      distReponse.setStatus(PathStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(distReponse).build();
+  }
+
+  /**
+   * Return the entity type of client's request, check path existence.
+   * If path doesn't exist, return Entity.UNKNOWN
+   * @param names the client's parsed request
+   * @return the entity type, unknown if path not found
+   */
+  @VisibleForTesting
+  public EntityType getEntityType(String[] names) throws IOException {
+    if (names.length == 0) {
+      return EntityType.UNKNOWN;
+    } else if (names.length == 1) { // volume level check
+      String volName = names[0];
+      if (!checkVolumeExistence(volName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.VOLUME;
+    } else if (names.length == 2) { // bucket level check
+      String volName = names[0];
+      String bucketName = names[1];
+      if (!checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      return EntityType.BUCKET;
+    } else { // length > 3. check dir or key existence (FSO-enabled)
+      String volName = names[0];
+      String bucketName = names[1];
+      String keyName = getKeyName(names);
+      // check if either volume or bucket doesn't exist
+      if (!checkVolumeExistence(volName)
+              || !checkBucketExistence(volName, bucketName)) {
+        return EntityType.UNKNOWN;
+      }
+      long bucketObjectId = getBucketObjectId(names);
+      return determineKeyPath(volName, bucketName, keyName, bucketObjectId);
+    }
+  }
+
+  /**
+   * Given a existent path, get the bucket object ID.
+   * @param names valid path request
+   * @return bucket objectID
+   * @throws IOException
+   */
+  private long getBucketObjectId(String[] names) throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(names[0], names[1]);
+    OmBucketInfo bucketInfo = omMetadataManager
+            .getBucketTable().getSkipCache(bucketKey);
+    return bucketInfo.getObjectID();
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  private long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    String dirKey = null;
+    for (int i = 2; i < cutoff; ++i) {
+      dirKey = omMetadataManager.getOzonePathKey(dirObjectId, names[i]);
+      OmDirectoryInfo dirInfo =
+              omMetadataManager.getDirectoryTable().getSkipCache(dirKey);
+      dirObjectId = dirInfo.getObjectID();
+    }
+    return dirObjectId;
+  }
+
+  @VisibleForTesting
+  public static String[] parseRequestPath(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      path = path.substring(1);
+    }
+    String[] names = path.split(OM_KEY_PREFIX);
+    return names;
+  }
+
+  /**
+   * Example: /vol1/buck1/a/b/c/d/e/file1.txt -> a/b/c/d/e/file1.txt.
+   * @param names parsed request
+   * @return key name
+   */
+  @VisibleForTesting
+  public static String getKeyName(String[] names) {
+    String[] keyArr = Arrays.copyOfRange(names, 2, names.length);
+    return String.join(OM_KEY_PREFIX, keyArr);
+  }
+
+  /**
+   * Format the path in a nice format with leading slash and without trailing
+   * slash.
+   * @param path
+   * @return
+   */
+  @VisibleForTesting
+  public static String reformatString(String path) {
+    if (!path.startsWith(OM_KEY_PREFIX)) {
+      path = OM_KEY_PREFIX + path;
+    }
+    return removeTrailingSlashIfNeeded(path);
+  }
+
+  private boolean checkVolumeExistence(String volName) throws IOException {
+    String volDBKey = omMetadataManager.getVolumeKey(volName);
+    if (omMetadataManager.getVolumeTable().getSkipCache(volDBKey) == null) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkBucketExistence(String volName, String bucketName)

Review comment:
       Rename to `bucketExists`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677681894



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DUResponse.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ * HTTP Response wrapped for Disk Usage requests.
+ */
+public class DUResponse {
+  /** Path status. */
+  @JsonProperty("status")
+  private ResponseStatus status;
+
+  /** The number of subpaths under the request path. */
+  @JsonProperty("count")
+  private int count;
+
+  /** Encapsulates a DU instance for a subpath. */
+  @JsonProperty("duData")

Review comment:
       ```suggestion
     @JsonProperty("data")
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677906878



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DUResponse.java
##########
@@ -31,13 +31,16 @@
   private ResponseStatus status;
 
   /** The number of subpaths under the request path. */
-  @JsonProperty("count")
+  @JsonProperty("subPathCount")
   private int count;
 
   /** Encapsulates a DU instance for a subpath. */
-  @JsonProperty("data")
+  @JsonProperty("subPaths")
   private List<DiskUsage> duData;
 
+  @JsonProperty("keySize")

Review comment:
       ```suggestion
     @JsonProperty("size")
   ```
   
   This field's display name should be in accordance with DiskUsage's size field.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677675832



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NSSummaryEndpoint.java
##########
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.BasicResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.helpers.OzoneFSUtils.removeTrailingSlashIfNeeded;
+/**
+ * REST APIs for namespace metadata summary.
+ */
+@Path("/namespace")
+@Produces(MediaType.APPLICATION_JSON)
+public class NSSummaryEndpoint {
+  @Inject
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  private ReconOMMetadataManager omMetadataManager;
+
+  @Inject
+  public NSSummaryEndpoint(ReconNamespaceSummaryManager namespaceSummaryManager,
+                           ReconOMMetadataManager omMetadataManager) {
+    this.reconNamespaceSummaryManager = namespaceSummaryManager;
+    this.omMetadataManager = omMetadataManager;
+  }
+
+  /**
+   * This endpoint will return the entity type and aggregate count of objects.
+   * @param path the request path.
+   * @return HTTP response with basic info: entity type, num of objects
+   * @throws IOException IOE
+   */
+  @GET
+  @Path("/summary")
+  public Response getBasicInfo(
+          @QueryParam("path") String path) throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+
+    EntityType type = getEntityType(path, names);
+
+    BasicResponse basicResponse = null;
+    switch (type) {
+    case ROOT:
+      basicResponse = new BasicResponse(EntityType.ROOT);
+      List<OmVolumeArgs> volumes = listVolumes();
+      basicResponse.setNumVolume(volumes.size());
+      List<OmBucketInfo> allBuckets = listBucketsUnderVolume(null);
+      basicResponse.setNumBucket(allBuckets.size());
+      int totalNumDir = 0;
+      int totalNumKey = 0;
+      for (OmBucketInfo bucket : allBuckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalNumDir += getTotalDirCount(bucketObjectId);
+        totalNumKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalNumDir);
+      basicResponse.setNumTotalKey(totalNumKey);
+      break;
+    case VOLUME:
+      basicResponse = new BasicResponse(EntityType.VOLUME);
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(names[0]);
+      basicResponse.setNumBucket(buckets.size());
+      int totalDir = 0;
+      int totalKey = 0;
+
+      // iterate all buckets to collect the total object count.
+      for (OmBucketInfo bucket : buckets) {
+        long bucketObjectId = bucket.getObjectID();
+        totalDir += getTotalDirCount(bucketObjectId);
+        totalKey += getTotalKeyCount(bucketObjectId);
+      }
+      basicResponse.setNumTotalDir(totalDir);
+      basicResponse.setNumTotalKey(totalKey);
+      break;
+    case BUCKET:
+      basicResponse = new BasicResponse(EntityType.BUCKET);
+      assert (names.length == 2);
+      long bucketObjectId = getBucketObjectId(names);
+      basicResponse.setNumTotalDir(getTotalDirCount(bucketObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(bucketObjectId));
+      break;
+    case DIRECTORY:
+      // path should exist so we don't need any extra verification/null check
+      long dirObjectId = getDirObjectId(names);
+      basicResponse = new BasicResponse(EntityType.DIRECTORY);
+      basicResponse.setNumTotalDir(getTotalDirCount(dirObjectId));
+      basicResponse.setNumTotalKey(getTotalKeyCount(dirObjectId));
+      break;
+    case KEY:
+      basicResponse = new BasicResponse(EntityType.KEY);
+      break;
+    case UNKNOWN:
+      basicResponse = new BasicResponse(EntityType.UNKNOWN);
+      basicResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(basicResponse).build();
+  }
+
+  /**
+   * DU endpoint to return datasize for subdirectory (bucket for volume).
+   * @param path request path
+   * @return DU response
+   * @throws IOException
+   */
+  @GET
+  @Path("/du")
+  public Response getDiskUsage(@QueryParam("path") String path)
+          throws IOException {
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    DUResponse duResponse = new DUResponse();
+    switch (type) {
+    case ROOT:
+      List<OmVolumeArgs> volumes = listVolumes();
+      duResponse.setCount(volumes.size());
+
+      List<DUResponse.DiskUsage> volumeDuData = new ArrayList<>();
+      for (OmVolumeArgs volume: volumes) {
+        String volumeName = volume.getVolume();
+        String subpath = omMetadataManager.getVolumeKey(volumeName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        long dataSize = 0;
+        diskUsage.setSubpath(subpath);
+        // iterate all buckets per volume to get total data size
+        for (OmBucketInfo bucket: listBucketsUnderVolume(volumeName)) {
+          long bucketObjectID = bucket.getObjectID();
+          dataSize += getTotalSize(bucketObjectID);
+        }
+        diskUsage.setSize(dataSize);
+        volumeDuData.add(diskUsage);
+      }
+      duResponse.setDuData(volumeDuData);
+      break;
+    case VOLUME:
+      String volName = names[0];
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(volName);
+      duResponse.setCount(buckets.size());
+
+      // List of DiskUsage data for all buckets
+      List<DUResponse.DiskUsage> bucketDuData = new ArrayList<>();
+      for (OmBucketInfo bucket : buckets) {
+        String bucketName = bucket.getBucketName();
+        long bucketObjectID = bucket.getObjectID();
+        String subpath = omMetadataManager.getBucketKey(volName, bucketName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(bucketObjectID);
+        diskUsage.setSize(dataSize);
+        bucketDuData.add(diskUsage);
+      }
+      duResponse.setDuData(bucketDuData);
+      break;
+    case BUCKET:
+      long bucketObjectId = getBucketObjectId(names);
+      NSSummary bucketNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(bucketObjectId);
+
+      // get object IDs for all its subdirectories
+      Set<Long> bucketSubdirs = bucketNSSummary.getChildDir();
+      duResponse.setCount(bucketSubdirs.size());
+      List<DUResponse.DiskUsage> dirDUData = new ArrayList<>();
+      for (long subdirObjectId: bucketSubdirs) {
+        NSSummary subdirNSSummary = reconNamespaceSummaryManager
+                .getNSSummary(subdirObjectId);
+
+        // get directory's name and generate the next-level subpath.
+        String dirName = subdirNSSummary.getDirName();
+        String subpath = buildSubpath(normalizedPath, dirName);
+        // we need to reformat the subpath in the response in a
+        // format with leading slash and without trailing slash
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        dirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(dirDUData);
+      break;
+    case DIRECTORY:
+      long dirObjectId = getDirObjectId(names);
+      NSSummary dirNSSummary =
+              reconNamespaceSummaryManager.getNSSummary(dirObjectId);
+      Set<Long> subdirs = dirNSSummary.getChildDir();
+
+      duResponse = new DUResponse();
+      duResponse.setCount(subdirs.size());
+      List<DUResponse.DiskUsage> subdirDUData = new ArrayList<>();
+      // iterate all subdirectories to get disk usage data
+      for (long subdirObjectId: subdirs) {
+        NSSummary subdirNSSummary =
+                reconNamespaceSummaryManager.getNSSummary(subdirObjectId);
+        String subdirName = subdirNSSummary.getDirName();
+        // build the path for subdirectory
+        String subpath = buildSubpath(normalizedPath, subdirName);
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        // reformat the response
+        diskUsage.setSubpath(subpath);
+        long dataSize = getTotalSize(subdirObjectId);
+        diskUsage.setSize(dataSize);
+        subdirDUData.add(diskUsage);
+      }
+      duResponse.setDuData(subdirDUData);
+      break;
+    case KEY:
+      // DU for key is the data size
+      duResponse.setCount(1);
+      DUResponse.DiskUsage keyDU = new DUResponse.DiskUsage();
+      // The object ID for the directory that the key is directly in
+      long parentObjectId = getDirObjectId(names, names.length - 1);
+      String fileName = names[names.length - 1];
+      String ozoneKey =
+              omMetadataManager.getOzonePathKey(parentObjectId, fileName);
+      OmKeyInfo keyInfo =
+              omMetadataManager.getFileTable().getSkipCache(ozoneKey);
+      String subpath = buildSubpath(normalizedPath, null);
+      keyDU.setSubpath(subpath);
+      keyDU.setSize(keyInfo.getDataSize());
+      duResponse.setDuData(Collections.singletonList(keyDU));
+      break;
+    case UNKNOWN:
+      duResponse.setStatus(ResponseStatus.PATH_NOT_FOUND);
+      break;
+    default:
+      break;
+    }
+    return Response.ok(duResponse).build();
+  }
+
+  /**
+   * Quota usage endpoint that summarize the quota allowed and quota used in
+   * bytes.
+   * @param path request path
+   * @return Quota Usage response
+   * @throws IOException
+   */
+  @GET
+  @Path("/quota")
+  public Response getQuotaUsage(@QueryParam("path") String path)
+          throws IOException {
+
+    if (path == null || path.length() == 0) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    String normalizedPath = OmUtils.normalizeKey(path, false);
+    String[] names = parseRequestPath(normalizedPath);
+    EntityType type = getEntityType(path, names);
+
+    QuotaUsageResponse quotaUsageResponse = new QuotaUsageResponse();
+    if (type == EntityType.ROOT) {
+      List<OmVolumeArgs> volumes = listVolumes();
+      List<OmBucketInfo> buckets = listBucketsUnderVolume(null);
+      long quotaInBytes = 0L;
+      long quotaUsedInBytes = 0L;
+
+      for (OmVolumeArgs volume: volumes) {
+        quotaInBytes += volume.getQuotaInBytes();

Review comment:
       ```suggestion
           final long quota = volume.getQuotaInBytes();
           assert(quota >= -1L);
           if (quota == -1L) {
             // If one volume has unlimited quota, the "root" quota is unlimited.
             quotaInBytes = -1L;
             break;
           }
           quotaInBytes += quota;
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677782020



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/BasicResponse.java
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * HTTP Response wrapped for a 'basic' request.
+ */
+public class BasicResponse {
+  /** The namespace the request path is on. */
+  @JsonProperty("type")
+  private EntityType entityType;
+
+  @JsonProperty("vol")

Review comment:
       ```suggestion
     @JsonProperty("numVolume")
   ```

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/BasicResponse.java
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * HTTP Response wrapped for a 'basic' request.
+ */
+public class BasicResponse {
+  /** The namespace the request path is on. */
+  @JsonProperty("type")
+  private EntityType entityType;
+
+  @JsonProperty("vol")
+  private int numVolume;
+
+  /** Total number of buckets for volume, 0 for other types. */
+  @JsonProperty("bucket")

Review comment:
       ```suggestion
     @JsonProperty("numBucket")
   ```

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/BasicResponse.java
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * HTTP Response wrapped for a 'basic' request.
+ */
+public class BasicResponse {
+  /** The namespace the request path is on. */
+  @JsonProperty("type")
+  private EntityType entityType;
+
+  @JsonProperty("vol")
+  private int numVolume;
+
+  /** Total number of buckets for volume, 0 for other types. */
+  @JsonProperty("bucket")
+  private int numBucket;
+
+  /** Total number of directories for a bucket or directory, 0 for others. */
+  @JsonProperty("dir")

Review comment:
       ```suggestion
     @JsonProperty("numDir")
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] yuangu002 commented on pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#issuecomment-887769884


   > Also I noticed a peculiar thing that, **before** Recon's initial sync with OM, `/namespace` endpoints will return HTTP 500. But `/containers` endpoint won't.
   > 
   > <img alt="500" width="500" src="https://user-images.githubusercontent.com/50227127/127209797-47f42aed-daf8-47a9-a7ee-945479e8b043.png">
   
   I believe it's a NPE on either Recon's OM or the Table isn't initialized.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a change in pull request #2417: HDDS-5378 Add APIs to retrieve Namespace Summary from Recon

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2417:
URL: https://github.com/apache/ozone/pull/2417#discussion_r677679368



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DUResponse.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ * HTTP Response wrapped for Disk Usage requests.
+ */
+public class DUResponse {
+  /** Path status. */
+  @JsonProperty("status")
+  private ResponseStatus status;
+
+  /** The number of subpaths under the request path. */
+  @JsonProperty("count")
+  private int count;
+
+  /** Encapsulates a DU instance for a subpath. */
+  @JsonProperty("duData")
+  private List<DiskUsage> duData;
+
+  public DUResponse() {
+    this.status = ResponseStatus.OK;
+  }
+
+  public ResponseStatus getStatus() {
+    return this.status;
+  }
+
+  public void setStatus(ResponseStatus status) {
+    this.status = status;
+  }
+
+  public int getCount() {
+    return count;
+  }
+
+  public void setCount(int count) {
+    this.count = count;
+  }
+
+  public List<DiskUsage> getDuData() {
+    return duData;
+  }
+
+  public void setDuData(List<DiskUsage> duData) {
+    this.duData = duData;
+  }
+
+  /**
+   * DU info for a path (path name, data size).
+   */
+  public static class DiskUsage {
+    /** The subpath name. */
+    @JsonProperty("subpath")

Review comment:
       Let's just call this field "path".
   
   ```suggestion
       @JsonProperty("path")
   ```
   
   Renaming this json property only should suffice for now, along with unit tests that checks this field from json, if any. You can leave the rest of the variable names subpath unchanged.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org