You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "Jackie-Jiang (via GitHub)" <gi...@apache.org> on 2023/06/16 00:15:45 UTC

[GitHub] [pinot] Jackie-Jiang opened a new pull request, #10928: Take upsert snapshot when creating new consuming segment

Jackie-Jiang opened a new pull request, #10928:
URL: https://github.com/apache/pinot/pull/10928

   Fix #10800
   
   - Track all the segments managed by the `BasePartitionUpsertMetadataManager`
   - Remove the tracking for replaced segments because that can be replaced by tracking all the segments
   - Introduce a read-write lock in `BasePartitionUpsertMetadataManager` to ensure segments are not changed when snapshot is taken
   - Take snapshot before a new consuming segment starts consuming instead of when the segment is removed (most commonly during server shut down)


-- 
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] walterddr commented on a diff in pull request #10928: Take upsert snapshot when creating new consuming segment

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/BasePartitionUpsertMetadataManager.java:
##########
@@ -61,8 +63,12 @@ public abstract class BasePartitionUpsertMetadataManager implements PartitionUps
   protected final ServerMetrics _serverMetrics;
   protected final Logger _logger;
 
-  @VisibleForTesting
-  public final Set<IndexSegment> _replacedSegments = ConcurrentHashMap.newKeySet();
+  // Tracks all the segments managed by this manager (excluding EmptySegment)
+  protected final Set<IndexSegment> _trackedSegments = ConcurrentHashMap.newKeySet();
+
+  // NOTE: We do not persist snapshot on the first consuming segment because most segments might not be loaded yet
+  protected volatile boolean _gotFirstConsumingSegment = false;
+  protected final ReadWriteLock _snapshotLock;

Review Comment:
   this lock is used (read-wise) by 3 methods `add/remove/replaceSegment`, therefore should this lock be private? 
   



-- 
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] Jackie-Jiang commented on pull request #10928: Take upsert snapshot when creating new consuming segment

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on PR #10928:
URL: https://github.com/apache/pinot/pull/10928#issuecomment-1599349616

   > @Jackie-Jiang the snapshot operation in prod can take till around 30 seconds as per my observation.
   > Are we fine with spending that much extra time between each segment commit?
   > There will be a momentary consumption lag while snapshot is being taken although I think pinot should be able to easily catch up.
   > One optimisation we can do is only take snapshot of the segment which were **changed** from the last commit.
   
   @KKcorps Good point. It will definitely increase the delay before a new consuming segment starts consuming events, but might not be the whole 30 seconds because we only persist snapshot for one partition, and it doesn't involve the segment destroy time. I added a TODO to consider optimizing it by only persist the changed snapshot, but want to see how long it usually takes to persist all the snapshot as a baseline.
   


-- 
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 #10928: Take upsert snapshot when creating new consuming segment

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

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/10928?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#10928](https://app.codecov.io/gh/apache/pinot/pull/10928?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (04f0839) into [master](https://app.codecov.io/gh/apache/pinot/commit/f1966d9fa01040774ff2e153ab69f9c69903fcbe?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (f1966d9) will **increase** coverage by `0.00%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #10928     +/-   ##
   =========================================
     Coverage    0.11%    0.11%             
   =========================================
     Files        2188     2134     -54     
     Lines      117754   115279   -2475     
     Branches    17791    17500    -291     
   =========================================
     Hits          137      137             
   + Misses     117597   115122   -2475     
     Partials       20       20             
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1temurin11 | `?` | |
   | integration1temurin17 | `?` | |
   | integration2temurin11 | `?` | |
   | integration2temurin17 | `?` | |
   | integration2temurin20 | `?` | |
   | unittests1temurin11 | `?` | |
   | unittests1temurin17 | `?` | |
   | unittests1temurin20 | `?` | |
   | unittests2temurin11 | `?` | |
   | unittests2temurin17 | `0.11% <0.00%> (-0.01%)` | :arrow_down: |
   | unittests2temurin20 | `?` | |
   
   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=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://app.codecov.io/gh/apache/pinot/pull/10928?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [...manager/realtime/LLRealtimeSegmentDataManager.java](https://app.codecov.io/gh/apache/pinot/pull/10928?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvTExSZWFsdGltZVNlZ21lbnREYXRhTWFuYWdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...l/indexsegment/immutable/ImmutableSegmentImpl.java](https://app.codecov.io/gh/apache/pinot/pull/10928?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvaW1tdXRhYmxlL0ltbXV0YWJsZVNlZ21lbnRJbXBsLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...cal/upsert/BasePartitionUpsertMetadataManager.java](https://app.codecov.io/gh/apache/pinot/pull/10928?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvQmFzZVBhcnRpdGlvblVwc2VydE1ldGFkYXRhTWFuYWdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   
   ... and [56 files with indirect coverage changes](https://app.codecov.io/gh/apache/pinot/pull/10928/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :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=apache)
   


-- 
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] walterddr commented on a diff in pull request #10928: Take upsert snapshot when creating new consuming segment

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/BasePartitionUpsertMetadataManager.java:
##########
@@ -61,8 +63,12 @@ public abstract class BasePartitionUpsertMetadataManager implements PartitionUps
   protected final ServerMetrics _serverMetrics;
   protected final Logger _logger;
 
-  @VisibleForTesting
-  public final Set<IndexSegment> _replacedSegments = ConcurrentHashMap.newKeySet();
+  // Tracks all the segments managed by this manager (excluding EmptySegment)
+  protected final Set<IndexSegment> _trackedSegments = ConcurrentHashMap.newKeySet();
+
+  // NOTE: We do not persist snapshot on the first consuming segment because most segments might not be loaded yet
+  protected volatile boolean _gotFirstConsumingSegment = false;
+  protected final ReadWriteLock _snapshotLock;

Review Comment:
   seems like all readLock are follow by start operation. do we still need the start operation? b/c there couldn't been any concurrent operations with the new lock right?



-- 
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] KKcorps commented on a diff in pull request #10928: Take upsert snapshot when creating new consuming segment

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


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1309,15 +1310,15 @@ public LLRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableCo
     _indexLoadingConfig = indexLoadingConfig;
     _schema = schema;
     _serverMetrics = serverMetrics;
+    _partitionUpsertMetadataManager = partitionUpsertMetadataManager;
     _isReadyToConsumeData = isReadyToConsumeData;
     _segmentVersion = indexLoadingConfig.getSegmentVersion();
     _instanceId = _realtimeTableDataManager.getServerInstance();
     _leaseExtender = SegmentBuildTimeLeaseExtender.getLeaseExtender(_tableNameWithType);
     _protocolHandler = new ServerSegmentCompletionProtocolHandler(_serverMetrics, _tableNameWithType);
     CompletionConfig completionConfig = _tableConfig.getValidationConfig().getCompletionConfig();
-    _segmentCompletionMode = completionConfig != null
-        && CompletionMode.DOWNLOAD.toString().equalsIgnoreCase(completionConfig.getCompletionMode())
-        ? CompletionMode.DOWNLOAD : CompletionMode.DEFAULT;
+    _segmentCompletionMode = completionConfig != null && CompletionMode.DOWNLOAD.toString()

Review Comment:
   nit: Unnecessary formatting



-- 
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] KKcorps commented on a diff in pull request #10928: Take upsert snapshot when creating new consuming segment

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


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1360,9 +1361,9 @@ public LLRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableCo
             firstSortedColumn, llcSegmentName);
         sortedColumn = firstSortedColumn;
       } else {
-        _segmentLogger
-            .warn("Sorted column name: {} from RealtimeDataResourceZKMetadata is not existed in schema for segment {}.",
-                firstSortedColumn, llcSegmentName);
+        _segmentLogger.warn(

Review Comment:
   nit: Unnecessary formatting



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1386,27 +1387,26 @@ public LLRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableCo
 
     _nullHandlingEnabled = indexingConfig.isNullHandlingEnabled();
 
-    _columnIndicesForRealtimeTable = new ColumnIndicesForRealtimeTable(sortedColumn,
-        new ArrayList<>(indexLoadingConfig.getInvertedIndexColumns()),
-        new ArrayList<>(indexLoadingConfig.getTextIndexColumns()),
-        new ArrayList<>(indexLoadingConfig.getFSTIndexColumns()),
-        new ArrayList<>(indexLoadingConfig.getNoDictionaryColumns()),
-        new ArrayList<>(indexLoadingConfig.getVarLengthDictionaryColumns()));
+    _columnIndicesForRealtimeTable =
+        new ColumnIndicesForRealtimeTable(sortedColumn, new ArrayList<>(indexLoadingConfig.getInvertedIndexColumns()),
+            new ArrayList<>(indexLoadingConfig.getTextIndexColumns()),
+            new ArrayList<>(indexLoadingConfig.getFSTIndexColumns()),
+            new ArrayList<>(indexLoadingConfig.getNoDictionaryColumns()),
+            new ArrayList<>(indexLoadingConfig.getVarLengthDictionaryColumns()));
 
     // Start new realtime segment
     String consumerDir = realtimeTableDataManager.getConsumerDir();
     RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder =
         new RealtimeSegmentConfig.Builder(indexLoadingConfig).setTableNameWithType(_tableNameWithType)
-            .setSegmentName(_segmentNameStr)
-            .setStreamName(streamTopic).setSchema(_schema).setTimeColumnName(timeColumnName)
-            .setCapacity(_segmentMaxRowCount).setAvgNumMultiValues(indexLoadingConfig.getRealtimeAvgMultiValueCount())
-            .setSegmentZKMetadata(segmentZKMetadata)
-            .setOffHeap(_isOffHeap).setMemoryManager(_memoryManager)
+            .setSegmentName(_segmentNameStr).setStreamName(streamTopic).setSchema(_schema)
+            .setTimeColumnName(timeColumnName).setCapacity(_segmentMaxRowCount)
+            .setAvgNumMultiValues(indexLoadingConfig.getRealtimeAvgMultiValueCount())
+            .setSegmentZKMetadata(segmentZKMetadata).setOffHeap(_isOffHeap).setMemoryManager(_memoryManager)
             .setStatsHistory(realtimeTableDataManager.getStatsHistory())
             .setAggregateMetrics(indexingConfig.isAggregateMetrics())
             .setIngestionAggregationConfigs(IngestionConfigUtils.getAggregationConfigs(tableConfig))
-            .setNullHandlingEnabled(_nullHandlingEnabled)
-            .setConsumerDir(consumerDir).setUpsertMode(tableConfig.getUpsertMode())
+            .setNullHandlingEnabled(_nullHandlingEnabled).setConsumerDir(consumerDir)

Review Comment:
   nit: Unnecessary formatting



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1451,17 +1451,16 @@ public LLRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableCo
       setConsumeEndTime(segmentZKMetadata, _consumeStartTime);
       _segmentCommitterFactory =
           new SegmentCommitterFactory(_segmentLogger, _protocolHandler, tableConfig, indexLoadingConfig, serverMetrics);
-      _segmentLogger
-          .info("Starting consumption on realtime consuming segment {} maxRowCount {} maxEndTime {}", llcSegmentName,
-              _segmentMaxRowCount, new DateTime(_consumeEndTime, DateTimeZone.UTC));
+      _segmentLogger.info("Starting consumption on realtime consuming segment {} maxRowCount {} maxEndTime {}",
+          llcSegmentName, _segmentMaxRowCount, new DateTime(_consumeEndTime, DateTimeZone.UTC));
       startConsumerThread();
     } catch (Exception e) {
       // In case of exception thrown here, segment goes to ERROR state. Then any attempt to reset the segment from
       // ERROR -> OFFLINE -> CONSUMING via Helix Admin fails because the semaphore is acquired, but not released.
       // Hence releasing the semaphore here to unblock reset operation via Helix Admin.
       _partitionGroupConsumerSemaphore.release();
-      _realtimeTableDataManager.addSegmentError(_segmentNameStr, new SegmentErrorInfo(now(),
-          "Failed to initialize segment data manager", e));
+      _realtimeTableDataManager.addSegmentError(_segmentNameStr,

Review Comment:
   nit: Unnecessary formatting



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1386,27 +1387,26 @@ public LLRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableCo
 
     _nullHandlingEnabled = indexingConfig.isNullHandlingEnabled();
 
-    _columnIndicesForRealtimeTable = new ColumnIndicesForRealtimeTable(sortedColumn,
-        new ArrayList<>(indexLoadingConfig.getInvertedIndexColumns()),
-        new ArrayList<>(indexLoadingConfig.getTextIndexColumns()),
-        new ArrayList<>(indexLoadingConfig.getFSTIndexColumns()),
-        new ArrayList<>(indexLoadingConfig.getNoDictionaryColumns()),
-        new ArrayList<>(indexLoadingConfig.getVarLengthDictionaryColumns()));
+    _columnIndicesForRealtimeTable =
+        new ColumnIndicesForRealtimeTable(sortedColumn, new ArrayList<>(indexLoadingConfig.getInvertedIndexColumns()),

Review Comment:
   nit: Unnecessary formatting



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1309,15 +1310,15 @@ public LLRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableCo
     _indexLoadingConfig = indexLoadingConfig;
     _schema = schema;
     _serverMetrics = serverMetrics;
+    _partitionUpsertMetadataManager = partitionUpsertMetadataManager;
     _isReadyToConsumeData = isReadyToConsumeData;
     _segmentVersion = indexLoadingConfig.getSegmentVersion();
     _instanceId = _realtimeTableDataManager.getServerInstance();
     _leaseExtender = SegmentBuildTimeLeaseExtender.getLeaseExtender(_tableNameWithType);
     _protocolHandler = new ServerSegmentCompletionProtocolHandler(_serverMetrics, _tableNameWithType);
     CompletionConfig completionConfig = _tableConfig.getValidationConfig().getCompletionConfig();
-    _segmentCompletionMode = completionConfig != null
-        && CompletionMode.DOWNLOAD.toString().equalsIgnoreCase(completionConfig.getCompletionMode())
-        ? CompletionMode.DOWNLOAD : CompletionMode.DEFAULT;
+    _segmentCompletionMode = completionConfig != null && CompletionMode.DOWNLOAD.toString()

Review Comment:
   nit: Unnecessary formatting



-- 
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] walterddr commented on a diff in pull request #10928: Take upsert snapshot when creating new consuming segment

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/BasePartitionUpsertMetadataManager.java:
##########
@@ -61,8 +63,12 @@ public abstract class BasePartitionUpsertMetadataManager implements PartitionUps
   protected final ServerMetrics _serverMetrics;
   protected final Logger _logger;
 
-  @VisibleForTesting
-  public final Set<IndexSegment> _replacedSegments = ConcurrentHashMap.newKeySet();
+  // Tracks all the segments managed by this manager (excluding EmptySegment)
+  protected final Set<IndexSegment> _trackedSegments = ConcurrentHashMap.newKeySet();
+
+  // NOTE: We do not persist snapshot on the first consuming segment because most segments might not be loaded yet
+  protected volatile boolean _gotFirstConsumingSegment = false;
+  protected final ReadWriteLock _snapshotLock;

Review Comment:
   nvm there's addRecord. my bad



-- 
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] KKcorps commented on pull request #10928: Take upsert snapshot when creating new consuming segment

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

   @Jackie-Jiang the snapshot operation in prod can take till around 30 seconds as per my observation. 
   
   Are we fine with spending that much time between each segment commit?
   
   There will be a momentary consumption lag while snapshot is being taken although I think pinot should be able to easily catch up.


-- 
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] Jackie-Jiang commented on pull request #10928: Take upsert snapshot when creating new consuming segment

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on PR #10928:
URL: https://github.com/apache/pinot/pull/10928#issuecomment-1593875061

   cc @deemoliu 


-- 
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] KKcorps commented on pull request #10928: Take upsert snapshot when creating new consuming segment

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

   @Jackie-Jiang should I go ahead and merge it?


-- 
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] Jackie-Jiang merged pull request #10928: Take upsert snapshot when creating new consuming segment

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


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