You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/03/31 18:46:56 UTC

[GitHub] [druid] zachjsh opened a new pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

zachjsh opened a new pull request #11056:
URL: https://github.com/apache/druid/pull/11056


   ### Description
   
   
   This change adds a query parameter `computeUsingClusterView` to loadstatus apis
   that if specified have the coordinator compute undereplication for segments based
   on the number of services available within cluster that the segment can be replicated
   on, instead of the configured replication count configured in load rule. A default
   load rule is created in all clusters that specified that all segments should be
   replicated 2 times. As replicas are forced to be on separate nodes in the cluster,
   this causes the loadstatus api to report that there are under-replicated segments
   when there is only 1 data server in the cluster. In this case, calling loadstatus
   api without this new query parameter will always result in a response indicating
   under-replication of segments
   
   * loadstatus api now returns 503 if either the cluster or segmentReplicantLookup objects arent
      yet initialized. This should only happen during the period of time before the coordinator run loop
      has executed at least once, as these objects are set during every instantiation of the coordinator
      run loop.
   
   Note: Still need to add tests, just wanted to make sure this approach makes sense.
   
   This PR has:
   - [x] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] zachjsh commented on a change in pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
zachjsh commented on a change in pull request #11056:
URL: https://github.com/apache/druid/pull/11056#discussion_r605953046



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java
##########
@@ -584,6 +580,59 @@ public void runCompactSegmentsDuty()
     compactSegmentsDuty.run();
   }
 
+  private Map<String, Object2LongMap<String>> computeUnderReplicationCountsPerDataSourcePerTierForSegmentsInternal(
+      Iterable<DataSegment> dataSegments,
+      boolean computeUsingClusterView
+  )
+  {
+    final Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTier = new HashMap<>();
+
+    if (segmentReplicantLookup == null) {
+      throw new ServiceUnavailableException(
+          "Coordinator segment replicant lookup is not initialized yet. Try again later.");
+    }
+
+    if (computeUsingClusterView && cluster == null) {
+      throw new ServiceUnavailableException(
+          "coordinator hasn't populated information about cluster yet, try again later");
+    }
+
+    final DateTime now = DateTimes.nowUtc();
+
+    for (final DataSegment segment : dataSegments) {
+      final List<Rule> rules = metadataRuleManager.getRulesWithDefault(segment.getDataSource());
+
+      for (final Rule rule : rules) {
+        if (!rule.appliesTo(segment, now)) {
+          // Rule did not match. Continue to the next Rule.
+          continue;
+        }
+        if (!rule.canLoadSegments()) {
+          // Rule matched but rule does not and cannot load segments.
+          // Hence, there is no need to update underReplicationCountsPerDataSourcePerTier map
+          break;
+        }
+
+        if (computeUsingClusterView && (rule instanceof LoadRule)) {

Review comment:
       Changed accordingly. Before though, for broadcast rule, it was falling through to else branch and effectively doing same thing. Thought it was weird to add implementation for Broadcast rule that didn't use the parameter, but I guess not a big deal.




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] zachjsh closed pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
zachjsh closed pull request #11056:
URL: https://github.com/apache/druid/pull/11056


   


-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] techdocsmith commented on a change in pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
techdocsmith commented on a change in pull request #11056:
URL: https://github.com/apache/druid/pull/11056#discussion_r606042905



##########
File path: docs/operations/api-reference.md
##########
@@ -102,6 +104,14 @@ Returns the number of segments left to load until segments that should be loaded
 
 Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
 
+* `/druid/coordinator/v1/loadstatus?full?computeUsingClusterView`
+
+Returns the number of segments not yet loaded for each tier until all segments to load in the cluster are available.

Review comment:
       ```suggestion
   Returns the number of segments not yet loaded for each tier until all segments loading in the cluster are available.
   ```

##########
File path: docs/operations/api-reference.md
##########
@@ -102,6 +104,14 @@ Returns the number of segments left to load until segments that should be loaded
 
 Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
 
+* `/druid/coordinator/v1/loadstatus?full?computeUsingClusterView`
+
+Returns the number of segments not yet loaded for each tier until all segments to load in the cluster are available.
+The result includes segment replication counts. It also factors in the number of available nodes that are of a service type that can load the segment when computing the number of segments remaining to load.
+A segment is considered fully loaded when:
+- Druid had replicated it the number of times configured in the corresponding load rule.
+- Or the number of replicas for the segment in each tier where it configured to be replicated equals the available nodes of a service type that are currently allowed to load the segment in the tier.

Review comment:
       ```suggestion
   - Or the number of replicas for the segment in each tier where it is configured to be replicated equals the available nodes of a service type that are currently allowed to load the segment in the tier.
   ```

##########
File path: docs/operations/api-reference.md
##########
@@ -102,6 +104,14 @@ Returns the number of segments left to load until segments that should be loaded
 
 Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
 
+* `/druid/coordinator/v1/loadstatus?full?computeUsingClusterView`
+
+Returns the number of segments not yet loaded for each tier until all segments to load in the cluster are available.
+The result includes segment replication counts. It also factors in the number of available nodes that are of a service type that can load the segment when computing the number of segments remaining to load.
+A segment is considered fully loaded when:
+- Druid had replicated it the number of times configured in the corresponding load rule.

Review comment:
       ```suggestion
   - Druid has replicated it the number of times configured in the corresponding load rule.
   ```

##########
File path: docs/operations/api-reference.md
##########
@@ -151,6 +161,8 @@ Setting `forceMetadataRefresh` to true will force the coordinator to poll latest
 (Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms 
 of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status)
 Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. 
+An optional query parameter `computeUsingClusterView` can be set to consider the cluster services available when computing
+the segments left to load. Please see [Coordinator Segment Loading](#coordinator-segment-loading) for a more detailed description of this parameter.

Review comment:
       ```suggestion
   the segments left to load. See [Coordinator Segment Loading](#coordinator-segment-loading) for details.
   ```

##########
File path: docs/operations/api-reference.md
##########
@@ -151,6 +161,8 @@ Setting `forceMetadataRefresh` to true will force the coordinator to poll latest
 (Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms 
 of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status)
 Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. 
+An optional query parameter `computeUsingClusterView` can be set to consider the cluster services available when computing

Review comment:
       ```suggestion
   You can pass the optional query parameter `computeUsingClusterView` to factor in the available cluster services when calculating
   ```




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #11056:
URL: https://github.com/apache/druid/pull/11056#discussion_r605322228



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java
##########
@@ -584,6 +580,59 @@ public void runCompactSegmentsDuty()
     compactSegmentsDuty.run();
   }
 
+  private Map<String, Object2LongMap<String>> computeUnderReplicationCountsPerDataSourcePerTierForSegmentsInternal(
+      Iterable<DataSegment> dataSegments,
+      boolean computeUsingClusterView
+  )
+  {
+    final Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTier = new HashMap<>();
+
+    if (segmentReplicantLookup == null) {
+      throw new ServiceUnavailableException(
+          "Coordinator segment replicant lookup is not initialized yet. Try again later.");
+    }
+
+    if (computeUsingClusterView && cluster == null) {
+      throw new ServiceUnavailableException(
+          "coordinator hasn't populated information about cluster yet, try again later");
+    }
+
+    final DateTime now = DateTimes.nowUtc();
+
+    for (final DataSegment segment : dataSegments) {
+      final List<Rule> rules = metadataRuleManager.getRulesWithDefault(segment.getDataSource());
+
+      for (final Rule rule : rules) {
+        if (!rule.appliesTo(segment, now)) {
+          // Rule did not match. Continue to the next Rule.
+          continue;
+        }
+        if (!rule.canLoadSegments()) {
+          // Rule matched but rule does not and cannot load segments.
+          // Hence, there is no need to update underReplicationCountsPerDataSourcePerTier map
+          break;
+        }
+
+        if (computeUsingClusterView && (rule instanceof LoadRule)) {

Review comment:
       The `(rule instanceof LoadRule)` should be removed, since `BroadcastDistributionRule` can also load segments (it also returns true for `canLoadSegments()`, which is already checked before this)

##########
File path: server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java
##########
@@ -119,6 +119,34 @@ public void updateUnderReplicated(
     });
   }
 
+  @Override
+  public void updateUnderReplicatedWithClusterView(

Review comment:
       Can you add an implementation for `BroadcastDistributionRule` as well? I think the implementation there could just call `updateUnderReplicated` and ignore the `cluster` parameter, since it's already looking at the available servers in the cluster.

##########
File path: server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java
##########
@@ -1201,7 +1201,7 @@ public void testMarkSegmentsAsUnusedInvalidPayloadBothArguments()
   public void testGetDatasourceLoadstatusForceMetadataRefreshNull()
   {
     DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null);
-    Response response = dataSourcesResource.getDatasourceLoadstatus("datasource1", null, null, null, null);
+    Response response = dataSourcesResource.getDatasourceLoadstatus("datasource1", null, null, null, null, null);

Review comment:
       Can you add tests that use the new parameter?




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] zachjsh merged pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
zachjsh merged pull request #11056:
URL: https://github.com/apache/druid/pull/11056


   


-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #11056:
URL: https://github.com/apache/druid/pull/11056#discussion_r605991809



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java
##########
@@ -584,6 +580,59 @@ public void runCompactSegmentsDuty()
     compactSegmentsDuty.run();
   }
 
+  private Map<String, Object2LongMap<String>> computeUnderReplicationCountsPerDataSourcePerTierForSegmentsInternal(
+      Iterable<DataSegment> dataSegments,
+      boolean computeUsingClusterView
+  )
+  {
+    final Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTier = new HashMap<>();
+
+    if (segmentReplicantLookup == null) {
+      throw new ServiceUnavailableException(
+          "Coordinator segment replicant lookup is not initialized yet. Try again later.");
+    }
+
+    if (computeUsingClusterView && cluster == null) {
+      throw new ServiceUnavailableException(
+          "coordinator hasn't populated information about cluster yet, try again later");
+    }
+
+    final DateTime now = DateTimes.nowUtc();
+
+    for (final DataSegment segment : dataSegments) {
+      final List<Rule> rules = metadataRuleManager.getRulesWithDefault(segment.getDataSource());
+
+      for (final Rule rule : rules) {
+        if (!rule.appliesTo(segment, now)) {
+          // Rule did not match. Continue to the next Rule.
+          continue;
+        }
+        if (!rule.canLoadSegments()) {
+          // Rule matched but rule does not and cannot load segments.
+          // Hence, there is no need to update underReplicationCountsPerDataSourcePerTier map
+          break;
+        }
+
+        if (computeUsingClusterView && (rule instanceof LoadRule)) {

Review comment:
       Cool, thanks, I thought it would be cleaner to have the new method be supported by all segment-loading rules, vs. the behavior specific to just LoadRule




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei commented on a change in pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #11056:
URL: https://github.com/apache/druid/pull/11056#discussion_r605995768



##########
File path: docs/operations/api-reference.md
##########
@@ -102,6 +102,15 @@ Returns the number of segments left to load until segments that should be loaded
 
 Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
 
+* `/druid/coordinator/v1/loadstatus?full?computeUsingClusterView`
+
+Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available.
+This includes segment replication counts. When computing the number of segments left to load, the number of nodes
+available in the cluster that are of a service type that can load the segment is considered. A segment is considered fully
+loaded if it is replicated as many times as configured in the corresponding load rule, or if there are as many replicas of

Review comment:
       Suggest "equals the number of available nodes" instead in the suggested revision




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] zachjsh commented on pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
zachjsh commented on pull request #11056:
URL: https://github.com/apache/druid/pull/11056#issuecomment-812189041


   @techdocsmith can you take a look at the change to docs.


-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] techdocsmith commented on a change in pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
techdocsmith commented on a change in pull request #11056:
URL: https://github.com/apache/druid/pull/11056#discussion_r605987115



##########
File path: docs/operations/api-reference.md
##########
@@ -102,6 +102,15 @@ Returns the number of segments left to load until segments that should be loaded
 
 Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
 
+* `/druid/coordinator/v1/loadstatus?full?computeUsingClusterView`
+
+Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available.

Review comment:
       ```suggestion
   Returns the number of segments not yet loaded for each tier until all segments to load in the cluster are available.
   ```

##########
File path: docs/operations/api-reference.md
##########
@@ -102,6 +102,15 @@ Returns the number of segments left to load until segments that should be loaded
 
 Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
 
+* `/druid/coordinator/v1/loadstatus?full?computeUsingClusterView`
+
+Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available.
+This includes segment replication counts. When computing the number of segments left to load, the number of nodes

Review comment:
       ```suggestion
   The result includes segment replication counts. It also factors in the number of available nodes that are of a service type that can load the segment when computing the number of segments remaining to load.
   ```

##########
File path: docs/operations/api-reference.md
##########
@@ -102,6 +102,15 @@ Returns the number of segments left to load until segments that should be loaded
 
 Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
 
+* `/druid/coordinator/v1/loadstatus?full?computeUsingClusterView`
+
+Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available.
+This includes segment replication counts. When computing the number of segments left to load, the number of nodes
+available in the cluster that are of a service type that can load the segment is considered. A segment is considered fully
+loaded if it is replicated as many times as configured in the corresponding load rule, or if there are as many replicas of
+the segment in each tier where the segment is configured to be replicated, as there are available nodes in the tier at the

Review comment:
       ```suggestion
   
   ```

##########
File path: docs/operations/api-reference.md
##########
@@ -102,6 +102,15 @@ Returns the number of segments left to load until segments that should be loaded
 
 Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
 
+* `/druid/coordinator/v1/loadstatus?full?computeUsingClusterView`
+
+Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available.
+This includes segment replication counts. When computing the number of segments left to load, the number of nodes
+available in the cluster that are of a service type that can load the segment is considered. A segment is considered fully
+loaded if it is replicated as many times as configured in the corresponding load rule, or if there are as many replicas of

Review comment:
       ```suggestion
   - Druid had replicated it the number of times configured in the corresponding load rule.
   - Or the number of replicas for the segment in each tier where it configured to be replicated equals the available nodes of a service type that are currently allowed to load the segment in the tier.
   ```

##########
File path: docs/operations/api-reference.md
##########
@@ -102,6 +102,15 @@ Returns the number of segments left to load until segments that should be loaded
 
 Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
 
+* `/druid/coordinator/v1/loadstatus?full?computeUsingClusterView`
+
+Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available.
+This includes segment replication counts. When computing the number of segments left to load, the number of nodes
+available in the cluster that are of a service type that can load the segment is considered. A segment is considered fully

Review comment:
       ```suggestion
   A segment is considered fully loaded when:
   ```

##########
File path: docs/operations/api-reference.md
##########
@@ -102,6 +102,15 @@ Returns the number of segments left to load until segments that should be loaded
 
 Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
 
+* `/druid/coordinator/v1/loadstatus?full?computeUsingClusterView`
+
+Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available.
+This includes segment replication counts. When computing the number of segments left to load, the number of nodes
+available in the cluster that are of a service type that can load the segment is considered. A segment is considered fully
+loaded if it is replicated as many times as configured in the corresponding load rule, or if there are as many replicas of
+the segment in each tier where the segment is configured to be replicated, as there are available nodes in the tier at the
+current time that are of a service type that is allowed to load the segment.

Review comment:
       ```suggestion
   ```




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] zachjsh commented on a change in pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
zachjsh commented on a change in pull request #11056:
URL: https://github.com/apache/druid/pull/11056#discussion_r605953124



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java
##########
@@ -119,6 +119,34 @@ public void updateUnderReplicated(
     });
   }
 
+  @Override
+  public void updateUnderReplicatedWithClusterView(

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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [druid] zachjsh commented on a change in pull request #11056: Add paramter to loadstatus API to compute underdeplication against cluster view

Posted by GitBox <gi...@apache.org>.
zachjsh commented on a change in pull request #11056:
URL: https://github.com/apache/druid/pull/11056#discussion_r605952500



##########
File path: server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java
##########
@@ -1201,7 +1201,7 @@ public void testMarkSegmentsAsUnusedInvalidPayloadBothArguments()
   public void testGetDatasourceLoadstatusForceMetadataRefreshNull()
   {
     DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null);
-    Response response = dataSourcesResource.getDatasourceLoadstatus("datasource1", null, null, null, null);
+    Response response = dataSourcesResource.getDatasourceLoadstatus("datasource1", null, null, null, null, null);

Review comment:
       Added test.




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org