You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "npuppala111 (via GitHub)" <gi...@apache.org> on 2023/06/16 20:51:56 UTC

[GitHub] [pinot] npuppala111 opened a new pull request, #10937: [WIP] New API to get tenant tags

npuppala111 opened a new pull request, #10937:
URL: https://github.com/apache/pinot/pull/10937

   (no comment)


-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] walterddr commented on a diff in pull request #10937: New API to get tenant tags

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1244338599


##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,69 @@
+/**
+ * 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.broker.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.broker.broker.BrokerAdminApiApplication;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {
+  @Inject
+  @Named(BrokerAdminApiApplication.BROKER_INSTANCE_ID)
+  private String _instanceId;
+  @Inject
+  private HelixManager _helixManager;
+
+  @GET
+  @Path("tags")
+  @ApiOperation(value = "Tenant tags for current instance")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success"),
+      @ApiResponse(code = 500, message = "Internal server error"),
+      @ApiResponse(code = 404, message = "Table not found")
+  })
+  @Produces(MediaType.APPLICATION_JSON)
+  public List<String> getInstanceTags() {
+    InstanceConfig config = HelixHelper.getInstanceConfig(_helixManager, _instanceId);
+    return config.getTags();

Review Comment:
   null check? zk helix admin util seems can return null 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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10937: New API to get tenant tags

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1258605384


##########
pinot-server/src/main/java/org/apache/pinot/server/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.server.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.server.api.AdminApiApplication;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags
+ */
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {
+    private final ScheduledExecutorService _executorService = Executors.newScheduledThreadPool(3);

Review Comment:
   Looks like `_executorService` is unused? Can you remove it?



##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.broker.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.Collections;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.broker.broker.BrokerAdminApiApplication;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags
+ */
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",

Review Comment:
   - I think the value should be `Constants.INSTANCE_TAG`.
   - We can skip the tags attribute since some of the other resources also don't use it.
   - We can change the description to "Metadata for this instance (like tenant tags)"



##########
pinot-server/src/main/java/org/apache/pinot/server/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.server.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.server.api.AdminApiApplication;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags
+ */
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {
+    private final ScheduledExecutorService _executorService = Executors.newScheduledThreadPool(3);
+    @Inject
+    @Named(AdminApiApplication.SERVER_INSTANCE_ID)
+    private String _instanceId;
+    @Inject
+    private HelixManager _helixManager;
+
+    @GET
+    @Path("tags")
+    @ApiOperation(value = "Tenant tags for current instance")
+    @ApiResponses(value = {
+            @ApiResponse(code = 200, message = "Success"),
+            @ApiResponse(code = 500, message = "Internal server error"),
+            @ApiResponse(code = 404, message = "Table not found")

Review Comment:
   I don't think we can remove the 404 message



-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10937: [WIP] New API to get tenant tags

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1232774116


##########
pinot-server/src/main/java/org/apache/pinot/server/api/resources/InstanceTagResource.java:
##########
@@ -0,0 +1,70 @@
+/**
+ * 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.server.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.server.api.AdminApiApplication;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+
+public class InstanceTagFetcher {

Review Comment:
   I think we can call the class `InstanceResource`? That way it could contain APIs for any instance level info.



-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] chenboat merged pull request #10937: New API to get tenant tags

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat merged PR #10937:
URL: https://github.com/apache/pinot/pull/10937


-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] tibrewalpratik17 commented on a diff in pull request #10937: New API to get tenant tags

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1258691856


##########
pinot-server/src/main/java/org/apache/pinot/server/api/resources/InstanceResource.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
+ *
+ *   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.server.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.server.api.AdminApiApplication;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags
+ */
+@Api(description = "Metadata for this instance (like tenant tags)", tags = "instance",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {
+    @Inject
+    @Named(AdminApiApplication.SERVER_INSTANCE_ID)
+    private String _instanceId;
+    @Inject
+    private HelixManager _helixManager;
+
+    @GET
+    @Path("tags")
+    @ApiOperation(value = "Tenant tags for current instance")
+    @ApiResponses(value = {
+            @ApiResponse(code = 200, message = "Success"),
+            @ApiResponse(code = 500, message = "Internal server error")
+    })
+    @Produces(MediaType.APPLICATION_JSON)
+    public List<String> getInstanceTags() {
+        InstanceConfig config = HelixHelper.getInstanceConfig(_helixManager, _instanceId);
+        return config.getTags();

Review Comment:
   Shouldn't we return an empty list here as well if config or config.getTags() is null.



##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,75 @@
+/**
+ * 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.broker.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.Collections;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.broker.broker.BrokerAdminApiApplication;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags
+ */
+@Api(description = "Metadata for this instance (like tenant tags)", tags = "instance",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {
+  @Inject
+  @Named(BrokerAdminApiApplication.BROKER_INSTANCE_ID)
+  private String _instanceId;
+  @Inject
+  private HelixManager _helixManager;
+
+  @GET
+  @Path("tags")
+  @ApiOperation(value = "Tenant tags for current instance")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success"),
+      @ApiResponse(code = 500, message = "Internal server error")
+  })
+  @Produces(MediaType.APPLICATION_JSON)
+  public List<String> getInstanceTags() {
+    InstanceConfig config = HelixHelper.getInstanceConfig(_helixManager, _instanceId);
+    if (config != null && config.getTags() != null) {
+      return config.getTags();

Review Comment:
   In case of `untagged` instances, are we planning to return `untagged` or should we return empty list? 



-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10937: New API to get tenant tags

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1258605384


##########
pinot-server/src/main/java/org/apache/pinot/server/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.server.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.server.api.AdminApiApplication;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags
+ */
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {
+    private final ScheduledExecutorService _executorService = Executors.newScheduledThreadPool(3);

Review Comment:
   Looks like `_executorService` is unused?



-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] chenboat commented on pull request #10937: New API to get tenant tags

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat commented on PR #10937:
URL: https://github.com/apache/pinot/pull/10937#issuecomment-1605171907

   @npuppala111 @ankitsultana the instance tag can be retrieved from Pinot controller using the controller's instance api. The tag is stored in Zookeeper and the Pinot controller is usually the right service to got the tag info. I wonder why we can not get these info from the controller?


-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] chenboat commented on a diff in pull request #10937: New API to get tenant tags

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1240527536


##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,69 @@
+/**
+ * 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.broker.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.broker.broker.BrokerAdminApiApplication;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {
+  @Inject
+  @Named(BrokerAdminApiApplication.BROKER_INSTANCE_ID)
+  private String _instanceId;
+  @Inject
+  private HelixManager _helixManager;
+
+  @GET
+  @Path("tags")
+  @ApiOperation(value = "Tenant tags for current instance")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success"),
+      @ApiResponse(code = 500, message = "Internal server error"),
+      @ApiResponse(code = 404, message = "Table not found")
+  })
+  @Produces(MediaType.APPLICATION_JSON)
+  public List<String> getInstanceTags() {
+    InstanceConfig config = HelixHelper.getInstanceConfig(_helixManager, _instanceId);
+    return config.getTags();

Review Comment:
   Can config be null?



##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,75 @@
+/**
+ * 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.broker.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.Collections;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.broker.broker.BrokerAdminApiApplication;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags

Review Comment:
   Can you add a . to complete the sentence?



##########
pinot-server/src/main/java/org/apache/pinot/server/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.server.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.Collections;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.server.api.AdminApiApplication;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags

Review Comment:
   Add . to finish the sentence.



-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10937: [WIP] New API to get tenant tags

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1232772020


##########
pinot-common/src/main/java/org/apache/pinot/common/metadata/ZKMetadataProvider.java:
##########
@@ -10,7 +10,7 @@
  *   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
+   * software distributed under the License is distributed on an

Review Comment:
   I think we can revert both changes in this file.



##########
pom.xml:
##########
@@ -23,11 +23,11 @@
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
 
-  <parent>
+  <!-- parent>

Review Comment:
   This needs to be reverted.



##########
pinot-server/src/main/java/org/apache/pinot/server/api/resources/InstanceTagResource.java:
##########
@@ -0,0 +1,70 @@
+/**
+ * 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.server.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.server.api.AdminApiApplication;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+

Review Comment:
   Empty line to be removed



-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10937: New API to get tenant tags

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1258901684


##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,75 @@
+/**
+ * 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.broker.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.Collections;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.broker.broker.BrokerAdminApiApplication;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags
+ */
+@Api(description = "Metadata for this instance (like tenant tags)", tags = "instance",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {
+  @Inject
+  @Named(BrokerAdminApiApplication.BROKER_INSTANCE_ID)
+  private String _instanceId;
+  @Inject
+  private HelixManager _helixManager;
+
+  @GET
+  @Path("tags")
+  @ApiOperation(value = "Tenant tags for current instance")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success"),
+      @ApiResponse(code = 500, message = "Internal server error")
+  })
+  @Produces(MediaType.APPLICATION_JSON)
+  public List<String> getInstanceTags() {
+    InstanceConfig config = HelixHelper.getInstanceConfig(_helixManager, _instanceId);
+    if (config != null && config.getTags() != null) {
+      return config.getTags();

Review Comment:
   We will mirror InstanceConfig so as not to introduce new business logic.



-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] ankitsultana commented on pull request #10937: New API to get tenant tags

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on PR #10937:
URL: https://github.com/apache/pinot/pull/10937#issuecomment-1605850109

   > @npuppala111 @ankitsultana the instance tag can be retrieved from Pinot controller using the controller's instance api. The tag is stored in Zookeeper and the Pinot controller is usually the right service to got the tag info. I wonder why we can not get these info from the controller?
   
   For some of our observability requirements we need to retrieve tenant tags at instance level (for both brokers and servers). This is done by a separate thread in the JVM which *acts* as a sidecar (it's still within the same process as the instance so not exactly a sidecar per usual definition).
   
   To do this we have the following options:
   
   1. **Hit Controller HTTP API**: This would require that the observability layer knows how to discover the controller for a given instance. It also adds a criss-cross dependency: observability for a given process relies on hitting another component (controller). It also increases controller HTTP load quite a bit as the number of instances increase.
   2. **Use ZK or Helix**: Using ZK directly is not clean since we would have to leak Pinot internal details about where the instance configs are stored to our sidecar. Using Helix directly is probably the closest to what we would be doing with this HTTP API approach as well, but I think the HTTP API approach is cleaner: the observability layer shouldn't need to know about Helix or other details, and this API can also be used generally in true sidecars in the future by not just us but even other users outside Uber.
   
   Given the above I think it's best to have a separate API.


-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] codecov-commenter commented on pull request #10937: New API to get tenant tags

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #10937:
URL: https://github.com/apache/pinot/pull/10937#issuecomment-1601859996

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/10937?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#10937](https://app.codecov.io/gh/apache/pinot/pull/10937?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (4ff3576) into [master](https://app.codecov.io/gh/apache/pinot/commit/30b5da9626a774f97812b924c917fe766eba4b53?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (30b5da9) will **increase** coverage by `0.00%`.
   > The diff coverage is `0.00%`.
   
   > :exclamation: Current head 4ff3576 differs from pull request most recent head 267ac4f. Consider uploading reports for the commit 267ac4f to get more accurate results
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #10937     +/-   ##
   =========================================
     Coverage    0.11%    0.11%             
   =========================================
     Files        2189     2136     -53     
     Lines      117928   115429   -2499     
     Branches    17834    17532    -302     
   =========================================
     Hits          137      137             
   + Misses     117771   115272   -2499     
     Partials       20       20             
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1temurin11 | `?` | |
   | integration1temurin17 | `?` | |
   | integration1temurin20 | `?` | |
   | integration2temurin11 | `?` | |
   | integration2temurin17 | `?` | |
   | integration2temurin20 | `?` | |
   | unittests1temurin11 | `?` | |
   | unittests1temurin17 | `?` | |
   | unittests1temurin20 | `?` | |
   | unittests2temurin11 | `0.11% <0.00%> (ø)` | |
   | unittests2temurin17 | `0.11% <0.00%> (ø)` | |
   | unittests2temurin20 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://app.codecov.io/gh/apache/pinot/pull/10937?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [...che/pinot/broker/api/resources/BrokerResource.java](https://app.codecov.io/gh/apache/pinot/pull/10937?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9Ccm9rZXJSZXNvdXJjZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...pinot/broker/broker/BrokerAdminApiApplication.java](https://app.codecov.io/gh/apache/pinot/pull/10937?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0Jyb2tlckFkbWluQXBpQXBwbGljYXRpb24uamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...e/pinot/broker/broker/helix/BaseBrokerStarter.java](https://app.codecov.io/gh/apache/pinot/pull/10937?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jhc2VCcm9rZXJTdGFydGVyLmphdmE=) | `0.00% <ø> (ø)` | |
   
   ... and [54 files with indirect coverage changes](https://app.codecov.io/gh/apache/pinot/pull/10937/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] chenboat commented on a diff in pull request #10937: New API to get tenant tags

Posted by "chenboat (via GitHub)" <gi...@apache.org>.
chenboat commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1240527020


##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,69 @@
+/**
+ * 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.broker.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.broker.broker.BrokerAdminApiApplication;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {

Review Comment:
   Please add javadoc to clearly explain the use of this new resource api



-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10937: New API to get tenant tags

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1258617777


##########
pinot-server/src/main/java/org/apache/pinot/server/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.server.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.server.api.AdminApiApplication;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags
+ */
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {
+    private final ScheduledExecutorService _executorService = Executors.newScheduledThreadPool(3);
+    @Inject
+    @Named(AdminApiApplication.SERVER_INSTANCE_ID)
+    private String _instanceId;
+    @Inject
+    private HelixManager _helixManager;
+
+    @GET
+    @Path("tags")
+    @ApiOperation(value = "Tenant tags for current instance")
+    @ApiResponses(value = {
+            @ApiResponse(code = 200, message = "Success"),
+            @ApiResponse(code = 500, message = "Internal server error"),
+            @ApiResponse(code = 404, message = "Table not found")

Review Comment:
   I think we can remove the 404 message



-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10937: New API to get tenant tags

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1258609346


##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/InstanceResource.java:
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.broker.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.Collections;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.broker.broker.BrokerAdminApiApplication;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags
+ */
+@Api(value = "instance-tags", description = "Instance tags", tags = "instance-tags",

Review Comment:
   - I think we can remove the value attribute since other resources in broker and server are not using it.
   - The `tags` attribute value can be changed to `instance`. i.e. `tags = "instance"`.
   - We can change the description to "Metadata for this instance (like tenant tags)"



-- 
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: commits-unsubscribe@pinot.apache.org

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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #10937: New API to get tenant tags

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on code in PR #10937:
URL: https://github.com/apache/pinot/pull/10937#discussion_r1258901124


##########
pinot-server/src/main/java/org/apache/pinot/server/api/resources/InstanceResource.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
+ *
+ *   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.server.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiKeyAuthDefinition;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import io.swagger.annotations.Authorization;
+import io.swagger.annotations.SecurityDefinition;
+import io.swagger.annotations.SwaggerDefinition;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.server.api.AdminApiApplication;
+
+import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
+
+/**
+ * This resource API can be used to retrieve instance level information like instance tags
+ */
+@Api(description = "Metadata for this instance (like tenant tags)", tags = "instance",
+    authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY)})
+@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = @ApiKeyAuthDefinition(name =
+    HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = SWAGGER_AUTHORIZATION_KEY)))
+@Path("instance")
+public class InstanceResource {
+    @Inject
+    @Named(AdminApiApplication.SERVER_INSTANCE_ID)
+    private String _instanceId;
+    @Inject
+    private HelixManager _helixManager;
+
+    @GET
+    @Path("tags")
+    @ApiOperation(value = "Tenant tags for current instance")
+    @ApiResponses(value = {
+            @ApiResponse(code = 200, message = "Success"),
+            @ApiResponse(code = 500, message = "Internal server error")
+    })
+    @Produces(MediaType.APPLICATION_JSON)
+    public List<String> getInstanceTags() {
+        InstanceConfig config = HelixHelper.getInstanceConfig(_helixManager, _instanceId);
+        return config.getTags();

Review Comment:
   +1. @npuppala111 can you do that here similar to how you are doing it for broker InstanceResource?



-- 
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: commits-unsubscribe@pinot.apache.org

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