You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/07/20 19:42:11 UTC

[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #9062: [Upsert] persist validDocsIndex snapshot for Pinot upsert optimization

Jackie-Jiang commented on code in PR #9062:
URL: https://github.com/apache/pinot/pull/9062#discussion_r925983121


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -183,8 +183,8 @@ protected void doInit() {
       Preconditions.checkState(schema != null, "Failed to find schema for table: %s", _tableNameWithType);
 
       _primaryKeyColumns = schema.getPrimaryKeyColumns();
-      Preconditions.checkState(!CollectionUtils.isEmpty(_primaryKeyColumns),
-          "Primary key columns must be configured for dedup");
+      Preconditions

Review Comment:
   Can you please re-apply the [Pinot Style](https://docs.pinot.apache.org/developers/developers-and-contributors/code-setup#setup-ide)?
   I don't know why you get different auto-formatting results, but these changes are not related to the PR, and make it quite hard to review. Let's also revert the unrelated auto-formatting changes in other classes



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -416,15 +415,18 @@ private void buildDedupMeta(ImmutableSegmentImpl immutableSegment) {
 
   private void handleUpsert(ImmutableSegmentImpl immutableSegment) {
     String segmentName = immutableSegment.getSegmentName();
-    Integer partitionGroupId =
-        SegmentUtils.getRealtimeSegmentPartitionId(segmentName, _tableNameWithType, _helixManager,
-            _primaryKeyColumns.get(0));
-    Preconditions.checkNotNull(partitionGroupId,
-        String.format("PartitionGroupId is not available for segment: '%s' (upsert-enabled table: %s)", segmentName,
+    Integer partitionGroupId = SegmentUtils
+        .getRealtimeSegmentPartitionId(segmentName, _tableNameWithType, _helixManager, _primaryKeyColumns.get(0));
+    Preconditions.checkNotNull(partitionGroupId, String
+        .format("PartitionGroupId is not available for segment: '%s' (upsert-enabled table: %s)", segmentName,
             _tableNameWithType));
     PartitionUpsertMetadataManager partitionUpsertMetadataManager =
         _tableUpsertMetadataManager.getOrCreatePartitionManager(partitionGroupId);
+
     ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
+    if (immutableSegment.getValidDocIds() != null) {

Review Comment:
   I don't think this is the correct way to handle it. We should get the persisted valid doc id, and iterate over it to get the `recordInfoIterator`



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java:
##########
@@ -198,6 +201,13 @@ public void init(SegmentGeneratorConfig config, SegmentCreationDataSource dataSo
     // Create a temporary directory used in segment creation
     _tempIndexDir = new File(indexDir, "tmp-" + UUID.randomUUID());
     LOGGER.debug("tempIndexDir:{}", _tempIndexDir);
+
+    _upsertSnapshotEnabled = _config.isUpsertSnapshotEnabled();

Review Comment:
   We don't have the valid doc id info during segment creation. The snapshot has to be taken periodically during consumption. In the current implementation, no valid doc is added



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