You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/01/25 18:51:12 UTC

[GitHub] [pinot] Jackie-Jiang commented on a change in pull request #8069: adding isInstantDelete API for segment deletion

Jackie-Jiang commented on a change in pull request #8069:
URL: https://github.com/apache/pinot/pull/8069#discussion_r792020328



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/SegmentDeletionManager.java
##########
@@ -83,21 +83,25 @@ public void stop() {
   }
 
   public void deleteSegments(final String tableName, final Collection<String> segmentIds) {
-    deleteSegmentsWithDelay(tableName, segmentIds, DEFAULT_DELETION_DELAY_SECONDS);
+    deleteSegments(tableName, segmentIds, false);
+  }
+
+  public void deleteSegments(final String tableName, final Collection<String> segmentIds, boolean isInstantDeletion) {
+    deleteSegmentsWithDelay(tableName, segmentIds, isInstantDeletion, DEFAULT_DELETION_DELAY_SECONDS);
   }
 
   protected void deleteSegmentsWithDelay(final String tableName, final Collection<String> segmentIds,
-      final long deletionDelaySeconds) {
+      final boolean isInstantDeletion, final long deletionDelaySeconds) {

Review comment:
       (format, minor) We don't usually put `final` in the method parameters

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/SegmentDeletionManager.java
##########
@@ -158,47 +162,68 @@ protected synchronized void deleteSegmentFromPropertyStoreAndLocal(String tableN
     if (!segmentsToRetryLater.isEmpty()) {
       long effectiveDeletionDelay = Math.min(deletionDelay * 2, MAX_DELETION_DELAY_SECONDS);
       LOGGER.info("Postponing deletion of {} segments from table {}", segmentsToRetryLater.size(), tableName);
-      deleteSegmentsWithDelay(tableName, segmentsToRetryLater, effectiveDeletionDelay);
+      deleteSegmentsWithDelay(tableName, segmentsToRetryLater, isInstantDeletion, effectiveDeletionDelay);
       return;
     }
   }
 
   public void removeSegmentsFromStore(String tableNameWithType, List<String> segments) {
+    removeSegmentsFromStore(tableNameWithType, segments, false);
+  }
+
+  public void removeSegmentsFromStore(String tableNameWithType, List<String> segments, boolean isInstantDeletion) {
     for (String segment : segments) {
-      removeSegmentFromStore(tableNameWithType, segment);
+      removeSegmentFromStore(tableNameWithType, segment, isInstantDeletion);
     }
   }
 
-  protected void removeSegmentFromStore(String tableNameWithType, String segmentId) {
+  protected void removeSegmentFromStore(String tableNameWithType, String segmentId, boolean isInstantDeletion) {
     // Ignore HLC segments as they are not stored in Pinot FS
     if (SegmentName.isHighLevelConsumerSegmentName(segmentId)) {
       return;
     }
     if (_dataDir != null) {
-      String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType);
-      URI fileToMoveURI = URIUtils.getUri(_dataDir, rawTableName, URIUtils.encode(segmentId));
-      URI deletedSegmentDestURI = URIUtils.getUri(_dataDir, DELETED_SEGMENTS, rawTableName, URIUtils.encode(segmentId));
-      PinotFS pinotFS = PinotFSFactory.create(fileToMoveURI.getScheme());
+      if (isInstantDeletion) {
+        String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType);
+        URI fileToDeleteURI = URIUtils.getUri(_dataDir, rawTableName, URIUtils.encode(segmentId));
+        PinotFS pinotFS = PinotFSFactory.create(fileToDeleteURI.getScheme());

Review comment:
       (minor) These 3 lines can be extracted out since the logic is the same

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
##########
@@ -571,12 +571,15 @@ public SuccessResponse reloadAllSegmentsDeprecated2(
   @ApiOperation(value = "Delete a segment", notes = "Delete a segment")
   public SuccessResponse deleteSegment(
       @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
-      @ApiParam(value = "Name of the segment", required = true) @PathParam("segmentName") @Encoded String segmentName) {
+      @ApiParam(value = "Name of the segment", required = true) @PathParam("segmentName") @Encoded String segmentName,
+      @ApiParam(value = "If instant delete without retention") @QueryParam("isInstantDelete") @DefaultValue("false")
+          String isInstantDelete) {

Review comment:
       Directly take a boolean?
   Suggest renaming it to `skipMovingToDeletedDir` to be more specific? Also update the description accordingly.




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