Copilot commented on code in PR #17622: URL: https://github.com/apache/pinot/pull/17622#discussion_r2767160129
########## pinot-controller/src/test/java/org/apache/pinot/controller/cursors/ResponseStoreCleanerTest.java: ########## @@ -0,0 +1,332 @@ +/** + * 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.controller.cursors; + +import com.sun.net.httpserver.HttpExchange; +import com.sun.net.httpserver.HttpHandler; +import java.io.IOException; +import java.io.OutputStream; +import java.net.URI; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.zookeeper.datamodel.ZNRecord; +import org.apache.pinot.common.metrics.ControllerMetrics; +import org.apache.pinot.common.response.broker.CursorResponseNative; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.LeadControllerManager; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.utils.FakeHttpServer; +import org.apache.pinot.spi.metrics.PinotMetricUtils; +import org.apache.pinot.spi.utils.JsonUtils; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + + +public class ResponseStoreCleanerTest { + private static final String RESPONSE_STORE_PATH = "/responseStore"; + private static final long CURRENT_TIME_MS = 1000000L; + + private final Executor _executor = Executors.newFixedThreadPool(4); Review Comment: The executor is created in a field initializer but never shut down. This can lead to resource leaks when tests complete. Consider creating it in `@BeforeClass` and properly shutting it down in `@AfterClass` alongside the broker servers and connection manager cleanup. ########## pinot-controller/src/main/java/org/apache/pinot/controller/cursors/ResponseStoreCleaner.java: ########## @@ -132,34 +131,110 @@ public void doClean(long currentTime) { Integer.parseInt(HelixHelper.getGrpcPort(broker)))); } + Map<String, String> requestHeaders; try { - Map<String, String> requestHeaders = AuthProviderUtils.makeAuthHeadersMap(_authProvider); + requestHeaders = AuthProviderUtils.makeAuthHeadersMap(_authProvider); + } catch (Exception e) { + LOGGER.error("Failed to create auth headers for response store cleanup", e); + return; + } - Map<String, List<CursorResponseNative>> brokerCursorsMap = getAllQueryResults(brokers, requestHeaders); + Map<String, List<CursorResponseNative>> brokerCursorsMap; + try { + brokerCursorsMap = getAllQueryResults(brokers, requestHeaders); + } catch (Exception e) { + LOGGER.error("Failed to get query results from brokers for cleanup", e); + return; + } + + String protocol = _controllerConf.getControllerBrokerProtocol(); + int portOverride = _controllerConf.getControllerBrokerPortOverride(); - String protocol = _controllerConf.getControllerBrokerProtocol(); - int portOverride = _controllerConf.getControllerBrokerPortOverride(); + // Process each broker independently to ensure partial failures don't block cleanup of other brokers + for (Map.Entry<String, List<CursorResponseNative>> entry : brokerCursorsMap.entrySet()) { + String brokerKey = entry.getKey(); + InstanceInfo broker = brokers.get(brokerKey); + if (broker == null) { + LOGGER.warn("Broker not found for key: {}", brokerKey); + continue; + } Review Comment: The null check for `broker` is defensive but unnecessary since `brokerCursorsMap` is built from the same `brokers` map in `getAllQueryResults()`. This condition should never be true unless there's a concurrent modification issue. Consider removing this check or adding a comment explaining why it's needed if there's a legitimate edge case. ```suggestion ``` ########## pinot-controller/src/main/java/org/apache/pinot/controller/cursors/ResponseStoreCleaner.java: ########## @@ -132,34 +131,110 @@ public void doClean(long currentTime) { Integer.parseInt(HelixHelper.getGrpcPort(broker)))); } + Map<String, String> requestHeaders; try { - Map<String, String> requestHeaders = AuthProviderUtils.makeAuthHeadersMap(_authProvider); + requestHeaders = AuthProviderUtils.makeAuthHeadersMap(_authProvider); + } catch (Exception e) { + LOGGER.error("Failed to create auth headers for response store cleanup", e); + return; + } - Map<String, List<CursorResponseNative>> brokerCursorsMap = getAllQueryResults(brokers, requestHeaders); + Map<String, List<CursorResponseNative>> brokerCursorsMap; + try { + brokerCursorsMap = getAllQueryResults(brokers, requestHeaders); + } catch (Exception e) { + LOGGER.error("Failed to get query results from brokers for cleanup", e); + return; + } + + String protocol = _controllerConf.getControllerBrokerProtocol(); + int portOverride = _controllerConf.getControllerBrokerPortOverride(); - String protocol = _controllerConf.getControllerBrokerProtocol(); - int portOverride = _controllerConf.getControllerBrokerPortOverride(); + // Process each broker independently to ensure partial failures don't block cleanup of other brokers + for (Map.Entry<String, List<CursorResponseNative>> entry : brokerCursorsMap.entrySet()) { + String brokerKey = entry.getKey(); + InstanceInfo broker = brokers.get(brokerKey); + if (broker == null) { + LOGGER.warn("Broker not found for key: {}", brokerKey); + continue; + } + // Collect URLs for expired responses for THIS broker only List<String> brokerUrls = new ArrayList<>(); - for (Map.Entry<String, List<CursorResponseNative>> entry : brokerCursorsMap.entrySet()) { - for (CursorResponse response : entry.getValue()) { - if (response.getExpirationTimeMs() <= currentTime) { - InstanceInfo broker = brokers.get(entry.getKey()); - int port = portOverride > 0 ? portOverride : broker.getPort(); - brokerUrls.add( - String.format(DELETE_QUERY_RESULT, protocol, broker.getHost(), port, response.getRequestId())); - } + for (CursorResponse response : entry.getValue()) { + if (response.getExpirationTimeMs() <= currentTime) { + int port = portOverride > 0 ? portOverride : broker.getPort(); + brokerUrls.add( + String.format(DELETE_QUERY_RESULT, protocol, broker.getHost(), port, response.getRequestId())); } - Map<String, String> deleteStatus = getResponseMap(requestHeaders, brokerUrls, "DELETE", HttpDelete::new); + } + if (brokerUrls.isEmpty()) { + LOGGER.debug("No expired responses to clean up for broker: {}", brokerKey); + continue; + } + + LOGGER.info("Cleaning up {} expired responses from broker: {}", brokerUrls.size(), brokerKey); + + try { + Map<String, String> deleteStatus = + deleteExpiredResponses(requestHeaders, brokerUrls); deleteStatus.forEach( (key, value) -> LOGGER.info("ResponseStore delete response - Broker: {}. Response: {}", key, value)); + } catch (Exception e) { + // Log error but continue with other brokers - don't let one broker failure block cleanup of others + LOGGER.error("Failed to delete expired responses from broker: {}. Will retry on next cleanup cycle.", + brokerKey, e); } - } catch (Exception e) { - LOGGER.error(e.getMessage()); } } + /** + * Delete expired responses from brokers. Treats 404 responses as success since the goal + * is to ensure the response doesn't exist (idempotent delete). + */ + private Map<String, String> deleteExpiredResponses(Map<String, String> requestHeaders, List<String> brokerUrls) + throws Exception { + List<Pair<String, String>> urlsAndRequestBodies = new ArrayList<>(brokerUrls.size()); + brokerUrls.forEach((url) -> urlsAndRequestBodies.add(Pair.of(url, ""))); + + CompletionService<MultiHttpRequestResponse> completionService = + new MultiHttpRequest(_executor, _connectionManager).execute(urlsAndRequestBodies, requestHeaders, + DELETE_TIMEOUT_MS, "DELETE", HttpDelete::new); + + Map<String, String> responseMap = new HashMap<>(); + List<String> errMessages = new ArrayList<>(); + + for (int i = 0; i < brokerUrls.size(); i++) { + try (MultiHttpRequestResponse httpRequestResponse = completionService.take().get()) { + URI uri = httpRequestResponse.getURI(); + int status = httpRequestResponse.getResponse().getCode(); + String responseString = EntityUtils.toString(httpRequestResponse.getResponse().getEntity()); + + if (status == 200) { + responseMap.put(getInstanceKey(uri.getHost(), Integer.toString(uri.getPort())), responseString); + } else if (status == 404) { + // 404 means the response is already deleted - this is acceptable for idempotent cleanup + LOGGER.debug("Response already deleted (404) for uri: {}", uri); + responseMap.put(getInstanceKey(uri.getHost(), Integer.toString(uri.getPort())), + "Already deleted (was 404)"); + } else { + // Other errors are unexpected and should be logged + LOGGER.warn("Unexpected status={} from uri='{}', response='{}'", status, uri, responseString); + errMessages.add(String.format("Unexpected status=%d from uri='%s'", status, uri)); + } + } catch (Exception e) { + LOGGER.error("Failed to execute DELETE op", e); + errMessages.add(e.getMessage()); + } + } + + if (!errMessages.isEmpty()) { + throw new Exception("Some delete operations failed: " + StringUtils.join(errMessages, ", ")); Review Comment: Throwing a generic `Exception` loses type information and makes exception handling harder for callers. Consider creating a specific exception type (e.g., `ResponseStoreCleanupException`) or using an existing appropriate exception type. ########## pinot-controller/src/test/java/org/apache/pinot/controller/cursors/ResponseStoreCleanerTest.java: ########## @@ -0,0 +1,332 @@ +/** + * 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.controller.cursors; + +import com.sun.net.httpserver.HttpExchange; +import com.sun.net.httpserver.HttpHandler; +import java.io.IOException; +import java.io.OutputStream; +import java.net.URI; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; +import org.apache.helix.model.InstanceConfig; +import org.apache.helix.zookeeper.datamodel.ZNRecord; +import org.apache.pinot.common.metrics.ControllerMetrics; +import org.apache.pinot.common.response.broker.CursorResponseNative; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.LeadControllerManager; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.utils.FakeHttpServer; +import org.apache.pinot.spi.metrics.PinotMetricUtils; +import org.apache.pinot.spi.utils.JsonUtils; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + + +public class ResponseStoreCleanerTest { + private static final String RESPONSE_STORE_PATH = "/responseStore"; + private static final long CURRENT_TIME_MS = 1000000L; + + private final Executor _executor = Executors.newFixedThreadPool(4); + private final PoolingHttpClientConnectionManager _connectionManager = new PoolingHttpClientConnectionManager(); Review Comment: The connection manager is created but never closed. Add `_connectionManager.close()` in the `@AfterClass` method to properly release resources. ########## pinot-controller/src/main/java/org/apache/pinot/controller/cursors/ResponseStoreCleaner.java: ########## @@ -170,51 +245,54 @@ private Map<String, List<CursorResponseNative>> getAllQueryResults(Map<String, I int port = portOverride > 0 ? portOverride : broker.getPort(); brokerUrls.add(String.format(QUERY_RESULT_STORE, protocol, broker.getHost(), port)); } - LOGGER.debug("Getting running queries via broker urls: {}", brokerUrls); - Map<String, String> strResponseMap = getResponseMap(requestHeaders, brokerUrls, "GET", HttpGet::new); - return strResponseMap.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> { - try { - return JsonUtils.stringToObject(e.getValue(), new TypeReference<>() { - }); - } catch (IOException ex) { - throw new RuntimeException(ex); - } - })); - } + LOGGER.debug("Getting stored responses via broker urls: {}", brokerUrls); - private <T extends HttpUriRequestBase> Map<String, String> getResponseMap(Map<String, String> requestHeaders, - List<String> brokerUrls, String methodName, Function<String, T> httpRequestBaseSupplier) - throws Exception { List<Pair<String, String>> urlsAndRequestBodies = new ArrayList<>(brokerUrls.size()); brokerUrls.forEach((url) -> urlsAndRequestBodies.add(Pair.of(url, ""))); CompletionService<MultiHttpRequestResponse> completionService = new MultiHttpRequest(_executor, _connectionManager).execute(urlsAndRequestBodies, requestHeaders, - ResponseStoreCleaner.TIMEOUT_MS, methodName, httpRequestBaseSupplier); - Map<String, String> responseMap = new HashMap<>(); - List<String> errMessages = new ArrayList<>(brokerUrls.size()); + GET_TIMEOUT_MS, "GET", HttpGet::new); + + Map<String, List<CursorResponseNative>> responseMap = new HashMap<>(); + List<String> errMessages = new ArrayList<>(); + for (int i = 0; i < brokerUrls.size(); i++) { try (MultiHttpRequestResponse httpRequestResponse = completionService.take().get()) { - // The completion order is different from brokerUrls, thus use uri in the response. URI uri = httpRequestResponse.getURI(); int status = httpRequestResponse.getResponse().getCode(); String responseString = EntityUtils.toString(httpRequestResponse.getResponse().getEntity()); - // Unexpected server responses are collected and returned as exception. + if (status != 200) { - throw new Exception( - String.format("Unexpected status=%d and response='%s' from uri='%s'", status, responseString, uri)); + errMessages.add(String.format("Unexpected status=%d from uri='%s', response='%s'", + status, uri, responseString)); + continue; + } + + String brokerKey = getInstanceKey(uri.getHost(), Integer.toString(uri.getPort())); + try { + List<CursorResponseNative> responses = JsonUtils.stringToObject(responseString, new TypeReference<>() { + }); + responseMap.put(brokerKey, responses); + LOGGER.debug("Got {} stored responses from broker: {}", responses.size(), brokerKey); + } catch (IOException ex) { + LOGGER.error("Failed to parse response from broker: {}", brokerKey, ex); + errMessages.add(String.format("Failed to parse response from broker '%s': %s", brokerKey, ex.getMessage())); } - responseMap.put((getInstanceKey(uri.getHost(), Integer.toString(uri.getPort()))), responseString); } catch (Exception e) { - LOGGER.error("Failed to execute {} op. ", methodName, e); - // Can't just throw exception from here as there is a need to release the other connections. - // So just collect the error msg to throw them together after the for-loop. + LOGGER.error("Failed to execute GET op", e); errMessages.add(e.getMessage()); } } + if (!errMessages.isEmpty()) { - throw new Exception("Unexpected responses from brokers: " + StringUtils.join(errMessages, ",")); + LOGGER.warn("Some brokers failed to respond: {}", errMessages); + // Only throw if ALL brokers failed - allow partial success + if (responseMap.isEmpty()) { + throw new Exception("All brokers failed to respond: " + StringUtils.join(errMessages, ", ")); Review Comment: Similar to the delete operations, throwing a generic `Exception` loses type information. Consider using a more specific exception type for better error handling. ```suggestion throw new IOException("All brokers failed to respond: " + StringUtils.join(errMessages, ", ")); ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
