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 2020/10/28 00:02:51 UTC

[GitHub] [incubator-pinot] yupeng9 opened a new pull request #6204: add upsert metadata metric

yupeng9 opened a new pull request #6204:
URL: https://github.com/apache/incubator-pinot/pull/6204


   ## Description
   
   Part of a series of PRs for #4261
   Check this [doc](https://docs.google.com/document/d/1qljEMndPMxbbKtjlVn9mn2toz7Qrk0TGQsHLfI--7h8/edit#heading=h.lsfmyoyyxtgt) out for the new design
   
   add upsert metadata metric of the primary key counts per partition metadata manager
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang merged pull request #6204: add upsert metadata metric

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged pull request #6204:
URL: https://github.com/apache/incubator-pinot/pull/6204


   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6204: add upsert metadata metric

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6204:
URL: https://github.com/apache/incubator-pinot/pull/6204#discussion_r513778525



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
##########
@@ -800,7 +800,8 @@ public FakeLLRealtimeSegmentDataManager(RealtimeSegmentZKMetadata segmentZKMetad
         throws Exception {
       super(segmentZKMetadata, tableConfig, realtimeTableDataManager, resourceDataDir,
           new IndexLoadingConfig(makeInstanceDataManagerConfig(), tableConfig), schema, llcSegmentName,
-          semaphoreMap.get(llcSegmentName.getPartitionId()), serverMetrics, new PartitionUpsertMetadataManager());
+          semaphoreMap.get(llcSegmentName.getPartitionId()), serverMetrics,
+          new PartitionUpsertMetadataManager("testTable", 0, serverMetrics));

Review comment:
       ```suggestion
             new PartitionUpsertMetadataManager("testTable_REALTIME", 0, serverMetrics));
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/upsert/PartitionUpsertMetadataManager.java
##########
@@ -85,7 +97,6 @@ public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<Re
               // committing a consuming segment, or reloading a completed segment.
 
               // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-              // Update the record location when there is a tie because the record locations should point to the new

Review comment:
       Revert this removal

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java
##########
@@ -119,9 +121,11 @@ public void buildSegment()
   public void loadSegment()
       throws Exception {
     _indexSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.heap);
+    ServerMetrics serverMetrics = Mockito.mock(ServerMetrics.class);
     _upsertIndexSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.heap);
     ((ImmutableSegmentImpl) _upsertIndexSegment)
-        .enableUpsert(new PartitionUpsertMetadataManager(), new ThreadSafeMutableRoaringBitmap());
+        .enableUpsert(new PartitionUpsertMetadataManager("testTable", 0, serverMetrics),

Review comment:
       ```suggestion
           .enableUpsert(new PartitionUpsertMetadataManager("testTable_REALTIME", 0, serverMetrics),
   ```

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/upsert/PartitionUpsertMetadataManagerTest.java
##########
@@ -34,10 +36,12 @@
 
 public class PartitionUpsertMetadataManagerTest {
   private static final String SEGMENT_PREFIX = "testSegment";
+  private static final String TEST_TABLE = "testTable";

Review comment:
       ```suggestion
     private static final String REALTIME_TABLE_NAME = "testTable_REALTIME";
   ```

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplUpsertTest.java
##########
@@ -58,7 +60,8 @@ public void setup()
         .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.FULL)).build();
     _recordTransformer = CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
     File jsonFile = new File(dataResourceUrl.getFile());
-    _partitionUpsertMetadataManager = new TableUpsertMetadataManager().getOrCreatePartitionManager(0);
+    _partitionUpsertMetadataManager =
+        new TableUpsertMetadataManager("testTable", Mockito.mock(ServerMetrics.class)).getOrCreatePartitionManager(0);

Review comment:
       ```suggestion
           new TableUpsertMetadataManager("testTable_REALTINE", Mockito.mock(ServerMetrics.class)).getOrCreatePartitionManager(0);
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #6204: add upsert metadata metric

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6204:
URL: https://github.com/apache/incubator-pinot/pull/6204#discussion_r513124672



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/upsert/PartitionUpsertMetadataManager.java
##########
@@ -85,8 +97,7 @@ public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<Re
               // committing a consuming segment, or reloading a completed segment.
 
               // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-              // Update the record location when there is a tie because the record locations should point to the new
-              // segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+               // segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old

Review comment:
       no.. auto 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.

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] [incubator-pinot] yupeng9 commented on pull request #6204: add upsert metadata metric

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on pull request #6204:
URL: https://github.com/apache/incubator-pinot/pull/6204#issuecomment-718246056


   @Jackie-Jiang comments addressed. PTAL


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6204: add upsert metadata metric

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6204:
URL: https://github.com/apache/incubator-pinot/pull/6204#discussion_r513777950



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/upsert/PartitionUpsertMetadataManager.java
##########
@@ -85,8 +97,7 @@ public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<Re
               // committing a consuming segment, or reloading a completed segment.
 
               // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-              // Update the record location when there is a tie because the record locations should point to the new
-              // segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+               // segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old

Review comment:
       This is not auto formatting. One line is mis-deleted. Please double check




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] codecov-io commented on pull request #6204: add upsert metadata metric

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #6204:
URL: https://github.com/apache/incubator-pinot/pull/6204#issuecomment-717652034


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6204?src=pr&el=h1) Report
   > Merging [#6204](https://codecov.io/gh/apache/incubator-pinot/pull/6204?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a?el=desc) will **decrease** coverage by `19.98%`.
   > The diff coverage is `49.14%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6204/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/6204?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #6204       +/-   ##
   ===========================================
   - Coverage   66.44%   46.46%   -19.99%     
   ===========================================
     Files        1075     1237      +162     
     Lines       54773    58559     +3786     
     Branches     8168     8663      +495     
   ===========================================
   - Hits        36396    27207     -9189     
   - Misses      15700    29135    +13435     
   + Partials     2677     2217      -460     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | #integration | `46.46% <49.14%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6204?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ot/broker/broker/AllowAllAccessControlFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0FsbG93QWxsQWNjZXNzQ29udHJvbEZhY3RvcnkuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `52.83% <0.00%> (-13.84%)` | :arrow_down: |
   | [...org/apache/pinot/broker/queryquota/HitCounter.java](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcXVlcnlxdW90YS9IaXRDb3VudGVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...che/pinot/broker/queryquota/MaxHitRateTracker.java](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcXVlcnlxdW90YS9NYXhIaXRSYXRlVHJhY2tlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ache/pinot/broker/queryquota/QueryQuotaEntity.java](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcXVlcnlxdW90YS9RdWVyeVF1b3RhRW50aXR5LmphdmE=) | `0.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/client/AbstractResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Fic3RyYWN0UmVzdWx0U2V0LmphdmE=) | `26.66% <0.00%> (-30.48%)` | :arrow_down: |
   | [.../main/java/org/apache/pinot/client/Connection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L0Nvbm5lY3Rpb24uamF2YQ==) | `22.22% <0.00%> (-26.62%)` | :arrow_down: |
   | [.../org/apache/pinot/client/ResultTableResultSet.java](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L1Jlc3VsdFRhYmxlUmVzdWx0U2V0LmphdmE=) | `24.00% <0.00%> (-10.29%)` | :arrow_down: |
   | [...not/common/assignment/InstancePartitionsUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vYXNzaWdubWVudC9JbnN0YW5jZVBhcnRpdGlvbnNVdGlscy5qYXZh) | `64.28% <ø> (-8.89%)` | :arrow_down: |
   | [.../apache/pinot/common/exception/QueryException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZXhjZXB0aW9uL1F1ZXJ5RXhjZXB0aW9uLmphdmE=) | `90.27% <ø> (+5.55%)` | :arrow_up: |
   | ... and [1224 more](https://codecov.io/gh/apache/incubator-pinot/pull/6204/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6204?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6204?src=pr&el=footer). Last update [57d292c...89602e7](https://codecov.io/gh/apache/incubator-pinot/pull/6204?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6204: add upsert metadata metric

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6204:
URL: https://github.com/apache/incubator-pinot/pull/6204#discussion_r513103531



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/upsert/PartitionUpsertMetadataManager.java
##########
@@ -85,8 +97,7 @@ public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<Re
               // committing a consuming segment, or reloading a completed segment.
 
               // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-              // Update the record location when there is a tie because the record locations should point to the new
-              // segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+               // segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old

Review comment:
       This line is mis-deleted?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/upsert/PartitionUpsertMetadataManager.java
##########
@@ -51,6 +53,16 @@
 public class PartitionUpsertMetadataManager {
   private static final Logger LOGGER = LoggerFactory.getLogger(PartitionUpsertMetadataManager.class);
 
+  private final String _tableName;

Review comment:
       Use `_tableNameWithType` for clarity

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/upsert/PartitionUpsertMetadataManager.java
##########
@@ -113,6 +124,9 @@ public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<Re
         }
       });
     }
+    // update metrics

Review comment:
       (nit) Capitalize the first character for consistency

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/upsert/TableUpsertMetadataManager.java
##########
@@ -29,8 +30,16 @@
 @ThreadSafe
 public class TableUpsertMetadataManager {
   private final Map<Integer, PartitionUpsertMetadataManager> _partitionMetadataManagerMap = new ConcurrentHashMap<>();
+  private final String _tableName;
+  private final ServerMetrics _serverMetrics;
+
+  public TableUpsertMetadataManager(String tableName, ServerMetrics serverMetrics) {
+    _tableName = tableName;
+    _serverMetrics = serverMetrics;
+  }
 
   public PartitionUpsertMetadataManager getOrCreatePartitionManager(int partitionId) {
-    return _partitionMetadataManagerMap.computeIfAbsent(partitionId, k -> new PartitionUpsertMetadataManager());
+    return _partitionMetadataManagerMap
+        .computeIfAbsent(partitionId, k -> new PartitionUpsertMetadataManager(_tableName, partitionId, _serverMetrics));

Review comment:
       (nit)
   ```suggestion
           .computeIfAbsent(partitionId, k -> new PartitionUpsertMetadataManager(_tableName, k, _serverMetrics));
   ```

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/upsert/PartitionUpsertMetadataManagerTest.java
##########
@@ -37,7 +39,8 @@
 
   @Test
   public void testAddSegment() {
-    PartitionUpsertMetadataManager upsertMetadataManager = new PartitionUpsertMetadataManager();
+    PartitionUpsertMetadataManager upsertMetadataManager =
+        new PartitionUpsertMetadataManager("testTable", 0, Mockito.mock(ServerMetrics.class));

Review comment:
       Make `"testTable"` a constant and change it to `testTable_REALTIME`?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/upsert/TableUpsertMetadataManager.java
##########
@@ -29,8 +30,16 @@
 @ThreadSafe
 public class TableUpsertMetadataManager {
   private final Map<Integer, PartitionUpsertMetadataManager> _partitionMetadataManagerMap = new ConcurrentHashMap<>();
+  private final String _tableName;

Review comment:
       `_tableNameWithType`

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java
##########
@@ -121,7 +125,8 @@ public void loadSegment()
     _indexSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.heap);
     _upsertIndexSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), ReadMode.heap);
     ((ImmutableSegmentImpl) _upsertIndexSegment)
-        .enableUpsert(new PartitionUpsertMetadataManager(), new ThreadSafeMutableRoaringBitmap());
+        .enableUpsert(new PartitionUpsertMetadataManager("testTable", 0, _serverMetrics),

Review comment:
       (nit) Mock a ServerMetrics here locally instead of having a member variable

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
##########
@@ -800,7 +801,8 @@ public FakeLLRealtimeSegmentDataManager(RealtimeSegmentZKMetadata segmentZKMetad
         throws Exception {
       super(segmentZKMetadata, tableConfig, realtimeTableDataManager, resourceDataDir,
           new IndexLoadingConfig(makeInstanceDataManagerConfig(), tableConfig), schema, llcSegmentName,
-          semaphoreMap.get(llcSegmentName.getPartitionId()), serverMetrics, new PartitionUpsertMetadataManager());
+          semaphoreMap.get(llcSegmentName.getPartitionId()), serverMetrics,
+          new PartitionUpsertMetadataManager("testTable", 0, Mockito.mock(ServerMetrics.class)));

Review comment:
       Use the serverMetrics available instead of mocking one




----------------------------------------------------------------
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:
users@infra.apache.org



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