clintropolis commented on a change in pull request #11766:
URL: https://github.com/apache/druid/pull/11766#discussion_r722007436



##########
File path: docs/configuration/index.md
##########
@@ -1794,6 +1794,7 @@ See [cache configuration](#cache-configuration) for how 
to configure cache setti
 |--------|---------------|-----------|-------|
 |`druid.serverview.type`|batch or http|Segment discovery method to use. "http" 
enables discovering segments using HTTP instead of zookeeper.|batch|
 |`druid.broker.segment.watchedTiers`|List of strings|Broker watches the 
segment announcements from processes serving segments to build cache of which 
process is serving which segments, this configuration allows to only consider 
segments being served from a whitelist of tiers. By default, Broker would 
consider all tiers. This can be used to partition your dataSources in specific 
Historical tiers and configure brokers in partitions so that they are only 
queryable for specific dataSources.|none|
+|`druid.broker.segment.ignoredTiers`|List of strings|The Broker watches the 
segment announcements from processes that serve segments to build a cache to 
relate each process to the segments it serves. This configuration allows the 
Broker to ignore the segments being served from a list of tiers. By default, 
Broker considers all tiers. This config is mutually exclusive from 
`druid.broker.segment.watchedTiers` and at most only one of these can be 
configured on a Broker.|none|

Review comment:
       should `druid.broker.segment.watchedTiers` docs also be updated to 
indicate that it is mutually exclusive with `ignoredTiers`?

##########
File path: 
server/src/test/java/org/apache/druid/client/BrokerSegmentWatcherConfigTest.java
##########
@@ -45,9 +45,10 @@ public void testSerde() throws Exception
     );
 
     Assert.assertNull(config.getWatchedTiers());
+    Assert.assertNull(config.getIgnoredTiers());
 
     //non-defaults
-    json = "{ \"watchedTiers\": [\"t1\", \"t2\"], \"watchedDataSources\": 
[\"ds1\", \"ds2\"] }";
+    json = "{ \"watchedTiers\": [\"t1\", \"t2\"], \"watchedDataSources\": 
[\"ds1\", \"ds2\"], \"ignoredTiers\": [\"t3\", \"t4\"] }";

Review comment:
       this test is sort of strange now that it is an invalid configuration, 
though technically it isn't enforced by the config itself so it isn't 
problematic here. maybe we should at least leave a comment that this config is 
illegal in practice?

##########
File path: server/src/main/java/org/apache/druid/client/BrokerServerView.java
##########
@@ -106,17 +106,34 @@ public BrokerServerView(
     this.baseView = baseView;
     this.tierSelectorStrategy = tierSelectorStrategy;
     this.emitter = emitter;
-    this.segmentWatcherConfig = segmentWatcherConfig;
     this.clients = new ConcurrentHashMap<>();
     this.selectors = new HashMap<>();
     this.timelines = new HashMap<>();
 
+    // Validate and set the segment watcher config
+    this.segmentWatcherConfig = segmentWatcherConfig;
+    if (segmentWatcherConfig.getWatchedTiers() != null

Review comment:
       i know this isn't new, but I wonder since we now doing some validation 
here if we should also explode if `watchedTiers` is empty, which also seems 
like an invalid configuration




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to