deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655733166



##########
File path: 
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +63,170 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int 
partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to 
improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> 
_primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int 
partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> 
getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
-  }
-
-  // TODO(upset): consider an off-heap KV store to persist this index to 
improve the recovery speed.
-  @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> 
_primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
   /**
-   * Initializes the upsert metadata for the given immutable segment, returns 
the valid doc ids for the segment.
+   * Initializes the upsert metadata for the given immutable segment.
    */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, 
Iterator<RecordInfo> recordInfoIterator) {
+  public void addSegment(IndexSegment segment, Iterator<RecordInfo> 
recordInfoIterator) {
+    String segmentName = segment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    ThreadSafeMutableRoaringBitmap validDocIds = segment.getValidDocIds();
+    assert validDocIds != null;
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new 
ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, 
(primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater 
than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same 
segment (the segment being added). In this case,
-            //      we want to 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.
-            //   2. The current record location is pointing to the 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.
+          // 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.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (segment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) 
{
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, 
recordInfo._timestamp, validDocIds);
+              return new RecordLocation(segment, recordInfo._docId, 
recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or 
the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the 
segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() 
|| (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && 
LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && 
LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > 
LLCSegmentName
-                    
.getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              
currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced

Review comment:
       @chenboat This case happens when reloading a completed segment. The old 
segment is being replaced.
   cc'ed @Jackie-Jiang 
   




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

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