Copilot commented on code in PR #17241:
URL: https://github.com/apache/pinot/pull/17241#discussion_r2553369232
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java:
##########
@@ -233,17 +233,26 @@ public
PinotLLCRealtimeSegmentManager(PinotHelixResourceManager helixResourceMan
helixResourceManager);
_flushThresholdUpdateManager = new FlushThresholdUpdateManager();
_isDeepStoreLLCSegmentUploadRetryEnabled =
controllerConf.isDeepStoreRetryUploadLLCSegmentEnabled();
+ if (_isDeepStoreLLCSegmentUploadRetryEnabled != null &&
_isDeepStoreLLCSegmentUploadRetryEnabled) {
+
enableDeepStoreSegmentUploadRetry(controllerConf.getDeepStoreRetryUploadParallelism());
+ }
_isTmpSegmentAsyncDeletionEnabled =
controllerConf.isTmpSegmentAsyncDeletionEnabled();
_deepstoreUploadRetryTimeoutMs =
controllerConf.getDeepStoreRetryUploadTimeoutMs();
- _fileUploadDownloadClient = _isDeepStoreLLCSegmentUploadRetryEnabled ?
initFileUploadDownloadClient() : null;
- _deepStoreUploadExecutor = _isDeepStoreLLCSegmentUploadRetryEnabled ?
Executors.newFixedThreadPool(
- controllerConf.getDeepStoreRetryUploadParallelism()) : null;
- _deepStoreUploadExecutorPendingSegments =
- _isDeepStoreLLCSegmentUploadRetryEnabled ?
ConcurrentHashMap.newKeySet() : null;
}
- public boolean isDeepStoreLLCSegmentUploadRetryEnabled() {
- return _isDeepStoreLLCSegmentUploadRetryEnabled;
+ private void enableDeepStoreSegmentUploadRetry(int
deepStoreRetryUploadParallelism) {
+ _fileUploadDownloadClient = initFileUploadDownloadClient();
+ _deepStoreUploadExecutor =
Executors.newFixedThreadPool(deepStoreRetryUploadParallelism);
+ _deepStoreUploadExecutorPendingSegments = ConcurrentHashMap.newKeySet();
+ }
+
+ public boolean isDeepStoreLLCSegmentUploadRetryEnabled(boolean
pauselessEnabled) {
+ if (_isDeepStoreLLCSegmentUploadRetryEnabled == null && pauselessEnabled) {
+ _isDeepStoreLLCSegmentUploadRetryEnabled = true;
+ enableDeepStoreSegmentUploadRetry(
+
ControllerConf.ControllerPeriodicTasksConf.DEFAULT_DEEP_STORE_RETRY_UPLOAD_PARALLELISM);
+ }
+ return _isDeepStoreLLCSegmentUploadRetryEnabled != null &&
_isDeepStoreLLCSegmentUploadRetryEnabled;
Review Comment:
The lazy initialization in `isDeepStoreLLCSegmentUploadRetryEnabled()` is
not thread-safe. Multiple threads could simultaneously check
`_isDeepStoreLLCSegmentUploadRetryEnabled == null`, resulting in multiple calls
to `enableDeepStoreSegmentUploadRetry()` and resource leaks (multiple thread
pools and HTTP clients). Use `synchronized` or implement double-checked locking
with `volatile` to ensure thread safety.
##########
pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java:
##########
@@ -1201,12 +1201,18 @@ public void testUploadToSegmentStore()
// init fake PinotLLCRealtimeSegmentManager
ControllerConf controllerConfig = new ControllerConf();
-
controllerConfig.setProperty(ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT,
- true);
+ boolean pauselessEnabled = false;
+ if (new Random().nextDouble() > 0.5) {
+ controllerConfig.setProperty(
+
ControllerConf.ControllerPeriodicTasksConf.ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT,
+ true);
+ } else {
+ pauselessEnabled = true;
+ }
Review Comment:
Using randomization in unit tests makes them non-deterministic and harder to
debug when they fail. The test should explicitly cover both code paths with
separate test methods or parameterized tests to ensure consistent and
reproducible behavior.
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java:
##########
@@ -209,13 +209,13 @@ public class PinotLLCRealtimeSegmentManager {
private final ControllerMetrics _controllerMetrics;
private final MetadataEventNotifierFactory _metadataEventNotifierFactory;
private final FlushThresholdUpdateManager _flushThresholdUpdateManager;
- private final boolean _isDeepStoreLLCSegmentUploadRetryEnabled;
+ private Boolean _isDeepStoreLLCSegmentUploadRetryEnabled;
Review Comment:
[nitpick] Using a nullable `Boolean` to represent a tri-state
(uninitialized/true/false) makes the logic harder to follow. Consider using an
explicit enum (e.g., `UNINITIALIZED`, `ENABLED`, `DISABLED`) to improve code
clarity and avoid potential confusion with null checks.
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java:
##########
@@ -209,13 +209,13 @@ public class PinotLLCRealtimeSegmentManager {
private final ControllerMetrics _controllerMetrics;
private final MetadataEventNotifierFactory _metadataEventNotifierFactory;
private final FlushThresholdUpdateManager _flushThresholdUpdateManager;
- private final boolean _isDeepStoreLLCSegmentUploadRetryEnabled;
+ private Boolean _isDeepStoreLLCSegmentUploadRetryEnabled;
private final boolean _isTmpSegmentAsyncDeletionEnabled;
private final int _deepstoreUploadRetryTimeoutMs;
- private final FileUploadDownloadClient _fileUploadDownloadClient;
+ private FileUploadDownloadClient _fileUploadDownloadClient;
private final AtomicInteger _numCompletingSegments = new AtomicInteger(0);
- private final ExecutorService _deepStoreUploadExecutor;
- private final Set<String> _deepStoreUploadExecutorPendingSegments;
+ private ExecutorService _deepStoreUploadExecutor;
+ private Set<String> _deepStoreUploadExecutorPendingSegments;
Review Comment:
These fields are modified after object construction without synchronization.
If multiple threads access these fields during or after lazy initialization, it
could lead to visibility issues. Mark these fields as `volatile` or ensure all
access is properly synchronized.
```suggestion
private volatile Set<String> _deepStoreUploadExecutorPendingSegments;
```
--
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]