zhtaoxiang commented on code in PR #13636:
URL: https://github.com/apache/pinot/pull/13636#discussion_r1712242432


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/dedup/BasePartitionDedupMetadataManager.java:
##########
@@ -0,0 +1,104 @@
+/**
+ * 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.dedup;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.spi.config.table.HashFunction;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BasePartitionDedupMetadataManager implements 
PartitionDedupMetadataManager {
+  protected final String _tableNameWithType;
+  protected final List<String> _primaryKeyColumns;
+  protected final int _partitionId;
+  protected final ServerMetrics _serverMetrics;
+  protected final HashFunction _hashFunction;
+  protected final double _metadataTTL;
+  protected final String _metadataTimeColumn;
+  protected final Logger _logger;
+
+  protected BasePartitionDedupMetadataManager(String tableNameWithType, int 
partitionId,
+      DedupContext dedupContext) {
+    _tableNameWithType = tableNameWithType;
+    _partitionId = partitionId;
+    _primaryKeyColumns = dedupContext.getPrimaryKeyColumns();
+    _hashFunction = dedupContext.getHashFunction();
+    _serverMetrics = dedupContext.getServerMetrics();
+    _metadataTTL = dedupContext.getMetadataTTL() >= 0 ? 
dedupContext.getMetadataTTL() : 0;
+    _metadataTimeColumn = dedupContext.getMetadataTimeColumn();
+    if (_metadataTTL > 0) {
+      Preconditions.checkArgument(_metadataTimeColumn != null,
+          "When metadataTTL is configured, metadata time column must be 
configured for dedup enabled table: %s",
+          tableNameWithType);
+    }
+    _logger = LoggerFactory.getLogger(tableNameWithType + "-" + partitionId + 
"-" + getClass().getSimpleName());
+  }
+
+  @Override
+  public boolean checkRecordPresentOrUpdate(PrimaryKey pk, IndexSegment 
indexSegment) {
+    throw new 
UnsupportedOperationException("checkRecordPresentOrUpdate(PrimaryKey pk, 
IndexSegment indexSegment) is "
+        + "deprecated!");
+  }
+
+  @Override
+  public void addSegment(IndexSegment segment) {
+    try (DedupUtils.DedupRecordInfoReader dedupRecordInfoReader = new 
DedupUtils.DedupRecordInfoReader(segment,
+        _primaryKeyColumns, _metadataTimeColumn)) {
+      Iterator<DedupRecordInfo> dedupRecordInfoIterator =
+          DedupUtils.getDedupRecordInfoIterator(dedupRecordInfoReader, 
segment.getSegmentMetadata().getTotalDocs());
+      addSegment(segment, dedupRecordInfoIterator);
+      int dedupPrimaryKeyCount = removeExpiredPrimaryKeys();
+      _serverMetrics.setValueOfPartitionGauge(_tableNameWithType, 
_partitionId, ServerGauge.DEDUP_PRIMARY_KEYS_COUNT,
+          dedupPrimaryKeyCount);
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Caught exception while adding 
segment: %s of table: %s to "
+          + "ConcurrentMapPartitionDedupMetadataManager", 
segment.getSegmentName(), _tableNameWithType), e);

Review Comment:
   good catch!



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/dedup/ConcurrentMapPartitionDedupMetadataManager.java:
##########
@@ -19,105 +19,80 @@
 package org.apache.pinot.segment.local.dedup;
 
 import com.google.common.annotations.VisibleForTesting;
-import java.util.HashMap;
+import com.google.common.util.concurrent.AtomicDouble;
 import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.pinot.common.metrics.ServerGauge;
-import org.apache.pinot.common.metrics.ServerMetrics;
-import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader;
 import org.apache.pinot.segment.local.utils.HashUtils;
 import org.apache.pinot.segment.spi.IndexSegment;
-import org.apache.pinot.spi.config.table.HashFunction;
-import org.apache.pinot.spi.data.readers.PrimaryKey;
-import org.apache.pinot.spi.utils.ByteArray;
 
-class ConcurrentMapPartitionDedupMetadataManager implements 
PartitionDedupMetadataManager {
-  private final String _tableNameWithType;
-  private final List<String> _primaryKeyColumns;
-  private final int _partitionId;
-  private final ServerMetrics _serverMetrics;
-  private final HashFunction _hashFunction;
 
+class ConcurrentMapPartitionDedupMetadataManager extends 
BasePartitionDedupMetadataManager {
   @VisibleForTesting
-  final ConcurrentHashMap<Object, IndexSegment> _primaryKeyToSegmentMap = new 
ConcurrentHashMap<>();
+  final AtomicDouble _largestSeenTime = new AtomicDouble(0);
+  @VisibleForTesting
+  final ConcurrentHashMap<Object, Pair<IndexSegment, Double>> 
_primaryKeyToSegmentAndTimeMap =
+      new ConcurrentHashMap<>();
 
-  public ConcurrentMapPartitionDedupMetadataManager(String tableNameWithType, 
List<String> primaryKeyColumns,
-      int partitionId, ServerMetrics serverMetrics, HashFunction hashFunction) 
{
-    _tableNameWithType = tableNameWithType;
-    _primaryKeyColumns = primaryKeyColumns;
-    _partitionId = partitionId;
-    _serverMetrics = serverMetrics;
-    _hashFunction = hashFunction;
+  protected ConcurrentMapPartitionDedupMetadataManager(String 
tableNameWithType, int partitionId,
+      DedupContext dedupContext) {
+    super(tableNameWithType, partitionId, dedupContext);
   }
 
-  public void addSegment(IndexSegment segment) {
-    // Add all PKs to _primaryKeyToSegmentMap
-    Iterator<PrimaryKey> primaryKeyIterator = getPrimaryKeyIterator(segment);
-    while (primaryKeyIterator.hasNext()) {
-      PrimaryKey pk = primaryKeyIterator.next();
-      _primaryKeyToSegmentMap.put(HashUtils.hashPrimaryKey(pk, _hashFunction), 
segment);
+  @Override
+  protected void addSegment(IndexSegment segment, Iterator<DedupRecordInfo> 
dedupRecordInfoIterator) {
+    while (dedupRecordInfoIterator.hasNext()) {
+      DedupRecordInfo dedupRecordInfo = dedupRecordInfoIterator.next();
+      double metadataTime = dedupRecordInfo.getDedupTime();
+      _largestSeenTime.getAndUpdate(time -> Math.max(time, metadataTime));
+      
_primaryKeyToSegmentAndTimeMap.compute(HashUtils.hashPrimaryKey(dedupRecordInfo.getPrimaryKey(),
 _hashFunction),
+            (primaryKey, segmentAndTime) -> {
+            if (segmentAndTime == null || segmentAndTime.getRight() < 
metadataTime) {

Review Comment:
   yes, it is possible:
   1. a record with the same key is ingested after the metadata TTL
   2. the segment is uploaded directly - build by other workflow



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/dedup/BaseTableDedupMetadataManager.java:
##########
@@ -26,32 +26,50 @@
 import org.apache.pinot.common.metrics.ServerMetrics;
 import org.apache.pinot.segment.local.data.manager.TableDataManager;
 import org.apache.pinot.spi.config.table.DedupConfig;
-import org.apache.pinot.spi.config.table.HashFunction;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.data.Schema;
 
 
-abstract class BaseTableDedupMetadataManager implements 
TableDedupMetadataManager {
+public abstract class BaseTableDedupMetadataManager implements 
TableDedupMetadataManager {
   protected final Map<Integer, PartitionDedupMetadataManager> 
_partitionMetadataManagerMap = new ConcurrentHashMap<>();
   protected String _tableNameWithType;
-  protected List<String> _primaryKeyColumns;
-  protected ServerMetrics _serverMetrics;
-  protected HashFunction _hashFunction;
+  protected DedupContext _dedupContext;
 
   @Override
   public void init(TableConfig tableConfig, Schema schema, TableDataManager 
tableDataManager,
       ServerMetrics serverMetrics) {
     _tableNameWithType = tableConfig.getTableName();
 
-    _primaryKeyColumns = schema.getPrimaryKeyColumns();
-    Preconditions.checkArgument(!CollectionUtils.isEmpty(_primaryKeyColumns),
+    List<String> primaryKeyColumns = schema.getPrimaryKeyColumns();
+    Preconditions.checkArgument(!CollectionUtils.isEmpty(primaryKeyColumns),
         "Primary key columns must be configured for dedup enabled table: %s", 
_tableNameWithType);
 
-    _serverMetrics = serverMetrics;
-
     DedupConfig dedupConfig = tableConfig.getDedupConfig();
     Preconditions.checkArgument(dedupConfig != null, "Dedup must be enabled 
for table: %s", _tableNameWithType);
-    _hashFunction = dedupConfig.getHashFunction();
+    double metadataTTL = dedupConfig.getMetadataTTL();
+    String metadataTimeColumn = dedupConfig.getMetadataTimeColumn();
+    if (metadataTTL > 0) {
+      metadataTimeColumn = dedupConfig.getMetadataTimeColumn();
+      if (metadataTimeColumn == null) {
+        metadataTimeColumn = 
tableConfig.getValidationConfig().getTimeColumnName();
+      }
+      Preconditions.checkArgument(metadataTimeColumn != null,
+          "When metadataTTL is configured, metadata time column or time column 
must be configured for "
+              + "dedup enabled table: %s", _tableNameWithType);
+    }
+
+    DedupContext.Builder dedupContextBuider = new DedupContext.Builder();
+    dedupContextBuider
+        .setTableConfig(tableConfig)
+        .setSchema(schema)
+        .setPrimaryKeyColumns(primaryKeyColumns)
+        .setHashFunction(dedupConfig.getHashFunction())
+        .setMetadataTTL(metadataTTL)
+        .setMetadataTimeColumn(metadataTimeColumn)

Review Comment:
   yes, they are the same. renamed MetadataTimeColumn to DedupTimeColumn



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/BasePartitionUpsertMetadataManager.java:
##########
@@ -743,7 +744,7 @@ public void replaceSegment(ImmutableSegment segment, 
@Nullable ThreadSafeMutable
   }
 
   protected void removeSegment(IndexSegment segment, MutableRoaringBitmap 
validDocIds) {
-    try (UpsertUtils.PrimaryKeyReader primaryKeyReader = new 
UpsertUtils.PrimaryKeyReader(segment,
+    try (PrimaryKeyReader primaryKeyReader = new PrimaryKeyReader(segment,
         _primaryKeyColumns)) {

Review Comment:
   good catch!



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/dedup/ConcurrentMapPartitionDedupMetadataManager.java:
##########
@@ -19,105 +19,80 @@
 package org.apache.pinot.segment.local.dedup;
 
 import com.google.common.annotations.VisibleForTesting;
-import java.util.HashMap;
+import com.google.common.util.concurrent.AtomicDouble;
 import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.pinot.common.metrics.ServerGauge;
-import org.apache.pinot.common.metrics.ServerMetrics;
-import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader;
 import org.apache.pinot.segment.local.utils.HashUtils;
 import org.apache.pinot.segment.spi.IndexSegment;
-import org.apache.pinot.spi.config.table.HashFunction;
-import org.apache.pinot.spi.data.readers.PrimaryKey;
-import org.apache.pinot.spi.utils.ByteArray;
 
-class ConcurrentMapPartitionDedupMetadataManager implements 
PartitionDedupMetadataManager {
-  private final String _tableNameWithType;
-  private final List<String> _primaryKeyColumns;
-  private final int _partitionId;
-  private final ServerMetrics _serverMetrics;
-  private final HashFunction _hashFunction;
 
+class ConcurrentMapPartitionDedupMetadataManager extends 
BasePartitionDedupMetadataManager {
   @VisibleForTesting
-  final ConcurrentHashMap<Object, IndexSegment> _primaryKeyToSegmentMap = new 
ConcurrentHashMap<>();
+  final AtomicDouble _largestSeenTime = new AtomicDouble(0);
+  @VisibleForTesting
+  final ConcurrentHashMap<Object, Pair<IndexSegment, Double>> 
_primaryKeyToSegmentAndTimeMap =
+      new ConcurrentHashMap<>();
 
-  public ConcurrentMapPartitionDedupMetadataManager(String tableNameWithType, 
List<String> primaryKeyColumns,
-      int partitionId, ServerMetrics serverMetrics, HashFunction hashFunction) 
{
-    _tableNameWithType = tableNameWithType;
-    _primaryKeyColumns = primaryKeyColumns;
-    _partitionId = partitionId;
-    _serverMetrics = serverMetrics;
-    _hashFunction = hashFunction;
+  protected ConcurrentMapPartitionDedupMetadataManager(String 
tableNameWithType, int partitionId,
+      DedupContext dedupContext) {
+    super(tableNameWithType, partitionId, dedupContext);
   }
 
-  public void addSegment(IndexSegment segment) {
-    // Add all PKs to _primaryKeyToSegmentMap
-    Iterator<PrimaryKey> primaryKeyIterator = getPrimaryKeyIterator(segment);
-    while (primaryKeyIterator.hasNext()) {
-      PrimaryKey pk = primaryKeyIterator.next();
-      _primaryKeyToSegmentMap.put(HashUtils.hashPrimaryKey(pk, _hashFunction), 
segment);
+  @Override
+  protected void addSegment(IndexSegment segment, Iterator<DedupRecordInfo> 
dedupRecordInfoIterator) {
+    while (dedupRecordInfoIterator.hasNext()) {
+      DedupRecordInfo dedupRecordInfo = dedupRecordInfoIterator.next();
+      double metadataTime = dedupRecordInfo.getDedupTime();
+      _largestSeenTime.getAndUpdate(time -> Math.max(time, metadataTime));
+      
_primaryKeyToSegmentAndTimeMap.compute(HashUtils.hashPrimaryKey(dedupRecordInfo.getPrimaryKey(),
 _hashFunction),
+            (primaryKey, segmentAndTime) -> {
+            if (segmentAndTime == null || segmentAndTime.getRight() < 
metadataTime) {
+              return Pair.of(segment, metadataTime);
+            } else {
+              return segmentAndTime;
+            }
+          });
     }
-    _serverMetrics.setValueOfPartitionGauge(_tableNameWithType, _partitionId, 
ServerGauge.DEDUP_PRIMARY_KEYS_COUNT,
-        _primaryKeyToSegmentMap.size());
   }
 
-  public void removeSegment(IndexSegment segment) {
-    // TODO(saurabh): Explain reload scenario here
-    Iterator<PrimaryKey> primaryKeyIterator = getPrimaryKeyIterator(segment);
-    while (primaryKeyIterator.hasNext()) {
-      PrimaryKey pk = primaryKeyIterator.next();
-      _primaryKeyToSegmentMap.compute(HashUtils.hashPrimaryKey(pk, 
_hashFunction), (primaryKey, currentSegment) -> {
-        if (currentSegment == segment) {
-          return null;
-        } else {
-          return currentSegment;
-        }
-      });
+  @Override
+  protected void removeSegment(IndexSegment segment, Iterator<DedupRecordInfo> 
dedupRecordInfoIterator) {
+    while (dedupRecordInfoIterator.hasNext()) {
+      DedupRecordInfo dedupRecordInfo = dedupRecordInfoIterator.next();
+      _primaryKeyToSegmentAndTimeMap.computeIfPresent(
+          HashUtils.hashPrimaryKey(dedupRecordInfo.getPrimaryKey(), 
_hashFunction), (primaryKey, segmentAndTime) -> {
+            if (segmentAndTime.getLeft() == segment && 
segmentAndTime.getRight() == dedupRecordInfo.getDedupTime()) {
+              return null;
+            } else {
+              return segmentAndTime;
+            }
+          });
     }
-    _serverMetrics.setValueOfPartitionGauge(_tableNameWithType, _partitionId, 
ServerGauge.DEDUP_PRIMARY_KEYS_COUNT,
-        _primaryKeyToSegmentMap.size());
   }
 
-  @VisibleForTesting
-  Iterator<PrimaryKey> getPrimaryKeyIterator(IndexSegment segment) {
-    Map<String, PinotSegmentColumnReader> columnToReaderMap = new HashMap<>();
-    for (String primaryKeyColumn : _primaryKeyColumns) {
-      columnToReaderMap.put(primaryKeyColumn, new 
PinotSegmentColumnReader(segment, primaryKeyColumn));
+  @Override
+  public int removeExpiredPrimaryKeys() {
+    if (_metadataTTL > 0) {
+      double smallestTimeToKeep = _largestSeenTime.get() - _metadataTTL;
+      _primaryKeyToSegmentAndTimeMap.entrySet().removeIf(entry -> 
entry.getValue().getRight() < smallestTimeToKeep);
     }
-    int numTotalDocs = segment.getSegmentMetadata().getTotalDocs();
-    int numPrimaryKeyColumns = _primaryKeyColumns.size();
-    return new Iterator<PrimaryKey>() {
-      private int _docId = 0;
-
-      @Override
-      public boolean hasNext() {
-        return _docId < numTotalDocs;
-      }
-
-      @Override
-      public PrimaryKey next() {
-        Object[] values = new Object[numPrimaryKeyColumns];
-        for (int i = 0; i < numPrimaryKeyColumns; i++) {
-          Object value = 
columnToReaderMap.get(_primaryKeyColumns.get(i)).getValue(_docId);
-          if (value instanceof byte[]) {
-            value = new ByteArray((byte[]) value);
-          }
-          values[i] = value;
-        }
-        _docId++;
-        return new PrimaryKey(values);
-      }
-    };
+    return _primaryKeyToSegmentAndTimeMap.size();
   }
 
-  public boolean checkRecordPresentOrUpdate(PrimaryKey pk, IndexSegment 
indexSegment) {
-    boolean present =
-        _primaryKeyToSegmentMap.putIfAbsent(HashUtils.hashPrimaryKey(pk, 
_hashFunction), indexSegment) != null;
+  @Override
+  public boolean dropOrAddRecord(DedupRecordInfo dedupRecordInfo, IndexSegment 
indexSegment) {

Review Comment:
   sounds good!



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/dedup/ConcurrentMapPartitionDedupMetadataManager.java:
##########
@@ -19,105 +19,80 @@
 package org.apache.pinot.segment.local.dedup;
 
 import com.google.common.annotations.VisibleForTesting;
-import java.util.HashMap;
+import com.google.common.util.concurrent.AtomicDouble;
 import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.pinot.common.metrics.ServerGauge;
-import org.apache.pinot.common.metrics.ServerMetrics;
-import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader;
 import org.apache.pinot.segment.local.utils.HashUtils;
 import org.apache.pinot.segment.spi.IndexSegment;
-import org.apache.pinot.spi.config.table.HashFunction;
-import org.apache.pinot.spi.data.readers.PrimaryKey;
-import org.apache.pinot.spi.utils.ByteArray;
 
-class ConcurrentMapPartitionDedupMetadataManager implements 
PartitionDedupMetadataManager {
-  private final String _tableNameWithType;
-  private final List<String> _primaryKeyColumns;
-  private final int _partitionId;
-  private final ServerMetrics _serverMetrics;
-  private final HashFunction _hashFunction;
 
+class ConcurrentMapPartitionDedupMetadataManager extends 
BasePartitionDedupMetadataManager {
   @VisibleForTesting
-  final ConcurrentHashMap<Object, IndexSegment> _primaryKeyToSegmentMap = new 
ConcurrentHashMap<>();
+  final AtomicDouble _largestSeenTime = new AtomicDouble(0);
+  @VisibleForTesting
+  final ConcurrentHashMap<Object, Pair<IndexSegment, Double>> 
_primaryKeyToSegmentAndTimeMap =

Review Comment:
   the dedup time column is not necessarily a long type, users can define it to 
be double or any number type.
   
   Do you feel we should restrict the time of the dedup time column?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/dedup/BasePartitionDedupMetadataManager.java:
##########
@@ -0,0 +1,104 @@
+/**
+ * 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.dedup;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.spi.config.table.HashFunction;
+import org.apache.pinot.spi.data.readers.PrimaryKey;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BasePartitionDedupMetadataManager implements 
PartitionDedupMetadataManager {
+  protected final String _tableNameWithType;
+  protected final List<String> _primaryKeyColumns;
+  protected final int _partitionId;
+  protected final ServerMetrics _serverMetrics;
+  protected final HashFunction _hashFunction;
+  protected final double _metadataTTL;
+  protected final String _metadataTimeColumn;
+  protected final Logger _logger;
+
+  protected BasePartitionDedupMetadataManager(String tableNameWithType, int 
partitionId,
+      DedupContext dedupContext) {
+    _tableNameWithType = tableNameWithType;
+    _partitionId = partitionId;
+    _primaryKeyColumns = dedupContext.getPrimaryKeyColumns();
+    _hashFunction = dedupContext.getHashFunction();
+    _serverMetrics = dedupContext.getServerMetrics();
+    _metadataTTL = dedupContext.getMetadataTTL() >= 0 ? 
dedupContext.getMetadataTTL() : 0;
+    _metadataTimeColumn = dedupContext.getMetadataTimeColumn();
+    if (_metadataTTL > 0) {
+      Preconditions.checkArgument(_metadataTimeColumn != null,
+          "When metadataTTL is configured, metadata time column must be 
configured for dedup enabled table: %s",
+          tableNameWithType);
+    }
+    _logger = LoggerFactory.getLogger(tableNameWithType + "-" + partitionId + 
"-" + getClass().getSimpleName());
+  }
+
+  @Override
+  public boolean checkRecordPresentOrUpdate(PrimaryKey pk, IndexSegment 
indexSegment) {
+    throw new 
UnsupportedOperationException("checkRecordPresentOrUpdate(PrimaryKey pk, 
IndexSegment indexSegment) is "
+        + "deprecated!");
+  }
+
+  @Override
+  public void addSegment(IndexSegment segment) {
+    try (DedupUtils.DedupRecordInfoReader dedupRecordInfoReader = new 
DedupUtils.DedupRecordInfoReader(segment,
+        _primaryKeyColumns, _metadataTimeColumn)) {
+      Iterator<DedupRecordInfo> dedupRecordInfoIterator =
+          DedupUtils.getDedupRecordInfoIterator(dedupRecordInfoReader, 
segment.getSegmentMetadata().getTotalDocs());
+      addSegment(segment, dedupRecordInfoIterator);
+      int dedupPrimaryKeyCount = removeExpiredPrimaryKeys();
+      _serverMetrics.setValueOfPartitionGauge(_tableNameWithType, 
_partitionId, ServerGauge.DEDUP_PRIMARY_KEYS_COUNT,
+          dedupPrimaryKeyCount);
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Caught exception while adding 
segment: %s of table: %s to "
+          + "ConcurrentMapPartitionDedupMetadataManager", 
segment.getSegmentName(), _tableNameWithType), e);
+    }
+  }
+
+  @VisibleForTesting

Review Comment:
   make sense



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