gianm commented on code in PR #16372:
URL: https://github.com/apache/druid/pull/16372#discussion_r1588038254


##########
processing/src/main/java/org/apache/druid/query/groupby/GroupByResourcesReservationPool.java:
##########
@@ -104,18 +104,23 @@ public GroupByResourcesReservationPool(
   }
 
   /**
-   * Reserves appropriate resources, and maps it to the queryResourceId 
(usually the query's resource id) in the internal map
+   * Reserves appropriate resources, and maps it to the queryResourceId 
(usually the query's resource id) in the internal map.
+   * This is a blocking call, and can block upto the given query's timeout

Review Comment:
   up to (spelling)



##########
processing/src/test/java/org/apache/druid/query/groupby/GroupByResourcesReservationPoolTest.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.groupby;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.collections.BlockingPool;
+import org.apache.druid.collections.DefaultBlockingPool;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.query.QueryResourceId;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+
+public class GroupByResourcesReservationPoolTest
+{
+
+  /**
+   * This test confirms that the interleaved 
GroupByResourcesReservationPool.reserve() and 
GroupByResourcesReservationPool.clean()
+   * between multiple threads succeed. It is specifically designed to test the 
case when the operations are interleaved in the
+   * following manner:
+   * <p>
+   * THREAD1                      THREAD2
+   * pool.reserve(query1)
+   *                              pool.reserve(query2)
+   * pool.clean(query1)
+   * <p>
+   * This test assumes a few things about the implementation of the 
interfaces, which are laid out in the comments.
+   * <p>
+   * The test should complete under 10 seconds, and the majority of the time 
would be consumed by waiting for the thread
+   * that sleeps for 5 seconds

Review Comment:
   I don't think we should add new unit tests that have sleeps; the test suite 
takes long enough to run already. They are also a sign of a test that is not 
robust.
   
   Is it possible to rewrite this test to not use a sleep? If not, I'd suggest 
having it be `@Ignore` so it doesn't run on every test suite run. Then include 
a comment in the code of `GroupByResourcesReservationPool` itself that says if 
a future developer is changing the logic, they should run this test manually to 
ensure they aren't introducing a deadlock.



##########
processing/src/main/java/org/apache/druid/query/groupby/GroupByResourcesReservationPool.java:
##########
@@ -104,18 +104,23 @@ public GroupByResourcesReservationPool(
   }
 
   /**
-   * Reserves appropriate resources, and maps it to the queryResourceId 
(usually the query's resource id) in the internal map
+   * Reserves appropriate resources, and maps it to the queryResourceId 
(usually the query's resource id) in the internal map.
+   * This is a blocking call, and can block upto the given query's timeout
    */
   public void reserve(QueryResourceId queryResourceId, GroupByQuery 
groupByQuery, boolean willMergeRunner)
   {
     if (queryResourceId == null) {
       throw DruidException.defensive("Query resource id must be populated");
     }
+    GroupByQueryResources resources =
+        GroupingEngine.prepareResource(groupByQuery, mergeBufferPool, 
willMergeRunner, groupByQueryConfig);
+
     pool.compute(queryResourceId, (id, existingResource) -> {
       if (existingResource != null) {
+        resources.close();

Review Comment:
   Rather than risk `resources.close()` holding the `compute` lock for too 
long, how about using `pool.putIfAbsent` instead, and then if `putIfAbsent` 
returns nonnull (signifying there was some existing resource), then call 
`resources.close()` and throw the defensive exception.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

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


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

Reply via email to