Jackie-Jiang commented on a change in pull request #6778:
URL: https://github.com/apache/pinot/pull/6778#discussion_r704772140
##########
File path:
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
##########
@@ -613,6 +687,16 @@ private boolean isPeerURL(String segmentLocation) {
.startsWith(CommonConstants.Segment.PEER_SEGMENT_DOWNLOAD_SCHEME);
}
+ /**
+ * Cache the LLC segment without deep store download uri to
+ * {@link PinotLLCRealtimeSegmentManager#_llcSegmentMapForUpload}.
+ */
+ @VisibleForTesting
+ void cacheLLCSegmentNameForUpload(String realtimeTableName, String
segmentName) {
+ String tableNameWithType =
TableNameBuilder.REALTIME.tableNameWithType(realtimeTableName);
Review comment:
`realtimeTableName` should already have the type suffix
##########
File path:
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
##########
@@ -135,9 +158,20 @@
private final Lock[] _idealStateUpdateLocks;
private final TableConfigCache _tableConfigCache;
private final FlushThresholdUpdateManager _flushThresholdUpdateManager;
+ private final boolean _isUploadingRealtimeMissingSegmentStoreCopyEnabled;
private volatile boolean _isStopping = false;
private AtomicInteger _numCompletingSegments = new AtomicInteger(0);
+ private FileUploadDownloadClient _fileUploadDownloadClient;
+ /**
+ * Map caching the LLC segment names without deep store download uri.
+ * Controller gets the LLC segment names from this map, and asks servers to
upload the segments to segment store.
+ * This helps to alleviates excessive ZK access when fetching LLC segment
list.
+ * Key: table name; Value: LLC segment names to be uploaded to segment store.
+ */
+ private Map<String, Queue<String>> _llcSegmentMapForUpload;
Review comment:
I don't really see how this can help reduce the ZK access. It is
actually adding a lot of unnecessary computation and memory overhead, and can
cause race condition as mentioned in the PR
##########
File path:
pinot-common/src/main/java/org/apache/pinot/common/utils/LLCSegmentName.java
##########
@@ -21,10 +21,14 @@
import org.apache.commons.lang3.StringUtils;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
public class LLCSegmentName extends SegmentName implements Comparable {
private final static String DATE_FORMAT = "yyyyMMdd'T'HHmm'Z'";
+ private final static DateTimeFormatter DATE_FORMATTER =
DateTimeFormat.forPattern(DATE_FORMAT).withZoneUTC();
Review comment:
Let's also change line 59 to `DATE_FORMATTER.print(msSinceEpoch)`
##########
File path:
pinot-common/src/main/java/org/apache/pinot/common/utils/FileUploadDownloadClient.java
##########
@@ -727,6 +727,23 @@ public SimpleHttpResponse uploadSegment(URI uri, String
segmentName, InputStream
return uploadSegment(uri, segmentName, inputStream, null, parameters,
DEFAULT_SOCKET_TIMEOUT_MS);
}
+ /**
+ * Controller periodic task uses this endpoint to ask servers to upload
committed llc segment to segment store if missing.
+ * @param uri The uri to ask servers to upload segment to segment store
+ * @return the uploaded segment download url from segment store
+ * @throws URISyntaxException
+ * @throws IOException
+ * @throws HttpErrorStatusException
+ */
+ public String uploadToSegmentStore(String uri)
Review comment:
Suggest modeling it as `public SimpleHttpResponse sendPostRequest(URI
uri)` (similar to `public SimpleHttpResponse sendGetRequest(URI uri)`) and
handle the response on the caller side. Passing the request uri to
`uploadToSegmentStore()` is kind of weird
##########
File path:
pinot-controller/src/main/java/org/apache/pinot/controller/ControllerConf.java
##########
@@ -173,6 +173,16 @@
public static final String SEGMENT_RELOCATOR_INITIAL_DELAY_IN_SECONDS =
"controller.segmentRelocator.initialDelayInSeconds";
+ // The flag to indicate if controller periodic job will fix the missing
LLC segment deep store copy.
+ // Default value is false.
+ public static final String ENABLE_DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT =
+ "controller.realtime.segment.deepStoreUploadRetryEnabled";
+ // For realtime ingestion, we care more about recent data. This config
indicates the fix range for missing LLC
+ // segment deep store copy, i.e. controller periodic job will only fix the
segments created within this
+ // range. Note that smaller value puts less pressure on servers and
zookeeper.
+ public static final String
DEEP_STORE_RETRY_UPLOAD_LLC_SEGMENT_CREATED_IN_DAYS =
Review comment:
I feel we should not make this configurable but always re-upload if the
segment is missing in deep store no matter how old the segment is. If the
intention for this is just to avoid reading all segment ZK metadata for the
table, that is unnecessary because this is triggered only in periodic task, and
we have multiple places where all segment ZK metadata are fetched and so far
there is no performance issue.
--
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]