You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "LakshSingla (via GitHub)" <gi...@apache.org> on 2023/03/20 06:30:45 UTC

[GitHub] [druid] LakshSingla opened a new pull request, #13952: Limit the subquery results by memory usage

LakshSingla opened a new pull request, #13952:
URL: https://github.com/apache/druid/pull/13952

   ### Description
   
   Currently, in the ClientQuerySegmentWalker, when the data sources get inlined, they can be limited by the number of rows to prevent a query (subquery) from hogging up the broker's memory. This however doesn't have a proper correspondence with the memory used, since a row can have multiple columns with varying amounts of data in them. Therefore it would be better if there is a memory limit also available, which prevents the subquery's results from exploding beyond a certain memory limit.
   
   This PR aims to use the Frame which were introduced along with the MSQ to store the inline results. Since the Frames are backed by memory, we can fetch the memory used by the frame, and correspondingly the data source to estimate the size that is consumed by the inline data source. This is a close estimate of the size taken by the subquery results.
   
   ##### Limitations
   Currently, if the row signatures contain columns with unknown types, then this falls back to the rows for generating the intermediate result.  
   
   #### Release note
   To be added
   
   
   <hr>
   
   ##### Key changed/added classes in this PR
    
   
   <hr>
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items from the checklist below are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   This PR has:
   
   - [x] been self-reviewed.
      - [x] 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.)
   - [x] added documentation for new or modified features or behaviors.
   - [ ] a release note entry in the PR description.
   - [ ] 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.
   - [x] 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.
   - [x] 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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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] abhishekagarwal87 commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1241746091


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -613,96 +638,149 @@ private static <T, QueryType extends Query<T>> DataSource toInlineDataSource(
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,
       final AtomicLong memoryLimitAccumulator,
+      final AtomicBoolean cannotMaterializeToFrames,
       final int limit,
-      long memoryLimit
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
   )
   {
-    final int limitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
-    boolean memoryLimitSet = memoryLimit >= 0;
-
-    if (limitAccumulator.get() >= limitToUse) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum[%d] reached",
-          limitToUse
-      );
-    }
-
-    if (memoryLimitSet && memoryLimitAccumulator.get() >= memoryLimit) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
-          memoryLimit
-      );
-    }
+    final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
 
     DataSource dataSource;
-    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
-    if (memoryLimitSet) {
-      try {
-        Optional<Sequence<FrameSignaturePair>> framesOptional = toolChest.resultsAsFrames(
+
+    switch (ClientQuerySegmentWalkerUtils.getLimitType(memoryLimit, cannotMaterializeToFrames.get())) {
+      case ROW_LIMIT:
+        if (limitAccumulator.get() >= rowLimitToUse) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] rows",
+              rowLimitToUse
+          );
+        }
+        dataSource = materializeResultsAsArray(
             query,
             results,
-            memoryLimit - memoryLimitAccumulator.get()
+            toolChest,
+            limitAccumulator,
+            limit
         );
-
-        if (!framesOptional.isPresent()) {
-          throw new ISE("The memory of the subqueries cannot be estimated correctly.");
+        break;
+      case MEMORY_LIMIT:
+        if (memoryLimitAccumulator.get() >= memoryLimit) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+              memoryLimit
+          );
         }
-
-        Sequence<FrameSignaturePair> frames = framesOptional.get();
-        List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
-        frames.forEach(
-            frame -> {
-              if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] bytes",
-                    memoryLimit
-                );
-
-              }
-              if (limitAccumulator.addAndGet(frame.getFrame().numRows()) >= limitToUse) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] rows",
-                    limitToUse
-                );
-              }
-              frameSignaturePairs.add(frame);
-            }
-        );
-        dataSource = new FramesBackedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query));
-      }
-      catch (ResourceLimitExceededException rlee) {
-        throw rlee;
-      }
-      catch (Exception e) {
-        log.info(
-            "Unable to write the subquery results to a frame. Results won't be accounted for in the memory "
-            + "calculation"
+        Optional<DataSource> maybeDataSource = materializeResultsAsFrames(
+            query,
+            results,
+            toolChest,
+            limitAccumulator,
+            memoryLimitAccumulator,
+            memoryLimit,
+            useNestedForUnknownTypeInSubquery
         );
-        throw e;
-      }
-    } else {
-      final RowSignature signature = toolChest.resultArraySignature(query);
-
-      final ArrayList<Object[]> resultList = new ArrayList<>();
-
-      toolChest.resultsAsArrays(query, results).accumulate(
-          resultList,
-          (acc, in) -> {
-            if (limitAccumulator.getAndIncrement() >= limitToUse) {
-              throw ResourceLimitExceededException.withMessage(
-                  "Subquery generated results beyond maximum[%d] rows",
-                  limitToUse
-              );
-            }
-            acc.add(in);
-            return acc;
+        if (!maybeDataSource.isPresent()) {
+          cannotMaterializeToFrames.set(true);
+          // Check if the previous row limit accumulator has exceeded the memory results
+          if (memoryLimitAccumulator.get() >= memoryLimit) {
+            throw ResourceLimitExceededException.withMessage(
+                "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+                memoryLimit
+            );
           }
-      );
-      dataSource = InlineDataSource.fromIterable(resultList, signature);
+          dataSource = materializeResultsAsArray(
+              query,
+              results,
+              toolChest,
+              limitAccumulator,
+              limit
+          );
+        } else {
+          dataSource = maybeDataSource.get();
+        }
+        break;
+      default:
+        throw new IAE("Only row based and memory based limiting is supported");
     }
     return dataSource;
   }
 
+  private static <T, QueryType extends Query<T>> Optional<DataSource> materializeResultsAsFrames(
+      final QueryType query,
+      final Sequence<T> results,
+      final QueryToolChest<T, QueryType> toolChest,
+      final AtomicInteger limitAccumulator,
+      final AtomicLong memoryLimitAccumulator,
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
+  )
+  {
+    Optional<Sequence<FrameSignaturePair>> framesOptional;
+
+    try {
+      framesOptional = toolChest.resultsAsFrames(
+          query,
+          results,
+          new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+          useNestedForUnknownTypeInSubquery
+      );
+    }
+    catch (Exception e) {
+      return Optional.empty();

Review Comment:
   Yeah. We shouldn't be doing 2nd option. We can do something like below
    - log the thing as Info. 
    - log the exception stack trace if debug is set in the query context. 



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe merged pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe merged PR #13952:
URL: https://github.com/apache/druid/pull/13952


-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1241583829


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -613,96 +638,149 @@ private static <T, QueryType extends Query<T>> DataSource toInlineDataSource(
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,
       final AtomicLong memoryLimitAccumulator,
+      final AtomicBoolean cannotMaterializeToFrames,
       final int limit,
-      long memoryLimit
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
   )
   {
-    final int limitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
-    boolean memoryLimitSet = memoryLimit >= 0;
-
-    if (limitAccumulator.get() >= limitToUse) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum[%d] reached",
-          limitToUse
-      );
-    }
-
-    if (memoryLimitSet && memoryLimitAccumulator.get() >= memoryLimit) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
-          memoryLimit
-      );
-    }
+    final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
 
     DataSource dataSource;
-    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
-    if (memoryLimitSet) {
-      try {
-        Optional<Sequence<FrameSignaturePair>> framesOptional = toolChest.resultsAsFrames(
+
+    switch (ClientQuerySegmentWalkerUtils.getLimitType(memoryLimit, cannotMaterializeToFrames.get())) {
+      case ROW_LIMIT:
+        if (limitAccumulator.get() >= rowLimitToUse) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] rows",
+              rowLimitToUse
+          );
+        }
+        dataSource = materializeResultsAsArray(
             query,
             results,
-            memoryLimit - memoryLimitAccumulator.get()
+            toolChest,
+            limitAccumulator,
+            limit
         );
-
-        if (!framesOptional.isPresent()) {
-          throw new ISE("The memory of the subqueries cannot be estimated correctly.");
+        break;
+      case MEMORY_LIMIT:
+        if (memoryLimitAccumulator.get() >= memoryLimit) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+              memoryLimit
+          );
         }
-
-        Sequence<FrameSignaturePair> frames = framesOptional.get();
-        List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
-        frames.forEach(
-            frame -> {
-              if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] bytes",
-                    memoryLimit
-                );
-
-              }
-              if (limitAccumulator.addAndGet(frame.getFrame().numRows()) >= limitToUse) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] rows",
-                    limitToUse
-                );
-              }
-              frameSignaturePairs.add(frame);
-            }
-        );
-        dataSource = new FramesBackedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query));
-      }
-      catch (ResourceLimitExceededException rlee) {
-        throw rlee;
-      }
-      catch (Exception e) {
-        log.info(
-            "Unable to write the subquery results to a frame. Results won't be accounted for in the memory "
-            + "calculation"
+        Optional<DataSource> maybeDataSource = materializeResultsAsFrames(
+            query,
+            results,
+            toolChest,
+            limitAccumulator,
+            memoryLimitAccumulator,
+            memoryLimit,
+            useNestedForUnknownTypeInSubquery
         );
-        throw e;
-      }
-    } else {
-      final RowSignature signature = toolChest.resultArraySignature(query);
-
-      final ArrayList<Object[]> resultList = new ArrayList<>();
-
-      toolChest.resultsAsArrays(query, results).accumulate(
-          resultList,
-          (acc, in) -> {
-            if (limitAccumulator.getAndIncrement() >= limitToUse) {
-              throw ResourceLimitExceededException.withMessage(
-                  "Subquery generated results beyond maximum[%d] rows",
-                  limitToUse
-              );
-            }
-            acc.add(in);
-            return acc;
+        if (!maybeDataSource.isPresent()) {
+          cannotMaterializeToFrames.set(true);
+          // Check if the previous row limit accumulator has exceeded the memory results
+          if (memoryLimitAccumulator.get() >= memoryLimit) {
+            throw ResourceLimitExceededException.withMessage(
+                "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+                memoryLimit
+            );
           }
-      );
-      dataSource = InlineDataSource.fromIterable(resultList, signature);
+          dataSource = materializeResultsAsArray(
+              query,
+              results,
+              toolChest,
+              limitAccumulator,
+              limit
+          );
+        } else {
+          dataSource = maybeDataSource.get();
+        }
+        break;
+      default:
+        throw new IAE("Only row based and memory based limiting is supported");
     }
     return dataSource;
   }
 
+  private static <T, QueryType extends Query<T>> Optional<DataSource> materializeResultsAsFrames(
+      final QueryType query,
+      final Sequence<T> results,
+      final QueryToolChest<T, QueryType> toolChest,
+      final AtomicInteger limitAccumulator,
+      final AtomicLong memoryLimitAccumulator,
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
+  )
+  {
+    Optional<Sequence<FrameSignaturePair>> framesOptional;
+
+    try {
+      framesOptional = toolChest.resultsAsFrames(
+          query,
+          results,
+          new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+          useNestedForUnknownTypeInSubquery
+      );
+    }
+    catch (Exception e) {
+      return Optional.empty();
+    }
+
+    if (!framesOptional.isPresent()) {
+      return Optional.empty();
+    }
+
+    Sequence<FrameSignaturePair> frames = framesOptional.get();
+    List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
+    frames.forEach(
+        frame -> {
+          limitAccumulator.addAndGet(frame.getFrame().numRows());
+          if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
+            throw ResourceLimitExceededException.withMessage(
+                "Subquery generated results beyond maximum[%d] bytes",

Review Comment:
   Updating with a more appropriate error message



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on PR #13952:
URL: https://github.com/apache/druid/pull/13952#issuecomment-1607389056

   Since this PR is liable to break due to merge conflicts, going ahead and merging this. 
   @LakshSingla Please address the logging feedback from @abhishekagarwal87 as part of a separate PR. 


-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1170874096


##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -217,77 +218,106 @@ public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
   )
   {
     final AtomicLong memoryLimitAccumulator = memoryLimitBytes != null ? new AtomicLong(memoryLimitBytes) : null;
-    Yielder<ScanResultValue> yielder = Yielders.each(resultSequence);
-    RowSignature prevSignature = null;
-    List<Cursor> unwrittenCursors = null;
-    List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
-    while (!yielder.isDone()) {
-      ScanResultValue scanResultValue = yielder.get();
 
-      final List rows = (List) scanResultValue.getEvents();
-      final Function<?, Object[]> mapper = getResultFormatMapper(query);
-      final Iterable<Object[]> formattedRows = Iterables.transform(rows, (Function) mapper);
+    Iterable<FrameSignaturePair> iterable = () -> new Iterator<FrameSignaturePair>()
+    {
+      ScanResultValue nextScanResultValue = null;
+      Yielder<ScanResultValue> yielder = Yielders.each(resultSequence);
+      boolean started = false;
+
+      @Override
+      public boolean hasNext()
+      {
+        return !yielder.isDone() || nextScanResultValue != null;

Review Comment:
   This looks to me a impl of peeking iterator. `com.google.common.collect.PeekingIterator`



##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -217,77 +218,106 @@ public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
   )
   {
     final AtomicLong memoryLimitAccumulator = memoryLimitBytes != null ? new AtomicLong(memoryLimitBytes) : null;
-    Yielder<ScanResultValue> yielder = Yielders.each(resultSequence);
-    RowSignature prevSignature = null;
-    List<Cursor> unwrittenCursors = null;
-    List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
-    while (!yielder.isDone()) {
-      ScanResultValue scanResultValue = yielder.get();
 
-      final List rows = (List) scanResultValue.getEvents();
-      final Function<?, Object[]> mapper = getResultFormatMapper(query);
-      final Iterable<Object[]> formattedRows = Iterables.transform(rows, (Function) mapper);
+    Iterable<FrameSignaturePair> iterable = () -> new Iterator<FrameSignaturePair>()
+    {
+      ScanResultValue nextScanResultValue = null;
+      Yielder<ScanResultValue> yielder = Yielders.each(resultSequence);
+      boolean started = false;
+
+      @Override
+      public boolean hasNext()
+      {
+        return !yielder.isDone() || nextScanResultValue != null;
+      }
 
-      if (prevSignature == null || !prevSignature.equals(scanResultValue.getRowSignature())) {
-
-        if (unwrittenCursors != null && prevSignature != null) {
-          FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
-              FrameType.ROW_BASED,
-              new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
-              prevSignature,
-              new ArrayList<>(),
-              true
-          );
-          Cursor concatCursor = new ConcatCursor(unwrittenCursors);
-          Frame frame = FrameCursorUtils.cursorToFrame(
-              concatCursor,
-              frameWriterFactory,
-              memoryLimitAccumulator != null ? memoryLimitAccumulator.get() : null
-          );
-          if (memoryLimitAccumulator != null) {
-            memoryLimitAccumulator.getAndAdd(-frame.numBytes());
+      @Override
+      public FrameSignaturePair next()
+      {
+        if (nextScanResultValue == null) {
+          if (!started) {
+            nextScanResultValue = yielder.get();
+            yielder = yielder.next(null);
+            started = true;
+            return next();
+          } else {
+            throw new NoSuchElementException();
+          }
+        }
+        List<ScanResultValue> batch = new ArrayList<>();
+        RowSignature signature = nextScanResultValue.getRowSignature();
+        batch.add(nextScanResultValue);
+        boolean updatedNextScanResultValue = false;
+        while (!yielder.isDone()) {
+          ScanResultValue potentiallyBatchableScanResultValue = yielder.get();
+          if (signature != null && signature.equals(potentiallyBatchableScanResultValue.getRowSignature())) {
+            batch.add(potentiallyBatchableScanResultValue);
+            yielder = yielder.next(null);
+          } else {
+            nextScanResultValue = potentiallyBatchableScanResultValue;
+            updatedNextScanResultValue = true;
+            yielder = yielder.next(null);
+            break;
           }
-          frameSignaturePairs.add(new FrameSignaturePair(frame, prevSignature));
         }
 
-        unwrittenCursors = new ArrayList<>();
-        unwrittenCursors.add(IterableRowsCursorHelper.getCursorFromIterable(
-            formattedRows,
-            scanResultValue.getRowSignature()
-        ));
+        // We are done iterating over the elements, and next call to hasNext() should return false
+        if (yielder.isDone() && !updatedNextScanResultValue) {
+          nextScanResultValue = null;
+        }
 
-      } else {
-        unwrittenCursors.add(IterableRowsCursorHelper.getCursorFromIterable(
-            formattedRows,
-            scanResultValue.getRowSignature()
-        ));
-      }
+        FrameSignaturePair retVal = convertScanResultValuesToFrame(batch, signature, query, memoryLimitAccumulator);
 
-      prevSignature = scanResultValue.getRowSignature();
-      yielder = yielder.next(null);
-    }
+        if (memoryLimitAccumulator != null) {

Review Comment:
   Nit: Please push memoryLimitAccumulator inside convertScanResultValuesToFrame



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1175931819


##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -187,11 +206,129 @@ public RowSignature resultArraySignature(final ScanQuery query)
     }
   }
 
+  /**
+   * This batches the fetched {@link ScanResultValue}s which have similar signatures and are consecutives. In best case
+   * it would return a single frame, and in the worst case, it would return as many frames as the number of {@link ScanResultValue}
+   * passed.
+   */
+  @Override
+  public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
+      final ScanQuery query,
+      final Sequence<ScanResultValue> resultSequence,
+      @Nullable Long memoryLimitBytes
+  )
+  {
+    final AtomicLong memoryLimitAccumulator = memoryLimitBytes != null ? new AtomicLong(memoryLimitBytes) : null;
+    Iterator<ScanResultValue> resultSequenceIterator = new Iterator<ScanResultValue>()
+    {
+      Yielder<ScanResultValue> yielder = Yielders.each(resultSequence);
+
+      @Override
+      public boolean hasNext()
+      {
+        return !yielder.isDone();
+      }
+
+      @Override
+      public ScanResultValue next()
+      {
+        ScanResultValue scanResultValue = yielder.get();
+        yielder = yielder.next(null);
+        return scanResultValue;
+      }
+    };
+
+    Iterable<FrameSignaturePair> retVal = () -> new Iterator<FrameSignaturePair>()
+    {
+      PeekingIterator<ScanResultValue> scanResultValuePeekingIterator = Iterators.peekingIterator(resultSequenceIterator);
+
+      @Override
+      public boolean hasNext()
+      {
+        return scanResultValuePeekingIterator.hasNext();
+      }
+
+      @Override
+      public FrameSignaturePair next()
+      {
+        final List<ScanResultValue> batch = new ArrayList<>();
+        final ScanResultValue scanResultValue = scanResultValuePeekingIterator.next();
+        batch.add(scanResultValue);
+        final RowSignature rowSignature = scanResultValue.getRowSignature();
+        while (scanResultValuePeekingIterator.hasNext()) {
+          final RowSignature nextRowSignature = scanResultValuePeekingIterator.peek().getRowSignature();
+          if (nextRowSignature != null && nextRowSignature.equals(rowSignature)) {
+            batch.add(scanResultValuePeekingIterator.next());
+          }

Review Comment:
   should there be an else here which breaks the while loop ?



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1168664165


##########
processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.query.scan;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.ListCursor;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ConcatCursorTest
+{
+  @Test
+  public void testCursor()
+  {
+    Cursor dummyCursor1 = new ListCursor(new ArrayList<>());

Review Comment:
   Test cases for
   * empty cursors
   * empty at end only 
   * empty at start
   * 2 empty cursors one after the other in all 3 places start , end , middle. 
   
   Same set of test cases for advanceUninterruptibly. 
   
   Maybe use parameterized tests. 



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on PR #13952:
URL: https://github.com/apache/druid/pull/13952#issuecomment-1607390459

   Thanks for the contribution @LakshSingla !!


-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1193402969


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -179,15 +190,19 @@ public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<In
     }
 
     // Now that we know the structure is workable, actually do the inlining (if necessary).
-    newQuery = newQuery.withDataSource(
-        inlineIfNecessary(
-            freeTradeDataSource,
-            toolChest,
-            new AtomicInteger(),
-            maxSubqueryRows,
-            false
-        )
+    AtomicLong memoryLimitAcc = new AtomicLong(0);
+    DataSource maybeInlinedDataSource = inlineIfNecessary(
+        freeTradeDataSource,
+        toolChest,
+        new AtomicInteger(),
+        memoryLimitAcc,
+        maxSubqueryRows,
+        maxSubqueryMemory,
+        false
     );
+    newQuery = newQuery.withDataSource(maybeInlinedDataSource);
+
+    log.info("Memory used by subqueries of query [%s] is [%d]", query, memoryLimitAcc.get());

Review Comment:
   Thanks for the pointer. Updated the logging level



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1184941668


##########
processing/src/main/java/org/apache/druid/frame/write/FrameWriters.java:
##########
@@ -53,24 +54,38 @@ private FrameWriters()
    * @param signature        signature of the frames
    * @param sortColumns      sort columns for the frames. If nonempty, {@link FrameSort#sort} is used to sort the
    *                         resulting frames.
+   * @param allowNullColumnTypes to allow null ColumnType in the signature. This should only be enabled when the user
+   *                             knows that the column objects exist as native Java POJOs (LinkedList, Maps etc), which
+   *                             can be serded using the Druid's nested columns
    */
   public static FrameWriterFactory makeFrameWriterFactory(
       final FrameType frameType,
       final MemoryAllocatorFactory allocatorFactory,
-      final RowSignature signature,
-      final List<KeyColumn> sortColumns
+      @Nonnull final RowSignature signature,

Review Comment:
   Thanks for the pointer! Will fix the clutter. 



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1142092017


##########
processing/src/main/java/org/apache/druid/query/InlineDataSource.java:
##########
@@ -300,7 +363,7 @@ public boolean equals(Object o)
       return false;
     }
     InlineDataSource that = (InlineDataSource) o;
-    return rowsEqual(rows, that.rows) &&

Review Comment:
   I'll check if this works since that would be faster.
   InlineDatasource's equality is done only in the tests afaik, therefore we should be fine if it doesn't work as such. 



-- 
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: commits-unsubscribe@druid.apache.org

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] github-code-scanning[bot] commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1141713394


##########
server/src/main/java/org/apache/druid/server/InlineResultsCursor.java:
##########
@@ -0,0 +1,425 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import org.apache.druid.frame.write.UnsupportedColumnTypeException;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.query.BaseQuery;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.filter.ValueMatcher;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.DimensionDictionarySelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.DoubleColumnSelector;
+import org.apache.druid.segment.FloatColumnSelector;
+import org.apache.druid.segment.IdLookup;
+import org.apache.druid.segment.LongColumnSelector;
+import org.apache.druid.segment.ObjectColumnSelector;
+import org.apache.druid.segment.SimpleAscendingOffset;
+import org.apache.druid.segment.SimpleSettableOffset;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.data.IndexedInts;
+import org.apache.druid.segment.data.Offset;
+import org.apache.druid.segment.data.RangeIndexedInts;
+import org.apache.druid.segment.serde.ComplexMetrics;
+import org.joda.time.DateTime;
+
+import javax.annotation.Nullable;
+import java.util.List;
+import java.util.Optional;
+
+public class InlineResultsCursor implements Cursor
+{
+
+  private final SimpleSettableOffset offset;
+  private final RowSignature rowSignature;
+  private final List<Object[]> results;
+
+
+  public InlineResultsCursor(List<Object[]> results, RowSignature rowSignature)
+  {
+    this.results = Preconditions.checkNotNull(results, "'results' cannot be null");
+    this.rowSignature = rowSignature;
+    this.offset = new SimpleAscendingOffset(results.size());
+  }
+
+  @Override
+  public ColumnSelectorFactory getColumnSelectorFactory()
+  {
+    return new ColumnSelectorFactory()
+    {
+      @Override
+      public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
+      {
+        return makeDimensionSelectorFor(dimensionSpec);
+      }
+
+      @Override
+      public ColumnValueSelector makeColumnValueSelector(String columnName)
+      {
+        int index = offset.getOffset();
+        return makeColumnValueSelectorFor(columnName, index);
+      }
+
+      @Nullable
+      @Override
+      public ColumnCapabilities getColumnCapabilities(String column)
+      {
+        return rowSignature.getColumnCapabilities(column);
+      }
+    };
+  }
+
+  @Override
+  public DateTime getTime()
+  {
+    return DateTimes.MIN;
+  }
+
+  @Override
+  public void advance()
+  {
+    offset.increment();
+    BaseQuery.checkInterrupted();
+  }
+
+  @Override
+  public void advanceUninterruptibly()
+  {
+    offset.increment();
+  }
+
+  @Override
+  public boolean isDone()
+  {
+    return !offset.withinBounds();
+  }
+
+  @Override
+  public boolean isDoneOrInterrupted()
+  {
+    return !offset.withinBounds() || Thread.currentThread().isInterrupted();
+  }
+
+  @Override
+  public void reset()
+  {
+    offset.reset();
+  }
+
+  private DimensionSelector makeDimensionSelectorFor(DimensionSpec dimensionSpec)
+  {
+    return dimensionSpec.decorate(makeUndecoratedDimensionSelectorFor(dimensionSpec.getDimension()));
+  }
+
+  private DimensionSelector makeUndecoratedDimensionSelectorFor(String dimensionSpec)
+  {
+    return new DimensionSelector()
+    {
+      @Override
+      public IndexedInts getRow()
+      {
+        Object stringValueObject = results.get(offset.getOffset())[rowSignature.indexOf(dimensionSpec)];
+        RangeIndexedInts rangeIndexedInts = new RangeIndexedInts();
+        if (stringValueObject instanceof String) {
+          rangeIndexedInts.setSize(1);
+        } else if (stringValueObject instanceof List) {
+          rangeIndexedInts.setSize(((List<?>) stringValueObject).size());
+        }
+        return rangeIndexedInts;
+      }
+
+      @Override
+      public ValueMatcher makeValueMatcher(@Nullable String value)
+      {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public ValueMatcher makeValueMatcher(Predicate<String> predicate)
+      {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+      {
+
+      }
+
+      @Nullable
+      @Override
+      public Object getObject()
+      {
+        Object stringValueObject = results.get(offset.getOffset())[rowSignature.indexOf(dimensionSpec)];
+        return stringValueObject;
+      }
+
+      @Override
+      public Class<?> classOfObject()
+      {
+        Object stringValueObject = results.get(offset.getOffset())[rowSignature.indexOf(dimensionSpec)];
+        if (stringValueObject instanceof String) {
+          return String.class;
+        } else if (stringValueObject instanceof List) {
+          return List.class;
+        } else {
+          return Object.class;
+        }
+      }
+
+      @Override
+      public int getValueCardinality()
+      {
+        return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
+      }
+
+      @Nullable
+      @Override
+      public String lookupName(int id)
+      {
+        Object stringValueObject = results.get(offset.getOffset())[rowSignature.indexOf(dimensionSpec)];
+        if (stringValueObject instanceof String) {
+          if (id == 0) {
+            return (String) stringValueObject;
+          } else {
+            throw new IndexOutOfBoundsException();
+          }
+        } else if (stringValueObject instanceof List) {
+          List<?> stringValueList = (List<?>) stringValueObject;
+          if (id < stringValueList.size()) {
+            return (String) stringValueList.get(id);
+          } else {
+            throw new IndexOutOfBoundsException();
+          }
+        } else {
+          throw new UnsupportedOperationException();
+        }
+      }
+
+      @Override
+      public boolean nameLookupPossibleInAdvance()
+      {
+        return false;
+      }
+
+      @Nullable
+      @Override
+      public IdLookup idLookup()
+      {
+        return null;
+      }
+    };
+  }
+
+  private ColumnValueSelector makeColumnValueSelectorFor(String columnName, int index)

Review Comment:
   ## Useless parameter
   
   The parameter 'index' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4410)



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1142035052


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +632,71 @@ private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSour
       );
     }
 
+    if (memoryLimitAccumulator.get() >= memoryLimitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
+          memoryLimitToUse
+      );
+    }
+
     final RowSignature signature = toolChest.resultArraySignature(query);
 
-    final ArrayList<Object[]> resultList = new ArrayList<>();
+    final List<Object[]> resultList = toolChest.resultsAsArrays(query, results).toList();
 
-    toolChest.resultsAsArrays(query, results).accumulate(
-        resultList,
-        (acc, in) -> {
-          if (limitAccumulator.getAndIncrement() >= limitToUse) {
-            throw ResourceLimitExceededException.withMessage(
-                "Subquery generated results beyond maximum[%d]",
-                limitToUse
-            );
+    if (limitAccumulator.addAndGet(resultList.size()) >= limitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Subquery generated results beyond maximum[%d] rows",
+          limitToUse
+      );
+    }
+
+    Frame frame = null;
+
+    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
+    if (memoryLimitSet) {

Review Comment:
   We should only serialize the frames only when memoryLimit Is set else the old code path should be invoked.  
   



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1241583406


##########
processing/src/main/java/org/apache/druid/frame/segment/FrameCursorUtils.java:
##########
@@ -67,4 +76,58 @@ public static Filter buildFilter(@Nullable Filter filter, Interval interval)
       );
     }
   }
+
+  /**
+   * Writes a {@link Cursor} to a sequence of {@link Frame}. This method iterates over the rows of the cursor,
+   * and writes the columns to the frames
+   *
+   * @param cursor                 Cursor to write to the frame
+   * @param frameWriterFactory     Frame writer factory to write to the frame.
+   *                               Determines the signature of the rows that are written to the frames
+   */
+  public static Sequence<Frame> cursorToFrames(
+      Cursor cursor,
+      FrameWriterFactory frameWriterFactory
+  )
+  {
+
+    return Sequences.simple(
+        () -> new Iterator<Frame>()
+        {
+          @Override
+          public boolean hasNext()
+          {
+            return !cursor.isDone();
+          }
+
+          @Override
+          public Frame next()
+          {
+            // Makes sure that cursor contains some elements prior. This ensures if no row is written, then the row size
+            // is larger than the MemoryAllocators returned by the provided factory
+            if (!hasNext()) {
+              throw new NoSuchElementException();
+            }
+            boolean firstRowWritten = false;
+            Frame frame;
+            try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) {
+              while (!cursor.isDone()) {
+                if (!frameWriter.addSelection()) {
+                  break;
+                }
+                firstRowWritten = true;
+                cursor.advance();
+              }
+
+              if (!firstRowWritten) {
+                throw new ISE("Row size is greater than the frame size.");

Review Comment:
   Yes this can be seen at the top level. Refactored with the DruidException and a more actionable error message 



-- 
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: commits-unsubscribe@druid.apache.org

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] gianm commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "gianm (via GitHub)" <gi...@apache.org>.
gianm commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1182869188


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +627,80 @@ private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSour
       );
     }
 
-    final RowSignature signature = toolChest.resultArraySignature(query);
+    if (memoryLimitSet && memoryLimitAccumulator.get() >= memoryLimit) {
+      throw ResourceLimitExceededException.withMessage(
+          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
+          memoryLimit
+      );
+    }
 
-    final ArrayList<Object[]> resultList = new ArrayList<>();
+    DataSource dataSource;
+    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
+    if (memoryLimitSet) {
+      try {
+        Optional<Sequence<FrameSignaturePair>> framesOptional = toolChest.resultsAsFrames(
+            query,
+            results,
+            memoryLimit - memoryLimitAccumulator.get()
+        );
 
-    toolChest.resultsAsArrays(query, results).accumulate(
-        resultList,
-        (acc, in) -> {
-          if (limitAccumulator.getAndIncrement() >= limitToUse) {
-            throw ResourceLimitExceededException.withMessage(
-                "Subquery generated results beyond maximum[%d]",
-                limitToUse
-            );
-          }
-          acc.add(in);
-          return acc;
+        if (!framesOptional.isPresent()) {
+          throw new ISE("The memory of the subqueries cannot be estimated correctly.");
         }
-    );
 
-    return InlineDataSource.fromIterable(resultList, signature);
+        Sequence<FrameSignaturePair> frames = framesOptional.get();
+        List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
+        frames.forEach(
+            frame -> {
+              if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
+                throw ResourceLimitExceededException.withMessage(
+                    "Subquery generated results beyond maximum[%d] bytes",
+                    memoryLimit
+                );
+
+              }
+              if (limitAccumulator.addAndGet(frame.getFrame().numRows()) >= limitToUse) {
+                throw ResourceLimitExceededException.withMessage(
+                    "Subquery generated results beyond maximum[%d] rows",
+                    limitToUse
+                );
+              }
+              frameSignaturePairs.add(frame);
+            }
+        );
+        dataSource = new FramesBackedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query));
+      }
+      catch (ResourceLimitExceededException rlee) {
+        throw rlee;
+      }
+      catch (Exception e) {
+        log.info(

Review Comment:
   In this `catch` case, the whole query fails, right? Are there known situations where it would happen?
   
   IMO we should be doing one of two things:
   
   1. If we believe this can never happen, we don't need a `catch` block at all. Just let the exception out and fail the query. Let's then also make sure we have enough testing to be confident it doesn't happen.
   2. If there's known situations where this would happen, we should detect them and avoid going down the frame path at all. If that's not feasible, then we should put in some logic here to recover the query rather than failing it.
   
   What're your thoughts?
   
   Either way, there's no reason to log anything, let's not clutter the logs. The standard exception handling is enough if the query is going to fail. And if it's not going to fail, we should recover quietly.



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -560,15 +607,18 @@ private DataSource insertSubqueryIds(
    *                         If zero, this method will throw an error immediately.
    * @throws ResourceLimitExceededException if the limit is exceeded
    */
-  private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSource(
+  private static <T, QueryType extends Query<T>> DataSource toInlineDataSource(
       final QueryType query,
       final Sequence<T> results,
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,
-      final int limit
+      final AtomicLong memoryLimitAccumulator,
+      final int limit,
+      long memoryLimit
   )
   {
     final int limitToUse = limit < 0 ? Integer.MAX_VALUE : limit;

Review Comment:
   Rename this `rowLimitToUse`, for clarity.



##########
processing/src/main/java/org/apache/druid/frame/write/FrameWriters.java:
##########
@@ -53,24 +54,38 @@ private FrameWriters()
    * @param signature        signature of the frames
    * @param sortColumns      sort columns for the frames. If nonempty, {@link FrameSort#sort} is used to sort the
    *                         resulting frames.
+   * @param allowNullColumnTypes to allow null ColumnType in the signature. This should only be enabled when the user
+   *                             knows that the column objects exist as native Java POJOs (LinkedList, Maps etc), which
+   *                             can be serded using the Druid's nested columns
    */
   public static FrameWriterFactory makeFrameWriterFactory(
       final FrameType frameType,
       final MemoryAllocatorFactory allocatorFactory,
-      final RowSignature signature,
-      final List<KeyColumn> sortColumns
+      @Nonnull final RowSignature signature,

Review Comment:
   Use `@Nullable` where appropriate, but avoid `@Nonnull` on parameters in general, since it creates an implication that unannotated things _can_ be null. But, they mostly can't, so it's misleading.
   
   If you're looking for benefit for static analyzers, better to use `@EverythingIsNonnullByDefault` at package level through `package-info.java`. We have that on various packages already and could add it to more as desired. That's like annotating every un-annotated parameter with `@Nonnull`, without the clutter.



##########
processing/src/main/java/org/apache/druid/query/QueryToolChest.java:
##########
@@ -330,4 +331,32 @@ public Sequence<Object[]> resultsAsArrays(QueryType query, Sequence<ResultType>
   {
     throw new UOE("Query type '%s' does not support returning results as arrays", query.getType());
   }
+
+  /**
+   * Converts a sequence of this query's ResultType into a sequence of {@link FrameSignaturePair}. The array signature
+   * is the one give by {@link #resultArraySignature(Query)}. If the toolchest doesn't support this method, then it can
+   * return an empty optional. It is the duty of the callees to throw an appropriate exception in that case or use an
+   * alternative fallback approach
+   *
+   * Check documentation of {@link #resultsAsArrays(Query, Sequence)} as the behaviour of the rows represented by the
+   * frame sequence is identical.
+   *
+   * Each Frame has a separate {@link RowSignature} because for some query types like the Scan query, every

Review Comment:
   This seems to contradict the statement earlier that "the array signature is the one given by `resultArraySignature`". Sounds like it could actually be different. In that case please update the earlier text accordingly.



##########
server/src/main/java/org/apache/druid/segment/FrameBasedInlineSegmentWrangler.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.segment;
+
+import org.apache.druid.frame.read.FrameReader;
+import org.apache.druid.frame.segment.FrameSegment;
+import org.apache.druid.query.DataSource;
+import org.apache.druid.query.FramesBackedInlineDataSource;
+import org.apache.druid.timeline.SegmentId;
+import org.joda.time.Interval;
+
+import java.util.stream.Collectors;
+
+public class FrameBasedInlineSegmentWrangler implements SegmentWrangler
+{
+
+  private static final String SEGMENT_ID = "inline";
+
+  @Override
+  public Iterable<Segment> getSegmentsForIntervals(
+      DataSource dataSource,
+      Iterable<Interval> intervals
+  )
+  {
+    final FramesBackedInlineDataSource framesBackedInlineDataSource = (FramesBackedInlineDataSource) dataSource;
+
+    return framesBackedInlineDataSource.getFrames().stream().map(

Review Comment:
   Better to avoid materializing the list: return an `Iterable` based on `.iterator()` instead of `.collect(Collectors.toList())`.



##########
processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java:
##########
@@ -705,6 +718,39 @@ public Sequence<Object[]> resultsAsArrays(final GroupByQuery query, final Sequen
     return resultSequence.map(ResultRow::getArray);
   }
 
+  /**
+   * This returns a single frame containing the results of the group by query.
+   */
+  @Override
+  public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
+      GroupByQuery query,
+      Sequence<ResultRow> resultSequence,
+      @Nullable Long memoryLimitBytes
+  )
+  {
+    RowSignature rowSignature = resultArraySignature(query);
+
+    FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
+        FrameType.ROW_BASED,
+        new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+        rowSignature,
+        new ArrayList<>(),
+        true
+    );
+
+
+    Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(
+        resultSequence.map(ResultRow::getArray),
+        rowSignature
+    );
+
+    Frame frame = FrameCursorUtils.cursorToFrame(cursor, frameWriterFactory, memoryLimitBytes);

Review Comment:
   With the changes to `resultsAsFrames`, this would start returning a sequence of multiple smaller frames, and wouldn't take `memoryLimitBytes`.



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -179,15 +190,19 @@ public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<In
     }
 
     // Now that we know the structure is workable, actually do the inlining (if necessary).
-    newQuery = newQuery.withDataSource(
-        inlineIfNecessary(
-            freeTradeDataSource,
-            toolChest,
-            new AtomicInteger(),
-            maxSubqueryRows,
-            false
-        )
+    AtomicLong memoryLimitAcc = new AtomicLong(0);
+    DataSource maybeInlinedDataSource = inlineIfNecessary(
+        freeTradeDataSource,
+        toolChest,
+        new AtomicInteger(),
+        memoryLimitAcc,
+        maxSubqueryRows,
+        maxSubqueryMemory,
+        false
     );
+    newQuery = newQuery.withDataSource(maybeInlinedDataSource);
+
+    log.info("Memory used by subqueries of query [%s] is [%d]", query, memoryLimitAcc.get());

Review Comment:
   We shouldn't have any `INFO` level logs that happen on a per-query basis. It generates too many log messages. Please reduce this to `DEBUG` level.



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -313,7 +328,9 @@ private DataSource inlineIfNecessary(
       final DataSource dataSource,
       @Nullable final QueryToolChest toolChestIfOutermost,
       final AtomicInteger subqueryRowLimitAccumulator,
+      final AtomicLong subqueryRowMemoryLimitAccumulator,

Review Comment:
   `subqueryMemoryLimitAccumulator` is a better name.



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +627,80 @@ private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSour
       );
     }
 
-    final RowSignature signature = toolChest.resultArraySignature(query);
+    if (memoryLimitSet && memoryLimitAccumulator.get() >= memoryLimit) {
+      throw ResourceLimitExceededException.withMessage(

Review Comment:
   Update the other message to refer to the row limit, for clarity.



##########
server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java:
##########
@@ -348,7 +350,7 @@ public void testTimeseriesOnGroupByOnTable()
 
     testQuery(
         query,
-        ImmutableList.of(
+        new ArrayList<>(ImmutableList.of(

Review Comment:
   Is there a reason this needs to be a mutable array?



##########
processing/src/main/java/org/apache/druid/query/QueryToolChest.java:
##########
@@ -330,4 +331,32 @@ public Sequence<Object[]> resultsAsArrays(QueryType query, Sequence<ResultType>
   {
     throw new UOE("Query type '%s' does not support returning results as arrays", query.getType());
   }
+
+  /**
+   * Converts a sequence of this query's ResultType into a sequence of {@link FrameSignaturePair}. The array signature
+   * is the one give by {@link #resultArraySignature(Query)}. If the toolchest doesn't support this method, then it can
+   * return an empty optional. It is the duty of the callees to throw an appropriate exception in that case or use an
+   * alternative fallback approach
+   *
+   * Check documentation of {@link #resultsAsArrays(Query, Sequence)} as the behaviour of the rows represented by the
+   * frame sequence is identical.
+   *
+   * Each Frame has a separate {@link RowSignature} because for some query types like the Scan query, every
+   * column in the final result might not be present in the individual ResultType (and subsequently Frame). Therefore,
+   * this is done to preserve the space by not populating the column in that particular Frame and omitting it from its
+   * signature
+   * 
+   * @param query Query being executed by the toolchest. Used to determine the rowSignature of the Frames
+   * @param resultSequence results of the form returned by {@link #mergeResults(QueryRunner)}
+   * @param memoryLimitBytes Limit the memory results. Throws {@link ResourceLimitExceededException} if the result exceed
+   *                    the memoryLimitBytes
+   */
+  public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(

Review Comment:
   Rather than accepting `memoryLimitBytes` and returning a jumbo frame, it'd be better to return a sequence of moderately-sized frames and have the caller decide when they've had too many frames. This makes the method simpler, easier to test/understand, and more flexible for potential future use cases. In particular I suggest the following changes:
   
   - Remove the `memoryLimitBytes` parameter.
   - Add a `MemoryAllocatorFactory` parameter so the caller can control the size of each frame.



##########
server/src/main/java/org/apache/druid/segment/FrameBasedInlineSegmentWrangler.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.segment;
+
+import org.apache.druid.frame.read.FrameReader;
+import org.apache.druid.frame.segment.FrameSegment;
+import org.apache.druid.query.DataSource;
+import org.apache.druid.query.FramesBackedInlineDataSource;
+import org.apache.druid.timeline.SegmentId;
+import org.joda.time.Interval;
+
+import java.util.stream.Collectors;
+
+public class FrameBasedInlineSegmentWrangler implements SegmentWrangler

Review Comment:
   Some of the classes are `FrameBased` and some are `FramesBacked`. Let's keep it consistent and use `FrameBased` for all of them.



##########
processing/src/main/java/org/apache/druid/query/QueryContexts.java:
##########
@@ -54,6 +54,7 @@
   public static final String VECTORIZE_VIRTUAL_COLUMNS_KEY = "vectorizeVirtualColumns";
   public static final String VECTOR_SIZE_KEY = "vectorSize";
   public static final String MAX_SUBQUERY_ROWS_KEY = "maxSubqueryRows";
+  public static final String MAX_SUBQUERY_MEMORY_BYTES_KEY = "maxSubqueryMemoryBytes";

Review Comment:
   `maxSubqueryBytes` is better, IMO, since it's just as clear and a bit more succinct.



##########
processing/src/main/java/org/apache/druid/frame/write/FrameWriters.java:
##########
@@ -53,24 +54,38 @@ private FrameWriters()
    * @param signature        signature of the frames
    * @param sortColumns      sort columns for the frames. If nonempty, {@link FrameSort#sort} is used to sort the
    *                         resulting frames.
+   * @param allowNullColumnTypes to allow null ColumnType in the signature. This should only be enabled when the user
+   *                             knows that the column objects exist as native Java POJOs (LinkedList, Maps etc), which
+   *                             can be serded using the Druid's nested columns
    */
   public static FrameWriterFactory makeFrameWriterFactory(

Review Comment:
   IMO adding `allowNullColumnTypes` as a parameter here isn't the right approach. Better for the caller to replace all the unknown types (`null`) in `signature` with `COMPLEX<json>`, if that's what's desired. It keeps the frame writer code simpler, easier to test/understand, etc.



##########
processing/src/main/java/org/apache/druid/segment/join/FramesBackedInlineJoinableFactory.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.segment.join;
+
+import org.apache.druid.query.DataSource;
+import org.apache.druid.query.FramesBackedInlineDataSource;
+import org.apache.druid.query.InlineDataSource;
+
+import java.util.Optional;
+
+/**
+ * Creates a joinable from the {@link FramesBackedInlineDataSource}. This materializes the datasource to an
+ * {@link InlineDataSource}, before creating the joinable on it, which carries the overhead of this conversion.
+ */
+public class FramesBackedInlineJoinableFactory implements JoinableFactory
+{
+  private final InlineJoinableFactory INLINE_JOINABLE_FACTORY = new InlineJoinableFactory();
+
+
+  @Override
+  public boolean isDirectlyJoinable(DataSource dataSource)
+  {
+    return dataSource instanceof FramesBackedInlineDataSource;
+  }
+
+  @Override
+  public Optional<Joinable> build(DataSource dataSource, JoinConditionAnalysis condition)
+  {
+    FramesBackedInlineDataSource framesBackedInlineDataSource = (FramesBackedInlineDataSource) dataSource;
+    InlineDataSource inlineDataSource = framesBackedInlineDataSource.toInlineDataSource();

Review Comment:
   Can we create a Joinable directly on the frames? Materialization to inline is going to be expensive and likely 2–3x memory usage as well. I don't think that is going to be viable.



##########
processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java:
##########
@@ -52,6 +54,8 @@ public DruidDefaultSerializersModule()
 
     JodaStuff.register(this);
 
+    addSerializer(FramesBackedInlineDataSource.class, new FramesBackedInlineDataSourceSerializer());

Review Comment:
   I would think we don't need to serialize this, as it should exist in-memory only. So I'm also wondering where this was needed.



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on PR #13952:
URL: https://github.com/apache/druid/pull/13952#issuecomment-1561537779

   Thanks for the reviews!
   Post the latest batch of changes, here are the major things that I have updated:
   1. Create a `FrameBasedIndexedTable` to create an indexed table based on Frame datasource. This also changed the Frame type to COLUMNAR since to index the table we need to get COLUMNs corresponding to the indexed keys. 
   2. Use a memory allocator factory to allow flexibility in converting the inline results to the frames
   3. Added configurations, need to provide the fallback code paths as well.
   4. Refactored the code and stylistic changes


-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1193428989


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +627,80 @@ private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSour
       );
     }
 
-    final RowSignature signature = toolChest.resultArraySignature(query);
+    if (memoryLimitSet && memoryLimitAccumulator.get() >= memoryLimit) {
+      throw ResourceLimitExceededException.withMessage(
+          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
+          memoryLimit
+      );
+    }
 
-    final ArrayList<Object[]> resultList = new ArrayList<>();
+    DataSource dataSource;
+    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
+    if (memoryLimitSet) {
+      try {
+        Optional<Sequence<FrameSignaturePair>> framesOptional = toolChest.resultsAsFrames(
+            query,
+            results,
+            memoryLimit - memoryLimitAccumulator.get()
+        );
 
-    toolChest.resultsAsArrays(query, results).accumulate(
-        resultList,
-        (acc, in) -> {
-          if (limitAccumulator.getAndIncrement() >= limitToUse) {
-            throw ResourceLimitExceededException.withMessage(
-                "Subquery generated results beyond maximum[%d]",
-                limitToUse
-            );
-          }
-          acc.add(in);
-          return acc;
+        if (!framesOptional.isPresent()) {
+          throw new ISE("The memory of the subqueries cannot be estimated correctly.");
         }
-    );
 
-    return InlineDataSource.fromIterable(resultList, signature);
+        Sequence<FrameSignaturePair> frames = framesOptional.get();
+        List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
+        frames.forEach(
+            frame -> {
+              if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
+                throw ResourceLimitExceededException.withMessage(
+                    "Subquery generated results beyond maximum[%d] bytes",
+                    memoryLimit
+                );
+
+              }
+              if (limitAccumulator.addAndGet(frame.getFrame().numRows()) >= limitToUse) {
+                throw ResourceLimitExceededException.withMessage(
+                    "Subquery generated results beyond maximum[%d] rows",
+                    limitToUse
+                );
+              }
+              frameSignaturePairs.add(frame);
+            }
+        );
+        dataSource = new FramesBackedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query));
+      }
+      catch (ResourceLimitExceededException rlee) {
+        throw rlee;
+      }
+      catch (Exception e) {
+        log.info(

Review Comment:
   This seems reasonable to me. I originally added this because of the fact that columns with null types might exist and this seemed like a simpler and lazier way to handle them. With the addition of the user flag `useNestedForUnknownTypesInSubquery` mentioned  in the [top-level comment
   ](https://github.com/apache/druid/pull/13952#pullrequestreview-1409574208)I the error handling can seem a lot less convoluted since the user/dev has control of the flag and what to do with it. 
   Also, the default `useNestedForUnknownTypesInSubquery` = false would make it so that we are not breaking what exists currently, just limiting the queries for which we can limit the subquery by bytes till the flag is set to `true`. 



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1159338874


##########
processing/src/main/java/org/apache/druid/frame/write/FrameWriters.java:
##########
@@ -58,19 +58,30 @@ public static FrameWriterFactory makeFrameWriterFactory(
       final FrameType frameType,
       final MemoryAllocatorFactory allocatorFactory,
       final RowSignature signature,
-      final List<KeyColumn> sortColumns
+      final List<KeyColumn> sortColumns,
+      final boolean allowNullColumnTypes
   )
   {
     switch (Preconditions.checkNotNull(frameType, "frameType")) {
       case COLUMNAR:
-        return new ColumnarFrameWriterFactory(allocatorFactory, signature, sortColumns);
+        return new ColumnarFrameWriterFactory(allocatorFactory, signature, sortColumns, allowNullColumnTypes);
       case ROW_BASED:
-        return new RowBasedFrameWriterFactory(allocatorFactory, signature, sortColumns);
+        return new RowBasedFrameWriterFactory(allocatorFactory, signature, sortColumns, allowNullColumnTypes);
       default:
         throw new ISE("Unrecognized frame type [%s]", frameType);
     }
   }
 
+  public static FrameWriterFactory makeFrameWriterFactory(

Review Comment:
   Added this as a separate method originally because we don't require the boolean in the rest of the cases (i.e. the original ones in MSQ). Therefore it made sense to me to hide this complexity from the callers of this method that arent residing in the broker. 



-- 
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: commits-unsubscribe@druid.apache.org

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] github-code-scanning[bot] commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1142014035


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +632,71 @@
       );
     }
 
+    if (memoryLimitAccumulator.get() >= memoryLimitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
+          memoryLimitToUse
+      );
+    }
+
     final RowSignature signature = toolChest.resultArraySignature(query);
 
-    final ArrayList<Object[]> resultList = new ArrayList<>();
+    final List<Object[]> resultList = toolChest.resultsAsArrays(query, results).toList();
 
-    toolChest.resultsAsArrays(query, results).accumulate(
-        resultList,
-        (acc, in) -> {
-          if (limitAccumulator.getAndIncrement() >= limitToUse) {
-            throw ResourceLimitExceededException.withMessage(
-                "Subquery generated results beyond maximum[%d]",
-                limitToUse
-            );
+    if (limitAccumulator.addAndGet(resultList.size()) >= limitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Subquery generated results beyond maximum[%d] rows",
+          limitToUse
+      );
+    }
+
+    Frame frame = null;
+
+    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
+    if (memoryLimitSet) {
+      try {
+        FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
+            FrameType.ROW_BASED,
+            new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+            signature,
+            new ArrayList<>()
+        );
+
+        final Cursor cursor = new InlineResultsCursor(resultList, signature);
+
+        try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) {
+          while (!cursor.isDone()) {
+            if (!frameWriter.addSelection()) {
+              throw new FrameRowTooLargeException(frameWriterFactory.allocatorCapacity());
+            }
+
+            cursor.advance();
           }
-          acc.add(in);
-          return acc;
+
+          frame = Frame.wrap(frameWriter.toByteArray());
         }
-    );
 
-    return InlineDataSource.fromIterable(resultList, signature);
+
+        if (memoryLimitAccumulator.addAndGet(frame.numBytes()) >= memoryLimitToUse) {
+          throw ResourceLimitExceededException.withMessage(
+              "Subquery generated results beyond maximum[%d] bytes",
+              memoryLimit
+          );
+        }
+      }
+      catch (ResourceLimitExceededException rlee) {
+        throw rlee;
+      }
+      catch (Exception e) {
+        log.info("Unable to write the subquery results to a frame. Results won't be accounted for in the memory"
+                 + "calculation");

Review Comment:
   ## Missing space in string literal
   
   This string appears to be missing a space after 'memory'.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4411)



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1142025452


##########
processing/src/main/java/org/apache/druid/query/InlineDataSource.java:
##########
@@ -53,14 +63,17 @@
 public class InlineDataSource implements DataSource
 {
   private final Iterable<Object[]> rows;
+  private final Frame frame;

Review Comment:
   Lets update the documentation of this class mentioning it can be backed by frames. 



##########
processing/src/main/java/org/apache/druid/query/InlineDataSource.java:
##########
@@ -190,15 +211,57 @@ public List<ColumnType> getColumnTypes()
   @JsonProperty("rows")
   public List<Object[]> getRowsAsList()
   {
+    Iterable<Object[]> rows = getRows();
     return rows instanceof List ? ((List<Object[]>) rows) : Lists.newArrayList(rows);
   }
 
+  public Frame getBackingFrame()
+  {
+    return frame;
+  }
+
+  public boolean isBackedByFrame()
+  {
+    return frame != null;
+  }
+
   /**
    * Returns rows as an Iterable.
    */
   @JsonIgnore
   public Iterable<Object[]> getRows()
   {
+    if (isBackedByFrame()) {
+      List<Object[]> frameRows = new ArrayList<>();
+      FrameReader frameReader = FrameReader.create(signature);
+      final Sequence<Cursor> cursorSequence = new FrameStorageAdapter(
+          frame,
+          frameReader,
+          Intervals.ETERNITY
+      ).makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null);
+      cursorSequence.accumulate(
+          null,
+          (accumulated, cursor) -> {
+            final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
+            final List<BaseObjectColumnValueSelector> selectors = frameReader.signature()
+                                                                             .getColumnNames()
+                                                                             .stream()
+                                                                             .map(columnSelectorFactory::makeColumnValueSelector)
+                                                                             .collect(Collectors.toList());
+            while (!cursor.isDone()) {
+              Object[] row = new Object[signature.size()];
+              for (int i = 0; i < signature.size(); ++i) {
+                row[i] = selectors.get(i).getObject();
+              }
+              frameRows.add(row);
+              cursor.advance();
+            }
+            return null;
+          }
+      );
+      return frameRows;
+    }
+
     return rows;

Review Comment:
   Nit: Lets add this in the else part ?
   



##########
server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java:
##########
@@ -798,7 +798,36 @@ public void testTimeseriesOnGroupByOnTableErrorTooManyRows()
                                 .withId(DUMMY_QUERY_ID);
 
     expectedException.expect(ResourceLimitExceededException.class);
-    expectedException.expectMessage("Subquery generated results beyond maximum[2]");
+    expectedException.expectMessage("Subquery generated results beyond maximum[2] rows");
+
+    testQuery(query, ImmutableList.of(), ImmutableList.of());
+  }
+
+
+  @Test
+  public void testTimeseriesOnGroupByOnTableErrorTooLarge()

Review Comment:
   I think tests around
   1. String cols
   2. Long cols
   3. Double/float cols
   4. Complex cols
   5. Array cols
   6. Nested cols 
   Would help us build confidence in the feature. 
   
   Some tests to checkout: 
   * `CalciteQueryTests#testMaxSubqueryRows`
   * `GroupByQueryRunnerTest#testGroupByMaxRowsLimitContextOverride`
   
   Ideally all tests which have a subquery should be executed using the new code path but since its feature flagged it might not be a hard requirement. 



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +632,71 @@ private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSour
       );
     }
 
+    if (memoryLimitAccumulator.get() >= memoryLimitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
+          memoryLimitToUse
+      );
+    }
+
     final RowSignature signature = toolChest.resultArraySignature(query);
 
-    final ArrayList<Object[]> resultList = new ArrayList<>();
+    final List<Object[]> resultList = toolChest.resultsAsArrays(query, results).toList();
 
-    toolChest.resultsAsArrays(query, results).accumulate(
-        resultList,
-        (acc, in) -> {
-          if (limitAccumulator.getAndIncrement() >= limitToUse) {
-            throw ResourceLimitExceededException.withMessage(
-                "Subquery generated results beyond maximum[%d]",
-                limitToUse
-            );
+    if (limitAccumulator.addAndGet(resultList.size()) >= limitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Subquery generated results beyond maximum[%d] rows",
+          limitToUse
+      );
+    }
+
+    Frame frame = null;
+
+    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
+    if (memoryLimitSet) {

Review Comment:
   We should only serialize the to frames only when memoryLimit Is set else the old code path should be invoked.  
   



##########
processing/src/main/java/org/apache/druid/query/InlineDataSource.java:
##########
@@ -300,7 +363,7 @@ public boolean equals(Object o)
       return false;
     }
     InlineDataSource that = (InlineDataSource) o;
-    return rowsEqual(rows, that.rows) &&

Review Comment:
   maybe compare frames directly if possible and save on the cost of initializing various frameReaders and then de serializing them to rows. 



##########
processing/src/main/java/org/apache/druid/query/InlineDataSource.java:
##########
@@ -53,14 +63,17 @@
 public class InlineDataSource implements DataSource
 {
   private final Iterable<Object[]> rows;
+  private final Frame frame;
   private final RowSignature signature;
 
   private InlineDataSource(
       final Iterable<Object[]> rows,
+      final Frame frame,
       final RowSignature signature
   )
   {
-    this.rows = Preconditions.checkNotNull(rows, "'rows' must be nonnull");
+    this.rows = rows;

Review Comment:
   The inline DS can either be backed by a frame or by a rows. 
   So lets error out if both are null. 



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -565,10 +616,14 @@ private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSour
       final Sequence<T> results,
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,

Review Comment:
   we should never be using both limits I guess. Can we remove the additional 3 params and just pass another param called type ? and re-use the same limit variables ?



-- 
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: commits-unsubscribe@druid.apache.org

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] abhishekagarwal87 commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1239700678


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -613,96 +638,149 @@ private static <T, QueryType extends Query<T>> DataSource toInlineDataSource(
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,
       final AtomicLong memoryLimitAccumulator,
+      final AtomicBoolean cannotMaterializeToFrames,
       final int limit,
-      long memoryLimit
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
   )
   {
-    final int limitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
-    boolean memoryLimitSet = memoryLimit >= 0;
-
-    if (limitAccumulator.get() >= limitToUse) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum[%d] reached",
-          limitToUse
-      );
-    }
-
-    if (memoryLimitSet && memoryLimitAccumulator.get() >= memoryLimit) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
-          memoryLimit
-      );
-    }
+    final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
 
     DataSource dataSource;
-    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
-    if (memoryLimitSet) {
-      try {
-        Optional<Sequence<FrameSignaturePair>> framesOptional = toolChest.resultsAsFrames(
+
+    switch (ClientQuerySegmentWalkerUtils.getLimitType(memoryLimit, cannotMaterializeToFrames.get())) {
+      case ROW_LIMIT:
+        if (limitAccumulator.get() >= rowLimitToUse) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] rows",
+              rowLimitToUse
+          );
+        }
+        dataSource = materializeResultsAsArray(
             query,
             results,
-            memoryLimit - memoryLimitAccumulator.get()
+            toolChest,
+            limitAccumulator,
+            limit
         );
-
-        if (!framesOptional.isPresent()) {
-          throw new ISE("The memory of the subqueries cannot be estimated correctly.");
+        break;
+      case MEMORY_LIMIT:
+        if (memoryLimitAccumulator.get() >= memoryLimit) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+              memoryLimit
+          );
         }
-
-        Sequence<FrameSignaturePair> frames = framesOptional.get();
-        List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
-        frames.forEach(
-            frame -> {
-              if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] bytes",
-                    memoryLimit
-                );
-
-              }
-              if (limitAccumulator.addAndGet(frame.getFrame().numRows()) >= limitToUse) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] rows",
-                    limitToUse
-                );
-              }
-              frameSignaturePairs.add(frame);
-            }
-        );
-        dataSource = new FramesBackedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query));
-      }
-      catch (ResourceLimitExceededException rlee) {
-        throw rlee;
-      }
-      catch (Exception e) {
-        log.info(
-            "Unable to write the subquery results to a frame. Results won't be accounted for in the memory "
-            + "calculation"
+        Optional<DataSource> maybeDataSource = materializeResultsAsFrames(
+            query,
+            results,
+            toolChest,
+            limitAccumulator,
+            memoryLimitAccumulator,
+            memoryLimit,
+            useNestedForUnknownTypeInSubquery
         );
-        throw e;
-      }
-    } else {
-      final RowSignature signature = toolChest.resultArraySignature(query);
-
-      final ArrayList<Object[]> resultList = new ArrayList<>();
-
-      toolChest.resultsAsArrays(query, results).accumulate(
-          resultList,
-          (acc, in) -> {
-            if (limitAccumulator.getAndIncrement() >= limitToUse) {
-              throw ResourceLimitExceededException.withMessage(
-                  "Subquery generated results beyond maximum[%d] rows",
-                  limitToUse
-              );
-            }
-            acc.add(in);
-            return acc;
+        if (!maybeDataSource.isPresent()) {
+          cannotMaterializeToFrames.set(true);
+          // Check if the previous row limit accumulator has exceeded the memory results
+          if (memoryLimitAccumulator.get() >= memoryLimit) {
+            throw ResourceLimitExceededException.withMessage(
+                "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+                memoryLimit
+            );
           }
-      );
-      dataSource = InlineDataSource.fromIterable(resultList, signature);
+          dataSource = materializeResultsAsArray(
+              query,
+              results,
+              toolChest,
+              limitAccumulator,
+              limit
+          );
+        } else {
+          dataSource = maybeDataSource.get();
+        }
+        break;
+      default:
+        throw new IAE("Only row based and memory based limiting is supported");
     }
     return dataSource;
   }
 
+  private static <T, QueryType extends Query<T>> Optional<DataSource> materializeResultsAsFrames(
+      final QueryType query,
+      final Sequence<T> results,
+      final QueryToolChest<T, QueryType> toolChest,
+      final AtomicInteger limitAccumulator,
+      final AtomicLong memoryLimitAccumulator,
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
+  )
+  {
+    Optional<Sequence<FrameSignaturePair>> framesOptional;
+
+    try {
+      framesOptional = toolChest.resultsAsFrames(
+          query,
+          results,
+          new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+          useNestedForUnknownTypeInSubquery
+      );
+    }
+    catch (Exception e) {
+      return Optional.empty();

Review Comment:
   why is this a debug log though? It should be WARN. 



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -613,96 +638,149 @@ private static <T, QueryType extends Query<T>> DataSource toInlineDataSource(
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,
       final AtomicLong memoryLimitAccumulator,
+      final AtomicBoolean cannotMaterializeToFrames,
       final int limit,
-      long memoryLimit
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
   )
   {
-    final int limitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
-    boolean memoryLimitSet = memoryLimit >= 0;
-
-    if (limitAccumulator.get() >= limitToUse) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum[%d] reached",
-          limitToUse
-      );
-    }
-
-    if (memoryLimitSet && memoryLimitAccumulator.get() >= memoryLimit) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
-          memoryLimit
-      );
-    }
+    final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
 
     DataSource dataSource;
-    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
-    if (memoryLimitSet) {
-      try {
-        Optional<Sequence<FrameSignaturePair>> framesOptional = toolChest.resultsAsFrames(
+
+    switch (ClientQuerySegmentWalkerUtils.getLimitType(memoryLimit, cannotMaterializeToFrames.get())) {
+      case ROW_LIMIT:
+        if (limitAccumulator.get() >= rowLimitToUse) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] rows",
+              rowLimitToUse
+          );
+        }
+        dataSource = materializeResultsAsArray(
             query,
             results,
-            memoryLimit - memoryLimitAccumulator.get()
+            toolChest,
+            limitAccumulator,
+            limit
         );
-
-        if (!framesOptional.isPresent()) {
-          throw new ISE("The memory of the subqueries cannot be estimated correctly.");
+        break;
+      case MEMORY_LIMIT:
+        if (memoryLimitAccumulator.get() >= memoryLimit) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+              memoryLimit
+          );
         }
-
-        Sequence<FrameSignaturePair> frames = framesOptional.get();
-        List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
-        frames.forEach(
-            frame -> {
-              if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] bytes",
-                    memoryLimit
-                );
-
-              }
-              if (limitAccumulator.addAndGet(frame.getFrame().numRows()) >= limitToUse) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] rows",
-                    limitToUse
-                );
-              }
-              frameSignaturePairs.add(frame);
-            }
-        );
-        dataSource = new FramesBackedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query));
-      }
-      catch (ResourceLimitExceededException rlee) {
-        throw rlee;
-      }
-      catch (Exception e) {
-        log.info(
-            "Unable to write the subquery results to a frame. Results won't be accounted for in the memory "
-            + "calculation"
+        Optional<DataSource> maybeDataSource = materializeResultsAsFrames(
+            query,
+            results,
+            toolChest,
+            limitAccumulator,
+            memoryLimitAccumulator,
+            memoryLimit,
+            useNestedForUnknownTypeInSubquery
         );
-        throw e;
-      }
-    } else {
-      final RowSignature signature = toolChest.resultArraySignature(query);
-
-      final ArrayList<Object[]> resultList = new ArrayList<>();
-
-      toolChest.resultsAsArrays(query, results).accumulate(
-          resultList,
-          (acc, in) -> {
-            if (limitAccumulator.getAndIncrement() >= limitToUse) {
-              throw ResourceLimitExceededException.withMessage(
-                  "Subquery generated results beyond maximum[%d] rows",
-                  limitToUse
-              );
-            }
-            acc.add(in);
-            return acc;
+        if (!maybeDataSource.isPresent()) {
+          cannotMaterializeToFrames.set(true);
+          // Check if the previous row limit accumulator has exceeded the memory results
+          if (memoryLimitAccumulator.get() >= memoryLimit) {
+            throw ResourceLimitExceededException.withMessage(
+                "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+                memoryLimit
+            );
           }
-      );
-      dataSource = InlineDataSource.fromIterable(resultList, signature);
+          dataSource = materializeResultsAsArray(
+              query,
+              results,
+              toolChest,
+              limitAccumulator,
+              limit
+          );
+        } else {
+          dataSource = maybeDataSource.get();
+        }
+        break;
+      default:
+        throw new IAE("Only row based and memory based limiting is supported");
     }
     return dataSource;
   }
 
+  private static <T, QueryType extends Query<T>> Optional<DataSource> materializeResultsAsFrames(
+      final QueryType query,
+      final Sequence<T> results,
+      final QueryToolChest<T, QueryType> toolChest,
+      final AtomicInteger limitAccumulator,
+      final AtomicLong memoryLimitAccumulator,
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
+  )
+  {
+    Optional<Sequence<FrameSignaturePair>> framesOptional;
+
+    try {
+      framesOptional = toolChest.resultsAsFrames(
+          query,
+          results,
+          new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+          useNestedForUnknownTypeInSubquery
+      );
+    }
+    catch (Exception e) {
+      return Optional.empty();
+    }
+
+    if (!framesOptional.isPresent()) {
+      return Optional.empty();
+    }
+
+    Sequence<FrameSignaturePair> frames = framesOptional.get();
+    List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
+    frames.forEach(
+        frame -> {
+          limitAccumulator.addAndGet(frame.getFrame().numRows());
+          if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
+            throw ResourceLimitExceededException.withMessage(
+                "Subquery generated results beyond maximum[%d] bytes",

Review Comment:
   whats the expected action from user? 



##########
processing/src/main/java/org/apache/druid/frame/segment/FrameCursorUtils.java:
##########
@@ -67,4 +76,58 @@ public static Filter buildFilter(@Nullable Filter filter, Interval interval)
       );
     }
   }
+
+  /**
+   * Writes a {@link Cursor} to a sequence of {@link Frame}. This method iterates over the rows of the cursor,
+   * and writes the columns to the frames
+   *
+   * @param cursor                 Cursor to write to the frame
+   * @param frameWriterFactory     Frame writer factory to write to the frame.
+   *                               Determines the signature of the rows that are written to the frames
+   */
+  public static Sequence<Frame> cursorToFrames(
+      Cursor cursor,
+      FrameWriterFactory frameWriterFactory
+  )
+  {
+
+    return Sequences.simple(
+        () -> new Iterator<Frame>()
+        {
+          @Override
+          public boolean hasNext()
+          {
+            return !cursor.isDone();
+          }
+
+          @Override
+          public Frame next()
+          {
+            // Makes sure that cursor contains some elements prior. This ensures if no row is written, then the row size
+            // is larger than the MemoryAllocators returned by the provided factory
+            if (!hasNext()) {
+              throw new NoSuchElementException();
+            }
+            boolean firstRowWritten = false;
+            Frame frame;
+            try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) {
+              while (!cursor.isDone()) {
+                if (!frameWriter.addSelection()) {
+                  break;
+                }
+                firstRowWritten = true;
+                cursor.advance();
+              }
+
+              if (!firstRowWritten) {
+                throw new ISE("Row size is greater than the frame size.");

Review Comment:
   will a user ever see this error message? Please use the `DruidException` class instead and add an error message thats more actionable. 



-- 
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: commits-unsubscribe@druid.apache.org

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] github-code-scanning[bot] commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1203557149


##########
processing/src/main/java/org/apache/druid/frame/segment/FrameCursorUtils.java:
##########
@@ -67,4 +76,59 @@
       );
     }
   }
+
+  /**
+   * Writes a {@link Cursor} to a sequence of {@link Frame}. This method iterates over the rows of the cursor,
+   * and writes the columns to the frames
+   *
+   * @param cursor                 Cursor to write to the frame
+   * @param frameWriterFactory     Frame writer factory to write to the frame.
+   *                               Determines the signature of the rows that are written to the frames
+   * @param memoryAllocatorFactory Allocator factory which creates the frames for the result sequence

Review Comment:
   ## Spurious Javadoc @param tags
   
   @param tag "memoryAllocatorFactory" does not match any actual parameter of method "cursorToFrames()".
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4982)



##########
processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java:
##########
@@ -0,0 +1,338 @@
+/*
+ * 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.segment.join.table;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.frame.Frame;
+import org.apache.druid.frame.read.FrameReader;
+import org.apache.druid.frame.read.columnar.FrameColumnReader;
+import org.apache.druid.frame.read.columnar.FrameColumnReaders;
+import org.apache.druid.frame.segment.FrameStorageAdapter;
+import org.apache.druid.frame.segment.columnar.FrameQueryableIndex;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.FrameBasedInlineDataSource;
+import org.apache.druid.query.FrameSignaturePair;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SimpleAscendingOffset;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.BaseColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+
+import javax.annotation.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class FrameBasedIndexedTable implements IndexedTable
+{
+  private static final Logger LOG = new Logger(BroadcastSegmentIndexedTable.class);
+  private static final byte CACHE_PREFIX = 0x01;
+
+  private final Set<String> keyColumns;
+  private final RowSignature rowSignature;
+  private final String version;
+  private final List<IndexedTable.Index> keyColumnsIndexes;
+  private final int numRows;
+  private final List<QueryableIndex> frameQueryableIndexes = new ArrayList<>();
+  private final List<Integer> cumulativeRowCount = new ArrayList<>();
+
+
+  public FrameBasedIndexedTable(
+      final FrameBasedInlineDataSource frameBasedInlineDataSource,
+      final Set<String> keyColumns,
+      final String version
+  )
+  {
+    this.keyColumns = keyColumns;
+    this.version = version;
+    this.rowSignature = frameBasedInlineDataSource.getRowSignature();
+
+    int rowCount = 0;
+    for (FrameSignaturePair frameSignaturePair : frameBasedInlineDataSource.getFrames()) {
+      Frame frame = frameSignaturePair.getFrame();
+      RowSignature frameRowSignature = frameSignaturePair.getRowSignature();
+      frameQueryableIndexes.add(new FrameQueryableIndex(
+          frame,
+          frameRowSignature,
+          createColumnReaders(frameRowSignature)
+      ));
+      rowCount += frame.numRows();
+      cumulativeRowCount.add(rowCount);
+    }
+
+    this.numRows = rowCount;
+
+    final ArrayList<RowBasedIndexBuilder> indexBuilders = new ArrayList<>(rowSignature.size());
+    final List<String> keyColumnNames = new ArrayList<>(keyColumns.size());
+
+    for (int i = 0; i < rowSignature.size(); i++) {
+      final RowBasedIndexBuilder m;
+      final String columnName = rowSignature.getColumnName(i);
+      if (keyColumns.contains(columnName)) {
+        final ColumnType keyType =
+            rowSignature.getColumnType(i).orElse(IndexedTableJoinMatcher.DEFAULT_KEY_TYPE);
+
+        m = new RowBasedIndexBuilder(keyType);
+        keyColumnNames.add(columnName);
+      } else {
+        m = null;
+      }
+      indexBuilders.add(m);
+    }
+
+    final Sequence<Pair<Cursor, Function<String, ColumnCapabilities>>> cursors = Sequences.concat(
+        frameBasedInlineDataSource
+            .getFrames()
+            .stream()
+            .map(frameSignaturePair -> {
+              Frame frame = frameSignaturePair.getFrame();
+              RowSignature rowSignature = frameSignaturePair.getRowSignature();
+              FrameStorageAdapter frameStorageAdapter =
+                  new FrameStorageAdapter(frame, FrameReader.create(rowSignature, true), Intervals.ETERNITY);
+              return frameStorageAdapter.makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null)
+                  .map(cursor ->
+                           Pair.<Cursor, Function<String, ColumnCapabilities>>of(cursor, frameStorageAdapter::getColumnCapabilities
+                  ));
+            })
+            .collect(Collectors.toList())
+    );
+
+    final Sequence<Integer> sequence = Sequences.map(
+        cursors,
+        cursorWithColumnCapabilities -> {
+          Cursor cursor = cursorWithColumnCapabilities.lhs;
+          Function<String, ColumnCapabilities> columnCapabilitiesFunction = cursorWithColumnCapabilities.rhs;
+          if (cursor == null) {
+            return 0;
+          }
+          int rowNumber = 0;
+          ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
+
+          // this should really be optimized to use dimension selectors where possible to populate indexes from bitmap
+          // indexes, but, an optimization for another day
+          final List<BaseObjectColumnValueSelector> selectors = keyColumnNames
+              .stream()
+              .map(columnName -> {
+                // multi-value dimensions are not currently supported
+                if (columnCapabilitiesFunction.apply(columnName).hasMultipleValues().isMaybeTrue()) {
+                  return NilColumnValueSelector.instance();
+                }
+                return columnSelectorFactory.makeColumnValueSelector(columnName);
+              })
+              .collect(Collectors.toList());
+
+          while (!cursor.isDone()) {
+            for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) {
+              final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex);
+              final int columnPosition = rowSignature.indexOf(keyColumnName);
+              final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition);
+              keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject());
+            }
+
+            if (rowNumber % 100_000 == 0) {
+              if (rowNumber == 0) {
+                LOG.debug("Indexed first row for frame based datasource");
+              } else {
+                LOG.debug("Indexed row %s for frame based datasource", rowNumber);
+              }
+            }
+            rowNumber++;
+            cursor.advance();
+          }
+          return rowNumber;
+        }
+    );
+
+    Integer totalRows = sequence.accumulate(0, (accumulated, in) -> accumulated += in);
+
+    this.keyColumnsIndexes = indexBuilders.stream()
+                                          .map(builder -> builder != null ? builder.build() : null)
+                                          .collect(Collectors.toList());
+
+    LOG.info("Created FrameBasedIndexedTable with %s rows.", totalRows);
+  }
+
+  @Override
+  public String version()
+  {
+    return version;
+  }
+
+  @Override
+  public Set<String> keyColumns()
+  {
+    return keyColumns;
+  }
+
+  @Override
+  public RowSignature rowSignature()
+  {
+    return rowSignature;
+  }
+
+  @Override
+  public int numRows()
+  {
+    return numRows;
+  }
+
+  @Override
+  public Index columnIndex(int column)
+  {
+    return RowBasedIndexedTable.getKeyColumnIndex(column, keyColumnsIndexes);
+
+  }
+
+  @Override
+  public Reader columnReader(int column)
+  {
+
+    if (!rowSignature.contains(column)) {
+      throw new IAE("Column[%d] is not a valid column for the frame based datasource");

Review Comment:
   ## Missing format argument
   
   This format call refers to 1 argument(s) but only supplies 0 argument(s).
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4983)



##########
processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.segment.join.table;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.frame.Frame;
+import org.apache.druid.frame.FrameType;
+import org.apache.druid.frame.allocation.HeapMemoryAllocator;
+import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory;
+import org.apache.druid.frame.segment.FrameCursorUtils;
+import org.apache.druid.frame.write.FrameWriterFactory;
+import org.apache.druid.frame.write.FrameWriters;
+import org.apache.druid.query.FrameBasedInlineDataSource;
+import org.apache.druid.query.FrameSignaturePair;
+import org.apache.druid.query.IterableRowsCursorHelper;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.testing.InitializedNullHandlingTest;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+public class FrameBasedIndexedTableTest extends InitializedNullHandlingTest
+{
+  private static final String STRING_COL_1 = "market";
+  private static final String LONG_COL_1 = "longNumericNull";
+  private static final String DOUBLE_COL_1 = "doubleNumericNull";
+  private static final String FLOAT_COL_1 = "floatNumericNull";
+  private static final String STRING_COL_2 = "market";
+  private static final String MULTI_VALUE_COLUMN = "placementish";
+  private static final String DIM_NOT_EXISTS = "DIM_NOT_EXISTS";
+
+  private static final List<Object[]> DATASOURCE_ROWS =
+      ImmutableList.<Object[]>builder()
+                   .add(new Object[]{"spot", 1L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 1L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 2L, 1.3d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 1L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 1L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 1L, 1.5d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 4L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 1L, 1.7d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 5L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .build();
+
+  private static final RowSignature ROW_SIGNATURE =
+      RowSignature.builder()
+                  .add(STRING_COL_1, ColumnType.STRING)
+                  .add(LONG_COL_1, ColumnType.LONG)
+                  .add(DOUBLE_COL_1, ColumnType.DOUBLE)
+                  .add(FLOAT_COL_1, ColumnType.FLOAT)
+                  .add(STRING_COL_1, ColumnType.STRING)
+                  .add(MULTI_VALUE_COLUMN, ColumnType.STRING_ARRAY)
+                  .build();
+
+  private static final Set<String> KEY_COLUMNS = ImmutableSet.<String>builder()
+                                                             .add(STRING_COL_1)
+                                                             .add(LONG_COL_1)
+                                                             .add(DOUBLE_COL_1)
+                                                             .add(FLOAT_COL_1)
+                                                             .add(MULTI_VALUE_COLUMN)
+                                                             .add(DIM_NOT_EXISTS)
+                                                             .build();
+
+
+  private FrameBasedInlineDataSource dataSource;
+  private FrameBasedIndexedTable frameBasedIndexedTable;
+
+  @Before
+  public void setup()
+  {
+    Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(DATASOURCE_ROWS, ROW_SIGNATURE);
+    FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
+        FrameType.COLUMNAR,
+        new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+        ROW_SIGNATURE,
+        new ArrayList<>()
+    );
+    Frame frame = Iterables.getOnlyElement(FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory).toList());
+
+    dataSource = new FrameBasedInlineDataSource(
+        ImmutableList.of(new FrameSignaturePair(frame, ROW_SIGNATURE)),
+        ROW_SIGNATURE
+    );
+
+    frameBasedIndexedTable = new FrameBasedIndexedTable(dataSource, KEY_COLUMNS, "test");
+
+  }
+
+  @Test
+  public void testInitShouldGenerateCorrectTable()
+  {
+    Assert.assertEquals(9, frameBasedIndexedTable.numRows());
+  }
+
+  @Test
+  public void testStringKeyColumn()
+  {
+    // lets try a few values out
+    final String[] vals = new String[]{"spot", "total_market", "upfront"};
+    checkIndexAndReader(STRING_COL_1, vals);
+  }
+
+  @Test
+  public void testLongKeyColumn()
+  {
+    final Long[] vals = new Long[]{NullHandling.replaceWithDefault() ? 0L : null, 10L, 20L};
+    checkIndexAndReader(LONG_COL_1, vals);
+  }
+
+  @Test
+  public void testFloatKeyColumn()
+  {
+    final Float[] vals = new Float[]{NullHandling.replaceWithDefault() ? 0.0f : null, 10.0f, 20.0f};
+    checkIndexAndReader(FLOAT_COL_1, vals);
+  }
+
+  @Test
+  public void testDoubleKeyColumn()
+  {
+    final Double[] vals = new Double[]{NullHandling.replaceWithDefault() ? 0.0 : null, 10.0, 20.0};
+    checkIndexAndReader(DOUBLE_COL_1, vals);
+  }
+
+  private void checkIndexAndReader(String columnName, Object[] vals)
+  {
+    checkIndexAndReader(columnName, vals, new Object[0]);
+  }
+
+  private void checkIndexAndReader(String columnName, Object[] vals, Object[] nonmatchingVals)

Review Comment:
   ## Useless parameter
   
   The parameter 'vals' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4984)



##########
processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java:
##########
@@ -0,0 +1,338 @@
+/*
+ * 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.segment.join.table;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.frame.Frame;
+import org.apache.druid.frame.read.FrameReader;
+import org.apache.druid.frame.read.columnar.FrameColumnReader;
+import org.apache.druid.frame.read.columnar.FrameColumnReaders;
+import org.apache.druid.frame.segment.FrameStorageAdapter;
+import org.apache.druid.frame.segment.columnar.FrameQueryableIndex;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.FrameBasedInlineDataSource;
+import org.apache.druid.query.FrameSignaturePair;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SimpleAscendingOffset;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.BaseColumn;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+
+import javax.annotation.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class FrameBasedIndexedTable implements IndexedTable
+{
+  private static final Logger LOG = new Logger(BroadcastSegmentIndexedTable.class);
+  private static final byte CACHE_PREFIX = 0x01;
+
+  private final Set<String> keyColumns;
+  private final RowSignature rowSignature;
+  private final String version;
+  private final List<IndexedTable.Index> keyColumnsIndexes;
+  private final int numRows;
+  private final List<QueryableIndex> frameQueryableIndexes = new ArrayList<>();
+  private final List<Integer> cumulativeRowCount = new ArrayList<>();
+
+
+  public FrameBasedIndexedTable(
+      final FrameBasedInlineDataSource frameBasedInlineDataSource,
+      final Set<String> keyColumns,
+      final String version
+  )
+  {
+    this.keyColumns = keyColumns;
+    this.version = version;
+    this.rowSignature = frameBasedInlineDataSource.getRowSignature();
+
+    int rowCount = 0;
+    for (FrameSignaturePair frameSignaturePair : frameBasedInlineDataSource.getFrames()) {
+      Frame frame = frameSignaturePair.getFrame();
+      RowSignature frameRowSignature = frameSignaturePair.getRowSignature();
+      frameQueryableIndexes.add(new FrameQueryableIndex(
+          frame,
+          frameRowSignature,
+          createColumnReaders(frameRowSignature)
+      ));
+      rowCount += frame.numRows();
+      cumulativeRowCount.add(rowCount);
+    }
+
+    this.numRows = rowCount;
+
+    final ArrayList<RowBasedIndexBuilder> indexBuilders = new ArrayList<>(rowSignature.size());
+    final List<String> keyColumnNames = new ArrayList<>(keyColumns.size());
+
+    for (int i = 0; i < rowSignature.size(); i++) {
+      final RowBasedIndexBuilder m;
+      final String columnName = rowSignature.getColumnName(i);
+      if (keyColumns.contains(columnName)) {
+        final ColumnType keyType =
+            rowSignature.getColumnType(i).orElse(IndexedTableJoinMatcher.DEFAULT_KEY_TYPE);
+
+        m = new RowBasedIndexBuilder(keyType);
+        keyColumnNames.add(columnName);
+      } else {
+        m = null;
+      }
+      indexBuilders.add(m);
+    }
+
+    final Sequence<Pair<Cursor, Function<String, ColumnCapabilities>>> cursors = Sequences.concat(
+        frameBasedInlineDataSource
+            .getFrames()
+            .stream()
+            .map(frameSignaturePair -> {
+              Frame frame = frameSignaturePair.getFrame();
+              RowSignature rowSignature = frameSignaturePair.getRowSignature();
+              FrameStorageAdapter frameStorageAdapter =
+                  new FrameStorageAdapter(frame, FrameReader.create(rowSignature, true), Intervals.ETERNITY);
+              return frameStorageAdapter.makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null)
+                  .map(cursor ->
+                           Pair.<Cursor, Function<String, ColumnCapabilities>>of(cursor, frameStorageAdapter::getColumnCapabilities
+                  ));
+            })
+            .collect(Collectors.toList())
+    );
+
+    final Sequence<Integer> sequence = Sequences.map(
+        cursors,
+        cursorWithColumnCapabilities -> {
+          Cursor cursor = cursorWithColumnCapabilities.lhs;
+          Function<String, ColumnCapabilities> columnCapabilitiesFunction = cursorWithColumnCapabilities.rhs;
+          if (cursor == null) {
+            return 0;
+          }
+          int rowNumber = 0;
+          ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
+
+          // this should really be optimized to use dimension selectors where possible to populate indexes from bitmap
+          // indexes, but, an optimization for another day
+          final List<BaseObjectColumnValueSelector> selectors = keyColumnNames
+              .stream()
+              .map(columnName -> {
+                // multi-value dimensions are not currently supported
+                if (columnCapabilitiesFunction.apply(columnName).hasMultipleValues().isMaybeTrue()) {
+                  return NilColumnValueSelector.instance();
+                }
+                return columnSelectorFactory.makeColumnValueSelector(columnName);
+              })
+              .collect(Collectors.toList());
+
+          while (!cursor.isDone()) {
+            for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) {
+              final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex);
+              final int columnPosition = rowSignature.indexOf(keyColumnName);
+              final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition);
+              keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject());
+            }
+
+            if (rowNumber % 100_000 == 0) {
+              if (rowNumber == 0) {
+                LOG.debug("Indexed first row for frame based datasource");
+              } else {
+                LOG.debug("Indexed row %s for frame based datasource", rowNumber);
+              }
+            }
+            rowNumber++;
+            cursor.advance();
+          }
+          return rowNumber;
+        }
+    );
+
+    Integer totalRows = sequence.accumulate(0, (accumulated, in) -> accumulated += in);
+
+    this.keyColumnsIndexes = indexBuilders.stream()
+                                          .map(builder -> builder != null ? builder.build() : null)
+                                          .collect(Collectors.toList());
+
+    LOG.info("Created FrameBasedIndexedTable with %s rows.", totalRows);
+  }
+
+  @Override
+  public String version()
+  {
+    return version;
+  }
+
+  @Override
+  public Set<String> keyColumns()
+  {
+    return keyColumns;
+  }
+
+  @Override
+  public RowSignature rowSignature()
+  {
+    return rowSignature;
+  }
+
+  @Override
+  public int numRows()
+  {
+    return numRows;
+  }
+
+  @Override
+  public Index columnIndex(int column)
+  {
+    return RowBasedIndexedTable.getKeyColumnIndex(column, keyColumnsIndexes);
+
+  }
+
+  @Override
+  public Reader columnReader(int column)
+  {
+
+    if (!rowSignature.contains(column)) {
+      throw new IAE("Column[%d] is not a valid column for the frame based datasource");
+    }
+
+    String columnName = rowSignature.getColumnName(column);
+    final SimpleAscendingOffset offset = new SimpleAscendingOffset(numRows());
+    final List<BaseObjectColumnValueSelector<?>> columnValueSelectors = new ArrayList<>();
+    final Set<Closeable> closeables = new HashSet<>();
+
+    for (QueryableIndex frameQueryableIndex : frameQueryableIndexes) {
+      ColumnHolder columnHolder = frameQueryableIndex.getColumnHolder(columnName);
+      if (columnHolder == null) {
+        columnValueSelectors.add(NilColumnValueSelector.instance());
+      } else {
+        BaseColumn baseColumn = columnHolder.getColumn();
+        columnValueSelectors.add(baseColumn.makeColumnValueSelector(offset));
+        closeables.add(baseColumn);
+      }
+    }
+
+    return new Reader()
+    {
+      @Nullable
+      @Override
+      public Object read(int row)
+      {
+        int frameIndex = binSearch(cumulativeRowCount, row);
+        if (frameIndex == frameQueryableIndexes.size()) {
+          throw new IndexOutOfBoundsException(
+              StringUtils.format("Requested row index [%d], Max row count [%d]", row, numRows())
+          );
+        }
+        // The offset needs to be changed as well
+        int adjustedOffset = frameIndex == 0 ? row : row - cumulativeRowCount.get(frameIndex - 1);

Review Comment:
   ## User-controlled data in arithmetic expression
   
   This arithmetic expression depends on a [user-provided value](1), potentially causing an underflow.
   This arithmetic expression depends on a [user-provided value](2), potentially causing an underflow.
   This arithmetic expression depends on a [user-provided value](3), potentially causing an underflow.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4986)



##########
processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.segment.join.table;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.frame.Frame;
+import org.apache.druid.frame.FrameType;
+import org.apache.druid.frame.allocation.HeapMemoryAllocator;
+import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory;
+import org.apache.druid.frame.segment.FrameCursorUtils;
+import org.apache.druid.frame.write.FrameWriterFactory;
+import org.apache.druid.frame.write.FrameWriters;
+import org.apache.druid.query.FrameBasedInlineDataSource;
+import org.apache.druid.query.FrameSignaturePair;
+import org.apache.druid.query.IterableRowsCursorHelper;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.testing.InitializedNullHandlingTest;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+public class FrameBasedIndexedTableTest extends InitializedNullHandlingTest
+{
+  private static final String STRING_COL_1 = "market";
+  private static final String LONG_COL_1 = "longNumericNull";
+  private static final String DOUBLE_COL_1 = "doubleNumericNull";
+  private static final String FLOAT_COL_1 = "floatNumericNull";
+  private static final String STRING_COL_2 = "market";
+  private static final String MULTI_VALUE_COLUMN = "placementish";
+  private static final String DIM_NOT_EXISTS = "DIM_NOT_EXISTS";
+
+  private static final List<Object[]> DATASOURCE_ROWS =
+      ImmutableList.<Object[]>builder()
+                   .add(new Object[]{"spot", 1L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 1L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 2L, 1.3d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 1L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 1L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 1L, 1.5d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 4L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 1L, 1.7d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .add(new Object[]{"spot", 5L, 1.1d, 3.1f, "preferred", new Object[]{"val1", "val2"}})
+                   .build();
+
+  private static final RowSignature ROW_SIGNATURE =
+      RowSignature.builder()
+                  .add(STRING_COL_1, ColumnType.STRING)
+                  .add(LONG_COL_1, ColumnType.LONG)
+                  .add(DOUBLE_COL_1, ColumnType.DOUBLE)
+                  .add(FLOAT_COL_1, ColumnType.FLOAT)
+                  .add(STRING_COL_1, ColumnType.STRING)
+                  .add(MULTI_VALUE_COLUMN, ColumnType.STRING_ARRAY)
+                  .build();
+
+  private static final Set<String> KEY_COLUMNS = ImmutableSet.<String>builder()
+                                                             .add(STRING_COL_1)
+                                                             .add(LONG_COL_1)
+                                                             .add(DOUBLE_COL_1)
+                                                             .add(FLOAT_COL_1)
+                                                             .add(MULTI_VALUE_COLUMN)
+                                                             .add(DIM_NOT_EXISTS)
+                                                             .build();
+
+
+  private FrameBasedInlineDataSource dataSource;
+  private FrameBasedIndexedTable frameBasedIndexedTable;
+
+  @Before
+  public void setup()
+  {
+    Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(DATASOURCE_ROWS, ROW_SIGNATURE);
+    FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
+        FrameType.COLUMNAR,
+        new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+        ROW_SIGNATURE,
+        new ArrayList<>()
+    );
+    Frame frame = Iterables.getOnlyElement(FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory).toList());
+
+    dataSource = new FrameBasedInlineDataSource(
+        ImmutableList.of(new FrameSignaturePair(frame, ROW_SIGNATURE)),
+        ROW_SIGNATURE
+    );
+
+    frameBasedIndexedTable = new FrameBasedIndexedTable(dataSource, KEY_COLUMNS, "test");
+
+  }
+
+  @Test
+  public void testInitShouldGenerateCorrectTable()
+  {
+    Assert.assertEquals(9, frameBasedIndexedTable.numRows());
+  }
+
+  @Test
+  public void testStringKeyColumn()
+  {
+    // lets try a few values out
+    final String[] vals = new String[]{"spot", "total_market", "upfront"};
+    checkIndexAndReader(STRING_COL_1, vals);
+  }
+
+  @Test
+  public void testLongKeyColumn()
+  {
+    final Long[] vals = new Long[]{NullHandling.replaceWithDefault() ? 0L : null, 10L, 20L};
+    checkIndexAndReader(LONG_COL_1, vals);
+  }
+
+  @Test
+  public void testFloatKeyColumn()
+  {
+    final Float[] vals = new Float[]{NullHandling.replaceWithDefault() ? 0.0f : null, 10.0f, 20.0f};
+    checkIndexAndReader(FLOAT_COL_1, vals);
+  }
+
+  @Test
+  public void testDoubleKeyColumn()
+  {
+    final Double[] vals = new Double[]{NullHandling.replaceWithDefault() ? 0.0 : null, 10.0, 20.0};
+    checkIndexAndReader(DOUBLE_COL_1, vals);
+  }
+
+  private void checkIndexAndReader(String columnName, Object[] vals)
+  {
+    checkIndexAndReader(columnName, vals, new Object[0]);
+  }
+
+  private void checkIndexAndReader(String columnName, Object[] vals, Object[] nonmatchingVals)

Review Comment:
   ## Useless parameter
   
   The parameter 'nonmatchingVals' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4985)



-- 
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: commits-unsubscribe@druid.apache.org

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] paul-rogers commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "paul-rogers (via GitHub)" <gi...@apache.org>.
paul-rogers commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1142587809


##########
processing/src/main/java/org/apache/druid/query/InlineDataSource.java:
##########
@@ -300,7 +363,7 @@ public boolean equals(Object o)
       return false;
     }
     InlineDataSource that = (InlineDataSource) o;
-    return rowsEqual(rows, that.rows) &&
+    return rowsEqual(getRowsAsList(), that.getRowsAsList()) &&

Review Comment:
   Does it even make semantic sense to compare two input sources for equality? Are we adding this become some static check told us we need it, but not because we actually use it?



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +632,71 @@
       );
     }
 
+    if (memoryLimitAccumulator.get() >= memoryLimitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
+          memoryLimitToUse
+      );
+    }
+
     final RowSignature signature = toolChest.resultArraySignature(query);
 
-    final ArrayList<Object[]> resultList = new ArrayList<>();
+    final List<Object[]> resultList = toolChest.resultsAsArrays(query, results).toList();
 
-    toolChest.resultsAsArrays(query, results).accumulate(
-        resultList,
-        (acc, in) -> {
-          if (limitAccumulator.getAndIncrement() >= limitToUse) {
-            throw ResourceLimitExceededException.withMessage(
-                "Subquery generated results beyond maximum[%d]",
-                limitToUse
-            );
+    if (limitAccumulator.addAndGet(resultList.size()) >= limitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Subquery generated results beyond maximum[%d] rows",
+          limitToUse
+      );
+    }
+
+    Frame frame = null;
+
+    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
+    if (memoryLimitSet) {
+      try {
+        FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
+            FrameType.ROW_BASED,
+            new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+            signature,
+            new ArrayList<>()
+        );
+
+        final Cursor cursor = new InlineResultsCursor(resultList, signature);
+
+        try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) {
+          while (!cursor.isDone()) {
+            if (!frameWriter.addSelection()) {
+              throw new FrameRowTooLargeException(frameWriterFactory.allocatorCapacity());
+            }
+
+            cursor.advance();
           }
-          acc.add(in);
-          return acc;
+
+          frame = Frame.wrap(frameWriter.toByteArray());
         }
-    );
 
-    return InlineDataSource.fromIterable(resultList, signature);
+
+        if (memoryLimitAccumulator.addAndGet(frame.numBytes()) >= memoryLimitToUse) {
+          throw ResourceLimitExceededException.withMessage(
+              "Subquery generated results beyond maximum[%d] bytes",
+              memoryLimit
+          );
+        }
+      }
+      catch (ResourceLimitExceededException rlee) {
+        throw rlee;
+      }
+      catch (Exception e) {
+        log.info("Unable to write the subquery results to a frame. Results won't be accounted for in the memory"
+                 + "calculation");

Review Comment:
   Our error rules state that you must interpolate something, such as the reason why the subquery can't be written to a frame. And, yes, that code scanning item above is real.



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -565,10 +616,14 @@ private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSour
       final Sequence<T> results,
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,

Review Comment:
   If we have two limits, we might want to give them names so we know which limit that `limitAccumulator` is accumulating: row count or memory bytes?



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +632,71 @@ private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSour
       );
     }
 
+    if (memoryLimitAccumulator.get() >= memoryLimitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
+          memoryLimitToUse
+      );
+    }
+
     final RowSignature signature = toolChest.resultArraySignature(query);
 
-    final ArrayList<Object[]> resultList = new ArrayList<>();
+    final List<Object[]> resultList = toolChest.resultsAsArrays(query, results).toList();

Review Comment:
   If we are under memory pressure, is it wise to materialize the entire result set? Can the results be processed incrementally in a streaming manner?



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +632,71 @@ private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSour
       );
     }
 
+    if (memoryLimitAccumulator.get() >= memoryLimitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
+          memoryLimitToUse
+      );
+    }
+
     final RowSignature signature = toolChest.resultArraySignature(query);
 
-    final ArrayList<Object[]> resultList = new ArrayList<>();
+    final List<Object[]> resultList = toolChest.resultsAsArrays(query, results).toList();
 
-    toolChest.resultsAsArrays(query, results).accumulate(
-        resultList,
-        (acc, in) -> {
-          if (limitAccumulator.getAndIncrement() >= limitToUse) {
-            throw ResourceLimitExceededException.withMessage(
-                "Subquery generated results beyond maximum[%d]",
-                limitToUse
-            );
+    if (limitAccumulator.addAndGet(resultList.size()) >= limitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Subquery generated results beyond maximum[%d] rows",
+          limitToUse
+      );
+    }
+
+    Frame frame = null;
+
+    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
+    if (memoryLimitSet) {
+      try {
+        FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
+            FrameType.ROW_BASED,
+            new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+            signature,
+            new ArrayList<>()
+        );
+
+        final Cursor cursor = new InlineResultsCursor(resultList, signature);

Review Comment:
   All of this seems too complex to put inside the segment walker. For one thing, it is hard to test if it is an implementation detail. Perhaps pull out this logic into a separate class that can be unit tested extensively. For example, we'd want tests for hitting each of the limits, for handling variable-width columns, etc.



##########
processing/src/main/java/org/apache/druid/query/InlineDataSource.java:
##########
@@ -190,15 +211,57 @@ public List<ColumnType> getColumnTypes()
   @JsonProperty("rows")
   public List<Object[]> getRowsAsList()
   {
+    Iterable<Object[]> rows = getRows();
     return rows instanceof List ? ((List<Object[]>) rows) : Lists.newArrayList(rows);
   }
 
+  public Frame getBackingFrame()
+  {
+    return frame;
+  }
+
+  public boolean isBackedByFrame()
+  {
+    return frame != null;
+  }
+
   /**
    * Returns rows as an Iterable.
    */
   @JsonIgnore
   public Iterable<Object[]> getRows()
   {
+    if (isBackedByFrame()) {

Review Comment:
   This is the kind of unnecessary complexity that comes when we have one class do two things, so would be cool to split this into three: base class, array implementation, frame implementation.



##########
processing/src/main/java/org/apache/druid/query/InlineDataSource.java:
##########
@@ -53,14 +63,17 @@
 public class InlineDataSource implements DataSource
 {
   private final Iterable<Object[]> rows;
+  private final Frame frame;
   private final RowSignature signature;
 
   private InlineDataSource(
       final Iterable<Object[]> rows,
+      final Frame frame,
       final RowSignature signature
   )
   {
-    this.rows = Preconditions.checkNotNull(rows, "'rows' must be nonnull");
+    this.rows = rows;

Review Comment:
   If the code can use either rows or frames, but not both, then a cleaner design is to have two distinct classes. That avoids a bunch if if/then statements and the need for a reader to think through both cases at the same time.



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1241591469


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -613,96 +638,149 @@ private static <T, QueryType extends Query<T>> DataSource toInlineDataSource(
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,
       final AtomicLong memoryLimitAccumulator,
+      final AtomicBoolean cannotMaterializeToFrames,
       final int limit,
-      long memoryLimit
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
   )
   {
-    final int limitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
-    boolean memoryLimitSet = memoryLimit >= 0;
-
-    if (limitAccumulator.get() >= limitToUse) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum[%d] reached",
-          limitToUse
-      );
-    }
-
-    if (memoryLimitSet && memoryLimitAccumulator.get() >= memoryLimit) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
-          memoryLimit
-      );
-    }
+    final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
 
     DataSource dataSource;
-    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
-    if (memoryLimitSet) {
-      try {
-        Optional<Sequence<FrameSignaturePair>> framesOptional = toolChest.resultsAsFrames(
+
+    switch (ClientQuerySegmentWalkerUtils.getLimitType(memoryLimit, cannotMaterializeToFrames.get())) {
+      case ROW_LIMIT:
+        if (limitAccumulator.get() >= rowLimitToUse) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] rows",
+              rowLimitToUse
+          );
+        }
+        dataSource = materializeResultsAsArray(
             query,
             results,
-            memoryLimit - memoryLimitAccumulator.get()
+            toolChest,
+            limitAccumulator,
+            limit
         );
-
-        if (!framesOptional.isPresent()) {
-          throw new ISE("The memory of the subqueries cannot be estimated correctly.");
+        break;
+      case MEMORY_LIMIT:
+        if (memoryLimitAccumulator.get() >= memoryLimit) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+              memoryLimit
+          );
         }
-
-        Sequence<FrameSignaturePair> frames = framesOptional.get();
-        List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
-        frames.forEach(
-            frame -> {
-              if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] bytes",
-                    memoryLimit
-                );
-
-              }
-              if (limitAccumulator.addAndGet(frame.getFrame().numRows()) >= limitToUse) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] rows",
-                    limitToUse
-                );
-              }
-              frameSignaturePairs.add(frame);
-            }
-        );
-        dataSource = new FramesBackedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query));
-      }
-      catch (ResourceLimitExceededException rlee) {
-        throw rlee;
-      }
-      catch (Exception e) {
-        log.info(
-            "Unable to write the subquery results to a frame. Results won't be accounted for in the memory "
-            + "calculation"
+        Optional<DataSource> maybeDataSource = materializeResultsAsFrames(
+            query,
+            results,
+            toolChest,
+            limitAccumulator,
+            memoryLimitAccumulator,
+            memoryLimit,
+            useNestedForUnknownTypeInSubquery
         );
-        throw e;
-      }
-    } else {
-      final RowSignature signature = toolChest.resultArraySignature(query);
-
-      final ArrayList<Object[]> resultList = new ArrayList<>();
-
-      toolChest.resultsAsArrays(query, results).accumulate(
-          resultList,
-          (acc, in) -> {
-            if (limitAccumulator.getAndIncrement() >= limitToUse) {
-              throw ResourceLimitExceededException.withMessage(
-                  "Subquery generated results beyond maximum[%d] rows",
-                  limitToUse
-              );
-            }
-            acc.add(in);
-            return acc;
+        if (!maybeDataSource.isPresent()) {
+          cannotMaterializeToFrames.set(true);
+          // Check if the previous row limit accumulator has exceeded the memory results
+          if (memoryLimitAccumulator.get() >= memoryLimit) {
+            throw ResourceLimitExceededException.withMessage(
+                "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+                memoryLimit
+            );
           }
-      );
-      dataSource = InlineDataSource.fromIterable(resultList, signature);
+          dataSource = materializeResultsAsArray(
+              query,
+              results,
+              toolChest,
+              limitAccumulator,
+              limit
+          );
+        } else {
+          dataSource = maybeDataSource.get();
+        }
+        break;
+      default:
+        throw new IAE("Only row based and memory based limiting is supported");
     }
     return dataSource;
   }
 
+  private static <T, QueryType extends Query<T>> Optional<DataSource> materializeResultsAsFrames(
+      final QueryType query,
+      final Sequence<T> results,
+      final QueryToolChest<T, QueryType> toolChest,
+      final AtomicInteger limitAccumulator,
+      final AtomicLong memoryLimitAccumulator,
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
+  )
+  {
+    Optional<Sequence<FrameSignaturePair>> framesOptional;
+
+    try {
+      framesOptional = toolChest.resultsAsFrames(
+          query,
+          results,
+          new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+          useNestedForUnknownTypeInSubquery
+      );
+    }
+    catch (Exception e) {
+      return Optional.empty();

Review Comment:
   This will be executed per query so DEBUG is more appropriate according to me, otherwise, the logs will be cluttered with the exception message. Either we should:
   1. Keep it as DEBUG info so that we don't have to see cluttered logs. This has the disadvantage that we won't be able to readily observe if we fallback to the default method/code
   2. Don't catch the exception and let it propagate. The user will then report the issue and we can fix it.
   
   2nd option means that there won't be a fallback in case we aren't able to convert it to frames. Since this is a newer feature, I think we should still have a fallback till we are confident that we can convert each query, and once it is more mature and the frames can handle array types (currently it can handle string arrays only), we can remove this fallback altogether and let the exception pass through.



-- 
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: commits-unsubscribe@druid.apache.org

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] github-code-scanning[bot] commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1208966736


##########
processing/src/main/java/org/apache/druid/query/QueryToolChest.java:
##########
@@ -330,4 +332,32 @@
   {
     throw new UOE("Query type '%s' does not support returning results as arrays", query.getType());
   }
+
+  /**
+   * Converts a sequence of this query's ResultType into a sequence of {@link FrameSignaturePair}. The array signature
+   * is the one give by {@link #resultArraySignature(Query)}. If the toolchest doesn't support this method, then it can
+   * return an empty optional. It is the duty of the callees to throw an appropriate exception in that case or use an
+   * alternative fallback approach
+   *
+   * Check documentation of {@link #resultsAsArrays(Query, Sequence)} as the behaviour of the rows represented by the
+   * frame sequence is identical.
+   *
+   * Each Frame has a separate {@link RowSignature} because for some query types like the Scan query, every
+   * column in the final result might not be present in the individual ResultType (and subsequently Frame). Therefore,
+   * this is done to preserve the space by not populating the column in that particular Frame and omitting it from its
+   * signature
+   *  @param query Query being executed by the toolchest. Used to determine the rowSignature of the Frames
+   * @param resultSequence results of the form returned by {@link #mergeResults(QueryRunner)}
+   * @param memoryAllocatorFactory
+   * @param useNestedForUnknownTypes true if the unknown types in the results can be serded using complex types
+   */
+  public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
+      QueryType query,
+      Sequence<ResultType> resultSequence,
+      MemoryAllocatorFactory memoryAllocatorFactory,
+      boolean useNestedForUnknownTypes

Review Comment:
   ## Useless parameter
   
   The parameter 'useNestedForUnknownTypes' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4988)



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1159339200


##########
processing/src/main/java/org/apache/druid/query/QueryToolChest.java:
##########
@@ -330,4 +330,29 @@ public Sequence<Object[]> resultsAsArrays(QueryType query, Sequence<ResultType>
   {
     throw new UOE("Query type '%s' does not support returning results as arrays", query.getType());
   }
+
+  /**
+   * Converts a sequence of this query's ResultType into a sequence of {@link FrameSignaturePair}. The array signature
+   * is the one give by {@link #resultArraySignature(Query)}.
+   *
+   * Check documentation of {@link #resultsAsArrays(Query, Sequence)} as the behaviour of the rows represented by the
+   * frame sequence is identical.
+   *
+   * Each Frame has a separate {@link RowSignature} because for some query types like the Scan query, every
+   * column in the final result might not be present in the individual ResultType (and subsequently Frame). Therefore,
+   * this is done to preserve the space by not populating the column in that particular Frame and omitting it from its
+   * signature
+   */
+  public Sequence<FrameSignaturePair> resultsAsFrames(QueryType query, Sequence<ResultType> resultSequence)

Review Comment:
   Updated



-- 
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: commits-unsubscribe@druid.apache.org

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] github-code-scanning[bot] commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1170844067


##########
processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.query.scan;
+
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.Cursor;
+import org.joda.time.DateTime;
+
+import java.util.List;
+
+/**
+ * Combines multiple cursors and iterates over them. It skips over the empty cursors
+ */
+public class ConcatCursor implements Cursor
+{
+
+  private final List<Cursor> cursors;
+  private int currentCursor;
+
+  public ConcatCursor(
+      List<Cursor> cursors
+  )
+  {
+    this.cursors = cursors;
+    currentCursor = 0;
+    skipEmptyCursors();
+  }
+
+  @Override
+  public ColumnSelectorFactory getColumnSelectorFactory()
+  {
+    return cursors.get(currentCursor).getColumnSelectorFactory();
+  }
+
+  @Override
+  public DateTime getTime()
+  {
+    return cursors.get(currentCursor).getTime();
+  }
+
+  @Override
+  public void advance()
+  {
+    if (currentCursor < cursors.size()) {
+      cursors.get(currentCursor).advance();
+      advanceCursor();
+    }
+  }
+
+  @Override
+  public void advanceUninterruptibly()
+  {
+    if (currentCursor < cursors.size()) {
+      cursors.get(currentCursor).advanceUninterruptibly();
+      advanceCursor();
+    }
+  }
+
+  @Override
+  public boolean isDone()
+  {
+    return currentCursor == cursors.size();
+  }
+
+  @Override
+  public boolean isDoneOrInterrupted()
+  {
+    return isDone() || Thread.currentThread().isInterrupted();
+  }
+
+  @Override
+  public void reset()
+  {
+    while (currentCursor >= 0) {
+      if (currentCursor < cursors.size()) {
+        cursors.get(currentCursor).reset();
+      }
+      --currentCursor;
+    }
+    currentCursor = 0;
+    skipEmptyCursors();
+  }
+
+  /**
+   * This method should be called whenever the currentCursor gets updated. It skips over the empty cursors so that the
+   * current pointer is pointing to a valid cursor
+   */
+  private void skipEmptyCursors()
+  {
+    while (currentCursor < cursors.size() && cursors.get(currentCursor).isDone()) {
+      ++currentCursor;
+    }
+  }
+
+  /**
+   * This method updates the current cursor. This is used to update the current cursor under question.
+   */
+  private void advanceCursor()
+  {
+    if (cursors.get(currentCursor).isDone()) {
+      ++currentCursor;

Review Comment:
   ## User-controlled data in arithmetic expression
   
   This arithmetic expression depends on a [user-provided value](1), potentially causing an overflow.
   This arithmetic expression depends on a [user-provided value](2), potentially causing an overflow.
   This arithmetic expression depends on a [user-provided value](3), potentially causing an overflow.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4832)



##########
processing/src/main/java/org/apache/druid/query/FramesBackedInlineDataSourceSerializer.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.jsontype.TypeSerializer;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.java.util.common.jackson.JacksonUtils;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * Serializes {@link FramesBackedInlineDataSource} to the representation of {@link InlineDataSource}
+ * so that the servers' on wire transfer data doesn't change. {@link FramesBackedInlineDataSource} is currently limited
+ * to the brokers only and therefore this aids in conversion of the object to a representation that the data servers
+ * can recognize
+ */
+public class FramesBackedInlineDataSourceSerializer extends StdSerializer<FramesBackedInlineDataSource>
+{
+  public FramesBackedInlineDataSourceSerializer()
+  {
+    super(FramesBackedInlineDataSource.class);
+  }
+
+  @Override
+  public void serialize(FramesBackedInlineDataSource value, JsonGenerator jg, SerializerProvider serializers)
+      throws IOException
+  {
+    jg.writeStartObject();
+    jg.writeStringField("type", "inline");
+
+    RowSignature rowSignature = value.getRowSignature();
+    jg.writeObjectField("columnNames", rowSignature.getColumnNames());
+    List<ColumnType> columnTypes = IntStream.range(0, rowSignature.size())
+                                            .mapToObj(i -> rowSignature.getColumnType(i).orElse(null))
+                                            .collect(Collectors.toList());
+    jg.writeObjectField("columnTypes", columnTypes);
+
+    jg.writeArrayFieldStart("rows");
+
+    value.getRowsAsSequence().forEach(row -> {
+      try {
+        JacksonUtils.writeObjectUsingSerializerProvider(jg, serializers, row);
+      }
+      catch (IOException e) {
+        // Ideally, this shouldn't be reachable.
+        // Wrap the IO exception in the runtime exception and propogate it forward
+        throw new RE(
+            e,
+            "Exception encountered while serializing [%s] in [%s]",
+            row,

Review Comment:
   ## Implicit conversion from array to string
   
   Implicit conversion from Array to String.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4831)



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1159337173


##########
processing/src/main/java/org/apache/druid/query/FramesBackedInlineDataSource.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.query;
+
+import org.apache.druid.frame.Frame;
+import org.apache.druid.frame.read.FrameReader;
+import org.apache.druid.frame.segment.FrameStorageAdapter;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.planning.DataSourceAnalysis;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.SegmentReference;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.RowSignature;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Represents an inline datasource where the rows are embedded within the DataSource object itself.
+ *
+ * The rows are backed by a sequence of {@link FrameSignaturePair}, which contain the Frame representation of the rows
+ * represented by the datasource. {@link #getRowsAsList()} and {@link #getRowsAsSequence()} return the iterables which
+ * read the rows that are encapsulated in the frames.
+ *
+ * Note that the signature of the datasource can be different from the signatures of the constituent frames that it
+ * consists of. While fetching the iterables, it is the job of this class to make sure that the rows correspond to the
+ * {@link #rowSignature}. For frames that donot contain the columns present in the {@link #rowSignature}, they are
+ * populated with {@code null}.
+ */
+public class FramesBackedInlineDataSource implements DataSource
+{
+
+  final Sequence<FrameSignaturePair> frames;
+  final RowSignature rowSignature;
+
+  public FramesBackedInlineDataSource(
+      Sequence<FrameSignaturePair> frames,
+      RowSignature rowSignature
+  )
+  {
+    this.frames = frames;
+    this.rowSignature = rowSignature;
+  }
+
+  public Sequence<FrameSignaturePair> getFrames()
+  {
+    return frames;
+  }
+
+  public RowSignature getRowSignature()
+  {
+    return rowSignature;
+  }
+
+  public List<Object[]> getRowsAsList()

Review Comment:
   Updated and refactored it further to reuse the `rowsAsSequence()`, so that it doesn't do any duplicate computation.



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1170822169


##########
processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.query.scan;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.ListCursor;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ConcatCursorTest
+{
+  @Test
+  public void testCursor()
+  {
+    Cursor dummyCursor1 = new ListCursor(new ArrayList<>());
+    Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b"));
+    Cursor dummyCursor2 = new ListCursor(new ArrayList<>());
+    Cursor cursor2 = new ListCursor(ImmutableList.of("c", "d"));
+    Cursor dummyCursor3 = new ListCursor(new ArrayList<>());
+
+    Cursor concatCursor = new ConcatCursor(ImmutableList.of(
+        dummyCursor1,
+        cursor1,
+        dummyCursor2,
+        cursor2,
+        dummyCursor3
+    ));
+
+    List<Object> tempList = new ArrayList<>();
+    // Initial iteration
+    while (!concatCursor.isDone()) {
+      tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());

Review Comment:
   I think this would be done implicitly when we assert the contents, since on each iteration we are adding the element to the list.



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1177358523


##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -187,11 +206,129 @@ public RowSignature resultArraySignature(final ScanQuery query)
     }
   }
 
+  /**
+   * This batches the fetched {@link ScanResultValue}s which have similar signatures and are consecutives. In best case
+   * it would return a single frame, and in the worst case, it would return as many frames as the number of {@link ScanResultValue}
+   * passed.
+   */
+  @Override
+  public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
+      final ScanQuery query,
+      final Sequence<ScanResultValue> resultSequence,
+      @Nullable Long memoryLimitBytes
+  )
+  {
+    final AtomicLong memoryLimitAccumulator = memoryLimitBytes != null ? new AtomicLong(memoryLimitBytes) : null;
+    Iterator<ScanResultValue> resultSequenceIterator = new Iterator<ScanResultValue>()
+    {
+      Yielder<ScanResultValue> yielder = Yielders.each(resultSequence);
+
+      @Override
+      public boolean hasNext()
+      {
+        return !yielder.isDone();
+      }
+
+      @Override
+      public ScanResultValue next()
+      {
+        ScanResultValue scanResultValue = yielder.get();
+        yielder = yielder.next(null);
+        return scanResultValue;
+      }
+    };
+
+    Iterable<FrameSignaturePair> retVal = () -> new Iterator<FrameSignaturePair>()
+    {
+      PeekingIterator<ScanResultValue> scanResultValuePeekingIterator = Iterators.peekingIterator(resultSequenceIterator);
+
+      @Override
+      public boolean hasNext()
+      {
+        return scanResultValuePeekingIterator.hasNext();
+      }
+
+      @Override
+      public FrameSignaturePair next()
+      {
+        final List<ScanResultValue> batch = new ArrayList<>();
+        final ScanResultValue scanResultValue = scanResultValuePeekingIterator.next();
+        batch.add(scanResultValue);
+        final RowSignature rowSignature = scanResultValue.getRowSignature();
+        while (scanResultValuePeekingIterator.hasNext()) {
+          final RowSignature nextRowSignature = scanResultValuePeekingIterator.peek().getRowSignature();
+          if (nextRowSignature != null && nextRowSignature.equals(rowSignature)) {
+            batch.add(scanResultValuePeekingIterator.next());
+          }

Review Comment:
   Good catch for the 1st comment, added test cases that encompass this as well other methods.



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1175932725


##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -187,11 +206,129 @@ public RowSignature resultArraySignature(final ScanQuery query)
     }
   }
 
+  /**
+   * This batches the fetched {@link ScanResultValue}s which have similar signatures and are consecutives. In best case
+   * it would return a single frame, and in the worst case, it would return as many frames as the number of {@link ScanResultValue}
+   * passed.
+   */
+  @Override
+  public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
+      final ScanQuery query,
+      final Sequence<ScanResultValue> resultSequence,
+      @Nullable Long memoryLimitBytes
+  )
+  {
+    final AtomicLong memoryLimitAccumulator = memoryLimitBytes != null ? new AtomicLong(memoryLimitBytes) : null;
+    Iterator<ScanResultValue> resultSequenceIterator = new Iterator<ScanResultValue>()
+    {
+      Yielder<ScanResultValue> yielder = Yielders.each(resultSequence);
+
+      @Override
+      public boolean hasNext()
+      {
+        return !yielder.isDone();
+      }
+
+      @Override
+      public ScanResultValue next()
+      {
+        ScanResultValue scanResultValue = yielder.get();
+        yielder = yielder.next(null);
+        return scanResultValue;
+      }
+    };
+
+    Iterable<FrameSignaturePair> retVal = () -> new Iterator<FrameSignaturePair>()
+    {
+      PeekingIterator<ScanResultValue> scanResultValuePeekingIterator = Iterators.peekingIterator(resultSequenceIterator);
+
+      @Override
+      public boolean hasNext()
+      {
+        return scanResultValuePeekingIterator.hasNext();
+      }
+
+      @Override
+      public FrameSignaturePair next()
+      {
+        final List<ScanResultValue> batch = new ArrayList<>();
+        final ScanResultValue scanResultValue = scanResultValuePeekingIterator.next();
+        batch.add(scanResultValue);
+        final RowSignature rowSignature = scanResultValue.getRowSignature();
+        while (scanResultValuePeekingIterator.hasNext()) {
+          final RowSignature nextRowSignature = scanResultValuePeekingIterator.peek().getRowSignature();
+          if (nextRowSignature != null && nextRowSignature.equals(rowSignature)) {
+            batch.add(scanResultValuePeekingIterator.next());
+          }

Review Comment:
   Please add tests cases in `ScanQueryQueryToolChestTest` for this method. 



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1193409958


##########
processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java:
##########
@@ -52,6 +54,8 @@ public DruidDefaultSerializersModule()
 
     JodaStuff.register(this);
 
+    addSerializer(FramesBackedInlineDataSource.class, new FramesBackedInlineDataSourceSerializer());

Review Comment:
   This serialization is required when the broker inlines the subquery results and sends the inlined query to the historicals. In that case, we serialize the Frames and FrameBasedInlineDatasource to behave as if an equivalent InlineDatasource (based on rows) would have been serialized. 
   



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1142094611


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -565,10 +616,14 @@ private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSour
       final Sequence<T> results,
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,

Review Comment:
   I am not sure of the use case. Theoretically, we can pass both the limits and error the query out if any one of them is reached. Is that the behavior we can encourage or do we want the user to give only one of the limit? 



-- 
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: commits-unsubscribe@druid.apache.org

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] github-code-scanning[bot] commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1153178104


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +623,47 @@
       );
     }
 
-    final RowSignature signature = toolChest.resultArraySignature(query);
-
-    final ArrayList<Object[]> resultList = new ArrayList<>();
-
-    toolChest.resultsAsArrays(query, results).accumulate(
-        resultList,
-        (acc, in) -> {
-          if (limitAccumulator.getAndIncrement() >= limitToUse) {
-            throw ResourceLimitExceededException.withMessage(
-                "Subquery generated results beyond maximum[%d]",
-                limitToUse
-            );
-          }
-          acc.add(in);
-          return acc;
+    if (memoryLimitAccumulator.get() >= memoryLimitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
+          memoryLimitToUse
+      );
+    }
+
+    final List<Object[]> resultList = toolChest.resultsAsArrays(query, results).toList();
+
+    if (limitAccumulator.addAndGet(resultList.size()) >= limitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Subquery generated results beyond maximum[%d] rows",
+          limitToUse
+      );
+    }
+
+    FramesBackedInlineDataSource dataSource = null;
+    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
+    if (memoryLimitSet || true) {

Review Comment:
   ## Expression always evaluates to the same value
   
   Expression always evaluates to the same value.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4491)



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -577,25 +623,47 @@
       );
     }
 
-    final RowSignature signature = toolChest.resultArraySignature(query);
-
-    final ArrayList<Object[]> resultList = new ArrayList<>();
-
-    toolChest.resultsAsArrays(query, results).accumulate(
-        resultList,
-        (acc, in) -> {
-          if (limitAccumulator.getAndIncrement() >= limitToUse) {
-            throw ResourceLimitExceededException.withMessage(
-                "Subquery generated results beyond maximum[%d]",
-                limitToUse
-            );
-          }
-          acc.add(in);
-          return acc;
+    if (memoryLimitAccumulator.get() >= memoryLimitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
+          memoryLimitToUse
+      );
+    }
+
+    final List<Object[]> resultList = toolChest.resultsAsArrays(query, results).toList();
+
+    if (limitAccumulator.addAndGet(resultList.size()) >= limitToUse) {
+      throw ResourceLimitExceededException.withMessage(
+          "Subquery generated results beyond maximum[%d] rows",
+          limitToUse
+      );
+    }
+
+    FramesBackedInlineDataSource dataSource = null;
+    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
+    if (memoryLimitSet || true) {
+      try {
+        Sequence<FrameSignaturePair> frames = toolChest.resultsAsFrames(query, results);
+        Long memoryUsed = frames.accumulate(0L, ((accumulated, in) -> accumulated + in.getFrame().numBytes()));

Review Comment:
   ## User-controlled data in arithmetic expression
   
   This arithmetic expression depends on a [user-provided value](1), potentially causing an overflow.
   This arithmetic expression depends on a [user-provided value](2), potentially causing an overflow.
   This arithmetic expression depends on a [user-provided value](3), potentially causing an overflow.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4499)



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1157183209


##########
processing/src/main/java/org/apache/druid/frame/field/FieldWriters.java:
##########
@@ -57,10 +58,14 @@ private FieldWriters()
   public static FieldWriter create(
       final ColumnSelectorFactory columnSelectorFactory,
       final String columnName,
-      final ColumnType columnType
+      final ColumnType columnType,
+      final boolean allowNullColumnType

Review Comment:
   We should mention the null handling in the java doc



##########
processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java:
##########
@@ -52,6 +54,8 @@ public DruidDefaultSerializersModule()
 
     JodaStuff.register(this);
 
+    addSerializer(FramesBackedInlineDataSource.class, new FramesBackedInlineDataSourceSerializer());

Review Comment:
   Can you let us know the reason why do you think adding the serializer here makes sense. 
   



##########
processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java:
##########
@@ -451,6 +462,49 @@ public Sequence<Object[]> resultsAsArrays(
     );
   }
 
+  @Override
+  public boolean canFetchResultsAsFrames()
+  {
+    return true;
+  }
+
+  @Override
+  public Sequence<FrameSignaturePair> resultsAsFrames(
+      TimeseriesQuery query,
+      Sequence<Result<TimeseriesResultValue>> resultSequence
+  )
+  {
+    final RowSignature rowSignature = resultArraySignature(query);

Review Comment:
   Similar memory logic trick here as discussed in the scan query tool chest



##########
processing/src/main/java/org/apache/druid/frame/field/FieldWriters.java:
##########
@@ -57,10 +58,14 @@ private FieldWriters()
   public static FieldWriter create(
       final ColumnSelectorFactory columnSelectorFactory,
       final String columnName,
-      final ColumnType columnType
+      final ColumnType columnType,
+      final boolean allowNullColumnType
   )
   {
     if (columnType == null) {
+      if (allowNullColumnType) {
+        return makeComplexWriter(columnSelectorFactory, columnName, NestedDataComplexTypeSerde.TYPE_NAME);

Review Comment:
   ```suggestion
           // returning complex writer since we do not know the type of column.
           return makeComplexWriter(columnSelectorFactory, columnName, NestedDataComplexTypeSerde.TYPE_NAME);
   ```



##########
processing/src/main/java/org/apache/druid/frame/read/FrameReader.java:
##########
@@ -85,12 +86,17 @@ public static FrameReader create(final RowSignature signature)
     final List<FieldReader> fieldReaders = new ArrayList<>(signature.size());
 
     for (int columnNumber = 0; columnNumber < signature.size(); columnNumber++) {
-      final ColumnType columnType =
-          Preconditions.checkNotNull(
-              signature.getColumnType(columnNumber).orElse(null),
-              "Type for column [%s]",
-              signature.getColumnName(columnNumber)
-          );
+      ColumnType columnType;
+      if (!allowNullTypes) {

Review Comment:
   Same lets mention the change of semantics when nullTypes are allowed in the java doc. 



##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -187,11 +199,75 @@ public RowSignature resultArraySignature(final ScanQuery query)
     }
   }
 
+  @Override
+  public boolean canFetchResultsAsFrames()
+  {
+    return true;
+  }
+
+  @Override
+  public Sequence<FrameSignaturePair> resultsAsFrames(
+      final ScanQuery query,
+      final Sequence<ScanResultValue> resultSequence
+  )
+  {
+    return resultSequence.map(
+        result -> {
+          final List rows = (List) result.getEvents();
+          final Function<?, Object[]> mapper = getResultFormatMapper(query);
+          final Iterable<Object[]> formattedRows = Iterables.transform(rows, (Function) mapper);
+
+          RowBasedCursor cursor = IterableRowsCursorHelper.getCursorFromIterable(
+              formattedRows,
+              result.getRowSignature()
+          );
+
+          FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
+              FrameType.ROW_BASED,
+              new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+              result.getRowSignature(),
+              new ArrayList<>(),
+              true
+          );
+
+          Frame frame;
+
+          try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) {
+            while (!cursor.isDone()) {
+              if (!frameWriter.addSelection()) {
+                throw new FrameRowTooLargeException(frameWriterFactory.allocatorCapacity());
+              }
+
+              cursor.advance();
+            }
+
+            frame = Frame.wrap(frameWriter.toByteArray());
+          }
+
+          return new FrameSignaturePair(frame, result.getRowSignature());

Review Comment:
   Since we can have different row sigs per segment, what we can do is only start a new frame when the row signature is different. This will reduce the number of frames by a lot. 



##########
processing/src/main/java/org/apache/druid/query/QueryToolChest.java:
##########
@@ -330,4 +330,29 @@ public Sequence<Object[]> resultsAsArrays(QueryType query, Sequence<ResultType>
   {
     throw new UOE("Query type '%s' does not support returning results as arrays", query.getType());
   }
+
+  /**
+   * Converts a sequence of this query's ResultType into a sequence of {@link FrameSignaturePair}. The array signature
+   * is the one give by {@link #resultArraySignature(Query)}.
+   *
+   * Check documentation of {@link #resultsAsArrays(Query, Sequence)} as the behaviour of the rows represented by the
+   * frame sequence is identical.
+   *
+   * Each Frame has a separate {@link RowSignature} because for some query types like the Scan query, every
+   * column in the final result might not be present in the individual ResultType (and subsequently Frame). Therefore,
+   * this is done to preserve the space by not populating the column in that particular Frame and omitting it from its
+   * signature
+   */
+  public Sequence<FrameSignaturePair> resultsAsFrames(QueryType query, Sequence<ResultType> resultSequence)

Review Comment:
   Can we return an Optional<Sequence> and remove the `canFetchResultsAsFrames` call from the interface ?



##########
processing/src/main/java/org/apache/druid/frame/write/FrameWriters.java:
##########
@@ -58,19 +58,30 @@ public static FrameWriterFactory makeFrameWriterFactory(
       final FrameType frameType,
       final MemoryAllocatorFactory allocatorFactory,
       final RowSignature signature,
-      final List<KeyColumn> sortColumns
+      final List<KeyColumn> sortColumns,
+      final boolean allowNullColumnTypes
   )
   {
     switch (Preconditions.checkNotNull(frameType, "frameType")) {
       case COLUMNAR:
-        return new ColumnarFrameWriterFactory(allocatorFactory, signature, sortColumns);
+        return new ColumnarFrameWriterFactory(allocatorFactory, signature, sortColumns, allowNullColumnTypes);
       case ROW_BASED:
-        return new RowBasedFrameWriterFactory(allocatorFactory, signature, sortColumns);
+        return new RowBasedFrameWriterFactory(allocatorFactory, signature, sortColumns, allowNullColumnTypes);
       default:
         throw new ISE("Unrecognized frame type [%s]", frameType);
     }
   }
 
+  public static FrameWriterFactory makeFrameWriterFactory(

Review Comment:
   This method seems weird. You can call the base method directly. 



##########
processing/src/main/java/org/apache/druid/query/FramesBackedInlineDataSourceSerializer.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.jsontype.TypeSerializer;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import org.apache.druid.java.util.common.jackson.JacksonUtils;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * Serializes {@link FramesBackedInlineDataSource} to the representation of {@link IterableBackedInlineDataSource}
+ * so that the servers' on wire transfer data doesn't change. {@link FramesBackedInlineDataSource} is currently limited
+ * to the brokers only and therefore this aids in conversion of the object to a representation that the data servers
+ * can recognize
+ */
+public class FramesBackedInlineDataSourceSerializer extends StdSerializer<FramesBackedInlineDataSource>
+{
+  public FramesBackedInlineDataSourceSerializer()
+  {
+    super(FramesBackedInlineDataSource.class);
+  }
+
+  @Override
+  public void serialize(FramesBackedInlineDataSource value, JsonGenerator jg, SerializerProvider serializers)
+      throws IOException
+  {
+    jg.writeStartObject();
+    jg.writeStringField("type", "inline");
+
+    RowSignature rowSignature = value.getRowSignature();
+    jg.writeObjectField("columnNames", rowSignature.getColumnNames());
+    List<ColumnType> columnTypes = IntStream.range(0, rowSignature.size())
+                                            .mapToObj(i -> rowSignature.getColumnType(i).orElse(null))
+                                            .collect(Collectors.toList());
+    jg.writeObjectField("columnTypes", columnTypes);
+
+    jg.writeArrayFieldStart("rows");
+
+    value.getRowsAsSequence().forEach(row -> {
+      try {
+        JacksonUtils.writeObjectUsingSerializerProvider(jg, serializers, row);
+      }
+      catch (IOException e) {
+        // Do nothing, shouldn't be possible for now

Review Comment:
   I think we should propagate the exceptions forward.



##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -187,11 +199,75 @@ public RowSignature resultArraySignature(final ScanQuery query)
     }
   }
 
+  @Override
+  public boolean canFetchResultsAsFrames()
+  {
+    return true;
+  }
+
+  @Override
+  public Sequence<FrameSignaturePair> resultsAsFrames(
+      final ScanQuery query,
+      final Sequence<ScanResultValue> resultSequence
+  )
+  {
+    return resultSequence.map(

Review Comment:
   We should get the memory limit on line 213 and pass initialize a remaining quota variable. 
   The remaining quota should be passed to 227 and then what ever the frame uses should be subtracted from the reaming quota variable in line 246. 
   
   This way we will limit the frame generation whenever the memory limit is hit. 



##########
processing/src/main/java/org/apache/druid/query/FramesBackedInlineDataSource.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.query;
+
+import org.apache.druid.frame.Frame;
+import org.apache.druid.frame.read.FrameReader;
+import org.apache.druid.frame.segment.FrameStorageAdapter;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.planning.DataSourceAnalysis;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.SegmentReference;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.RowSignature;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Represents an inline datasource where the rows are embedded within the DataSource object itself.
+ *
+ * The rows are backed by a sequence of {@link FrameSignaturePair}, which contain the Frame representation of the rows
+ * represented by the datasource. {@link #getRowsAsList()} and {@link #getRowsAsSequence()} return the iterables which
+ * read the rows that are encapsulated in the frames.
+ *
+ * Note that the signature of the datasource can be different from the signatures of the constituent frames that it
+ * consists of. While fetching the iterables, it is the job of this class to make sure that the rows correspond to the
+ * {@link #rowSignature}. For frames that donot contain the columns present in the {@link #rowSignature}, they are
+ * populated with {@code null}.
+ */
+public class FramesBackedInlineDataSource implements DataSource
+{
+
+  final Sequence<FrameSignaturePair> frames;
+  final RowSignature rowSignature;
+
+  public FramesBackedInlineDataSource(
+      Sequence<FrameSignaturePair> frames,
+      RowSignature rowSignature
+  )
+  {
+    this.frames = frames;
+    this.rowSignature = rowSignature;
+  }
+
+  public Sequence<FrameSignaturePair> getFrames()
+  {
+    return frames;
+  }
+
+  public RowSignature getRowSignature()
+  {
+    return rowSignature;
+  }
+
+  public List<Object[]> getRowsAsList()

Review Comment:
   Can this be a private method ?



##########
processing/src/main/java/org/apache/druid/query/FramesBackedInlineDataSource.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.query;
+
+import org.apache.druid.frame.Frame;
+import org.apache.druid.frame.read.FrameReader;
+import org.apache.druid.frame.segment.FrameStorageAdapter;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.query.planning.DataSourceAnalysis;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.SegmentReference;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.RowSignature;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Represents an inline datasource where the rows are embedded within the DataSource object itself.
+ *
+ * The rows are backed by a sequence of {@link FrameSignaturePair}, which contain the Frame representation of the rows
+ * represented by the datasource. {@link #getRowsAsList()} and {@link #getRowsAsSequence()} return the iterables which
+ * read the rows that are encapsulated in the frames.
+ *
+ * Note that the signature of the datasource can be different from the signatures of the constituent frames that it
+ * consists of. While fetching the iterables, it is the job of this class to make sure that the rows correspond to the
+ * {@link #rowSignature}. For frames that donot contain the columns present in the {@link #rowSignature}, they are
+ * populated with {@code null}.
+ */
+public class FramesBackedInlineDataSource implements DataSource
+{
+
+  final Sequence<FrameSignaturePair> frames;
+  final RowSignature rowSignature;
+
+  public FramesBackedInlineDataSource(
+      Sequence<FrameSignaturePair> frames,
+      RowSignature rowSignature
+  )
+  {
+    this.frames = frames;
+    this.rowSignature = rowSignature;
+  }
+
+  public Sequence<FrameSignaturePair> getFrames()
+  {
+    return frames;
+  }
+
+  public RowSignature getRowSignature()
+  {
+    return rowSignature;
+  }
+
+  public List<Object[]> getRowsAsList()
+  {
+    List<Object[]> frameRows = new ArrayList<>();
+
+    final Sequence<Cursor> cursorSequence = frames
+        .flatMap(
+            frameSignaturePair -> {

Review Comment:
   We can extract this to a method since getRowsAsSequence also uses the same code



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -163,12 +169,15 @@ 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.getMaxSubqueryMemory());

Review Comment:
   Lets undocument the server property since it will enable all q's running with the new query path. 
   Also we should benchmark the new query path. 



##########
processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java:
##########
@@ -548,6 +561,46 @@ public Sequence<Object[]> resultsAsArrays(TopNQuery query, Sequence<Result<TopNR
     );
   }
 
+  @Override
+  public boolean canFetchResultsAsFrames()
+  {
+    return true;
+  }
+
+  @Override
+  public Sequence<FrameSignaturePair> resultsAsFrames(TopNQuery query, Sequence<Result<TopNResultValue>> resultSequence)
+  {
+    final RowSignature rowSignature = resultArraySignature(query);

Review Comment:
   Also mention the number of frames returned in each query type. 



##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -187,11 +199,75 @@ public RowSignature resultArraySignature(final ScanQuery query)
     }
   }
 
+  @Override
+  public boolean canFetchResultsAsFrames()
+  {
+    return true;
+  }
+
+  @Override
+  public Sequence<FrameSignaturePair> resultsAsFrames(
+      final ScanQuery query,
+      final Sequence<ScanResultValue> resultSequence
+  )
+  {
+    return resultSequence.map(
+        result -> {
+          final List rows = (List) result.getEvents();
+          final Function<?, Object[]> mapper = getResultFormatMapper(query);
+          final Iterable<Object[]> formattedRows = Iterables.transform(rows, (Function) mapper);
+
+          RowBasedCursor cursor = IterableRowsCursorHelper.getCursorFromIterable(
+              formattedRows,
+              result.getRowSignature()
+          );
+
+          FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
+              FrameType.ROW_BASED,
+              new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+              result.getRowSignature(),
+              new ArrayList<>(),
+              true
+          );
+
+          Frame frame;
+
+          try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) {
+            while (!cursor.isDone()) {
+              if (!frameWriter.addSelection()) {
+                throw new FrameRowTooLargeException(frameWriterFactory.allocatorCapacity());
+              }
+
+              cursor.advance();
+            }
+
+            frame = Frame.wrap(frameWriter.toByteArray());
+          }
+
+          return new FrameSignaturePair(frame, result.getRowSignature());

Review Comment:
   We are making one frame per result sequence and each result sequence represents one segment. This does not seem very scalable. 
   Lets leave a note here so that we can get back to this in a future PR. 



-- 
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: commits-unsubscribe@druid.apache.org

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] github-code-scanning[bot] commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1163674379


##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java:
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.sql.calcite;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.UOE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.JoinDataSource;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.ResourceLimitExceededException;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
+import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.ExtractionDimensionSpec;
+import org.apache.druid.query.extraction.SubstringDimExtractionFn;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.ordering.StringComparators;
+import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.topn.DimensionTopNMetricSpec;
+import org.apache.druid.query.topn.TopNQueryBuilder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinType;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Period;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Calcite tests which involve subqueries and materializing the intermediate results on {@link org.apache.druid.server.ClientQuerySegmentWalker}
+ * The tests are run with two different
+ */
+@RunWith(Parameterized.class)
+public class CalciteSubqueryTest extends BaseCalciteQueryTest
+{
+
+  public String testName;
+  public Map<String, Object> queryContext;
+
+  public CalciteSubqueryTest(
+      String testName,
+      Map<String, Object> queryContext
+  )
+  {
+    this.testName = testName;
+    this.queryContext = queryContext;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Iterable<Object[]> constructorFeeder()
+  {
+    final List<Object[]> constructors = new ArrayList<>();
+    constructors.add(
+        new Object[]{"without memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    constructors.add(
+        new Object[]{"with memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    return constructors;
+  }
+
+  @Test
+  public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter()
+  {
+    msqCompatible();
+    // Cannot vectorize topN operator.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT\n"
+        + "  SUM(cnt),\n"
+        + "  COUNT(*)\n"
+        + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
+        + "WHERE cnt > 0",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                new TopNQueryBuilder()
+                                    .dataSource(CalciteTests.DATASOURCE1)
+                                    .intervals(querySegmentSpec(Filtration.eternity()))
+                                    .granularity(Granularities.ALL)
+                                    .dimension(new DefaultDimensionSpec("dim2", "d0"))
+                                    .aggregators(new LongSumAggregatorFactory("a0", "cnt"))
+                                    .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
+                                    .threshold(1)
+                                    .build()
+                            )
+                        )
+                        .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(
+                            new LongSumAggregatorFactory("_a0", "a0"),
+                            new CountAggregatorFactory("_a1")
+                        ))
+                        .setContext(queryContext)
+                        .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{3L, 1L}
+        ) :
+        ImmutableList.of(
+            new Object[]{2L, 1L}
+        )
+    );
+  }
+
+  @Test
+  public void testExactCountDistinctOfSemiJoinResult()
+  {
+    msqCompatible();
+    // Cannot vectorize due to extraction dimension spec.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT COUNT(*)\n"
+        + "FROM (\n"
+        + "  SELECT DISTINCT dim2\n"
+        + "  FROM druid.foo\n"
+        + "  WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+        + "    SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+        + "  ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n"
+        + ")",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                GroupByQuery.builder()
+                                            .setDataSource(
+                                                join(
+                                                    new TableDataSource(CalciteTests.DATASOURCE1),
+                                                    new QueryDataSource(
+                                                        GroupByQuery
+                                                            .builder()
+                                                            .setDataSource(CalciteTests.DATASOURCE1)
+                                                            .setInterval(querySegmentSpec(Filtration.eternity()))
+                                                            .setGranularity(Granularities.ALL)
+                                                            .setDimFilter(not(selector("dim1", "", null)))
+                                                            .setDimensions(
+                                                                dimensions(
+                                                                    new ExtractionDimensionSpec(
+                                                                        "dim1",
+                                                                        "d0",
+                                                                        new SubstringDimExtractionFn(0, 1)
+                                                                    )
+                                                                )
+                                                            )
+                                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                                            .build()
+                                                    ),
+                                                    "j0.",
+                                                    equalsCondition(
+                                                        makeExpression("substring(\"dim2\", 0, 1)"),
+                                                        DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")
+                                                    ),
+                                                    JoinType.INNER
+                                                )
+                                            )
+                                            .setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
+                                            .setGranularity(Granularities.ALL)
+                                            .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                            .build()
+                            )
+                        )
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{2L}
+        )
+    );
+  }
+
+  @Test
+  public void testTwoExactCountDistincts()
+  {
+    requireMergeBuffers(6);
+    msqCompatible();
+    testQuery(
+        PLANNER_CONFIG_NO_HLL,
+        queryContext,
+        "SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo",
+        CalciteTests.REGULAR_USER_AUTH_RESULT,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        "j0.",
+                        "1",
+                        JoinType.INNER
+                    )
+                )
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("a0", "j0.a0")
+                .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+        ),
+        ImmutableList.of(
+            new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L}
+        )
+    );
+  }
+
+  @Test
+  public void testViewAndJoin()
+  {
+    cannotVectorize();
+    Map<String, Object> queryContextModified = withLeftDirectAccessEnabled(queryContext);
+    testQuery(
+        "SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ",
+        queryContextModified,
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(
+                      join(
+                          join(
+                              new TableDataSource(CalciteTests.DATASOURCE1),
+                              new QueryDataSource(
+                                  newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3)
+                                                       .intervals(querySegmentSpec(Filtration.eternity()))
+                                                       .columns("dim2")
+                                                       .context(queryContextModified)
+                                                       .build()
+                              ),
+                              "j0.",
+                              "(\"dim2\" == \"j0.dim2\")",
+                              JoinType.INNER,
+                              bound("dim2", "a", "a", false, false, null, null)
+                          ),
+                          new QueryDataSource(
+                              newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1)
+                                                   .intervals(querySegmentSpec(Filtration.eternity()))
+                                                   .columns("dim2")
+                                                   .context(queryContextModified)
+                                                   .build()
+                          ),
+                          "_j0.",
+                          "('a' == \"_j0.dim2\")",
+                          JoinType.INNER
+                      )
+                  )
+                  .intervals(querySegmentSpec(Filtration.eternity()))
+                  .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))))
+                  .granularity(Granularities.ALL)
+                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                  .context(queryContextModified)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{8L}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries()
+  {
+    msqCompatible();
+    cannotVectorize();
+
+    testQuery(
+        "SELECT TIME_FORMAT(\"date\", 'yyyy-MM'), SUM(x)\n"
+        + "FROM (\n"
+        + "    SELECT\n"
+        + "        FLOOR(__time to hour) as \"date\",\n"
+        + "        COUNT(*) as x\n"
+        + "    FROM foo\n"
+        + "    GROUP BY 1\n"
+        + ")\n"
+        + "GROUP BY 1",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            Druids.newTimeseriesQueryBuilder()
+                                  .dataSource(CalciteTests.DATASOURCE1)
+                                  .intervals(querySegmentSpec(Filtration.eternity()))
+                                  .granularity(Granularities.HOUR)
+                                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
+                                  .build()
+                        )
+                        .setInterval(querySegmentSpec(Intervals.ETERNITY))
+                        .setVirtualColumns(
+                            expressionVirtualColumn(
+                                "v0",
+                                "timestamp_format(\"d0\",'yyyy-MM','UTC')",
+                                ColumnType.STRING
+                            )
+                        )
+                        .setGranularity(Granularities.ALL)
+                        .addDimension(new DefaultDimensionSpec("v0", "_d0"))
+                        .addAggregator(new LongSumAggregatorFactory("_a0", "a0"))
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"2000-01", 3L},
+            new Object[]{"2001-01", 3L}
+        )
+    );
+  }
+
+  @Test
+  public void testUsingSubqueryAsFilterWithInnerSort()
+  {
+    // Regression test for https://github.com/apache/druid/issues/4208
+
+    testQuery(
+        "SELECT dim1, dim2 FROM druid.foo\n"
+        + " WHERE dim2 IN (\n"
+        + "   SELECT dim2\n"
+        + "   FROM druid.foo\n"
+        + "   GROUP BY dim2\n"
+        + "   ORDER BY dim2 DESC\n"
+        + " )",
+        queryContext,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new TableDataSource(CalciteTests.DATASOURCE1),
+                        new QueryDataSource(
+                            GroupByQuery.builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                        ),
+                        "j0.",
+                        equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.d0")),
+                        JoinType.INNER
+                    )
+                )
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("dim1", "dim2")
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{"", "a"},
+            new Object[]{"1", "a"},
+            new Object[]{"def", "abc"}
+        ) :
+        ImmutableList.of(
+            new Object[]{"", "a"},
+            new Object[]{"2", ""},
+            new Object[]{"1", "a"},
+            new Object[]{"def", "abc"}
+        )
+    );
+  }
+
+  @Test
+  public void testUsingSubqueryAsFilterOnTwoColumns()
+  {
+    msqCompatible();
+    testQuery(
+        "SELECT __time, cnt, dim1, dim2 FROM druid.foo "
+        + " WHERE (dim1, dim2) IN ("
+        + "   SELECT dim1, dim2 FROM ("
+        + "     SELECT dim1, dim2, COUNT(*)"
+        + "     FROM druid.foo"
+        + "     WHERE dim2 = 'abc'"
+        + "     GROUP BY dim1, dim2"
+        + "     HAVING COUNT(*) = 1"
+        + "   )"
+        + " )",
+        queryContext,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new TableDataSource(CalciteTests.DATASOURCE1),
+                        new QueryDataSource(
+                            GroupByQuery.builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimFilter(selector("dim2", "abc", null))
+                                        .setDimensions(dimensions(
+                                            new DefaultDimensionSpec("dim1", "d0"),
+                                            new DefaultDimensionSpec("dim2", "d1")
+                                        ))
+                                        .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+                                        .setPostAggregatorSpecs(
+                                            ImmutableList.of(expressionPostAgg("p0", "'abc'"))
+                                        )
+                                        .setHavingSpec(having(selector("a0", "1", null)))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                        ),
+                        "j0.",
+                        StringUtils.format(
+                            "(%s && %s)",
+                            equalsCondition(makeColumnExpression("dim1"), makeColumnExpression("j0.d0")),
+                            equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.p0"))

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [CalciteTestBase.makeColumnExpression](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4786)



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java:
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.sql.calcite;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.UOE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.JoinDataSource;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.ResourceLimitExceededException;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
+import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.ExtractionDimensionSpec;
+import org.apache.druid.query.extraction.SubstringDimExtractionFn;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.ordering.StringComparators;
+import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.topn.DimensionTopNMetricSpec;
+import org.apache.druid.query.topn.TopNQueryBuilder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinType;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Period;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Calcite tests which involve subqueries and materializing the intermediate results on {@link org.apache.druid.server.ClientQuerySegmentWalker}
+ * The tests are run with two different
+ */
+@RunWith(Parameterized.class)
+public class CalciteSubqueryTest extends BaseCalciteQueryTest
+{
+
+  public String testName;
+  public Map<String, Object> queryContext;
+
+  public CalciteSubqueryTest(
+      String testName,
+      Map<String, Object> queryContext
+  )
+  {
+    this.testName = testName;
+    this.queryContext = queryContext;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Iterable<Object[]> constructorFeeder()
+  {
+    final List<Object[]> constructors = new ArrayList<>();
+    constructors.add(
+        new Object[]{"without memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    constructors.add(
+        new Object[]{"with memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    return constructors;
+  }
+
+  @Test
+  public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter()
+  {
+    msqCompatible();
+    // Cannot vectorize topN operator.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT\n"
+        + "  SUM(cnt),\n"
+        + "  COUNT(*)\n"
+        + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
+        + "WHERE cnt > 0",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                new TopNQueryBuilder()
+                                    .dataSource(CalciteTests.DATASOURCE1)
+                                    .intervals(querySegmentSpec(Filtration.eternity()))
+                                    .granularity(Granularities.ALL)
+                                    .dimension(new DefaultDimensionSpec("dim2", "d0"))
+                                    .aggregators(new LongSumAggregatorFactory("a0", "cnt"))
+                                    .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
+                                    .threshold(1)
+                                    .build()
+                            )
+                        )
+                        .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(
+                            new LongSumAggregatorFactory("_a0", "a0"),
+                            new CountAggregatorFactory("_a1")
+                        ))
+                        .setContext(queryContext)
+                        .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{3L, 1L}
+        ) :
+        ImmutableList.of(
+            new Object[]{2L, 1L}
+        )
+    );
+  }
+
+  @Test
+  public void testExactCountDistinctOfSemiJoinResult()
+  {
+    msqCompatible();
+    // Cannot vectorize due to extraction dimension spec.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT COUNT(*)\n"
+        + "FROM (\n"
+        + "  SELECT DISTINCT dim2\n"
+        + "  FROM druid.foo\n"
+        + "  WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+        + "    SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+        + "  ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n"
+        + ")",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                GroupByQuery.builder()
+                                            .setDataSource(
+                                                join(
+                                                    new TableDataSource(CalciteTests.DATASOURCE1),
+                                                    new QueryDataSource(
+                                                        GroupByQuery
+                                                            .builder()
+                                                            .setDataSource(CalciteTests.DATASOURCE1)
+                                                            .setInterval(querySegmentSpec(Filtration.eternity()))
+                                                            .setGranularity(Granularities.ALL)
+                                                            .setDimFilter(not(selector("dim1", "", null)))
+                                                            .setDimensions(
+                                                                dimensions(
+                                                                    new ExtractionDimensionSpec(
+                                                                        "dim1",
+                                                                        "d0",
+                                                                        new SubstringDimExtractionFn(0, 1)
+                                                                    )
+                                                                )
+                                                            )
+                                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                                            .build()
+                                                    ),
+                                                    "j0.",
+                                                    equalsCondition(
+                                                        makeExpression("substring(\"dim2\", 0, 1)"),

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [CalciteTestBase.makeExpression](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4781)



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java:
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.sql.calcite;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.UOE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.JoinDataSource;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.ResourceLimitExceededException;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
+import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.ExtractionDimensionSpec;
+import org.apache.druid.query.extraction.SubstringDimExtractionFn;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.ordering.StringComparators;
+import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.topn.DimensionTopNMetricSpec;
+import org.apache.druid.query.topn.TopNQueryBuilder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinType;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Period;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Calcite tests which involve subqueries and materializing the intermediate results on {@link org.apache.druid.server.ClientQuerySegmentWalker}
+ * The tests are run with two different
+ */
+@RunWith(Parameterized.class)
+public class CalciteSubqueryTest extends BaseCalciteQueryTest
+{
+
+  public String testName;
+  public Map<String, Object> queryContext;
+
+  public CalciteSubqueryTest(
+      String testName,
+      Map<String, Object> queryContext
+  )
+  {
+    this.testName = testName;
+    this.queryContext = queryContext;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Iterable<Object[]> constructorFeeder()
+  {
+    final List<Object[]> constructors = new ArrayList<>();
+    constructors.add(
+        new Object[]{"without memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    constructors.add(
+        new Object[]{"with memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    return constructors;
+  }
+
+  @Test
+  public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter()
+  {
+    msqCompatible();
+    // Cannot vectorize topN operator.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT\n"
+        + "  SUM(cnt),\n"
+        + "  COUNT(*)\n"
+        + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
+        + "WHERE cnt > 0",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                new TopNQueryBuilder()
+                                    .dataSource(CalciteTests.DATASOURCE1)
+                                    .intervals(querySegmentSpec(Filtration.eternity()))
+                                    .granularity(Granularities.ALL)
+                                    .dimension(new DefaultDimensionSpec("dim2", "d0"))
+                                    .aggregators(new LongSumAggregatorFactory("a0", "cnt"))
+                                    .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
+                                    .threshold(1)
+                                    .build()
+                            )
+                        )
+                        .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(
+                            new LongSumAggregatorFactory("_a0", "a0"),
+                            new CountAggregatorFactory("_a1")
+                        ))
+                        .setContext(queryContext)
+                        .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{3L, 1L}
+        ) :
+        ImmutableList.of(
+            new Object[]{2L, 1L}
+        )
+    );
+  }
+
+  @Test
+  public void testExactCountDistinctOfSemiJoinResult()
+  {
+    msqCompatible();
+    // Cannot vectorize due to extraction dimension spec.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT COUNT(*)\n"
+        + "FROM (\n"
+        + "  SELECT DISTINCT dim2\n"
+        + "  FROM druid.foo\n"
+        + "  WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+        + "    SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+        + "  ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n"
+        + ")",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                GroupByQuery.builder()
+                                            .setDataSource(
+                                                join(
+                                                    new TableDataSource(CalciteTests.DATASOURCE1),
+                                                    new QueryDataSource(
+                                                        GroupByQuery
+                                                            .builder()
+                                                            .setDataSource(CalciteTests.DATASOURCE1)
+                                                            .setInterval(querySegmentSpec(Filtration.eternity()))
+                                                            .setGranularity(Granularities.ALL)
+                                                            .setDimFilter(not(selector("dim1", "", null)))
+                                                            .setDimensions(
+                                                                dimensions(
+                                                                    new ExtractionDimensionSpec(
+                                                                        "dim1",
+                                                                        "d0",
+                                                                        new SubstringDimExtractionFn(0, 1)
+                                                                    )
+                                                                )
+                                                            )
+                                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                                            .build()
+                                                    ),
+                                                    "j0.",
+                                                    equalsCondition(
+                                                        makeExpression("substring(\"dim2\", 0, 1)"),
+                                                        DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")
+                                                    ),
+                                                    JoinType.INNER
+                                                )
+                                            )
+                                            .setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
+                                            .setGranularity(Granularities.ALL)
+                                            .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                            .build()
+                            )
+                        )
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{2L}
+        )
+    );
+  }
+
+  @Test
+  public void testTwoExactCountDistincts()
+  {
+    requireMergeBuffers(6);
+    msqCompatible();
+    testQuery(
+        PLANNER_CONFIG_NO_HLL,
+        queryContext,
+        "SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo",
+        CalciteTests.REGULAR_USER_AUTH_RESULT,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        "j0.",
+                        "1",
+                        JoinType.INNER
+                    )
+                )
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("a0", "j0.a0")
+                .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+        ),
+        ImmutableList.of(
+            new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L}
+        )
+    );
+  }
+
+  @Test
+  public void testViewAndJoin()
+  {
+    cannotVectorize();
+    Map<String, Object> queryContextModified = withLeftDirectAccessEnabled(queryContext);
+    testQuery(
+        "SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ",
+        queryContextModified,
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(
+                      join(
+                          join(
+                              new TableDataSource(CalciteTests.DATASOURCE1),
+                              new QueryDataSource(
+                                  newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3)
+                                                       .intervals(querySegmentSpec(Filtration.eternity()))
+                                                       .columns("dim2")
+                                                       .context(queryContextModified)
+                                                       .build()
+                              ),
+                              "j0.",
+                              "(\"dim2\" == \"j0.dim2\")",
+                              JoinType.INNER,
+                              bound("dim2", "a", "a", false, false, null, null)
+                          ),
+                          new QueryDataSource(
+                              newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1)
+                                                   .intervals(querySegmentSpec(Filtration.eternity()))
+                                                   .columns("dim2")
+                                                   .context(queryContextModified)
+                                                   .build()
+                          ),
+                          "_j0.",
+                          "('a' == \"_j0.dim2\")",
+                          JoinType.INNER
+                      )
+                  )
+                  .intervals(querySegmentSpec(Filtration.eternity()))
+                  .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))))
+                  .granularity(Granularities.ALL)
+                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                  .context(queryContextModified)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{8L}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries()
+  {
+    msqCompatible();
+    cannotVectorize();
+
+    testQuery(
+        "SELECT TIME_FORMAT(\"date\", 'yyyy-MM'), SUM(x)\n"
+        + "FROM (\n"
+        + "    SELECT\n"
+        + "        FLOOR(__time to hour) as \"date\",\n"
+        + "        COUNT(*) as x\n"
+        + "    FROM foo\n"
+        + "    GROUP BY 1\n"
+        + ")\n"
+        + "GROUP BY 1",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            Druids.newTimeseriesQueryBuilder()
+                                  .dataSource(CalciteTests.DATASOURCE1)
+                                  .intervals(querySegmentSpec(Filtration.eternity()))
+                                  .granularity(Granularities.HOUR)
+                                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
+                                  .build()
+                        )
+                        .setInterval(querySegmentSpec(Intervals.ETERNITY))
+                        .setVirtualColumns(
+                            expressionVirtualColumn(
+                                "v0",
+                                "timestamp_format(\"d0\",'yyyy-MM','UTC')",
+                                ColumnType.STRING
+                            )
+                        )
+                        .setGranularity(Granularities.ALL)
+                        .addDimension(new DefaultDimensionSpec("v0", "_d0"))
+                        .addAggregator(new LongSumAggregatorFactory("_a0", "a0"))
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"2000-01", 3L},
+            new Object[]{"2001-01", 3L}
+        )
+    );
+  }
+
+  @Test
+  public void testUsingSubqueryAsFilterWithInnerSort()
+  {
+    // Regression test for https://github.com/apache/druid/issues/4208
+
+    testQuery(
+        "SELECT dim1, dim2 FROM druid.foo\n"
+        + " WHERE dim2 IN (\n"
+        + "   SELECT dim2\n"
+        + "   FROM druid.foo\n"
+        + "   GROUP BY dim2\n"
+        + "   ORDER BY dim2 DESC\n"
+        + " )",
+        queryContext,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new TableDataSource(CalciteTests.DATASOURCE1),
+                        new QueryDataSource(
+                            GroupByQuery.builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                        ),
+                        "j0.",
+                        equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.d0")),
+                        JoinType.INNER
+                    )
+                )
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("dim1", "dim2")
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{"", "a"},
+            new Object[]{"1", "a"},
+            new Object[]{"def", "abc"}
+        ) :
+        ImmutableList.of(
+            new Object[]{"", "a"},
+            new Object[]{"2", ""},
+            new Object[]{"1", "a"},
+            new Object[]{"def", "abc"}
+        )
+    );
+  }
+
+  @Test
+  public void testUsingSubqueryAsFilterOnTwoColumns()
+  {
+    msqCompatible();
+    testQuery(
+        "SELECT __time, cnt, dim1, dim2 FROM druid.foo "
+        + " WHERE (dim1, dim2) IN ("
+        + "   SELECT dim1, dim2 FROM ("
+        + "     SELECT dim1, dim2, COUNT(*)"
+        + "     FROM druid.foo"
+        + "     WHERE dim2 = 'abc'"
+        + "     GROUP BY dim1, dim2"
+        + "     HAVING COUNT(*) = 1"
+        + "   )"
+        + " )",
+        queryContext,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new TableDataSource(CalciteTests.DATASOURCE1),
+                        new QueryDataSource(
+                            GroupByQuery.builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimFilter(selector("dim2", "abc", null))
+                                        .setDimensions(dimensions(
+                                            new DefaultDimensionSpec("dim1", "d0"),
+                                            new DefaultDimensionSpec("dim2", "d1")
+                                        ))
+                                        .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+                                        .setPostAggregatorSpecs(
+                                            ImmutableList.of(expressionPostAgg("p0", "'abc'"))
+                                        )
+                                        .setHavingSpec(having(selector("a0", "1", null)))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                        ),
+                        "j0.",
+                        StringUtils.format(
+                            "(%s && %s)",
+                            equalsCondition(makeColumnExpression("dim1"), makeColumnExpression("j0.d0")),
+                            equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.p0"))

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [CalciteTestBase.makeColumnExpression](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4787)



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java:
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.sql.calcite;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.UOE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.JoinDataSource;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.ResourceLimitExceededException;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
+import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.ExtractionDimensionSpec;
+import org.apache.druid.query.extraction.SubstringDimExtractionFn;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.ordering.StringComparators;
+import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.topn.DimensionTopNMetricSpec;
+import org.apache.druid.query.topn.TopNQueryBuilder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinType;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Period;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Calcite tests which involve subqueries and materializing the intermediate results on {@link org.apache.druid.server.ClientQuerySegmentWalker}
+ * The tests are run with two different
+ */
+@RunWith(Parameterized.class)
+public class CalciteSubqueryTest extends BaseCalciteQueryTest
+{
+
+  public String testName;
+  public Map<String, Object> queryContext;
+
+  public CalciteSubqueryTest(
+      String testName,
+      Map<String, Object> queryContext
+  )
+  {
+    this.testName = testName;
+    this.queryContext = queryContext;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Iterable<Object[]> constructorFeeder()
+  {
+    final List<Object[]> constructors = new ArrayList<>();
+    constructors.add(
+        new Object[]{"without memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    constructors.add(
+        new Object[]{"with memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    return constructors;
+  }
+
+  @Test
+  public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter()
+  {
+    msqCompatible();
+    // Cannot vectorize topN operator.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT\n"
+        + "  SUM(cnt),\n"
+        + "  COUNT(*)\n"
+        + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
+        + "WHERE cnt > 0",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                new TopNQueryBuilder()
+                                    .dataSource(CalciteTests.DATASOURCE1)
+                                    .intervals(querySegmentSpec(Filtration.eternity()))
+                                    .granularity(Granularities.ALL)
+                                    .dimension(new DefaultDimensionSpec("dim2", "d0"))
+                                    .aggregators(new LongSumAggregatorFactory("a0", "cnt"))
+                                    .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
+                                    .threshold(1)
+                                    .build()
+                            )
+                        )
+                        .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(
+                            new LongSumAggregatorFactory("_a0", "a0"),
+                            new CountAggregatorFactory("_a1")
+                        ))
+                        .setContext(queryContext)
+                        .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{3L, 1L}
+        ) :
+        ImmutableList.of(
+            new Object[]{2L, 1L}
+        )
+    );
+  }
+
+  @Test
+  public void testExactCountDistinctOfSemiJoinResult()
+  {
+    msqCompatible();
+    // Cannot vectorize due to extraction dimension spec.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT COUNT(*)\n"
+        + "FROM (\n"
+        + "  SELECT DISTINCT dim2\n"
+        + "  FROM druid.foo\n"
+        + "  WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+        + "    SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+        + "  ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n"
+        + ")",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                GroupByQuery.builder()
+                                            .setDataSource(
+                                                join(
+                                                    new TableDataSource(CalciteTests.DATASOURCE1),
+                                                    new QueryDataSource(
+                                                        GroupByQuery
+                                                            .builder()
+                                                            .setDataSource(CalciteTests.DATASOURCE1)
+                                                            .setInterval(querySegmentSpec(Filtration.eternity()))
+                                                            .setGranularity(Granularities.ALL)
+                                                            .setDimFilter(not(selector("dim1", "", null)))
+                                                            .setDimensions(
+                                                                dimensions(
+                                                                    new ExtractionDimensionSpec(
+                                                                        "dim1",
+                                                                        "d0",
+                                                                        new SubstringDimExtractionFn(0, 1)
+                                                                    )
+                                                                )
+                                                            )
+                                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                                            .build()
+                                                    ),
+                                                    "j0.",
+                                                    equalsCondition(
+                                                        makeExpression("substring(\"dim2\", 0, 1)"),
+                                                        DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")
+                                                    ),
+                                                    JoinType.INNER
+                                                )
+                                            )
+                                            .setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
+                                            .setGranularity(Granularities.ALL)
+                                            .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                            .build()
+                            )
+                        )
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{2L}
+        )
+    );
+  }
+
+  @Test
+  public void testTwoExactCountDistincts()
+  {
+    requireMergeBuffers(6);
+    msqCompatible();
+    testQuery(
+        PLANNER_CONFIG_NO_HLL,
+        queryContext,
+        "SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo",
+        CalciteTests.REGULAR_USER_AUTH_RESULT,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        "j0.",
+                        "1",
+                        JoinType.INNER
+                    )
+                )
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("a0", "j0.a0")
+                .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+        ),
+        ImmutableList.of(
+            new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L}
+        )
+    );
+  }
+
+  @Test
+  public void testViewAndJoin()
+  {
+    cannotVectorize();
+    Map<String, Object> queryContextModified = withLeftDirectAccessEnabled(queryContext);
+    testQuery(
+        "SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ",
+        queryContextModified,
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(
+                      join(
+                          join(
+                              new TableDataSource(CalciteTests.DATASOURCE1),
+                              new QueryDataSource(
+                                  newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3)
+                                                       .intervals(querySegmentSpec(Filtration.eternity()))
+                                                       .columns("dim2")
+                                                       .context(queryContextModified)
+                                                       .build()
+                              ),
+                              "j0.",
+                              "(\"dim2\" == \"j0.dim2\")",
+                              JoinType.INNER,
+                              bound("dim2", "a", "a", false, false, null, null)
+                          ),
+                          new QueryDataSource(
+                              newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1)
+                                                   .intervals(querySegmentSpec(Filtration.eternity()))
+                                                   .columns("dim2")
+                                                   .context(queryContextModified)
+                                                   .build()
+                          ),
+                          "_j0.",
+                          "('a' == \"_j0.dim2\")",
+                          JoinType.INNER
+                      )
+                  )
+                  .intervals(querySegmentSpec(Filtration.eternity()))
+                  .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))))
+                  .granularity(Granularities.ALL)
+                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                  .context(queryContextModified)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{8L}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries()
+  {
+    msqCompatible();
+    cannotVectorize();
+
+    testQuery(
+        "SELECT TIME_FORMAT(\"date\", 'yyyy-MM'), SUM(x)\n"
+        + "FROM (\n"
+        + "    SELECT\n"
+        + "        FLOOR(__time to hour) as \"date\",\n"
+        + "        COUNT(*) as x\n"
+        + "    FROM foo\n"
+        + "    GROUP BY 1\n"
+        + ")\n"
+        + "GROUP BY 1",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            Druids.newTimeseriesQueryBuilder()
+                                  .dataSource(CalciteTests.DATASOURCE1)
+                                  .intervals(querySegmentSpec(Filtration.eternity()))
+                                  .granularity(Granularities.HOUR)
+                                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
+                                  .build()
+                        )
+                        .setInterval(querySegmentSpec(Intervals.ETERNITY))
+                        .setVirtualColumns(
+                            expressionVirtualColumn(
+                                "v0",
+                                "timestamp_format(\"d0\",'yyyy-MM','UTC')",
+                                ColumnType.STRING
+                            )
+                        )
+                        .setGranularity(Granularities.ALL)
+                        .addDimension(new DefaultDimensionSpec("v0", "_d0"))
+                        .addAggregator(new LongSumAggregatorFactory("_a0", "a0"))
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"2000-01", 3L},
+            new Object[]{"2001-01", 3L}
+        )
+    );
+  }
+
+  @Test
+  public void testUsingSubqueryAsFilterWithInnerSort()
+  {
+    // Regression test for https://github.com/apache/druid/issues/4208
+
+    testQuery(
+        "SELECT dim1, dim2 FROM druid.foo\n"
+        + " WHERE dim2 IN (\n"
+        + "   SELECT dim2\n"
+        + "   FROM druid.foo\n"
+        + "   GROUP BY dim2\n"
+        + "   ORDER BY dim2 DESC\n"
+        + " )",
+        queryContext,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new TableDataSource(CalciteTests.DATASOURCE1),
+                        new QueryDataSource(
+                            GroupByQuery.builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                        ),
+                        "j0.",
+                        equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.d0")),
+                        JoinType.INNER
+                    )
+                )
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("dim1", "dim2")
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{"", "a"},
+            new Object[]{"1", "a"},
+            new Object[]{"def", "abc"}
+        ) :
+        ImmutableList.of(
+            new Object[]{"", "a"},
+            new Object[]{"2", ""},
+            new Object[]{"1", "a"},
+            new Object[]{"def", "abc"}
+        )
+    );
+  }
+
+  @Test
+  public void testUsingSubqueryAsFilterOnTwoColumns()
+  {
+    msqCompatible();
+    testQuery(
+        "SELECT __time, cnt, dim1, dim2 FROM druid.foo "
+        + " WHERE (dim1, dim2) IN ("
+        + "   SELECT dim1, dim2 FROM ("
+        + "     SELECT dim1, dim2, COUNT(*)"
+        + "     FROM druid.foo"
+        + "     WHERE dim2 = 'abc'"
+        + "     GROUP BY dim1, dim2"
+        + "     HAVING COUNT(*) = 1"
+        + "   )"
+        + " )",
+        queryContext,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new TableDataSource(CalciteTests.DATASOURCE1),
+                        new QueryDataSource(
+                            GroupByQuery.builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimFilter(selector("dim2", "abc", null))
+                                        .setDimensions(dimensions(
+                                            new DefaultDimensionSpec("dim1", "d0"),
+                                            new DefaultDimensionSpec("dim2", "d1")
+                                        ))
+                                        .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+                                        .setPostAggregatorSpecs(
+                                            ImmutableList.of(expressionPostAgg("p0", "'abc'"))
+                                        )
+                                        .setHavingSpec(having(selector("a0", "1", null)))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                        ),
+                        "j0.",
+                        StringUtils.format(
+                            "(%s && %s)",
+                            equalsCondition(makeColumnExpression("dim1"), makeColumnExpression("j0.d0")),

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [CalciteTestBase.makeColumnExpression](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4784)



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java:
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.sql.calcite;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.UOE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.JoinDataSource;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.ResourceLimitExceededException;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
+import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.ExtractionDimensionSpec;
+import org.apache.druid.query.extraction.SubstringDimExtractionFn;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.ordering.StringComparators;
+import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.topn.DimensionTopNMetricSpec;
+import org.apache.druid.query.topn.TopNQueryBuilder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinType;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Period;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Calcite tests which involve subqueries and materializing the intermediate results on {@link org.apache.druid.server.ClientQuerySegmentWalker}
+ * The tests are run with two different
+ */
+@RunWith(Parameterized.class)
+public class CalciteSubqueryTest extends BaseCalciteQueryTest
+{
+
+  public String testName;
+  public Map<String, Object> queryContext;
+
+  public CalciteSubqueryTest(
+      String testName,
+      Map<String, Object> queryContext
+  )
+  {
+    this.testName = testName;
+    this.queryContext = queryContext;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Iterable<Object[]> constructorFeeder()
+  {
+    final List<Object[]> constructors = new ArrayList<>();
+    constructors.add(
+        new Object[]{"without memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    constructors.add(
+        new Object[]{"with memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    return constructors;
+  }
+
+  @Test
+  public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter()
+  {
+    msqCompatible();
+    // Cannot vectorize topN operator.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT\n"
+        + "  SUM(cnt),\n"
+        + "  COUNT(*)\n"
+        + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
+        + "WHERE cnt > 0",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                new TopNQueryBuilder()
+                                    .dataSource(CalciteTests.DATASOURCE1)
+                                    .intervals(querySegmentSpec(Filtration.eternity()))
+                                    .granularity(Granularities.ALL)
+                                    .dimension(new DefaultDimensionSpec("dim2", "d0"))
+                                    .aggregators(new LongSumAggregatorFactory("a0", "cnt"))
+                                    .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
+                                    .threshold(1)
+                                    .build()
+                            )
+                        )
+                        .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(
+                            new LongSumAggregatorFactory("_a0", "a0"),
+                            new CountAggregatorFactory("_a1")
+                        ))
+                        .setContext(queryContext)
+                        .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{3L, 1L}
+        ) :
+        ImmutableList.of(
+            new Object[]{2L, 1L}
+        )
+    );
+  }
+
+  @Test
+  public void testExactCountDistinctOfSemiJoinResult()
+  {
+    msqCompatible();
+    // Cannot vectorize due to extraction dimension spec.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT COUNT(*)\n"
+        + "FROM (\n"
+        + "  SELECT DISTINCT dim2\n"
+        + "  FROM druid.foo\n"
+        + "  WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+        + "    SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+        + "  ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n"
+        + ")",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                GroupByQuery.builder()
+                                            .setDataSource(
+                                                join(
+                                                    new TableDataSource(CalciteTests.DATASOURCE1),
+                                                    new QueryDataSource(
+                                                        GroupByQuery
+                                                            .builder()
+                                                            .setDataSource(CalciteTests.DATASOURCE1)
+                                                            .setInterval(querySegmentSpec(Filtration.eternity()))
+                                                            .setGranularity(Granularities.ALL)
+                                                            .setDimFilter(not(selector("dim1", "", null)))
+                                                            .setDimensions(
+                                                                dimensions(
+                                                                    new ExtractionDimensionSpec(
+                                                                        "dim1",
+                                                                        "d0",
+                                                                        new SubstringDimExtractionFn(0, 1)
+                                                                    )
+                                                                )
+                                                            )
+                                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                                            .build()
+                                                    ),
+                                                    "j0.",
+                                                    equalsCondition(
+                                                        makeExpression("substring(\"dim2\", 0, 1)"),
+                                                        DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")
+                                                    ),
+                                                    JoinType.INNER
+                                                )
+                                            )
+                                            .setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
+                                            .setGranularity(Granularities.ALL)
+                                            .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                            .build()
+                            )
+                        )
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{2L}
+        )
+    );
+  }
+
+  @Test
+  public void testTwoExactCountDistincts()
+  {
+    requireMergeBuffers(6);
+    msqCompatible();
+    testQuery(
+        PLANNER_CONFIG_NO_HLL,
+        queryContext,
+        "SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo",
+        CalciteTests.REGULAR_USER_AUTH_RESULT,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        "j0.",
+                        "1",
+                        JoinType.INNER
+                    )
+                )
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("a0", "j0.a0")
+                .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+        ),
+        ImmutableList.of(
+            new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L}
+        )
+    );
+  }
+
+  @Test
+  public void testViewAndJoin()
+  {
+    cannotVectorize();
+    Map<String, Object> queryContextModified = withLeftDirectAccessEnabled(queryContext);
+    testQuery(
+        "SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ",
+        queryContextModified,
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(
+                      join(
+                          join(
+                              new TableDataSource(CalciteTests.DATASOURCE1),
+                              new QueryDataSource(
+                                  newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3)
+                                                       .intervals(querySegmentSpec(Filtration.eternity()))
+                                                       .columns("dim2")
+                                                       .context(queryContextModified)
+                                                       .build()
+                              ),
+                              "j0.",
+                              "(\"dim2\" == \"j0.dim2\")",
+                              JoinType.INNER,
+                              bound("dim2", "a", "a", false, false, null, null)
+                          ),
+                          new QueryDataSource(
+                              newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1)
+                                                   .intervals(querySegmentSpec(Filtration.eternity()))
+                                                   .columns("dim2")
+                                                   .context(queryContextModified)
+                                                   .build()
+                          ),
+                          "_j0.",
+                          "('a' == \"_j0.dim2\")",
+                          JoinType.INNER
+                      )
+                  )
+                  .intervals(querySegmentSpec(Filtration.eternity()))
+                  .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))))
+                  .granularity(Granularities.ALL)
+                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                  .context(queryContextModified)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{8L}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries()
+  {
+    msqCompatible();
+    cannotVectorize();
+
+    testQuery(
+        "SELECT TIME_FORMAT(\"date\", 'yyyy-MM'), SUM(x)\n"
+        + "FROM (\n"
+        + "    SELECT\n"
+        + "        FLOOR(__time to hour) as \"date\",\n"
+        + "        COUNT(*) as x\n"
+        + "    FROM foo\n"
+        + "    GROUP BY 1\n"
+        + ")\n"
+        + "GROUP BY 1",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            Druids.newTimeseriesQueryBuilder()
+                                  .dataSource(CalciteTests.DATASOURCE1)
+                                  .intervals(querySegmentSpec(Filtration.eternity()))
+                                  .granularity(Granularities.HOUR)
+                                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
+                                  .build()
+                        )
+                        .setInterval(querySegmentSpec(Intervals.ETERNITY))
+                        .setVirtualColumns(
+                            expressionVirtualColumn(
+                                "v0",
+                                "timestamp_format(\"d0\",'yyyy-MM','UTC')",
+                                ColumnType.STRING
+                            )
+                        )
+                        .setGranularity(Granularities.ALL)
+                        .addDimension(new DefaultDimensionSpec("v0", "_d0"))
+                        .addAggregator(new LongSumAggregatorFactory("_a0", "a0"))
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"2000-01", 3L},
+            new Object[]{"2001-01", 3L}
+        )
+    );
+  }
+
+  @Test
+  public void testUsingSubqueryAsFilterWithInnerSort()
+  {
+    // Regression test for https://github.com/apache/druid/issues/4208
+
+    testQuery(
+        "SELECT dim1, dim2 FROM druid.foo\n"
+        + " WHERE dim2 IN (\n"
+        + "   SELECT dim2\n"
+        + "   FROM druid.foo\n"
+        + "   GROUP BY dim2\n"
+        + "   ORDER BY dim2 DESC\n"
+        + " )",
+        queryContext,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new TableDataSource(CalciteTests.DATASOURCE1),
+                        new QueryDataSource(
+                            GroupByQuery.builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                        ),
+                        "j0.",
+                        equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.d0")),

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [CalciteTestBase.makeColumnExpression](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4783)



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java:
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.sql.calcite;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.UOE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.JoinDataSource;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.ResourceLimitExceededException;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
+import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.ExtractionDimensionSpec;
+import org.apache.druid.query.extraction.SubstringDimExtractionFn;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.ordering.StringComparators;
+import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.topn.DimensionTopNMetricSpec;
+import org.apache.druid.query.topn.TopNQueryBuilder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinType;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Period;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Calcite tests which involve subqueries and materializing the intermediate results on {@link org.apache.druid.server.ClientQuerySegmentWalker}
+ * The tests are run with two different
+ */
+@RunWith(Parameterized.class)
+public class CalciteSubqueryTest extends BaseCalciteQueryTest
+{
+
+  public String testName;
+  public Map<String, Object> queryContext;
+
+  public CalciteSubqueryTest(
+      String testName,
+      Map<String, Object> queryContext
+  )
+  {
+    this.testName = testName;
+    this.queryContext = queryContext;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Iterable<Object[]> constructorFeeder()
+  {
+    final List<Object[]> constructors = new ArrayList<>();
+    constructors.add(
+        new Object[]{"without memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    constructors.add(
+        new Object[]{"with memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    return constructors;
+  }
+
+  @Test
+  public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter()
+  {
+    msqCompatible();
+    // Cannot vectorize topN operator.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT\n"
+        + "  SUM(cnt),\n"
+        + "  COUNT(*)\n"
+        + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
+        + "WHERE cnt > 0",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                new TopNQueryBuilder()
+                                    .dataSource(CalciteTests.DATASOURCE1)
+                                    .intervals(querySegmentSpec(Filtration.eternity()))
+                                    .granularity(Granularities.ALL)
+                                    .dimension(new DefaultDimensionSpec("dim2", "d0"))
+                                    .aggregators(new LongSumAggregatorFactory("a0", "cnt"))
+                                    .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
+                                    .threshold(1)
+                                    .build()
+                            )
+                        )
+                        .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(
+                            new LongSumAggregatorFactory("_a0", "a0"),
+                            new CountAggregatorFactory("_a1")
+                        ))
+                        .setContext(queryContext)
+                        .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{3L, 1L}
+        ) :
+        ImmutableList.of(
+            new Object[]{2L, 1L}
+        )
+    );
+  }
+
+  @Test
+  public void testExactCountDistinctOfSemiJoinResult()
+  {
+    msqCompatible();
+    // Cannot vectorize due to extraction dimension spec.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT COUNT(*)\n"
+        + "FROM (\n"
+        + "  SELECT DISTINCT dim2\n"
+        + "  FROM druid.foo\n"
+        + "  WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+        + "    SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+        + "  ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n"
+        + ")",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                GroupByQuery.builder()
+                                            .setDataSource(
+                                                join(
+                                                    new TableDataSource(CalciteTests.DATASOURCE1),
+                                                    new QueryDataSource(
+                                                        GroupByQuery
+                                                            .builder()
+                                                            .setDataSource(CalciteTests.DATASOURCE1)
+                                                            .setInterval(querySegmentSpec(Filtration.eternity()))
+                                                            .setGranularity(Granularities.ALL)
+                                                            .setDimFilter(not(selector("dim1", "", null)))
+                                                            .setDimensions(
+                                                                dimensions(
+                                                                    new ExtractionDimensionSpec(
+                                                                        "dim1",
+                                                                        "d0",
+                                                                        new SubstringDimExtractionFn(0, 1)
+                                                                    )
+                                                                )
+                                                            )
+                                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                                            .build()
+                                                    ),
+                                                    "j0.",
+                                                    equalsCondition(
+                                                        makeExpression("substring(\"dim2\", 0, 1)"),
+                                                        DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")
+                                                    ),
+                                                    JoinType.INNER
+                                                )
+                                            )
+                                            .setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
+                                            .setGranularity(Granularities.ALL)
+                                            .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                            .build()
+                            )
+                        )
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{2L}
+        )
+    );
+  }
+
+  @Test
+  public void testTwoExactCountDistincts()
+  {
+    requireMergeBuffers(6);
+    msqCompatible();
+    testQuery(
+        PLANNER_CONFIG_NO_HLL,
+        queryContext,
+        "SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo",
+        CalciteTests.REGULAR_USER_AUTH_RESULT,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        "j0.",
+                        "1",
+                        JoinType.INNER
+                    )
+                )
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("a0", "j0.a0")
+                .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+        ),
+        ImmutableList.of(
+            new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L}
+        )
+    );
+  }
+
+  @Test
+  public void testViewAndJoin()
+  {
+    cannotVectorize();
+    Map<String, Object> queryContextModified = withLeftDirectAccessEnabled(queryContext);
+    testQuery(
+        "SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ",
+        queryContextModified,
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(
+                      join(
+                          join(
+                              new TableDataSource(CalciteTests.DATASOURCE1),
+                              new QueryDataSource(
+                                  newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3)
+                                                       .intervals(querySegmentSpec(Filtration.eternity()))
+                                                       .columns("dim2")
+                                                       .context(queryContextModified)
+                                                       .build()
+                              ),
+                              "j0.",
+                              "(\"dim2\" == \"j0.dim2\")",
+                              JoinType.INNER,
+                              bound("dim2", "a", "a", false, false, null, null)
+                          ),
+                          new QueryDataSource(
+                              newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1)
+                                                   .intervals(querySegmentSpec(Filtration.eternity()))
+                                                   .columns("dim2")
+                                                   .context(queryContextModified)
+                                                   .build()
+                          ),
+                          "_j0.",
+                          "('a' == \"_j0.dim2\")",
+                          JoinType.INNER
+                      )
+                  )
+                  .intervals(querySegmentSpec(Filtration.eternity()))
+                  .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))))
+                  .granularity(Granularities.ALL)
+                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                  .context(queryContextModified)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{8L}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries()
+  {
+    msqCompatible();
+    cannotVectorize();
+
+    testQuery(
+        "SELECT TIME_FORMAT(\"date\", 'yyyy-MM'), SUM(x)\n"
+        + "FROM (\n"
+        + "    SELECT\n"
+        + "        FLOOR(__time to hour) as \"date\",\n"
+        + "        COUNT(*) as x\n"
+        + "    FROM foo\n"
+        + "    GROUP BY 1\n"
+        + ")\n"
+        + "GROUP BY 1",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            Druids.newTimeseriesQueryBuilder()
+                                  .dataSource(CalciteTests.DATASOURCE1)
+                                  .intervals(querySegmentSpec(Filtration.eternity()))
+                                  .granularity(Granularities.HOUR)
+                                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
+                                  .build()
+                        )
+                        .setInterval(querySegmentSpec(Intervals.ETERNITY))
+                        .setVirtualColumns(
+                            expressionVirtualColumn(
+                                "v0",
+                                "timestamp_format(\"d0\",'yyyy-MM','UTC')",
+                                ColumnType.STRING
+                            )
+                        )
+                        .setGranularity(Granularities.ALL)
+                        .addDimension(new DefaultDimensionSpec("v0", "_d0"))
+                        .addAggregator(new LongSumAggregatorFactory("_a0", "a0"))
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"2000-01", 3L},
+            new Object[]{"2001-01", 3L}
+        )
+    );
+  }
+
+  @Test
+  public void testUsingSubqueryAsFilterWithInnerSort()
+  {
+    // Regression test for https://github.com/apache/druid/issues/4208
+
+    testQuery(
+        "SELECT dim1, dim2 FROM druid.foo\n"
+        + " WHERE dim2 IN (\n"
+        + "   SELECT dim2\n"
+        + "   FROM druid.foo\n"
+        + "   GROUP BY dim2\n"
+        + "   ORDER BY dim2 DESC\n"
+        + " )",
+        queryContext,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new TableDataSource(CalciteTests.DATASOURCE1),
+                        new QueryDataSource(
+                            GroupByQuery.builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                        ),
+                        "j0.",
+                        equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.d0")),
+                        JoinType.INNER
+                    )
+                )
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("dim1", "dim2")
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{"", "a"},
+            new Object[]{"1", "a"},
+            new Object[]{"def", "abc"}
+        ) :
+        ImmutableList.of(
+            new Object[]{"", "a"},
+            new Object[]{"2", ""},
+            new Object[]{"1", "a"},
+            new Object[]{"def", "abc"}
+        )
+    );
+  }
+
+  @Test
+  public void testUsingSubqueryAsFilterOnTwoColumns()
+  {
+    msqCompatible();
+    testQuery(
+        "SELECT __time, cnt, dim1, dim2 FROM druid.foo "
+        + " WHERE (dim1, dim2) IN ("
+        + "   SELECT dim1, dim2 FROM ("
+        + "     SELECT dim1, dim2, COUNT(*)"
+        + "     FROM druid.foo"
+        + "     WHERE dim2 = 'abc'"
+        + "     GROUP BY dim1, dim2"
+        + "     HAVING COUNT(*) = 1"
+        + "   )"
+        + " )",
+        queryContext,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new TableDataSource(CalciteTests.DATASOURCE1),
+                        new QueryDataSource(
+                            GroupByQuery.builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimFilter(selector("dim2", "abc", null))
+                                        .setDimensions(dimensions(
+                                            new DefaultDimensionSpec("dim1", "d0"),
+                                            new DefaultDimensionSpec("dim2", "d1")
+                                        ))
+                                        .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+                                        .setPostAggregatorSpecs(
+                                            ImmutableList.of(expressionPostAgg("p0", "'abc'"))
+                                        )
+                                        .setHavingSpec(having(selector("a0", "1", null)))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                        ),
+                        "j0.",
+                        StringUtils.format(
+                            "(%s && %s)",
+                            equalsCondition(makeColumnExpression("dim1"), makeColumnExpression("j0.d0")),

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [CalciteTestBase.makeColumnExpression](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4785)



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java:
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.sql.calcite;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.UOE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.JoinDataSource;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.ResourceLimitExceededException;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
+import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
+import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.ExtractionDimensionSpec;
+import org.apache.druid.query.extraction.SubstringDimExtractionFn;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
+import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.ordering.StringComparators;
+import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.topn.DimensionTopNMetricSpec;
+import org.apache.druid.query.topn.TopNQueryBuilder;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.join.JoinType;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Period;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Calcite tests which involve subqueries and materializing the intermediate results on {@link org.apache.druid.server.ClientQuerySegmentWalker}
+ * The tests are run with two different
+ */
+@RunWith(Parameterized.class)
+public class CalciteSubqueryTest extends BaseCalciteQueryTest
+{
+
+  public String testName;
+  public Map<String, Object> queryContext;
+
+  public CalciteSubqueryTest(
+      String testName,
+      Map<String, Object> queryContext
+  )
+  {
+    this.testName = testName;
+    this.queryContext = queryContext;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Iterable<Object[]> constructorFeeder()
+  {
+    final List<Object[]> constructors = new ArrayList<>();
+    constructors.add(
+        new Object[]{"without memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    constructors.add(
+        new Object[]{"with memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
+    );
+    return constructors;
+  }
+
+  @Test
+  public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter()
+  {
+    msqCompatible();
+    // Cannot vectorize topN operator.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT\n"
+        + "  SUM(cnt),\n"
+        + "  COUNT(*)\n"
+        + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
+        + "WHERE cnt > 0",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                new TopNQueryBuilder()
+                                    .dataSource(CalciteTests.DATASOURCE1)
+                                    .intervals(querySegmentSpec(Filtration.eternity()))
+                                    .granularity(Granularities.ALL)
+                                    .dimension(new DefaultDimensionSpec("dim2", "d0"))
+                                    .aggregators(new LongSumAggregatorFactory("a0", "cnt"))
+                                    .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
+                                    .threshold(1)
+                                    .build()
+                            )
+                        )
+                        .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(
+                            new LongSumAggregatorFactory("_a0", "a0"),
+                            new CountAggregatorFactory("_a1")
+                        ))
+                        .setContext(queryContext)
+                        .build()
+        ),
+        NullHandling.replaceWithDefault() ?
+        ImmutableList.of(
+            new Object[]{3L, 1L}
+        ) :
+        ImmutableList.of(
+            new Object[]{2L, 1L}
+        )
+    );
+  }
+
+  @Test
+  public void testExactCountDistinctOfSemiJoinResult()
+  {
+    msqCompatible();
+    // Cannot vectorize due to extraction dimension spec.
+    cannotVectorize();
+
+    testQuery(
+        "SELECT COUNT(*)\n"
+        + "FROM (\n"
+        + "  SELECT DISTINCT dim2\n"
+        + "  FROM druid.foo\n"
+        + "  WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+        + "    SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+        + "  ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n"
+        + ")",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            new QueryDataSource(
+                                GroupByQuery.builder()
+                                            .setDataSource(
+                                                join(
+                                                    new TableDataSource(CalciteTests.DATASOURCE1),
+                                                    new QueryDataSource(
+                                                        GroupByQuery
+                                                            .builder()
+                                                            .setDataSource(CalciteTests.DATASOURCE1)
+                                                            .setInterval(querySegmentSpec(Filtration.eternity()))
+                                                            .setGranularity(Granularities.ALL)
+                                                            .setDimFilter(not(selector("dim1", "", null)))
+                                                            .setDimensions(
+                                                                dimensions(
+                                                                    new ExtractionDimensionSpec(
+                                                                        "dim1",
+                                                                        "d0",
+                                                                        new SubstringDimExtractionFn(0, 1)
+                                                                    )
+                                                                )
+                                                            )
+                                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                                            .build()
+                                                    ),
+                                                    "j0.",
+                                                    equalsCondition(
+                                                        makeExpression("substring(\"dim2\", 0, 1)"),
+                                                        DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")
+                                                    ),
+                                                    JoinType.INNER
+                                                )
+                                            )
+                                            .setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
+                                            .setGranularity(Granularities.ALL)
+                                            .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                            .setContext(QUERY_CONTEXT_DEFAULT)
+                                            .build()
+                            )
+                        )
+                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{2L}
+        )
+    );
+  }
+
+  @Test
+  public void testTwoExactCountDistincts()
+  {
+    requireMergeBuffers(6);
+    msqCompatible();
+    testQuery(
+        PLANNER_CONFIG_NO_HLL,
+        queryContext,
+        "SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo",
+        CalciteTests.REGULAR_USER_AUTH_RESULT,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        new QueryDataSource(
+                            GroupByQuery
+                                .builder()
+                                .setDataSource(
+                                    GroupByQuery
+                                        .builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                                )
+                                .setInterval(querySegmentSpec(Filtration.eternity()))
+                                .setGranularity(Granularities.ALL)
+                                .setAggregatorSpecs(
+                                    new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        not(selector("d0", null, null))
+                                    )
+                                )
+                                .setContext(QUERY_CONTEXT_DEFAULT)
+                                .build()
+                        ),
+                        "j0.",
+                        "1",
+                        JoinType.INNER
+                    )
+                )
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("a0", "j0.a0")
+                .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+        ),
+        ImmutableList.of(
+            new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L}
+        )
+    );
+  }
+
+  @Test
+  public void testViewAndJoin()
+  {
+    cannotVectorize();
+    Map<String, Object> queryContextModified = withLeftDirectAccessEnabled(queryContext);
+    testQuery(
+        "SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ",
+        queryContextModified,
+        ImmutableList.of(
+            Druids.newTimeseriesQueryBuilder()
+                  .dataSource(
+                      join(
+                          join(
+                              new TableDataSource(CalciteTests.DATASOURCE1),
+                              new QueryDataSource(
+                                  newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3)
+                                                       .intervals(querySegmentSpec(Filtration.eternity()))
+                                                       .columns("dim2")
+                                                       .context(queryContextModified)
+                                                       .build()
+                              ),
+                              "j0.",
+                              "(\"dim2\" == \"j0.dim2\")",
+                              JoinType.INNER,
+                              bound("dim2", "a", "a", false, false, null, null)
+                          ),
+                          new QueryDataSource(
+                              newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1)
+                                                   .intervals(querySegmentSpec(Filtration.eternity()))
+                                                   .columns("dim2")
+                                                   .context(queryContextModified)
+                                                   .build()
+                          ),
+                          "_j0.",
+                          "('a' == \"_j0.dim2\")",
+                          JoinType.INNER
+                      )
+                  )
+                  .intervals(querySegmentSpec(Filtration.eternity()))
+                  .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))))
+                  .granularity(Granularities.ALL)
+                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                  .context(queryContextModified)
+                  .build()
+        ),
+        ImmutableList.of(
+            new Object[]{8L}
+        )
+    );
+  }
+
+  @Test
+  public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries()
+  {
+    msqCompatible();
+    cannotVectorize();
+
+    testQuery(
+        "SELECT TIME_FORMAT(\"date\", 'yyyy-MM'), SUM(x)\n"
+        + "FROM (\n"
+        + "    SELECT\n"
+        + "        FLOOR(__time to hour) as \"date\",\n"
+        + "        COUNT(*) as x\n"
+        + "    FROM foo\n"
+        + "    GROUP BY 1\n"
+        + ")\n"
+        + "GROUP BY 1",
+        queryContext,
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(
+                            Druids.newTimeseriesQueryBuilder()
+                                  .dataSource(CalciteTests.DATASOURCE1)
+                                  .intervals(querySegmentSpec(Filtration.eternity()))
+                                  .granularity(Granularities.HOUR)
+                                  .aggregators(aggregators(new CountAggregatorFactory("a0")))
+                                  .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
+                                  .build()
+                        )
+                        .setInterval(querySegmentSpec(Intervals.ETERNITY))
+                        .setVirtualColumns(
+                            expressionVirtualColumn(
+                                "v0",
+                                "timestamp_format(\"d0\",'yyyy-MM','UTC')",
+                                ColumnType.STRING
+                            )
+                        )
+                        .setGranularity(Granularities.ALL)
+                        .addDimension(new DefaultDimensionSpec("v0", "_d0"))
+                        .addAggregator(new LongSumAggregatorFactory("_a0", "a0"))
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"2000-01", 3L},
+            new Object[]{"2001-01", 3L}
+        )
+    );
+  }
+
+  @Test
+  public void testUsingSubqueryAsFilterWithInnerSort()
+  {
+    // Regression test for https://github.com/apache/druid/issues/4208
+
+    testQuery(
+        "SELECT dim1, dim2 FROM druid.foo\n"
+        + " WHERE dim2 IN (\n"
+        + "   SELECT dim2\n"
+        + "   FROM druid.foo\n"
+        + "   GROUP BY dim2\n"
+        + "   ORDER BY dim2 DESC\n"
+        + " )",
+        queryContext,
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(
+                    join(
+                        new TableDataSource(CalciteTests.DATASOURCE1),
+                        new QueryDataSource(
+                            GroupByQuery.builder()
+                                        .setDataSource(CalciteTests.DATASOURCE1)
+                                        .setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+                                        .setContext(QUERY_CONTEXT_DEFAULT)
+                                        .build()
+                        ),
+                        "j0.",
+                        equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.d0")),

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [CalciteTestBase.makeColumnExpression](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4782)



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1168626818


##########
processing/src/main/java/org/apache/druid/frame/segment/FrameCursorUtils.java:
##########
@@ -67,4 +74,45 @@ public static Filter buildFilter(@Nullable Filter filter, Interval interval)
       );
     }
   }
+
+  /**
+   * Writes a {@link Cursor} to a {@link Frame}. This method iterates over the rows of the cursor, and writes the columns
+   * to the cursor
+   * @param cursor Cursor to write to the frame
+   * @param frameWriterFactory Frame writer factory to write to the frame.
+   *                           Determines the signature of the rows that are written to the frames
+   * @param memoryLimitBytes Limit in bytes, if needs to be enforced while converting the cursor to the frame. If adding
+   *                         a row causes the frame size to exceed this limit, we throw an {@link ResourceLimitExceededException}
+   */
+  public static Frame cursorToFrame(
+      Cursor cursor,
+      FrameWriterFactory frameWriterFactory,
+      @Nullable Long memoryLimitBytes
+  )
+  {
+    Frame frame;
+
+    try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) {
+      while (!cursor.isDone()) {
+        if (!frameWriter.addSelection()) {
+          throw new FrameRowTooLargeException(frameWriterFactory.allocatorCapacity());

Review Comment:
   This doesn't seem correct. Maybe you are adding too many rows to the frame and thats why frame runs out of space ? Maybe throw a new exception type?



##########
processing/src/main/java/org/apache/druid/frame/read/FrameReader.java:
##########
@@ -85,12 +88,17 @@ public static FrameReader create(final RowSignature signature)
     final List<FieldReader> fieldReaders = new ArrayList<>(signature.size());
 
     for (int columnNumber = 0; columnNumber < signature.size(); columnNumber++) {
-      final ColumnType columnType =
-          Preconditions.checkNotNull(
-              signature.getColumnType(columnNumber).orElse(null),
-              "Type for column [%s]",
-              signature.getColumnName(columnNumber)
-          );
+      ColumnType columnType;
+      if (!allowNullTypes) {
+        columnType =
+            Preconditions.checkNotNull(
+                signature.getColumnType(columnNumber).orElse(null),
+                "Type for column [%s]",
+                signature.getColumnName(columnNumber)
+            );
+      } else {
+        columnType = signature.getColumnType(columnNumber).orElse(ColumnType.NESTED_DATA);

Review Comment:
   Can you please mention why this is nested data 



##########
processing/src/main/java/org/apache/druid/query/FramesBackedInlineDataSource.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.query;
+
+import org.apache.druid.frame.Frame;
+import org.apache.druid.frame.read.FrameReader;
+import org.apache.druid.frame.segment.FrameStorageAdapter;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.query.planning.DataSourceAnalysis;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.SegmentReference;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.RowSignature;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Represents an inline datasource where the rows are embedded within the DataSource object itself.
+ * <p>
+ * The rows are backed by a sequence of {@link FrameSignaturePair}, which contain the Frame representation of the rows
+ * represented by the datasource. {@link #getRowsAsList()} and {@link #getRowsAsSequence()} return the iterables which
+ * read the rows that are encapsulated in the frames.
+ * <p>
+ * Note that the signature of the datasource can be different from the signatures of the constituent frames that it
+ * consists of. While fetching the iterables, it is the job of this class to make sure that the rows correspond to the
+ * {@link #rowSignature}. For frames that donot contain the columns present in the {@link #rowSignature}, they are
+ * populated with {@code null}.
+ */
+public class FramesBackedInlineDataSource implements DataSource
+{
+
+  final List<FrameSignaturePair> frames;
+  final RowSignature rowSignature;
+
+  public FramesBackedInlineDataSource(
+      List<FrameSignaturePair> frames,
+      RowSignature rowSignature
+  )
+  {
+    this.frames = frames;
+    this.rowSignature = rowSignature;
+  }
+
+  public List<FrameSignaturePair> getFrames()
+  {
+    return frames;
+  }
+
+  public RowSignature getRowSignature()
+  {
+    return rowSignature;
+  }
+
+  private List<Object[]> getRowsAsList()
+  {
+    List<Object[]> frameRows = new ArrayList<>();
+    Yielder<Object[]> rowsYielder = Yielders.each(getRowsAsSequence());
+    while (!rowsYielder.isDone()) {
+      Object[] row = rowsYielder.get();
+      frameRows.add(row);
+      rowsYielder = rowsYielder.next(null);
+    }
+    return frameRows;
+  }
+
+  public Sequence<Object[]> getRowsAsSequence()
+  {
+
+    final Sequence<Cursor> cursorSequence =
+        Sequences.simple(frames)
+                 .flatMap(
+                     frameSignaturePair -> {
+                       Frame frame = frameSignaturePair.getFrame();
+                       RowSignature frameSignature = frameSignaturePair.getRowSignature();
+                       FrameReader frameReader = FrameReader.create(frameSignature, true);
+                       return new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY)
+                           .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null);
+                     }
+                 );
+
+    return cursorSequence.flatMap(
+        (cursor) -> {
+          final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
+          final List<BaseObjectColumnValueSelector> selectors = rowSignature
+              .getColumnNames()
+              .stream()
+              .map(columnSelectorFactory::makeColumnValueSelector)
+              .collect(Collectors.toList());
+
+          return Sequences.simple(
+              () -> new Iterator<Object[]>()
+              {
+                @Override
+                public boolean hasNext()
+                {
+                  return !cursor.isDone();
+                }
+
+                @Override
+                public Object[] next()
+                {
+
+                  Object[] row = new Object[rowSignature.size()];
+                  for (int i = 0; i < rowSignature.size(); ++i) {
+                    row[i] = selectors.get(i).getObject();
+                  }
+
+                  cursor.advance();
+
+                  return row;
+                }
+              }
+          );
+        }
+    );
+  }
+
+  @Override
+  public Set<String> getTableNames()
+  {
+    return Collections.emptySet();
+  }
+
+  @Override
+  public List<DataSource> getChildren()
+  {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public DataSource withChildren(List<DataSource> children)
+  {
+    if (!children.isEmpty()) {
+      throw new IAE("Cannot accept children");
+    }
+
+    return this;
+  }
+
+  @Override
+  public boolean isCacheable(boolean isBroker)
+  {
+    return false;
+  }
+
+  @Override
+  public boolean isGlobal()
+  {
+    return true;
+  }
+
+  @Override
+  public boolean isConcrete()
+  {
+    return true;
+  }
+
+  @Override
+  public Function<SegmentReference, SegmentReference> createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc)
+  {
+    return Function.identity();
+  }
+
+  @Override
+  public DataSource withUpdatedDataSource(DataSource newSource)
+  {
+    return newSource;
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    return null;
+  }
+
+  @Override
+  public DataSourceAnalysis getAnalysis()
+  {
+    return new DataSourceAnalysis(this, null, null, Collections.emptyList());
+  }
+
+  public InlineDataSource toIterableBackedInlineDataSource()

Review Comment:
   Please mention the usages of this method and the limitations like materialization of the entire results so that callers are aware when to make this expensive call.  



##########
processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.query.scan;
+
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.Cursor;
+import org.joda.time.DateTime;
+
+import java.util.List;
+
+/**
+ * Combines multiple cursors and iterates over them. It skips over the empty cursors
+ */
+public class ConcatCursor implements Cursor
+{
+
+  private final List<Cursor> cursors;
+  private int currentCursor;
+
+  public ConcatCursor(
+      List<Cursor> cursors
+  )
+  {
+    this.cursors = cursors;
+    currentCursor = 0;
+    skipEmptyCursors();
+  }
+
+  @Override
+  public ColumnSelectorFactory getColumnSelectorFactory()
+  {
+    return cursors.get(currentCursor).getColumnSelectorFactory();
+  }
+
+  @Override
+  public DateTime getTime()
+  {
+    return cursors.get(currentCursor).getTime();
+  }
+
+  @Override
+  public void advance()
+  {
+    if (currentCursor < cursors.size()) {
+      cursors.get(currentCursor).advance();
+      if (cursors.get(currentCursor).isDone()) {
+        ++currentCursor;

Review Comment:
   I think this is a common method. Maybe extract it out to `moveCursorIfDonetoNextNonEmpty()`



##########
processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.query.scan;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.ListCursor;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ConcatCursorTest
+{
+  @Test
+  public void testCursor()
+  {
+    Cursor dummyCursor1 = new ListCursor(new ArrayList<>());
+    Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b"));
+    Cursor dummyCursor2 = new ListCursor(new ArrayList<>());
+    Cursor cursor2 = new ListCursor(ImmutableList.of("c", "d"));
+    Cursor dummyCursor3 = new ListCursor(new ArrayList<>());
+
+    Cursor concatCursor = new ConcatCursor(ImmutableList.of(
+        dummyCursor1,
+        cursor1,
+        dummyCursor2,
+        cursor2,
+        dummyCursor3
+    ));
+
+    List<Object> tempList = new ArrayList<>();
+    // Initial iteration
+    while (!concatCursor.isDone()) {
+      tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());

Review Comment:
   Also assert on the iteration count. 



##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -187,11 +203,112 @@ public RowSignature resultArraySignature(final ScanQuery query)
     }
   }
 
+  /**
+   * This batches the fetched {@link ScanResultValue}s which have similar signatures and are consecutives. In best case
+   * it would return a single frame, and in the worst case, it would return as many frames as the number of results
+   * passed. Note: Batching requires all the frames to be materialized before they are propogated further and this might
+   * be improved
+   */
+  @Override
+  public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
+      final ScanQuery query,
+      final Sequence<ScanResultValue> resultSequence,
+      @Nullable Long memoryLimitBytes
+  )
+  {
+    final AtomicLong memoryLimitAccumulator = memoryLimitBytes != null ? new AtomicLong(memoryLimitBytes) : null;
+    Yielder<ScanResultValue> yielder = Yielders.each(resultSequence);
+    RowSignature prevSignature = null;
+    List<Cursor> unwrittenCursors = null;
+    List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();

Review Comment:
   All of this is prematerialized. We need to make this lazy. For next() of the iterable, we always return one frame. That frame is generated lazily. 



##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -187,11 +203,112 @@ public RowSignature resultArraySignature(final ScanQuery query)
     }
   }
 
+  /**
+   * This batches the fetched {@link ScanResultValue}s which have similar signatures and are consecutives. In best case
+   * it would return a single frame, and in the worst case, it would return as many frames as the number of results

Review Comment:
   ```suggestion
      * it would return a single frame, and in the worst case, it would return as many frames as the number of {#link ScanResultValue}
   ```



##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java:
##########
@@ -187,11 +203,112 @@ public RowSignature resultArraySignature(final ScanQuery query)
     }
   }
 
+  /**
+   * This batches the fetched {@link ScanResultValue}s which have similar signatures and are consecutives. In best case
+   * it would return a single frame, and in the worst case, it would return as many frames as the number of results
+   * passed. Note: Batching requires all the frames to be materialized before they are propogated further and this might
+   * be improved
+   */
+  @Override
+  public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
+      final ScanQuery query,
+      final Sequence<ScanResultValue> resultSequence,
+      @Nullable Long memoryLimitBytes
+  )
+  {
+    final AtomicLong memoryLimitAccumulator = memoryLimitBytes != null ? new AtomicLong(memoryLimitBytes) : null;
+    Yielder<ScanResultValue> yielder = Yielders.each(resultSequence);
+    RowSignature prevSignature = null;
+    List<Cursor> unwrittenCursors = null;
+    List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();

Review Comment:
   Can we add final here



##########
processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.query.scan;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.ListCursor;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ConcatCursorTest
+{
+  @Test
+  public void testCursor()
+  {
+    Cursor dummyCursor1 = new ListCursor(new ArrayList<>());

Review Comment:
   Test case for
   * empty cursors
   * empty at end only 
   * empty at start
   * 2 empty cursors one after the other in all 3 places start , end , middle. 
   
   Same set of test cases for advanceUninterruptibly. 
   
   Maybe use parameterized tests. 



##########
processing/src/main/java/org/apache/druid/frame/segment/FrameCursorUtils.java:
##########
@@ -67,4 +74,45 @@ public static Filter buildFilter(@Nullable Filter filter, Interval interval)
       );
     }
   }
+
+  /**
+   * Writes a {@link Cursor} to a {@link Frame}. This method iterates over the rows of the cursor, and writes the columns
+   * to the cursor
+   * @param cursor Cursor to write to the frame
+   * @param frameWriterFactory Frame writer factory to write to the frame.
+   *                           Determines the signature of the rows that are written to the frames
+   * @param memoryLimitBytes Limit in bytes, if needs to be enforced while converting the cursor to the frame. If adding
+   *                         a row causes the frame size to exceed this limit, we throw an {@link ResourceLimitExceededException}
+   */
+  public static Frame cursorToFrame(
+      Cursor cursor,
+      FrameWriterFactory frameWriterFactory,
+      @Nullable Long memoryLimitBytes
+  )
+  {
+    Frame frame;
+
+    try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) {
+      while (!cursor.isDone()) {
+        if (!frameWriter.addSelection()) {
+          throw new FrameRowTooLargeException(frameWriterFactory.allocatorCapacity());
+        }
+
+        if (memoryLimitBytes != null && memoryLimitBytes < frameWriter.getTotalSize()) {
+          throw new ResourceLimitExceededException(
+              StringUtils.format(
+                  "The row limit exceeded the bytes allocated for the conversion (allocated = [%d] bytes)",

Review Comment:
   Exceeded total bytes allocated for this subquery. Current size [%d], total row count [%d] allocated size [%d]. Try the preferred method of limiting the results of the subquery to xxx bytes or increase the limit via context param xxx.



##########
processing/src/main/java/org/apache/druid/query/FramesBackedInlineDataSourceSerializer.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.jsontype.TypeSerializer;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import org.apache.druid.java.util.common.jackson.JacksonUtils;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * Serializes {@link FramesBackedInlineDataSource} to the representation of {@link InlineDataSource}
+ * so that the servers' on wire transfer data doesn't change. {@link FramesBackedInlineDataSource} is currently limited
+ * to the brokers only and therefore this aids in conversion of the object to a representation that the data servers
+ * can recognize
+ */
+public class FramesBackedInlineDataSourceSerializer extends StdSerializer<FramesBackedInlineDataSource>
+{
+  public FramesBackedInlineDataSourceSerializer()
+  {
+    super(FramesBackedInlineDataSource.class);
+  }
+
+  @Override
+  public void serialize(FramesBackedInlineDataSource value, JsonGenerator jg, SerializerProvider serializers)
+      throws IOException
+  {
+    jg.writeStartObject();
+    jg.writeStringField("type", "inline");
+
+    RowSignature rowSignature = value.getRowSignature();
+    jg.writeObjectField("columnNames", rowSignature.getColumnNames());
+    List<ColumnType> columnTypes = IntStream.range(0, rowSignature.size())
+                                            .mapToObj(i -> rowSignature.getColumnType(i).orElse(null))
+                                            .collect(Collectors.toList());
+    jg.writeObjectField("columnTypes", columnTypes);
+
+    jg.writeArrayFieldStart("rows");
+
+    value.getRowsAsSequence().forEach(row -> {
+      try {
+        JacksonUtils.writeObjectUsingSerializerProvider(jg, serializers, row);
+      }
+      catch (IOException e) {
+        // Ideally, this shouldn't be reachable.
+        // Wrap the IO exception in the runtime exception and propogate it forward
+        throw new RuntimeException(e);

Review Comment:
   Please add a exception message which says ```exception encountered while serializing `FramesBackedInlineDataSource` ```



-- 
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: commits-unsubscribe@druid.apache.org

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] cryptoe commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "cryptoe (via GitHub)" <gi...@apache.org>.
cryptoe commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1229354540


##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+public class ClientQuerySegmentWalkerUtils
+{
+  public enum SubqueryResultLimit
+  {
+    ROW_LIMIT,

Review Comment:
   Please add java docs here.



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -613,96 +638,149 @@ private static <T, QueryType extends Query<T>> DataSource toInlineDataSource(
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,
       final AtomicLong memoryLimitAccumulator,
+      final AtomicBoolean cannotMaterializeToFrames,
       final int limit,
-      long memoryLimit
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
   )
   {
-    final int limitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
-    boolean memoryLimitSet = memoryLimit >= 0;
-
-    if (limitAccumulator.get() >= limitToUse) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum[%d] reached",
-          limitToUse
-      );
-    }
-
-    if (memoryLimitSet && memoryLimitAccumulator.get() >= memoryLimit) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
-          memoryLimit
-      );
-    }
+    final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
 
     DataSource dataSource;
-    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
-    if (memoryLimitSet) {
-      try {
-        Optional<Sequence<FrameSignaturePair>> framesOptional = toolChest.resultsAsFrames(
+
+    switch (ClientQuerySegmentWalkerUtils.getLimitType(memoryLimit, cannotMaterializeToFrames.get())) {
+      case ROW_LIMIT:
+        if (limitAccumulator.get() >= rowLimitToUse) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] rows",
+              rowLimitToUse
+          );
+        }
+        dataSource = materializeResultsAsArray(
             query,
             results,
-            memoryLimit - memoryLimitAccumulator.get()
+            toolChest,
+            limitAccumulator,
+            limit
         );
-
-        if (!framesOptional.isPresent()) {
-          throw new ISE("The memory of the subqueries cannot be estimated correctly.");
+        break;
+      case MEMORY_LIMIT:
+        if (memoryLimitAccumulator.get() >= memoryLimit) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+              memoryLimit
+          );
         }
-
-        Sequence<FrameSignaturePair> frames = framesOptional.get();
-        List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
-        frames.forEach(
-            frame -> {
-              if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] bytes",
-                    memoryLimit
-                );
-
-              }
-              if (limitAccumulator.addAndGet(frame.getFrame().numRows()) >= limitToUse) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] rows",
-                    limitToUse
-                );
-              }
-              frameSignaturePairs.add(frame);
-            }
-        );
-        dataSource = new FramesBackedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query));
-      }
-      catch (ResourceLimitExceededException rlee) {
-        throw rlee;
-      }
-      catch (Exception e) {
-        log.info(
-            "Unable to write the subquery results to a frame. Results won't be accounted for in the memory "
-            + "calculation"
+        Optional<DataSource> maybeDataSource = materializeResultsAsFrames(
+            query,
+            results,
+            toolChest,
+            limitAccumulator,
+            memoryLimitAccumulator,
+            memoryLimit,
+            useNestedForUnknownTypeInSubquery
         );
-        throw e;
-      }
-    } else {
-      final RowSignature signature = toolChest.resultArraySignature(query);
-
-      final ArrayList<Object[]> resultList = new ArrayList<>();
-
-      toolChest.resultsAsArrays(query, results).accumulate(
-          resultList,
-          (acc, in) -> {
-            if (limitAccumulator.getAndIncrement() >= limitToUse) {
-              throw ResourceLimitExceededException.withMessage(
-                  "Subquery generated results beyond maximum[%d] rows",
-                  limitToUse
-              );
-            }
-            acc.add(in);
-            return acc;
+        if (!maybeDataSource.isPresent()) {
+          cannotMaterializeToFrames.set(true);
+          // Check if the previous row limit accumulator has exceeded the memory results
+          if (memoryLimitAccumulator.get() >= memoryLimit) {
+            throw ResourceLimitExceededException.withMessage(
+                "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+                memoryLimit
+            );
           }
-      );
-      dataSource = InlineDataSource.fromIterable(resultList, signature);
+          dataSource = materializeResultsAsArray(
+              query,
+              results,
+              toolChest,
+              limitAccumulator,
+              limit
+          );
+        } else {
+          dataSource = maybeDataSource.get();
+        }
+        break;
+      default:
+        throw new IAE("Only row based and memory based limiting is supported");
     }
     return dataSource;
   }
 
+  private static <T, QueryType extends Query<T>> Optional<DataSource> materializeResultsAsFrames(
+      final QueryType query,
+      final Sequence<T> results,
+      final QueryToolChest<T, QueryType> toolChest,
+      final AtomicInteger limitAccumulator,
+      final AtomicLong memoryLimitAccumulator,
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
+  )
+  {
+    Optional<Sequence<FrameSignaturePair>> framesOptional;
+
+    try {
+      framesOptional = toolChest.resultsAsFrames(
+          query,
+          results,
+          new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+          useNestedForUnknownTypeInSubquery
+      );
+    }
+    catch (Exception e) {
+      return Optional.empty();
+    }
+
+    if (!framesOptional.isPresent()) {
+      return Optional.empty();
+    }
+
+    Sequence<FrameSignaturePair> frames = framesOptional.get();
+    List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
+    frames.forEach(
+        frame -> {
+          limitAccumulator.addAndGet(frame.getFrame().numRows());
+          if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
+            throw ResourceLimitExceededException.withMessage(
+                "Subquery generated results beyond maximum[%d] bytes",

Review Comment:
   This method needs java docs. Line 730 is eating up exceptions to fall back. Lets document this. 



##########
server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java:
##########
@@ -613,96 +638,149 @@ private static <T, QueryType extends Query<T>> DataSource toInlineDataSource(
       final QueryToolChest<T, QueryType> toolChest,
       final AtomicInteger limitAccumulator,
       final AtomicLong memoryLimitAccumulator,
+      final AtomicBoolean cannotMaterializeToFrames,
       final int limit,
-      long memoryLimit
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
   )
   {
-    final int limitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
-    boolean memoryLimitSet = memoryLimit >= 0;
-
-    if (limitAccumulator.get() >= limitToUse) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum[%d] reached",
-          limitToUse
-      );
-    }
-
-    if (memoryLimitSet && memoryLimitAccumulator.get() >= memoryLimit) {
-      throw ResourceLimitExceededException.withMessage(
-          "Cannot issue subquery, maximum subquery result bytes[%d] reached",
-          memoryLimit
-      );
-    }
+    final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
 
     DataSource dataSource;
-    // Try to serialize the results into a frame only if the memory limit is set on the server or the query
-    if (memoryLimitSet) {
-      try {
-        Optional<Sequence<FrameSignaturePair>> framesOptional = toolChest.resultsAsFrames(
+
+    switch (ClientQuerySegmentWalkerUtils.getLimitType(memoryLimit, cannotMaterializeToFrames.get())) {
+      case ROW_LIMIT:
+        if (limitAccumulator.get() >= rowLimitToUse) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] rows",
+              rowLimitToUse
+          );
+        }
+        dataSource = materializeResultsAsArray(
             query,
             results,
-            memoryLimit - memoryLimitAccumulator.get()
+            toolChest,
+            limitAccumulator,
+            limit
         );
-
-        if (!framesOptional.isPresent()) {
-          throw new ISE("The memory of the subqueries cannot be estimated correctly.");
+        break;
+      case MEMORY_LIMIT:
+        if (memoryLimitAccumulator.get() >= memoryLimit) {
+          throw ResourceLimitExceededException.withMessage(
+              "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+              memoryLimit
+          );
         }
-
-        Sequence<FrameSignaturePair> frames = framesOptional.get();
-        List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
-        frames.forEach(
-            frame -> {
-              if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] bytes",
-                    memoryLimit
-                );
-
-              }
-              if (limitAccumulator.addAndGet(frame.getFrame().numRows()) >= limitToUse) {
-                throw ResourceLimitExceededException.withMessage(
-                    "Subquery generated results beyond maximum[%d] rows",
-                    limitToUse
-                );
-              }
-              frameSignaturePairs.add(frame);
-            }
-        );
-        dataSource = new FramesBackedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query));
-      }
-      catch (ResourceLimitExceededException rlee) {
-        throw rlee;
-      }
-      catch (Exception e) {
-        log.info(
-            "Unable to write the subquery results to a frame. Results won't be accounted for in the memory "
-            + "calculation"
+        Optional<DataSource> maybeDataSource = materializeResultsAsFrames(
+            query,
+            results,
+            toolChest,
+            limitAccumulator,
+            memoryLimitAccumulator,
+            memoryLimit,
+            useNestedForUnknownTypeInSubquery
         );
-        throw e;
-      }
-    } else {
-      final RowSignature signature = toolChest.resultArraySignature(query);
-
-      final ArrayList<Object[]> resultList = new ArrayList<>();
-
-      toolChest.resultsAsArrays(query, results).accumulate(
-          resultList,
-          (acc, in) -> {
-            if (limitAccumulator.getAndIncrement() >= limitToUse) {
-              throw ResourceLimitExceededException.withMessage(
-                  "Subquery generated results beyond maximum[%d] rows",
-                  limitToUse
-              );
-            }
-            acc.add(in);
-            return acc;
+        if (!maybeDataSource.isPresent()) {
+          cannotMaterializeToFrames.set(true);
+          // Check if the previous row limit accumulator has exceeded the memory results
+          if (memoryLimitAccumulator.get() >= memoryLimit) {
+            throw ResourceLimitExceededException.withMessage(
+                "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
+                memoryLimit
+            );
           }
-      );
-      dataSource = InlineDataSource.fromIterable(resultList, signature);
+          dataSource = materializeResultsAsArray(
+              query,
+              results,
+              toolChest,
+              limitAccumulator,
+              limit
+          );
+        } else {
+          dataSource = maybeDataSource.get();
+        }
+        break;
+      default:
+        throw new IAE("Only row based and memory based limiting is supported");
     }
     return dataSource;
   }
 
+  private static <T, QueryType extends Query<T>> Optional<DataSource> materializeResultsAsFrames(
+      final QueryType query,
+      final Sequence<T> results,
+      final QueryToolChest<T, QueryType> toolChest,
+      final AtomicInteger limitAccumulator,
+      final AtomicLong memoryLimitAccumulator,
+      long memoryLimit,
+      boolean useNestedForUnknownTypeInSubquery
+  )
+  {
+    Optional<Sequence<FrameSignaturePair>> framesOptional;
+
+    try {
+      framesOptional = toolChest.resultsAsFrames(
+          query,
+          results,
+          new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
+          useNestedForUnknownTypeInSubquery
+      );
+    }
+    catch (Exception e) {
+      return Optional.empty();

Review Comment:
   Please add some debug line so that we know the exception here.



-- 
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: commits-unsubscribe@druid.apache.org

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] LakshSingla commented on a diff in pull request #13952: Limit the subquery results by memory usage

Posted by "LakshSingla (via GitHub)" <gi...@apache.org>.
LakshSingla commented on code in PR #13952:
URL: https://github.com/apache/druid/pull/13952#discussion_r1203307982


##########
processing/src/main/java/org/apache/druid/query/QueryToolChest.java:
##########
@@ -330,4 +331,32 @@ public Sequence<Object[]> resultsAsArrays(QueryType query, Sequence<ResultType>
   {
     throw new UOE("Query type '%s' does not support returning results as arrays", query.getType());
   }
+
+  /**
+   * Converts a sequence of this query's ResultType into a sequence of {@link FrameSignaturePair}. The array signature
+   * is the one give by {@link #resultArraySignature(Query)}. If the toolchest doesn't support this method, then it can
+   * return an empty optional. It is the duty of the callees to throw an appropriate exception in that case or use an
+   * alternative fallback approach
+   *
+   * Check documentation of {@link #resultsAsArrays(Query, Sequence)} as the behaviour of the rows represented by the
+   * frame sequence is identical.
+   *
+   * Each Frame has a separate {@link RowSignature} because for some query types like the Scan query, every
+   * column in the final result might not be present in the individual ResultType (and subsequently Frame). Therefore,
+   * this is done to preserve the space by not populating the column in that particular Frame and omitting it from its
+   * signature
+   * 
+   * @param query Query being executed by the toolchest. Used to determine the rowSignature of the Frames
+   * @param resultSequence results of the form returned by {@link #mergeResults(QueryRunner)}
+   * @param memoryLimitBytes Limit the memory results. Throws {@link ResourceLimitExceededException} if the result exceed
+   *                    the memoryLimitBytes
+   */
+  public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(

Review Comment:
   I have updated the code with the suggestion here. 
   While this is a much better way to tackle this and give a lot more autonomy to the callers, this suffers from one issue that we see with MSQ ingestions containing large data sketches, that Frame size is insufficient to hold a single row. We can set the limit higher here to something like 128MB, however, then it would mean that we would potentially be materializing 128MBs more than what we require (instead of fine graning it at a singular row level). 
   Any ways to counteract this? 



-- 
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: commits-unsubscribe@druid.apache.org

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