cryptoe commented on code in PR #14808:
URL: https://github.com/apache/druid/pull/14808#discussion_r1300994917


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -127,6 +137,12 @@ public ClientQuerySegmentWalker(
     this.serverConfig = serverConfig;
     this.cache = cache;
     this.cacheConfig = cacheConfig;
+    this.subqueryLimitUtils = new SubqueryLimitUtils(

Review Comment:
   Generally utils donot have state ie only static methods. Does doing it this 
way aid testing ?



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -114,7 +121,10 @@ public ClientQuerySegmentWalker(
       ObjectMapper objectMapper,
       ServerConfig serverConfig,
       Cache cache,
-      CacheConfig cacheConfig
+      CacheConfig cacheConfig,
+      LookupExtractorFactoryContainerProvider lookupManager,
+      DruidHttpClientConfig httpClientConfig,
+      SubqueryCountStatsProvider statsProvider

Review Comment:
   ```suggestion
         SubqueryStatsProvider subQueryStatsProvider
   ```



##########
server/src/main/java/org/apache/druid/server/SubqueryLimitUtils.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server;
+
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.lookup.LookupExtractor;
+import org.apache.druid.query.lookup.LookupExtractorFactoryContainer;
+import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
+
+public class SubqueryLimitUtils
+{
+  private static final double SUBQUERY_MEMORY_BYTES_FRACTION = 0.5;
+  private static final Logger log = new Logger(SubqueryLimitUtils.class);
+
+
+  public static final String UNLIMITED_LIMIT_VALUE = "unlimited";
+  public static final String AUTO_LIMIT_VALUE = "auto";
+
+  public static final Long UNLIMITED_LIMIT_REPRESENTATION = -1L;
+
+  private final long autoLimitBytes;
+
+  public SubqueryLimitUtils(
+      final LookupExtractorFactoryContainerProvider lookupManager,
+      final long maxMemoryInJvm,
+      final int brokerNumHttpConnections
+  )
+  {
+    autoLimitBytes = computeLimitBytesForAuto(lookupManager, maxMemoryInJvm, 
brokerNumHttpConnections);
+  }
+
+  public long convertSubqueryLimitStringToLong(final String maxSubqueryLimit)
+  {
+    if (UNLIMITED_LIMIT_VALUE.equalsIgnoreCase(maxSubqueryLimit)) {
+      return UNLIMITED_LIMIT_REPRESENTATION;
+    }
+    if (AUTO_LIMIT_VALUE.equalsIgnoreCase(maxSubqueryLimit)) {
+      return autoLimitBytes;
+    }
+
+    long retVal;
+    try {
+      retVal = Long.parseLong(maxSubqueryLimit);
+    }
+    catch (NumberFormatException e) {
+      throw InvalidInput.exception(
+          e,
+          "Unable to parse the provided maxSubqueryLimit [%s] to a valid 
number. Valid values for the "
+          + "maxSubqueryLimits can be 'auto', 'unlimited' or a positive number 
representing bytes to reserve.",
+          maxSubqueryLimit
+      );
+    }
+
+    // This can happen if the provided number is greater than Longs.MAX_VALUE
+    if (retVal < 0) {
+      throw InvalidInput.exception("Limit too large");
+    }
+
+    return retVal;
+  }
+
+  /**
+   * Computes the byte limit when 'auto' is passed as a parameter. This 
computes the total heap space available
+   * for the subquery inlining by getting a fraction of the total heap space 
in JVM, removing the size of the lookups,
+   * and dividing it by the maximum concurrent queries that can run. Maximum 
concurrent queries that Druid can
+   * run is usually limited by its broker's http threadpool size

Review Comment:
   Please add an example above 



##########
server/src/main/java/org/apache/druid/server/metrics/SubqueryCountStatsMonitor.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.metrics;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.java.util.metrics.AbstractMonitor;
+import org.apache.druid.java.util.metrics.KeyedDiff;
+
+import java.util.Map;
+
+public class SubqueryCountStatsMonitor extends AbstractMonitor
+{
+
+  private final KeyedDiff keyedDiff = new KeyedDiff();
+  private final SubqueryCountStatsProvider statsProvider;
+
+  @Inject
+  public SubqueryCountStatsMonitor(SubqueryCountStatsProvider statsProvider)
+  {
+    this.statsProvider = statsProvider;
+  }
+
+  @Override
+  public boolean doMonitor(ServiceEmitter emitter)
+  {
+    final ServiceMetricEvent.Builder builder = new 
ServiceMetricEvent.Builder();
+    final long subqueriesWithRowBasedLimit = 
statsProvider.subqueriesWithRowBasedLimit();
+    final long subqueriesWithByteBasedLimit = 
statsProvider.subqueriesWithByteBasedLimit();
+    final long subqueriesFallingBackToRowBasedLimit = 
statsProvider.subqueriesFallingBackToRowBasedLimit();
+    final long subqueriesFallingBackDueToUnsufficientTypeInfo = 
statsProvider.subqueriesFallingBackDueToUnsufficientTypeInfo();
+
+    Map<String, Long> diff = keyedDiff.to(
+        "subqueryCountStats",
+        ImmutableMap.of(
+            "subquery/rowLimit/count", subqueriesWithRowBasedLimit,
+            "subquery/byteLimit/count", subqueriesWithByteBasedLimit,

Review Comment:
   Also Please document this in metrics.md



##########
server/src/main/java/org/apache/druid/server/metrics/SubqueryCountStatsMonitor.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.metrics;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.java.util.metrics.AbstractMonitor;
+import org.apache.druid.java.util.metrics.KeyedDiff;
+
+import java.util.Map;
+
+public class SubqueryCountStatsMonitor extends AbstractMonitor
+{
+
+  private final KeyedDiff keyedDiff = new KeyedDiff();
+  private final SubqueryCountStatsProvider statsProvider;
+
+  @Inject
+  public SubqueryCountStatsMonitor(SubqueryCountStatsProvider statsProvider)
+  {
+    this.statsProvider = statsProvider;
+  }
+
+  @Override
+  public boolean doMonitor(ServiceEmitter emitter)
+  {
+    final ServiceMetricEvent.Builder builder = new 
ServiceMetricEvent.Builder();
+    final long subqueriesWithRowBasedLimit = 
statsProvider.subqueriesWithRowBasedLimit();
+    final long subqueriesWithByteBasedLimit = 
statsProvider.subqueriesWithByteBasedLimit();
+    final long subqueriesFallingBackToRowBasedLimit = 
statsProvider.subqueriesFallingBackToRowBasedLimit();
+    final long subqueriesFallingBackDueToUnsufficientTypeInfo = 
statsProvider.subqueriesFallingBackDueToUnsufficientTypeInfo();
+
+    Map<String, Long> diff = keyedDiff.to(
+        "subqueryCountStats",
+        ImmutableMap.of(
+            "subquery/rowLimit/count", subqueriesWithRowBasedLimit,

Review Comment:
   Does this metric mean We have hit the row limit or we are using numRowBased 
guardrail is unclear from the name.
   Since we are touching this code, we can also add the number of q's going 
over the limit but it can be a follow up PR as well. 
   



##########
server/src/main/java/org/apache/druid/server/SubqueryLimitUtils.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server;
+
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.lookup.LookupExtractor;
+import org.apache.druid.query.lookup.LookupExtractorFactoryContainer;
+import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
+
+public class SubqueryLimitUtils

Review Comment:
   Nit:: Instead of Limit lets use the word SubQueryGuardRail so the reader 
instantly figures out that this class is being used for adding guardRails to 
subQuery and not something like 
   `Select * from select * from foo limit 100`



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -730,6 +763,11 @@ private static <T, QueryType extends Query<T>> 
Optional<DataSource> materializeR
           useNestedForUnknownTypeInSubquery
       );
     }
+    catch (UnsupportedColumnTypeException e) {
+      statsProvider.incrementSubqueriesFallingBackDueToUnsufficientTypeInfo();
+      log.debug(e, "Type info in signature insufficient to materialize rows as 
frames.");
+      return Optional.empty();
+    }
     catch (Exception e) {
       log.debug(e, "Unable to materialize the results as frames due to an 
unhandleable exception "

Review Comment:
   We should also add a unkownException to the statsProvider?



##########
extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java:
##########
@@ -311,6 +312,17 @@ public void testQuery() throws IOException
     Assert.assertNotNull(expectedResults);
     Assert.assertThat(expectedResults, IsInstanceOf.instanceOf(List.class));
 
+    DruidHttpClientConfig httpClientConfig = new DruidHttpClientConfig()
+    {
+      @Override
+      public long getMaxQueuedBytes()
+      {
+        return 0L;
+      }
+

Review Comment:
   Nit: extra spaces



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -175,10 +197,13 @@ public <T> QueryRunner<T> 
getQueryRunnerForIntervals(Query<T> query, Iterable<In
     final DataSource freeTradeDataSource = 
globalizeIfPossible(newQuery.getDataSource());
     // do an inlining dry run to see if any inlining is necessary, without 
actually running the queries.
     final int maxSubqueryRows = 
query.context().getMaxSubqueryRows(serverConfig.getMaxSubqueryRows());
-    final long maxSubqueryMemory = 
query.context().getMaxSubqueryMemoryBytes(serverConfig.getMaxSubqueryBytes());
+    final String maxSubqueryMemoryString = query.context()
+                                                
.getMaxSubqueryMemoryBytes(serverConfig.getMaxSubqueryBytes());
+    final long maxSubqueryMemory = 
subqueryLimitUtils.convertSubqueryLimitStringToLong(maxSubqueryMemoryString);

Review Comment:
   202 and 200 line can be merged. 



##########
server/src/main/java/org/apache/druid/server/SubqueryLimitUtils.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server;
+
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.lookup.LookupExtractor;
+import org.apache.druid.query.lookup.LookupExtractorFactoryContainer;
+import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
+
+public class SubqueryLimitUtils
+{
+  private static final double SUBQUERY_MEMORY_BYTES_FRACTION = 0.5;
+  private static final Logger log = new Logger(SubqueryLimitUtils.class);
+
+
+  public static final String UNLIMITED_LIMIT_VALUE = "unlimited";
+  public static final String AUTO_LIMIT_VALUE = "auto";
+
+  public static final Long UNLIMITED_LIMIT_REPRESENTATION = -1L;
+
+  private final long autoLimitBytes;
+
+  public SubqueryLimitUtils(
+      final LookupExtractorFactoryContainerProvider lookupManager,
+      final long maxMemoryInJvm,
+      final int brokerNumHttpConnections
+  )
+  {
+    autoLimitBytes = computeLimitBytesForAuto(lookupManager, maxMemoryInJvm, 
brokerNumHttpConnections);
+  }
+
+  public long convertSubqueryLimitStringToLong(final String maxSubqueryLimit)
+  {
+    if (UNLIMITED_LIMIT_VALUE.equalsIgnoreCase(maxSubqueryLimit)) {
+      return UNLIMITED_LIMIT_REPRESENTATION;
+    }
+    if (AUTO_LIMIT_VALUE.equalsIgnoreCase(maxSubqueryLimit)) {
+      return autoLimitBytes;
+    }
+
+    long retVal;
+    try {
+      retVal = Long.parseLong(maxSubqueryLimit);
+    }
+    catch (NumberFormatException e) {
+      throw InvalidInput.exception(
+          e,
+          "Unable to parse the provided maxSubqueryLimit [%s] to a valid 
number. Valid values for the "
+          + "maxSubqueryLimits can be 'auto', 'unlimited' or a positive number 
representing bytes to reserve.",
+          maxSubqueryLimit
+      );
+    }
+
+    // This can happen if the provided number is greater than Longs.MAX_VALUE
+    if (retVal < 0) {
+      throw InvalidInput.exception("Limit too large");
+    }
+
+    return retVal;
+  }
+
+  /**
+   * Computes the byte limit when 'auto' is passed as a parameter. This 
computes the total heap space available
+   * for the subquery inlining by getting a fraction of the total heap space 
in JVM, removing the size of the lookups,
+   * and dividing it by the maximum concurrent queries that can run. Maximum 
concurrent queries that Druid can
+   * run is usually limited by its broker's http threadpool size
+   */
+  private static long computeLimitBytesForAuto(
+      final LookupExtractorFactoryContainerProvider lookupManager,
+      final long maxMemoryInJvm,
+      final int brokerNumHttpConnections
+  )
+  {
+    long memoryInJvmWithoutLookups = maxMemoryInJvm - 
computeLookupFootprint(lookupManager);
+    long memoryInJvmForSubqueryResultsInlining = (long) 
(memoryInJvmWithoutLookups * SUBQUERY_MEMORY_BYTES_FRACTION);
+    long memoryInJvmForSubqueryResultsInliningPerQuery = 
memoryInJvmForSubqueryResultsInlining
+                                                         / 
brokerNumHttpConnections;
+    return Math.max(memoryInJvmForSubqueryResultsInliningPerQuery, 1L);
+  }
+
+  /**
+   * Computes the size occupied by the lookups. If the size of the lookup 
cannot be computed, it skips over the lookup
+   */
+  private static long computeLookupFootprint(final 
LookupExtractorFactoryContainerProvider lookupManager)
+  {
+
+    if (lookupManager == null || lookupManager.getAllLookupNames() == null) {
+      log.warn("Failed to get the lookupManager for estimating lookup size. 
Skipping.");
+      return 0;
+    }
+
+    int lookupCount = 0;
+    long lookupFootprint = 0;
+
+    for (final String lookupName : lookupManager.getAllLookupNames()) {
+      final LookupExtractorFactoryContainer container = 
lookupManager.get(lookupName).orElse(null);
+
+      if (container != null) {
+        try {
+          final LookupExtractor extractor = 
container.getLookupExtractorFactory().get();

Review Comment:
   Questions:
   What ensures lookups are loaded completely.
   `lookupFootprint += extractor.estimateHeapFootprint();` seems to be 
traversing each key value pair so it can become slow leading to slower broker 
startups times no ?
   
   
   



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