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:18:35 UTC

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

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