tibrewalpratik17 commented on code in PR #13347:
URL: https://github.com/apache/pinot/pull/13347#discussion_r1714403027


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/ConcurrentMapPartitionUpsertMetadataManagerForConsistentDeletes.java:
##########
@@ -0,0 +1,391 @@
+/**
+ * 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.segment.local.upsert;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.pinot.common.metrics.ServerMeter;
+import 
org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentImpl;
+import org.apache.pinot.segment.local.segment.readers.LazyRow;
+import org.apache.pinot.segment.local.utils.HashUtils;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.MutableSegment;
+import 
org.apache.pinot.segment.spi.index.mutable.ThreadSafeMutableRoaringBitmap;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+
+
+/**
+ * Implementation of {@link PartitionUpsertMetadataManager} that is backed by 
a {@link ConcurrentHashMap} and ensures
+ * consistent deletions. This should be used when the table is configured with 
'enableConsistentDeletes' set to true.
+ *
+ * Consistent deletion ensures that when deletedKeysTTL is enabled with 
UpsertCompaction, the key metadata is
+ * removed from the HashMap only after all other records in the old segments 
are compacted. This guarantees
+ * data consistency. Without this, there can be a scenario where a deleted 
record is compacted first, while an
+ * old record remains non-compacted in a previous segment. During a server 
restart, this could lead to the old
+ * record reappearing. For the end-user, this would result in a data loss or 
inconsistency scenario, as the
+ * record was marked for deletion.
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+@ThreadSafe
+public class ConcurrentMapPartitionUpsertMetadataManagerForConsistentDeletes
+    extends BasePartitionUpsertMetadataManager {
+
+  @VisibleForTesting
+  final ConcurrentHashMap<Object, 
ConcurrentMapPartitionUpsertMetadataManagerForConsistentDeletes.RecordLocation>
+      _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+  // Used to initialize a reference to previous row for merging in partial 
upsert
+  private final LazyRow _reusePreviousRow = new LazyRow();
+  private final Map<String, Object> _reuseMergeResultHolder = new HashMap<>();
+
+  public 
ConcurrentMapPartitionUpsertMetadataManagerForConsistentDeletes(String 
tableNameWithType, int partitionId,
+      UpsertContext context) {
+    super(tableNameWithType, partitionId, context);
+  }
+
+  @Override
+  protected long getNumPrimaryKeys() {
+    return _primaryKeyToRecordLocationMap.size();
+  }
+
+  @Override
+  protected void doAddOrReplaceSegment(ImmutableSegmentImpl segment, 
ThreadSafeMutableRoaringBitmap validDocIds,
+      @Nullable ThreadSafeMutableRoaringBitmap queryableDocIds, 
Iterator<RecordInfo> recordInfoIterator,
+      @Nullable IndexSegment oldSegment, @Nullable MutableRoaringBitmap 
validDocIdsForOldSegment) {
+    String segmentName = segment.getSegmentName();
+    segment.enableUpsert(this, validDocIds, queryableDocIds);
+
+    AtomicInteger numKeysInWrongSegment = new AtomicInteger();
+    while (recordInfoIterator.hasNext()) {
+      RecordInfo recordInfo = recordInfoIterator.next();
+      int newDocId = recordInfo.getDocId();
+      Comparable newComparisonValue = recordInfo.getComparisonValue();
+      
_primaryKeyToRecordLocationMap.compute(HashUtils.hashPrimaryKey(recordInfo.getPrimaryKey(),
 _hashFunction),
+          (primaryKey, currentRecordLocation) -> {
+            if (currentRecordLocation != null) {
+              // Existing primary key
+              IndexSegment currentSegment = currentRecordLocation.getSegment();
+              int currentDocId = currentRecordLocation.getDocId();
+              Comparable currentComparisonValue = 
currentRecordLocation.getComparisonValue();
+              int comparisonResult = 
newComparisonValue.compareTo(currentComparisonValue);
+              int currentDistinctSegmentCount = 
currentRecordLocation.getDistinctSegmentCount();
+
+              // The current record is in the same segment
+              // Update the record location when there is a tie to keep the 
newer record. Note that the record info
+              // iterator will return records with incremental doc ids.
+              if (currentSegment == segment) {
+                if (comparisonResult >= 0) {
+                  replaceDocId(segment, validDocIds, queryableDocIds, 
currentDocId, newDocId, recordInfo);
+                  return new 
ConcurrentMapPartitionUpsertMetadataManagerForConsistentDeletes.RecordLocation(segment,
+                      newDocId, newComparisonValue, 
currentDistinctSegmentCount);
+                } else {
+                  return currentRecordLocation;
+                }
+              }
+
+              // The current record is in an old segment being replaced
+              // This could happen when committing a consuming segment, or 
reloading a completed segment. In this
+              // case, we want to update the record location when there is a 
tie because the record locations should
+              // point to the new added segment instead of the old segment 
being replaced. Also, do not update the valid
+              // doc ids for the old segment because it has not been replaced 
yet. We pass in an optional valid doc ids
+              // snapshot for the old segment, which can be updated and used 
to track the docs not replaced yet.
+              if (currentSegment == oldSegment) {
+                if (comparisonResult >= 0) {
+                  addDocId(segment, validDocIds, queryableDocIds, newDocId, 
recordInfo);
+                  if (validDocIdsForOldSegment != null) {
+                    validDocIdsForOldSegment.remove(currentDocId);
+                  }
+                  return new RecordLocation(segment, newDocId, 
newComparisonValue,
+                      
RecordLocation.incrementSegmentCount(currentDistinctSegmentCount));
+                } else {
+                  return new RecordLocation(currentSegment, currentDocId, 
currentComparisonValue,
+                      
RecordLocation.incrementSegmentCount(currentDistinctSegmentCount));
+                }
+              }
+
+              // This should not happen because the previously replaced 
segment should have all keys removed. We still
+              // handle it here, and also track the number of keys not 
properly replaced previously.
+              String currentSegmentName = currentSegment.getSegmentName();
+              if (currentSegmentName.equals(segmentName)) {
+                numKeysInWrongSegment.getAndIncrement();
+                if (comparisonResult >= 0) {
+                  addDocId(segment, validDocIds, queryableDocIds, newDocId, 
recordInfo);
+                  return new RecordLocation(segment, newDocId, 
newComparisonValue,
+                      
RecordLocation.incrementSegmentCount(currentDistinctSegmentCount));
+                } else {
+                  return currentRecordLocation;
+                }
+              }
+
+              // The current record is in a different segment
+              // Update the record location when getting a newer comparison 
value, or the value is the same as the
+              // current value, but the segment has a larger sequence number 
(the segment is newer than the current
+              // segment).
+              if (comparisonResult > 0 || (comparisonResult == 0 && 
shouldReplaceOnComparisonTie(segmentName,
+                  currentSegmentName, 
segment.getSegmentMetadata().getIndexCreationTime(),
+                  
currentSegment.getSegmentMetadata().getIndexCreationTime()))) {
+                replaceDocId(segment, validDocIds, queryableDocIds, 
currentSegment, currentDocId, newDocId, recordInfo);
+                return new RecordLocation(segment, newDocId, 
newComparisonValue,
+                    
RecordLocation.incrementSegmentCount(currentDistinctSegmentCount));
+              } else {
+                return new RecordLocation(currentSegment, currentDocId, 
currentComparisonValue,
+                    
RecordLocation.incrementSegmentCount(currentDistinctSegmentCount));
+              }
+            } else {
+              // New primary key
+              addDocId(segment, validDocIds, queryableDocIds, newDocId, 
recordInfo);
+              return new RecordLocation(segment, newDocId, newComparisonValue, 
1);
+            }
+          });
+    }
+    int numKeys = numKeysInWrongSegment.get();
+    if (numKeys > 0) {
+      _logger.warn("Found {} primary keys in the wrong segment when adding 
segment: {}", numKeys, segmentName);
+      _serverMetrics.addMeteredTableValue(_tableNameWithType, 
ServerMeter.UPSERT_KEYS_IN_WRONG_SEGMENT, numKeys);
+    }
+  }
+
+  @Override
+  protected void addSegmentWithoutUpsert(ImmutableSegmentImpl segment, 
ThreadSafeMutableRoaringBitmap validDocIds,
+      @Nullable ThreadSafeMutableRoaringBitmap queryableDocIds, 
Iterator<RecordInfo> recordInfoIterator) {
+    throw new UnsupportedOperationException("Consistent-deletion does not 
support preloading of segments.");
+  }
+
+  @Override
+  protected void doRemoveSegment(IndexSegment segment) {
+    String segmentName = segment.getSegmentName();
+    _logger.info("Removing {} segment: {}, current primary key count: {}",
+        segment instanceof ImmutableSegment ? "immutable" : "mutable", 
segmentName, getNumPrimaryKeys());
+    long startTimeMs = System.currentTimeMillis();
+
+    try (
+        UpsertUtils.PrimaryKeyReader primaryKeyReader = new 
UpsertUtils.PrimaryKeyReader(segment, _primaryKeyColumns)) {
+      removeSegment(segment,
+          UpsertUtils.getPrimaryKeyIterator(primaryKeyReader, 
segment.getSegmentMetadata().getTotalDocs()));
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("Caught exception while removing segment: %s, table: 
%s", segment.getSegmentName(),
+              _tableNameWithType), e);
+    }
+
+    // Update metrics
+    long numPrimaryKeys = getNumPrimaryKeys();
+    updatePrimaryKeyGauge(numPrimaryKeys);
+    _logger.info("Finished removing segment: {} in {}ms, current primary key 
count: {}", segmentName,
+        System.currentTimeMillis() - startTimeMs, numPrimaryKeys);
+  }
+
+  @Override
+  protected void removeSegment(IndexSegment segment, Iterator<PrimaryKey> 
primaryKeyIterator) {
+    Set<Object> deDuplicatedPrimaryKeys = new HashSet<>();
+    while (primaryKeyIterator.hasNext()) {
+      PrimaryKey primaryKey = primaryKeyIterator.next();
+      
_primaryKeyToRecordLocationMap.computeIfPresent(HashUtils.hashPrimaryKey(primaryKey,
 _hashFunction),
+          (pk, recordLocation) -> {
+            if (deDuplicatedPrimaryKeys.contains(pk)) {
+              return recordLocation;
+            }
+            deDuplicatedPrimaryKeys.add(pk);
+            if (recordLocation.getSegment() == segment) {
+              return null;
+            }
+            return new RecordLocation(recordLocation.getSegment(), 
recordLocation.getDocId(),
+                recordLocation.getComparisonValue(),
+                
RecordLocation.decrementSegmentCount(recordLocation.getDistinctSegmentCount()));
+          });
+    }
+  }
+
+  @Override
+  public void doRemoveExpiredPrimaryKeys() {
+    AtomicInteger numDeletedTTLKeysRemoved = new AtomicInteger();
+    double largestSeenComparisonValue = _largestSeenComparisonValue.get();
+    double deletedKeysThreshold;
+    if (_deletedKeysTTL > 0) {
+      deletedKeysThreshold = largestSeenComparisonValue - _deletedKeysTTL;
+    } else {
+      deletedKeysThreshold = Double.MIN_VALUE;
+    }
+
+    _primaryKeyToRecordLocationMap.forEach((primaryKey, recordLocation) -> {
+      double comparisonValue = ((Number) 
recordLocation.getComparisonValue()).doubleValue();
+      // We need to verify that the record belongs to only one segment. If a 
record is part of multiple segments,
+      // an issue can arise where the upsert compaction might first process 
the segment containing the delete record
+      // while the previous segment(s) are not compacted. Upon restart, this 
can inadvertently revive the key
+      // that was originally marked for deletion.
+      if (_deletedKeysTTL > 0 && comparisonValue < deletedKeysThreshold
+          && recordLocation.getDistinctSegmentCount() <= 1) {
+        ThreadSafeMutableRoaringBitmap currentQueryableDocIds = 
recordLocation.getSegment().getQueryableDocIds();
+        // if key not part of queryable doc id, it means it is deleted
+        if (currentQueryableDocIds != null && 
!currentQueryableDocIds.contains(recordLocation.getDocId())) {
+          _primaryKeyToRecordLocationMap.remove(primaryKey, recordLocation);
+          removeDocId(recordLocation.getSegment(), recordLocation.getDocId());
+          numDeletedTTLKeysRemoved.getAndIncrement();
+        }
+      }
+    });
+
+    // Update metrics
+    updatePrimaryKeyGauge();
+    int numDeletedTTLKeys = numDeletedTTLKeysRemoved.get();
+    if (numDeletedTTLKeys > 0) {
+      _logger.info("Deleted {} primary keys based on deletedKeysTTL", 
numDeletedTTLKeys);
+      _serverMetrics.addMeteredTableValue(_tableNameWithType, 
ServerMeter.DELETED_KEYS_TTL_PRIMARY_KEYS_REMOVED,
+          numDeletedTTLKeys);
+    }
+  }
+
+  @Override
+  protected boolean doAddRecord(MutableSegment segment, RecordInfo recordInfo) 
{
+    AtomicBoolean isOutOfOrderRecord = new AtomicBoolean(false);
+    ThreadSafeMutableRoaringBitmap validDocIds = 
Objects.requireNonNull(segment.getValidDocIds());
+    ThreadSafeMutableRoaringBitmap queryableDocIds = 
segment.getQueryableDocIds();
+    int newDocId = recordInfo.getDocId();
+    Comparable newComparisonValue = recordInfo.getComparisonValue();
+
+    // When TTL is enabled, update largestSeenComparisonValue when adding new 
record
+    if (_deletedKeysTTL > 0) {
+      double comparisonValue = ((Number) newComparisonValue).doubleValue();
+      _largestSeenComparisonValue.getAndUpdate(v -> Math.max(v, 
comparisonValue));
+    }
+
+    
_primaryKeyToRecordLocationMap.compute(HashUtils.hashPrimaryKey(recordInfo.getPrimaryKey(),
 _hashFunction),
+        (primaryKey, currentRecordLocation) -> {
+          if (currentRecordLocation != null) {
+            // Existing primary key
+            IndexSegment currentSegment = currentRecordLocation.getSegment();
+            // Update the record location when the new comparison value is 
greater than or equal to the current value.
+            // Update the record location when there is a tie to keep the 
newer record.
+            if 
(newComparisonValue.compareTo(currentRecordLocation.getComparisonValue()) >= 0) 
{
+              int currentDocId = currentRecordLocation.getDocId();
+              if (segment == currentSegment) {
+                replaceDocId(segment, validDocIds, queryableDocIds, 
currentDocId, newDocId, recordInfo);
+                return new RecordLocation(segment, newDocId, 
newComparisonValue,
+                    currentRecordLocation.getDistinctSegmentCount());
+              } else {
+                replaceDocId(segment, validDocIds, queryableDocIds, 
currentSegment, currentDocId, newDocId, recordInfo);
+                return new RecordLocation(segment, newDocId, 
newComparisonValue,
+                    
RecordLocation.incrementSegmentCount(currentRecordLocation.getDistinctSegmentCount()));
+              }
+            } else {
+              // Out-of-order record
+              
handleOutOfOrderEvent(currentRecordLocation.getComparisonValue(), 
recordInfo.getComparisonValue());
+              isOutOfOrderRecord.set(true);
+              if (segment == currentSegment) {
+                return currentRecordLocation;
+              } else {
+                return new RecordLocation(currentSegment, 
currentRecordLocation.getDocId(),
+                    currentRecordLocation.getComparisonValue(),
+                    _context.isDropOutOfOrderRecord() ? 
currentRecordLocation.getDistinctSegmentCount()
+                        : 
RecordLocation.incrementSegmentCount(currentRecordLocation.getDistinctSegmentCount()));
+              }
+            }
+          } else {
+            // New primary key
+            addDocId(segment, validDocIds, queryableDocIds, newDocId, 
recordInfo);
+            return new RecordLocation(segment, newDocId, newComparisonValue, 
1);
+          }
+        });
+
+    updatePrimaryKeyGauge();
+    return !isOutOfOrderRecord.get();
+  }
+
+  @Override
+  protected void removeSegment(IndexSegment segment, MutableRoaringBitmap 
validDocIds) {
+  }
+
+  @Override
+  protected GenericRow doUpdateRecord(GenericRow record, RecordInfo 
recordInfo) {
+    assert _partialUpsertHandler != null;
+    
_primaryKeyToRecordLocationMap.computeIfPresent(HashUtils.hashPrimaryKey(recordInfo.getPrimaryKey(),
 _hashFunction),
+        (pk, recordLocation) -> {
+          // Read the previous record if the following conditions are met:
+          // - New record is not a DELETE record
+          // - New record is not out-of-order
+          // - Previous record is not deleted
+          if (!recordInfo.isDeleteRecord()
+              && 
recordInfo.getComparisonValue().compareTo(recordLocation.getComparisonValue()) 
>= 0) {
+            IndexSegment currentSegment = recordLocation.getSegment();
+            ThreadSafeMutableRoaringBitmap currentQueryableDocIds = 
currentSegment.getQueryableDocIds();
+            int currentDocId = recordLocation.getDocId();
+            if (currentQueryableDocIds == null || 
currentQueryableDocIds.contains(currentDocId)) {
+              _reusePreviousRow.init(currentSegment, currentDocId);
+              _partialUpsertHandler.merge(_reusePreviousRow, record, 
_reuseMergeResultHolder);
+              _reuseMergeResultHolder.clear();
+            }
+          }
+          return recordLocation;
+        });
+    return record;
+  }
+
+  @VisibleForTesting
+  static class RecordLocation {
+    private final IndexSegment _segment;
+    private final int _docId;
+    private final Comparable _comparisonValue;
+    // The number of distinct segments in which the record is present. If this 
count is less than or equal to 1,
+    // we proceed to remove the record from the primary hashmap during the 
deletedKeysTTL process.
+    private final int _distinctSegmentCount;
+
+    public RecordLocation(IndexSegment indexSegment, int docId, Comparable 
comparisonValue, int distinctSegmentCount) {
+      _segment = indexSegment;
+      _docId = docId;
+      _comparisonValue = comparisonValue;
+      _distinctSegmentCount = distinctSegmentCount;
+    }
+
+    public static int incrementSegmentCount(int count) {
+      return count + 1;
+    }
+
+    public static int decrementSegmentCount(int count) {
+      return count - 1;
+    }

Review Comment:
   can we do it in a follow-up? initially this will help me in debugging if we 
are missing out on any edge cases while doing addition / subtraction? 😅 



##########
pinot-spi/src/main/java/org/apache/pinot/spi/config/table/UpsertConfig.java:
##########
@@ -79,6 +79,9 @@ public enum ConsistencyMode {
   @JsonPropertyDescription("Whether to preload segments for fast upsert 
metadata recovery")
   private boolean _enablePreload;
 
+  @JsonPropertyDescription("If we are using deletion + compaction we need to 
enable this for data consistency")

Review Comment:
   Updated! 



-- 
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-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org

Reply via email to