You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "jtao15 (via GitHub)" <gi...@apache.org> on 2023/04/24 17:25:48 UTC

[GitHub] [pinot] jtao15 opened a new pull request, #10679: Allow custom lineage update and gc

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

   This pr add LineageManager which:
   1. allow custom lineage updates before committing any lineage change
   2. allow custom lineage gc
   


-- 
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] jtao15 commented on a diff in pull request #10679: Allow custom lineage update and gc

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


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/lineage/LineageManager.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * 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.api.lineage;

Review Comment:
   Good point, will move this to `org.apache.pinot.controller.helix.core`



-- 
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] snleee commented on a diff in pull request #10679: Allow custom lineage update and gc

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


##########
pinot-common/src/main/java/org/apache/pinot/common/lineage/SegmentLineage.java:
##########
@@ -46,18 +47,22 @@
  */
 public class SegmentLineage {
   private static final String COMMA_SEPARATOR = ",";
+  private static final String CUSTOM_MAP = "custom.map";

Review Comment:
   `CUSTOM_MAP_KEY`



-- 
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] snleee merged pull request #10679: Allow custom lineage update and gc

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


-- 
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] jtao15 commented on a diff in pull request #10679: Allow custom lineage update and gc

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


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -241,12 +248,25 @@ public String load(String instanceId) {
     for (int i = 0; i < _tableUpdaterLocks.length; i++) {
       _tableUpdaterLocks[i] = new Object();
     }
+    _lineageManager = lineageManager;
   }
 
   public PinotHelixResourceManager(ControllerConf controllerConf) {
     this(controllerConf.getZkStr(), controllerConf.getHelixClusterName(), controllerConf.getDataDir(),
         controllerConf.tenantIsolationEnabled(), controllerConf.getEnableBatchMessageMode(),
-        controllerConf.getHLCTablesAllowed(), controllerConf.getDeletedSegmentsRetentionInDays());
+        controllerConf.getHLCTablesAllowed(), controllerConf.getDeletedSegmentsRetentionInDays(),
+        getLineageManager(controllerConf));
+  }
+
+  public static LineageManager getLineageManager(ControllerConf controllerConf) {

Review Comment:
   Updated to use the Factory class.



-- 
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 #10679: Allow custom lineage update and gc

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

   ## [Codecov](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10679](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (4b23402) into [master](https://codecov.io/gh/apache/pinot/commit/d806b5f8f2750fa5611584537ae1451b332f649f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d806b5f) will **decrease** coverage by `44.00%`.
   > The diff coverage is `57.84%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #10679       +/-   ##
   =============================================
   - Coverage     68.44%   24.45%   -44.00%     
   + Complexity     6421       49     -6372     
   =============================================
     Files          2107     2093       -14     
     Lines        113792   113418      -374     
     Branches      17199    17147       -52     
   =============================================
   - Hits          77887    27733    -50154     
   - Misses        30369    82765    +52396     
   + Partials       5536     2920     -2616     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `24.45% <57.84%> (+0.02%)` | :arrow_up: |
   | unittests1 | `?` | |
   | unittests2 | `?` | |
   
   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=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...n/restlet/resources/EndReplaceSegmentsRequest.java](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vcmVzdGxldC9yZXNvdXJjZXMvRW5kUmVwbGFjZVNlZ21lbnRzUmVxdWVzdC5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...estlet/resources/RevertReplaceSegmentsRequest.java](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vcmVzdGxldC9yZXNvdXJjZXMvUmV2ZXJ0UmVwbGFjZVNlZ21lbnRzUmVxdWVzdC5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/utils/ConsistentDataPushUtils.java](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9Db25zaXN0ZW50RGF0YVB1c2hVdGlscy5qYXZh) | `0.00% <ø> (ø)` | |
   | [...va/org/apache/pinot/controller/ControllerConf.java](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9Db250cm9sbGVyQ29uZi5qYXZh) | `53.04% <33.33%> (-6.74%)` | :arrow_down: |
   | [...ntroller/helix/core/PinotHelixResourceManager.java](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL1Bpbm90SGVsaXhSZXNvdXJjZU1hbmFnZXIuamF2YQ==) | `39.94% <51.51%> (-29.47%)` | :arrow_down: |
   | [.../controller/api/lineage/DefaultLineageManager.java](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvbGluZWFnZS9EZWZhdWx0TGluZWFnZU1hbmFnZXIuamF2YQ==) | `55.88% <55.88%> (ø)` | |
   | [...ces/PinotSegmentUploadDownloadRestletResource.java](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90U2VnbWVudFVwbG9hZERvd25sb2FkUmVzdGxldFJlc291cmNlLmphdmE=) | `50.41% <66.66%> (-3.53%)` | :arrow_down: |
   | [...rg/apache/pinot/common/lineage/SegmentLineage.java](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbGluZWFnZS9TZWdtZW50TGluZWFnZS5qYXZh) | `89.55% <85.71%> (-6.82%)` | :arrow_down: |
   | [...restlet/resources/StartReplaceSegmentsRequest.java](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vcmVzdGxldC9yZXNvdXJjZXMvU3RhcnRSZXBsYWNlU2VnbWVudHNSZXF1ZXN0LmphdmE=) | `66.66% <100.00%> (+9.52%)` | :arrow_up: |
   | [...troller/helix/core/retention/RetentionManager.java](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL3JldGVudGlvbi9SZXRlbnRpb25NYW5hZ2VyLmphdmE=) | `45.00% <100.00%> (-34.04%)` | :arrow_down: |
   | ... and [1 more](https://codecov.io/gh/apache/pinot/pull/10679?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ... and [1496 files with indirect coverage changes](https://codecov.io/gh/apache/pinot/pull/10679/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   :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=The+Apache+Software+Foundation)
   


-- 
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] jtao15 commented on a diff in pull request #10679: Allow custom lineage update and gc

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


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/lineage/DefaultLineageManager.java:
##########
@@ -0,0 +1,125 @@
+/**
+ * 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.api.lineage;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import org.apache.pinot.common.lineage.LineageEntry;
+import org.apache.pinot.common.lineage.LineageEntryState;
+import org.apache.pinot.common.lineage.SegmentLineage;
+import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.utils.IngestionConfigUtils;
+
+
+public class DefaultLineageManager implements LineageManager {
+  private static final long REPLACED_SEGMENTS_RETENTION_IN_MILLIS = TimeUnit.DAYS.toMillis(1L); // 1 day
+  private static final long LINEAGE_ENTRY_CLEANUP_RETENTION_IN_MILLIS = TimeUnit.DAYS.toMillis(1L); // 1 day
+
+  protected ControllerConf _controllerConf;
+
+  public DefaultLineageManager(ControllerConf controllerConf) {
+    _controllerConf = controllerConf;
+  }
+
+  @Override
+  public void updateLineageForStartReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForEndReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForRevertReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForRetention(TableConfig tableConfig, SegmentLineage lineage, List<String> allSegments,
+      List<String> segmentsToDelete, Set<String> consumingSegments) {
+    // 1. The original segments can be deleted once the merged segments are successfully uploaded
+    // 2. The zombie lineage entry & merged segments should be deleted if the segment replacement failed in
+    //    the middle
+    Set<String> segmentsForTable = new HashSet<>(allSegments);
+    Iterator<LineageEntry> lineageEntryIterator = lineage.getLineageEntries().values().iterator();
+    while (lineageEntryIterator.hasNext()) {
+      LineageEntry lineageEntry = lineageEntryIterator.next();
+      if (lineageEntry.getState() == LineageEntryState.COMPLETED) {
+        Set<String> sourceSegments = new HashSet<>(lineageEntry.getSegmentsFrom());
+        sourceSegments.retainAll(segmentsForTable);
+        if (sourceSegments.isEmpty()) {
+          // If the lineage state is 'COMPLETED' and segmentFrom are removed, it is safe clean up the lineage entry
+          lineageEntryIterator.remove();
+        } else {
+          // If the lineage state is 'COMPLETED' and we already preserved the original segments for the required
+          // retention, it is safe to delete all segments from 'segmentsFrom'
+          if (shouldDeleteReplacedSegments(tableConfig, lineageEntry)) {
+            segmentsToDelete.addAll(sourceSegments);
+          }
+        }
+      } else if (lineageEntry.getState() == LineageEntryState.REVERTED || (
+          lineageEntry.getState() == LineageEntryState.IN_PROGRESS && lineageEntry.getTimestamp()
+              < System.currentTimeMillis() - LINEAGE_ENTRY_CLEANUP_RETENTION_IN_MILLIS)) {
+        // If the lineage state is 'IN_PROGRESS' or 'REVERTED', we need to clean up the zombie lineage
+        // entry and its segments
+        Set<String> destinationSegments = new HashSet<>(lineageEntry.getSegmentsTo());
+        destinationSegments.retainAll(segmentsForTable);
+        if (destinationSegments.isEmpty()) {
+          // If the lineage state is 'IN_PROGRESS or REVERTED' and source segments are already removed, it is safe
+          // to clean up the lineage entry. Deleting lineage will allow the task scheduler to re-schedule the source

Review Comment:
   This is destination segments, for REVERTED/IN_PROGRESS (more than 24hrs) states, we make sure all segmentsTo are deleted before cleaning up the lineage entries, else both segmentsFrom and segmentsTo are used to answer queries.



-- 
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] jtao15 commented on a diff in pull request #10679: Allow custom lineage update and gc

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


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/lineage/DefaultLineageManager.java:
##########
@@ -0,0 +1,125 @@
+/**
+ * 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.api.lineage;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import org.apache.pinot.common.lineage.LineageEntry;
+import org.apache.pinot.common.lineage.LineageEntryState;
+import org.apache.pinot.common.lineage.SegmentLineage;
+import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.utils.IngestionConfigUtils;
+
+
+public class DefaultLineageManager implements LineageManager {
+  private static final long REPLACED_SEGMENTS_RETENTION_IN_MILLIS = TimeUnit.DAYS.toMillis(1L); // 1 day
+  private static final long LINEAGE_ENTRY_CLEANUP_RETENTION_IN_MILLIS = TimeUnit.DAYS.toMillis(1L); // 1 day
+
+  protected ControllerConf _controllerConf;
+
+  public DefaultLineageManager(ControllerConf controllerConf) {
+    _controllerConf = controllerConf;
+  }
+
+  @Override
+  public void updateLineageForStartReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForEndReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForRevertReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForRetention(TableConfig tableConfig, SegmentLineage lineage, List<String> allSegments,

Review Comment:
   I'm only moving the retention manager's logic to `DefaultLineageManager`



-- 
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] jtao15 commented on a diff in pull request #10679: Allow custom lineage update and gc

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


##########
pinot-common/src/main/java/org/apache/pinot/common/lineage/SegmentLineage.java:
##########
@@ -49,15 +49,22 @@ public class SegmentLineage {
 
   private final String _tableNameWithType;
   private final Map<String, LineageEntry> _lineageEntries;
+  private final Map<String, String> _simpleFields;

Review Comment:
   Discussed offline and updated to use `customMap` instead. @snleee Please take a look again.



-- 
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] snleee commented on a diff in pull request #10679: Allow custom lineage update and gc

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


##########
pinot-common/src/main/java/org/apache/pinot/common/lineage/SegmentLineage.java:
##########
@@ -49,15 +49,22 @@ public class SegmentLineage {
 
   private final String _tableNameWithType;
   private final Map<String, LineageEntry> _lineageEntries;
+  private final Map<String, String> _simpleFields;

Review Comment:
   Instead of having `simpleFields/mapFields`, I think that it's better to have a single `Map<String, String> _customMap`?



##########
pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/EndReplaceSegmentsRequest.java:
##########
@@ -21,21 +21,31 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import javax.annotation.Nullable;
 
 /**
- * segmentsTo: The new segments that actually get created. Sometimes not all segments that are passed into
+ * 1. segmentsTo: The new segments that actually get created. Sometimes not all segments that are passed into

Review Comment:
   Can we add the title saying that this class is defining `EndReplaceSegmentsRequest`?
   
   e.g. `Request object for startReplaceSegments API.`



##########
pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/RevertReplaceSegmentsRequest.java:
##########
@@ -0,0 +1,39 @@
+/**
+ * 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.common.restlet.resources;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Map;
+import javax.annotation.Nullable;
+
+
+/**
+ * customMap : custom map.

Review Comment:
   Can we add the documentation that indicates this class is for `RevertReplaceSegmentsRequest`
   
   e.g. `Request object for startReplaceSegments API.`



##########
pinot-controller/src/main/java/org/apache/pinot/controller/ControllerConf.java:
##########
@@ -258,6 +258,7 @@ private static long getRandomInitialDelayInSeconds() {
   public static final String ACCESS_CONTROL_FACTORY_CLASS = "controller.admin.access.control.factory.class";
   public static final String ACCESS_CONTROL_USERNAME = "access.control.init.username";
   public static final String ACCESS_CONTROL_PASSWORD = "access.control.init.password";
+  public static final String LINEAGE_MANAGER_CLASS = "controller.lineage.manager";

Review Comment:
   `controller.lineage.manager.class`



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -241,12 +248,25 @@ public String load(String instanceId) {
     for (int i = 0; i < _tableUpdaterLocks.length; i++) {
       _tableUpdaterLocks[i] = new Object();
     }
+    _lineageManager = lineageManager;
   }
 
   public PinotHelixResourceManager(ControllerConf controllerConf) {
     this(controllerConf.getZkStr(), controllerConf.getHelixClusterName(), controllerConf.getDataDir(),
         controllerConf.tenantIsolationEnabled(), controllerConf.getEnableBatchMessageMode(),
-        controllerConf.getHLCTablesAllowed(), controllerConf.getDeletedSegmentsRetentionInDays());
+        controllerConf.getHLCTablesAllowed(), controllerConf.getDeletedSegmentsRetentionInDays(),
+        getLineageManager(controllerConf));
+  }
+
+  public static LineageManager getLineageManager(ControllerConf controllerConf) {

Review Comment:
   Let's consider to put this logic to `LineageManagerFactory`



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/lineage/LineageManager.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * 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.api.lineage;

Review Comment:
   Is there a better place to put this? 
   
   RetentionManager is in `org.apache.pinot.controller.helix.core` and LineageManager is mainly utilized under `PinotHelixResourceManager`, which is also under `org.apache.pinot.controller.helix.core`.
   
   Can you double check if this is a better candidate?
   
   



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -185,6 +187,10 @@ public class PinotHelixResourceManager {
   private static final int DEFAULT_TABLE_UPDATER_LOCKERS_SIZE = 100;
   private static final String API_REQUEST_ID_PREFIX = "api-";
 
+  private static enum ReplaceSegmentType {

Review Comment:
   Maybe the better naming is `LineageUpdateType`?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/lineage/DefaultLineageManager.java:
##########
@@ -0,0 +1,125 @@
+/**
+ * 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.api.lineage;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import org.apache.pinot.common.lineage.LineageEntry;
+import org.apache.pinot.common.lineage.LineageEntryState;
+import org.apache.pinot.common.lineage.SegmentLineage;
+import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.utils.IngestionConfigUtils;
+
+
+public class DefaultLineageManager implements LineageManager {
+  private static final long REPLACED_SEGMENTS_RETENTION_IN_MILLIS = TimeUnit.DAYS.toMillis(1L); // 1 day
+  private static final long LINEAGE_ENTRY_CLEANUP_RETENTION_IN_MILLIS = TimeUnit.DAYS.toMillis(1L); // 1 day
+
+  protected ControllerConf _controllerConf;
+
+  public DefaultLineageManager(ControllerConf controllerConf) {
+    _controllerConf = controllerConf;
+  }
+
+  @Override
+  public void updateLineageForStartReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForEndReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForRevertReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForRetention(TableConfig tableConfig, SegmentLineage lineage, List<String> allSegments,

Review Comment:
   Can I assume that we are simply moving the retention manager's logic? Or, are we adding any extra logic here?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -3723,13 +3764,26 @@ private boolean writeLineageEntryWithTightLoop(String tableNameWithType, String
       if (!currentLineageEntry.equals(lineageEntryToMatch)) {
         String errorMsg = String.format(
             "Aborting the to update lineage entry since we find that the entry has been modified for table %s, "
-                + "entry id: %s", tableNameWithType, lineageEntryId);
+                + "entry id: %s", tableConfig, lineageEntryId);
         LOGGER.error(errorMsg);
         throw new RuntimeException(errorMsg);
       }
 
       // Update lineage entry
       segmentLineageToUpdate.updateLineageEntry(lineageEntryId, lineageEntryToUpdate);
+      switch (replaceSegmentType) {
+        case START:
+          _lineageManager.updateLineageForStartReplaceSegments(null, lineageEntryId, customMap, segmentLineageToUpdate);

Review Comment:
   Can we pass `tableConfig` since it's available? The implementation of `LineageManager` can choose whether to use it or not.



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -3723,13 +3764,26 @@ private boolean writeLineageEntryWithTightLoop(String tableNameWithType, String
       if (!currentLineageEntry.equals(lineageEntryToMatch)) {
         String errorMsg = String.format(
             "Aborting the to update lineage entry since we find that the entry has been modified for table %s, "
-                + "entry id: %s", tableNameWithType, lineageEntryId);
+                + "entry id: %s", tableConfig, lineageEntryId);
         LOGGER.error(errorMsg);
         throw new RuntimeException(errorMsg);
       }
 
       // Update lineage entry
       segmentLineageToUpdate.updateLineageEntry(lineageEntryId, lineageEntryToUpdate);
+      switch (replaceSegmentType) {
+        case START:

Review Comment:
   Should we throw exception in this case since it's not the valid path? Or, this is fine?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -3723,13 +3764,26 @@ private boolean writeLineageEntryWithTightLoop(String tableNameWithType, String
       if (!currentLineageEntry.equals(lineageEntryToMatch)) {
         String errorMsg = String.format(
             "Aborting the to update lineage entry since we find that the entry has been modified for table %s, "
-                + "entry id: %s", tableNameWithType, lineageEntryId);
+                + "entry id: %s", tableConfig, lineageEntryId);

Review Comment:
   `tableConfig -> tableConfig.getTableName()`



-- 
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] PrachiKhobragade commented on a diff in pull request #10679: Allow custom lineage update and gc

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


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -241,12 +248,25 @@ public String load(String instanceId) {
     for (int i = 0; i < _tableUpdaterLocks.length; i++) {
       _tableUpdaterLocks[i] = new Object();
     }
+    _lineageManager = lineageManager;
   }
 
   public PinotHelixResourceManager(ControllerConf controllerConf) {
     this(controllerConf.getZkStr(), controllerConf.getHelixClusterName(), controllerConf.getDataDir(),
         controllerConf.tenantIsolationEnabled(), controllerConf.getEnableBatchMessageMode(),
-        controllerConf.getHLCTablesAllowed(), controllerConf.getDeletedSegmentsRetentionInDays());
+        controllerConf.getHLCTablesAllowed(), controllerConf.getDeletedSegmentsRetentionInDays(),
+        getLineageManager(controllerConf));
+  }
+
+  public static LineageManager getLineageManager(ControllerConf controllerConf) {
+    String lineageManagerClassName = controllerConf.getLineageManagerClass();
+    try {
+      return (LineageManager) Class.forName(lineageManagerClassName).getConstructor(ControllerConf.class)
+          .newInstance(controllerConf);
+    } catch (Exception e) {
+      LOGGER.error("LinageModifier not found: {}", lineageManagerClassName);

Review Comment:
   Spell check, LineageManager



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/lineage/DefaultLineageManager.java:
##########
@@ -0,0 +1,125 @@
+/**
+ * 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.api.lineage;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import org.apache.pinot.common.lineage.LineageEntry;
+import org.apache.pinot.common.lineage.LineageEntryState;
+import org.apache.pinot.common.lineage.SegmentLineage;
+import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.utils.IngestionConfigUtils;
+
+
+public class DefaultLineageManager implements LineageManager {
+  private static final long REPLACED_SEGMENTS_RETENTION_IN_MILLIS = TimeUnit.DAYS.toMillis(1L); // 1 day
+  private static final long LINEAGE_ENTRY_CLEANUP_RETENTION_IN_MILLIS = TimeUnit.DAYS.toMillis(1L); // 1 day
+
+  protected ControllerConf _controllerConf;
+
+  public DefaultLineageManager(ControllerConf controllerConf) {
+    _controllerConf = controllerConf;
+  }
+
+  @Override
+  public void updateLineageForStartReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForEndReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForRevertReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForRetention(TableConfig tableConfig, SegmentLineage lineage, List<String> allSegments,
+      List<String> segmentsToDelete, Set<String> consumingSegments) {
+    // 1. The original segments can be deleted once the merged segments are successfully uploaded
+    // 2. The zombie lineage entry & merged segments should be deleted if the segment replacement failed in
+    //    the middle
+    Set<String> segmentsForTable = new HashSet<>(allSegments);
+    Iterator<LineageEntry> lineageEntryIterator = lineage.getLineageEntries().values().iterator();
+    while (lineageEntryIterator.hasNext()) {
+      LineageEntry lineageEntry = lineageEntryIterator.next();
+      if (lineageEntry.getState() == LineageEntryState.COMPLETED) {
+        Set<String> sourceSegments = new HashSet<>(lineageEntry.getSegmentsFrom());
+        sourceSegments.retainAll(segmentsForTable);
+        if (sourceSegments.isEmpty()) {
+          // If the lineage state is 'COMPLETED' and segmentFrom are removed, it is safe clean up the lineage entry
+          lineageEntryIterator.remove();
+        } else {
+          // If the lineage state is 'COMPLETED' and we already preserved the original segments for the required
+          // retention, it is safe to delete all segments from 'segmentsFrom'
+          if (shouldDeleteReplacedSegments(tableConfig, lineageEntry)) {
+            segmentsToDelete.addAll(sourceSegments);
+          }
+        }
+      } else if (lineageEntry.getState() == LineageEntryState.REVERTED || (
+          lineageEntry.getState() == LineageEntryState.IN_PROGRESS && lineageEntry.getTimestamp()
+              < System.currentTimeMillis() - LINEAGE_ENTRY_CLEANUP_RETENTION_IN_MILLIS)) {
+        // If the lineage state is 'IN_PROGRESS' or 'REVERTED', we need to clean up the zombie lineage
+        // entry and its segments
+        Set<String> destinationSegments = new HashSet<>(lineageEntry.getSegmentsTo());
+        destinationSegments.retainAll(segmentsForTable);
+        if (destinationSegments.isEmpty()) {
+          // If the lineage state is 'IN_PROGRESS or REVERTED' and source segments are already removed, it is safe
+          // to clean up the lineage entry. Deleting lineage will allow the task scheduler to re-schedule the source

Review Comment:
   Is it source segments or destinations segments in the comments here?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/lineage/DefaultLineageManager.java:
##########
@@ -0,0 +1,125 @@
+/**
+ * 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.api.lineage;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import org.apache.pinot.common.lineage.LineageEntry;
+import org.apache.pinot.common.lineage.LineageEntryState;
+import org.apache.pinot.common.lineage.SegmentLineage;
+import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.utils.IngestionConfigUtils;
+
+
+public class DefaultLineageManager implements LineageManager {
+  private static final long REPLACED_SEGMENTS_RETENTION_IN_MILLIS = TimeUnit.DAYS.toMillis(1L); // 1 day
+  private static final long LINEAGE_ENTRY_CLEANUP_RETENTION_IN_MILLIS = TimeUnit.DAYS.toMillis(1L); // 1 day
+
+  protected ControllerConf _controllerConf;
+
+  public DefaultLineageManager(ControllerConf controllerConf) {
+    _controllerConf = controllerConf;
+  }
+
+  @Override
+  public void updateLineageForStartReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForEndReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForRevertReplaceSegments(TableConfig tableConfig, String lineageEntryId,
+      Map<String, String> customMap, SegmentLineage lineage) {
+  }
+
+  @Override
+  public void updateLineageForRetention(TableConfig tableConfig, SegmentLineage lineage, List<String> allSegments,

Review Comment:
   This method is doing more than one task, hence it was difficult to follow, generally it should do one task, but I don't know a better way to break this up
   maybe comments about the things it does will help
   1) it updates lineage, or cleans up lineage entries
   2) It finds all the segments that could be deleted
   



##########
pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/RevertReplaceSegmentsRequest.java:
##########
@@ -0,0 +1,36 @@
+/**
+ * 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.common.restlet.resources;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Map;
+import javax.annotation.Nullable;
+
+
+public class RevertReplaceSegmentsRequest {

Review Comment:
   A javadoc for when this class will be used will be helpful



-- 
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] jtao15 commented on a diff in pull request #10679: Allow custom lineage update and gc

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


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -3723,13 +3764,26 @@ private boolean writeLineageEntryWithTightLoop(String tableNameWithType, String
       if (!currentLineageEntry.equals(lineageEntryToMatch)) {
         String errorMsg = String.format(
             "Aborting the to update lineage entry since we find that the entry has been modified for table %s, "
-                + "entry id: %s", tableNameWithType, lineageEntryId);
+                + "entry id: %s", tableConfig, lineageEntryId);
         LOGGER.error(errorMsg);
         throw new RuntimeException(errorMsg);
       }
 
       // Update lineage entry
       segmentLineageToUpdate.updateLineageEntry(lineageEntryId, lineageEntryToUpdate);
+      switch (replaceSegmentType) {
+        case START:

Review Comment:
   We are not using `writeLineageEntryWithTightLoop()` for startReplacement api currently, but I think it's fine to have it there. 



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