You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/09/07 21:50:21 UTC

[GitHub] [druid] paul-rogers commented on a diff in pull request #13031: ScanQuery supports multi column orderBy queries

paul-rogers commented on code in PR #13031:
URL: https://github.com/apache/druid/pull/13031#discussion_r965322734


##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java:
##########
@@ -252,7 +260,295 @@ public void cleanup(Iterator<ScanResultValue> iterFromMake)
             ))
     );
   }
+  private Sequence<ScanResultValue> getScanOrderByResultValueSequence(

Review Comment:
   Is the scan query engine the right place for a sort? Scan queries are distributed. Each data node scans a separate segment. Should we do a sort per data node? I suppose yes, since then the Broker need only merge.
   
   The Druid pattern seems to be to insert another `QueryRunner`/`Sequence` pair for a new task. So, we'd want to insert a "sorting query runner" somewhere so that the sorting operation can be separate from the "segment scan" operation. This split avoids making the scan query engine more complex than it already is.
   
   Also, where should the sort occur? Per cursor? Per segment? Per node? If per node, then only the Broker merges. If per node (i.e., if node 1 scans 5 segments, sort the results of all 5), then the operation has to be done above the scan query engine level: at the point where we have visibility to all the data for the node.
   
   If per segment, then we have to merge per-node and in the Broker.
   
   I wonder, does this draft handle these cases?



##########
processing/src/main/java/org/apache/druid/query/scan/ScanQueryOrderByLimitRowIterator.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.Iterators;
+import org.apache.druid.collections.MultiColumnSorter;
+import org.apache.druid.java.util.common.UOE;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryRunner;
+import org.apache.druid.query.context.ResponseContext;
+
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class ScanQueryOrderByLimitRowIterator extends ScanQueryLimitRowIterator

Review Comment:
   I would suggest not combing the order by operation with the limit operation. The typical way this is done is to sort in the leaf nodes (or, if we like, in the root node), then start pulling from the result set. The limit (and offset) sit on top, and control what we pull.



##########
processing/src/main/java/org/apache/druid/collections/MultiColumnSorter.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.collections;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.MinMaxPriorityQueue;
+import com.google.common.collect.Ordering;
+import org.apache.druid.java.util.common.ISE;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+
+public class MultiColumnSorter<T>

Review Comment:
   This class is based on a priority queue. Thus, it would seem to be a "multi column merger" of previously-sorted values. Or, are we using a priority queue to sort?



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