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/26 17:32:06 UTC

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

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