You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/07/29 17:51:43 UTC

[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #5718: Feature/#5390 segment indexing reload status api

mcvsubbu commented on a change in pull request #5718:
URL: https://github.com/apache/incubator-pinot/pull/5718#discussion_r457517934



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
##########
@@ -485,4 +493,91 @@ private void deleteSegmentsInternal(String tableNameWithType, List<String> segme
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.FORBIDDEN);
     }
   }
+
+  @GET
+  @Path("segments/{tableName}/reload-status")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Status of segment reload", notes = "Status of segment reload")
+  public Map<String, TableMetadataReader.TableReloadStatus> getReloadStatus(
+          @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
+          @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr) {
+    List<String> tableNamesWithType = getExistingTableNamesWithType(tableName, Constants.validateTableType(tableTypeStr));
+    Map<String, TableMetadataReader.TableReloadStatus> reloadStatusMap = new HashMap<>();
+    for (String tableNameWithType : tableNamesWithType) {
+      TableMetadataReader.TableReloadStatus tableReloadStatus = null;
+      try {
+        tableReloadStatus = getSegmentsReloadStatus(tableNameWithType);
+      } catch (InvalidConfigException e) {
+        throw new ControllerApplicationException(LOGGER,
+                "Failed to load segment reload status for table: " + tableName, Status.NOT_FOUND);

Review comment:
       If you are returning 404 (NOT_FOUND) then please do not use "Failed" in the exception message. Since the exception is invalid config, determine what is invalid and throw that exception, may be as 400 (BAD_REQUEST)

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
##########
@@ -485,4 +493,91 @@ private void deleteSegmentsInternal(String tableNameWithType, List<String> segme
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.FORBIDDEN);
     }
   }
+
+  @GET
+  @Path("segments/{tableName}/reload-status")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Status of segment reload", notes = "Status of segment reload")
+  public Map<String, TableMetadataReader.TableReloadStatus> getReloadStatus(
+          @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
+          @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr) {
+    List<String> tableNamesWithType = getExistingTableNamesWithType(tableName, Constants.validateTableType(tableTypeStr));
+    Map<String, TableMetadataReader.TableReloadStatus> reloadStatusMap = new HashMap<>();

Review comment:
       If there are no tables found, then this is the place to throw 404

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
##########
@@ -176,23 +198,23 @@
   @Produces(MediaType.APPLICATION_JSON)
   @ApiOperation(value = "Get a map from server to segments hosted by the server (deprecated, use 'GET /segments/{tableName}/servers' instead)", notes = "Get a map from server to segments hosted by the server (deprecated, use 'GET /segments/{tableName}/servers' instead)")
   public List<Map<String, String>> getServerToSegmentsMapDeprecated1(
-      @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
-      @ApiParam(value = "MUST be null") @QueryParam("state") String stateStr,
-      @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr)
-      throws JsonProcessingException {
+          @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,

Review comment:
       Is your IDE set to pinot coding guidelines?

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentStatus.java
##########
@@ -0,0 +1,64 @@
+/**
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.pinot.common.restlet.resources;
+
+import java.util.Objects;
+
+public class SegmentStatus {

Review comment:
       Please document each member in this object clearly, what it contains in various situations

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentStatus.java
##########
@@ -0,0 +1,43 @@
+/**
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.pinot.common.restlet.resources;
+
+/**
+ * Holds segment last reload time status along with any errors for a segment with unsuccessful call to get reload times.
+ */
+public class SegmentStatus {
+  public String _segmentName;
+  public String _segmentReloadTime;

Review comment:
       * Why is this a "String"? Why not 'long'?
   * Please clearly indicate the time unit in the field name. e.g. `segmentReloadTimeUTCMs`
   * What is the value in this field if the segment was never reloaded, or the server restarts after the reload? Or, the reload faced an error and failed?

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
##########
@@ -485,4 +503,80 @@ private void deleteSegmentsInternal(String tableNameWithType, List<String> segme
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.FORBIDDEN);
     }
   }
+
+  @GET
+  @Path("segments/{tableName}/reload-status")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Status of segment reload", notes = "Status of segment reload")
+  public Map<String, ServerSegmentMetadataReader.TableReloadStatus> getReloadStatus(
+      @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
+      @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr) {
+    TableType tableType = Constants.validateTableType(tableTypeStr);
+    if (tableType == TableType.REALTIME) {
+      throw new ControllerApplicationException(LOGGER,
+          "Table type : " + tableTypeStr + " not yet supported.", Status.NOT_IMPLEMENTED);
+    }
+
+    List<String> tableNamesWithType = getExistingTableNamesWithType(tableName, Constants.validateTableType(tableTypeStr));
+    Map<String, ServerSegmentMetadataReader.TableReloadStatus> reloadStatusMap = new HashMap<>();
+    for (String tableNameWithType : tableNamesWithType) {
+      ServerSegmentMetadataReader.TableReloadStatus tableReloadStatus;
+      try {
+        tableReloadStatus = getSegmentsReloadStatus(tableNameWithType);
+      } catch (InvalidConfigException e) {
+        throw new ControllerApplicationException(LOGGER, e.getMessage(), Status.BAD_REQUEST);
+      }
+      reloadStatusMap.put(tableNameWithType, tableReloadStatus);
+    }
+    return reloadStatusMap;
+  }
+
+  private ServerSegmentMetadataReader.TableReloadStatus getSegmentsReloadStatus(String tableNameWithType)
+      throws InvalidConfigException {
+    TableMetadataReader tableMetadataReader =
+        new TableMetadataReader(_executor, _connectionManager, _pinotHelixResourceManager);
+    return tableMetadataReader.getReloadStatus(tableNameWithType,
+        _controllerConf.getServerAdminRequestTimeoutSeconds() * 1000);
+  }
+
+  @GET
+  @Path("segments/{tableName}/metadata")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Get the server metadata for all table segments", notes = "Get the server metadata for all table segments")
+  public Map<String, String> getServerMetadata(@ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
+                                               @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr) {
+    LOGGER.info("Received a request to fetch metadata for all segments for table {}", tableName);

Review comment:
       Seems like a debug level log.

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentStatus.java
##########
@@ -0,0 +1,64 @@
+/**
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.pinot.common.restlet.resources;
+
+import java.util.Objects;
+
+public class SegmentStatus {

Review comment:
       I strongly suggest we return a different object to the user instead of returning this one. It will enable us to evolve the internal interface independent of interface exposed to the user.
   Also, the controller returned status could include other run-time status about a segment. For example:
   1. Helix externalview of the server so we know if it is online or not.
   2. The crc, size, date uploaded/refreshed, etc. from segment metadata
   3. Other info as wee want to add later (e.g. number of times segment was hit/searched/selected since reboot, etc.)

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
##########
@@ -485,4 +503,80 @@ private void deleteSegmentsInternal(String tableNameWithType, List<String> segme
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.FORBIDDEN);
     }
   }
+
+  @GET
+  @Path("segments/{tableName}/reload-status")

Review comment:
       Let us call it "loadStatus" or "runTimeStatus" (I think all our APIs are in camel case). And change the comments/String below appropriately

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentStatus.java
##########
@@ -0,0 +1,64 @@
+/**
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.pinot.common.restlet.resources;
+
+import java.util.Objects;
+
+public class SegmentStatus {

Review comment:
       So, you are using this same class in both Controller as well as Server APIs. That is nice but has its pit falls. Imagine a case when we add a new field to this object. We cannot upgrade controllers and servers at the same time. So, there can exist a situation where servers are sending the old object (serialized) and the controller is trying to deserialize them using the new object. Or, vice versa.
   
   At the minimum:
   1. Write a block of comment at the top of the class that this class is upgrade sensitive explaining what may happen if it is changed without regard to upgrade consideration.
   2. Specifically mention that fields cannot be removed from the class (I suppose, unless there are proper defaults)
   3. Add an annotation to ignore unknown fields at the class level.

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
##########
@@ -485,4 +503,80 @@ private void deleteSegmentsInternal(String tableNameWithType, List<String> segme
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.FORBIDDEN);
     }
   }
+
+  @GET
+  @Path("segments/{tableName}/reload-status")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Status of segment reload", notes = "Status of segment reload")
+  public Map<String, ServerSegmentMetadataReader.TableReloadStatus> getReloadStatus(
+      @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
+      @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr) {
+    TableType tableType = Constants.validateTableType(tableTypeStr);
+    if (tableType == TableType.REALTIME) {
+      throw new ControllerApplicationException(LOGGER,
+          "Table type : " + tableTypeStr + " not yet supported.", Status.NOT_IMPLEMENTED);
+    }
+
+    List<String> tableNamesWithType = getExistingTableNamesWithType(tableName, Constants.validateTableType(tableTypeStr));
+    Map<String, ServerSegmentMetadataReader.TableReloadStatus> reloadStatusMap = new HashMap<>();
+    for (String tableNameWithType : tableNamesWithType) {
+      ServerSegmentMetadataReader.TableReloadStatus tableReloadStatus;
+      try {
+        tableReloadStatus = getSegmentsReloadStatus(tableNameWithType);
+      } catch (InvalidConfigException e) {
+        throw new ControllerApplicationException(LOGGER, e.getMessage(), Status.BAD_REQUEST);
+      }
+      reloadStatusMap.put(tableNameWithType, tableReloadStatus);
+    }
+    return reloadStatusMap;
+  }
+
+  private ServerSegmentMetadataReader.TableReloadStatus getSegmentsReloadStatus(String tableNameWithType)
+      throws InvalidConfigException {
+    TableMetadataReader tableMetadataReader =
+        new TableMetadataReader(_executor, _connectionManager, _pinotHelixResourceManager);
+    return tableMetadataReader.getReloadStatus(tableNameWithType,
+        _controllerConf.getServerAdminRequestTimeoutSeconds() * 1000);
+  }
+
+  @GET
+  @Path("segments/{tableName}/metadata")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Get the server metadata for all table segments", notes = "Get the server metadata for all table segments")
+  public Map<String, String> getServerMetadata(@ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
+                                               @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr) {
+    LOGGER.info("Received a request to fetch metadata for all segments for table {}", tableName);
+    TableType tableType = Constants.validateTableType(tableTypeStr);
+    if (tableType == TableType.REALTIME) {
+      throw new ControllerApplicationException(LOGGER,
+          "Table type : " + tableTypeStr + " not yet supported.", Status.NOT_IMPLEMENTED);
+    }
+
+    String tableNameWithType = getExistingTableNamesWithType(tableName, tableType).get(0);
+    Map<String, String> segmentsMetadata;
+    try {
+      segmentsMetadata = getSegmentsMetadataFromServer(tableNameWithType);
+    } catch (InvalidConfigException e) {
+      throw new ControllerApplicationException(LOGGER, e.getMessage(), Status.BAD_REQUEST);
+    } catch (IOException ioe) {
+      throw new ControllerApplicationException(LOGGER,
+          "Error parsing Pinot server response: " + ioe.getMessage(), Status.INTERNAL_SERVER_ERROR, ioe);

Review comment:
       Indicate the server name here that caused the error (unless that is logged elsewhere)

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ServerSegmentMetadataReader.java
##########
@@ -0,0 +1,121 @@
+package org.apache.pinot.controller.api.resources;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.collect.BiMap;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.Executor;
+import org.apache.commons.httpclient.HttpConnectionManager;
+import org.apache.commons.httpclient.URI;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.apache.pinot.common.http.MultiGetRequest;
+import org.apache.pinot.common.restlet.resources.SegmentStatus;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ServerSegmentMetadataReader {

Review comment:
       thy is this class in controller/api/resources? It should be inside controller/util

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
##########
@@ -485,4 +493,91 @@ private void deleteSegmentsInternal(String tableNameWithType, List<String> segme
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.FORBIDDEN);
     }
   }
+
+  @GET
+  @Path("segments/{tableName}/reload-status")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Status of segment reload", notes = "Status of segment reload")
+  public Map<String, TableMetadataReader.TableReloadStatus> getReloadStatus(
+          @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
+          @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr) {
+    List<String> tableNamesWithType = getExistingTableNamesWithType(tableName, Constants.validateTableType(tableTypeStr));
+    Map<String, TableMetadataReader.TableReloadStatus> reloadStatusMap = new HashMap<>();
+    for (String tableNameWithType : tableNamesWithType) {
+      TableMetadataReader.TableReloadStatus tableReloadStatus = null;
+      try {
+        tableReloadStatus = getSegmentsReloadStatus(tableNameWithType);
+      } catch (InvalidConfigException e) {
+        throw new ControllerApplicationException(LOGGER,
+                "Failed to load segment reload status for table: " + tableName, Status.NOT_FOUND);
+      }
+      if (Objects.isNull(tableReloadStatus))
+        throw new ControllerApplicationException(LOGGER,

Review comment:
       The exception message reads as if the table is not found. That is not the case, right?

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
##########
@@ -485,4 +493,91 @@ private void deleteSegmentsInternal(String tableNameWithType, List<String> segme
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.FORBIDDEN);
     }
   }
+
+  @GET
+  @Path("segments/{tableName}/reload-status")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Status of segment reload", notes = "Status of segment reload")
+  public Map<String, TableMetadataReader.TableReloadStatus> getReloadStatus(
+          @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
+          @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr) {
+    List<String> tableNamesWithType = getExistingTableNamesWithType(tableName, Constants.validateTableType(tableTypeStr));
+    Map<String, TableMetadataReader.TableReloadStatus> reloadStatusMap = new HashMap<>();
+    for (String tableNameWithType : tableNamesWithType) {
+      TableMetadataReader.TableReloadStatus tableReloadStatus = null;
+      try {
+        tableReloadStatus = getSegmentsReloadStatus(tableNameWithType);
+      } catch (InvalidConfigException e) {
+        throw new ControllerApplicationException(LOGGER,
+                "Failed to load segment reload status for table: " + tableName, Status.NOT_FOUND);
+      }
+      if (Objects.isNull(tableReloadStatus))
+        throw new ControllerApplicationException(LOGGER,
+                "Table: " + tableName + " not found.", Status.NOT_FOUND);
+      reloadStatusMap.put(tableNameWithType, tableReloadStatus);
+    }
+    return reloadStatusMap;
+  }
+
+  private TableMetadataReader.TableReloadStatus getSegmentsReloadStatus(String tableNameWithType)
+          throws InvalidConfigException {
+    final Map<String, List<String>> serversToSegmentsMap =
+            _pinotHelixResourceManager.getServerToSegmentsMap(tableNameWithType);
+    TableMetadataReader tableMetadataReader =
+            new TableMetadataReader(_executor, _connectionManager, _pinotHelixResourceManager);
+    return tableMetadataReader.getReloadStatus(tableNameWithType, serversToSegmentsMap,
+            _controllerConf.getServerAdminRequestTimeoutSeconds() * 1000);
+  }
+
+  @GET
+  @Path("segments/{tableName}/metadata")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Get the metadata for a segment", notes = "Get the metadata for a segment")
+  public Map<String, String> getServerMetadata(@ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
+                                               @ApiParam(value = "OFFLINE|REALTIME") @QueryParam("type") String tableTypeStr) {
+    TableType tableType = Constants.validateTableType(tableTypeStr);
+    if (tableType == TableType.REALTIME)
+      throw new ControllerApplicationException(LOGGER,
+              "Table type : " + tableTypeStr + " not yet supported.", Status.NOT_IMPLEMENTED);
+
+    List<String> tableNamesWithType = getExistingTableNamesWithType(tableName, Constants.validateTableType(tableTypeStr));
+    Map<String, String> segmentsMetadata = null;
+    try {
+      segmentsMetadata = getSegmentsMetadataFromServer(tableNamesWithType.get(0));
+    } catch (InvalidConfigException e) {
+      throw new ControllerApplicationException(LOGGER,
+              "Failed to load segment reload status for table: " + tableName, Status.NOT_FOUND);
+    } catch (IOException ioe) {
+      throw new ControllerApplicationException(LOGGER,
+              "Error parsing response to cluster config!", Response.Status.BAD_REQUEST, ioe);
+    }
+    if (segmentsMetadata == null)
+      throw new ControllerApplicationException(LOGGER,
+              "Table: " + tableName + " not found.", Status.NOT_FOUND);
+    return segmentsMetadata;
+  }
+
+  private Map<String, String> getSegmentsMetadataFromServer(String tableNameWithType)
+          throws InvalidConfigException, IOException {
+    LOGGER.trace("Inside getSegmentsMetadataFromServer() entry");

Review comment:
       remove these trace logs please

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
##########
@@ -485,4 +503,80 @@ private void deleteSegmentsInternal(String tableNameWithType, List<String> segme
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.FORBIDDEN);
     }
   }
+
+  @GET
+  @Path("segments/{tableName}/reload-status")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "Status of segment reload", notes = "Status of segment reload")

Review comment:
       I strongly suggest adding a verbosity level and/or a limit here. Can be added later if you wish. Imagine a table with a million segments. Do we really want to kill the servers trying to query all the segments? Or, output them only to let the client time out?
   
   An example could be: limit=100 by default, verbosity=5. A level of 4, 3, 2,1 will show less information for each segment. Maybe 0 will only show how many segments that are online/offline etc.?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org