You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ilooner <gi...@git.apache.org> on 2017/11/03 20:25:09 UTC

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Lead. DRILL-5926 ...

GitHub user ilooner opened a pull request:

    https://github.com/apache/drill/pull/1023

    DRILL-5922 Fixed Child Allocator Lead. DRILL-5926 Stabalize TestValueVector tests.

    ##DRILL-5922
    
     - QueryContext was never closed when the Foreman finished. So the query child allocator was never closed.
     - The PlanSplitter created a QueryContext temporarily to construct an RPC message but never closed it.
     - The waitForExit method was error prone. Changed it to use the standard condition variable pattern.
    
    ##DRILL-5926
    
    The TestValueVector tests would run out of memory. I simple increased the MaxDirectMemorySize for the forked test processes in the pom to avoid this.
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ilooner/drill DRILL-5922

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/1023.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1023
    
----
commit f4058a93e70b21328f1f06e1e1ede5451d00b499
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-11-02T18:27:56Z

     - Deleted commented out code in unit test

commit 56fd6929d3ef738e1d756c067e4e4f80cf965e37
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-11-02T20:34:47Z

     - Removed non-existant parameter from java docs

commit a32c3bed4cf52f15bcf2d0e284c04329827b32b7
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-11-03T16:11:04Z

     - Initial attempt at fix

commit c0afcc8853e4449ec5d7de6dfeee51efc8ad70ab
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-11-03T18:19:08Z

     - Fixed hanging unit tests

commit d1cd5672e1da1553b0d8cf94c5b3c47354d417a3
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-11-03T18:28:47Z

     - More wait to exit improvements

commit ce3a9555482b02a19579c2388a65c4dd29353588
Author: Timothy Farkas <ti...@apache.org>
Date:   2017-11-03T19:14:24Z

     - Deleted unused logger
     - Increased direct memory to prevent TestValueVector tests from running out of memory

----


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148894158
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,51 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +
    +    try {
    +      exitLock.lock();
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            logger.warn("Timed out after %d millis while waiting to exit.", EXIT_TIMEOUT);
    +            exitLock.lock();
    --- End diff --
    
    If the await times out, the condition variable does not automatically reacquire the lock for us, so we must do it manually.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/drill/pull/1023


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149257347
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    final long endTime = startTime + EXIT_TIMEOUT;
    +
    +    exitLock.lock();
    +
    +    try {
    +      long currentTime;
    +      while ((currentTime = System.currentTimeMillis()) < endTime) {
    --- End diff --
    
    Done


---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabali...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    @arina-ielchiieva Please let me know if you have any additional comments.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148997637
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java ---
    @@ -79,6 +80,15 @@ public QueryPlanFragments planFragments(DrillbitContext dContext, QueryId queryI
           responseBuilder.setStatus(QueryState.FAILED);
           responseBuilder.setError(error);
         }
    +
    +    try {
    +      queryContext.close();
    +    } catch (Exception e) {
    +      final String message = String.format("Error closing QueryContext when getting plan fragments for query %s",
    +        QueryIdHelper.getQueryId(queryId));
    +      logger.error(message, e);
    --- End diff --
    
    See above re log message formatting.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149257480
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    final long endTime = startTime + EXIT_TIMEOUT;
    +
    +    exitLock.lock();
    +
    +    try {
    +      long currentTime;
    +      while ((currentTime = System.currentTimeMillis()) < endTime) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          if (!exitCondition.await(endTime - currentTime, TimeUnit.MILLISECONDS)) {
    +            break;
    +          }
    +        } catch (InterruptedException e) {
    +          logger.error("Interrupted while waiting to exit");
    +        }
           }
     
    -      exitLatch = new ExtendedLatch();
    -    }
    +      if (!(queries.isEmpty() && runningFragments.isEmpty())) {
    +        logger.warn("Timed out after {} millis. Shutting down before all fragments and foremen " +
    +          "have completed.", EXIT_TIMEOUT);
    +      }
     
    -    // Wait for at most 5 seconds or until the latch is released.
    -    exitLatch.awaitUninterruptibly(5000);
    +    } finally {
    +      exitLock.unlock();
    +    }
       }
     
       /**
    -   * If it is safe to exit, and the exitLatch is in use, signals it so that waitToExit() will
    -   * unblock.
    +   * A thread calling the {@link #waitToExit()} method is notified when a foreman is retired.
        */
       private void indicateIfSafeToExit() {
    -    synchronized(this) {
    -      if (exitLatch != null) {
    -        if (queries.isEmpty() && runningFragments.isEmpty()) {
    -          exitLatch.countDown();
    -        }
    -      }
    -    }
    +    exitLock.lock();
    +    exitCondition.signal();
    --- End diff --
    
    *isEmpty* reminds me of a method name. I renamed it to *isEmptyCondition* instead.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r150327067
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
    I will move DRILL-5926 into a separate commit.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner closed the pull request at:

    https://github.com/apache/drill/pull/1023


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148896551
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,51 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +
    +    try {
    +      exitLock.lock();
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            logger.warn("Timed out after %d millis while waiting to exit.", EXIT_TIMEOUT);
    +            exitLock.lock();
    +          }
    +        } catch (InterruptedException e) {
    +          logger.warn("Interrupted while waiting to exit");
    +          exitLock.lock();
    +        }
           }
     
    -      exitLatch = new ExtendedLatch();
    -    }
    +      if (!(queries.isEmpty() && runningFragments.isEmpty())) {
    +        logger.warn("Timed out after %d millis. Shutting down before all fragments and foremen " +
    +          "have completed.", EXIT_TIMEOUT);
    --- End diff --
    
    This should actually just break from the loop instead of print a message, since EXIT_TIMEOUT millis have elapsed in this case, and the Timed out message is already printed below. I've updated the PR with this change.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148997959
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    exitLock.lock();
    +
    +    try {
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    --- End diff --
    
    Simpler:
    ```
    final long endTime = System.currentTimeMillis() + EXIT_TIMEOUT;
    ...
    while (System.currentTimeMillis() < endTime) {
    ...
    ```


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r150348155
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
    I am in favor of 1 as well.
    
    @paul-rogers One of the test that fails initially tries to allocate 2GB (without re-allocation). The test intermittently fails due to Pooled Allocator not releasing memory back to the system. I don't know if it is supposed to return memory back to the system when it is closed and whether it is supposed to be closed at all during the unit tests.


---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabali...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    @paul-rogers Can you take another pass at reviewing this? I have resolved conflicts with the graceful shutdown work. The graceful shutdown tests were also timing out on my laptop so I fixed them. 


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148893046
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,51 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +
    +    try {
    +      exitLock.lock();
    --- End diff --
    
    Fixed


---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabali...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    @paul-rogers Applied review comments. I'll squash commits and mark this as ready to commit



---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r155938262
  
    --- Diff: common/src/test/java/org/apache/drill/test/DrillTest.java ---
    @@ -54,7 +54,7 @@
       static MemWatcher memWatcher;
       static String className;
     
    -  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(100000);
    +  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(180000);
    --- End diff --
    
    Two comments. One a nit: I find it hard to read 180000. Suggestion, use 180_000.
    
    More substantial comment: 180K ms = 180 seconds = 3 minutes. This seems FAR too long for a default timeout.
    
    I wonder, should we make this much smaller? Then, find the tests that need extra time and mark them specially? Maybe tag them with "SlowTests" and set a larger limit for just those tests?
    
    This will cause tests to fail if they are too slow. That, in turn, will force us to either fix those tests, or label them as slow.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148891238
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
     Did all the regression tests pass ?  This change could have some impact on other tests (e.g., if was expecting an OOM, now gone).



---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148997788
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -59,12 +61,14 @@
     public class WorkManager implements AutoCloseable {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkManager.class);
     
    +  public static final long EXIT_TIMEOUT = 30000L;
    --- End diff --
    
    `30000L` --> `30_000L` for quicker reading.


---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabali...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    Travis build is flakey. Rerunning build.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r155938285
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -61,12 +63,14 @@
     public class WorkManager implements AutoCloseable {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkManager.class);
     
    +  public static final long EXIT_TIMEOUT = 5_000L;
    --- End diff --
    
    Three nits:
    
    1. We'll never wait decades, so this need not be a long.
    2. Please encode units in the name: `EXIT_TIMEOUT_MS`
    3. The underscore is great, but not entirely needed for four-digit numbers.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149167742
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    exitLock.lock();
    +
    +    try {
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            break;
    +          }
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149257392
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    final long endTime = startTime + EXIT_TIMEOUT;
    +
    +    exitLock.lock();
    +
    +    try {
    +      long currentTime;
    +      while ((currentTime = System.currentTimeMillis()) < endTime) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          if (!exitCondition.await(endTime - currentTime, TimeUnit.MILLISECONDS)) {
    +            break;
    +          }
    +        } catch (InterruptedException e) {
    +          logger.error("Interrupted while waiting to exit");
    +        }
           }
     
    -      exitLatch = new ExtendedLatch();
    -    }
    +      if (!(queries.isEmpty() && runningFragments.isEmpty())) {
    +        logger.warn("Timed out after {} millis. Shutting down before all fragments and foremen " +
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149230560
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    final long endTime = startTime + EXIT_TIMEOUT;
    +
    +    exitLock.lock();
    +
    +    try {
    +      long currentTime;
    +      while ((currentTime = System.currentTimeMillis()) < endTime) {
    --- End diff --
    
    I'd recommend changing `while` condition to `queries` and `runningFragments` not empty check as a primary condition for exiting. Timeout (waitTime >0) is the secondary condition. 


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148894504
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,51 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +
    +    try {
    +      exitLock.lock();
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            logger.warn("Timed out after %d millis while waiting to exit.", EXIT_TIMEOUT);
    +            exitLock.lock();
    +          }
    +        } catch (InterruptedException e) {
    +          logger.warn("Interrupted while waiting to exit");
    +          exitLock.lock();
    --- End diff --
    
    1. If await is interrupted the condition variable does not acquire the lock for us, so we have to acquire it manually.
    2. I'll change from warning to error.


---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabali...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    @paul-rogers For some reason this wan't included in a batch commit. After rebasing onto master I noticed some tests were failing sporadically so I've included a 1 line fix for DRILL-6003 in the last commit. Can you please take a look and +1 again? Hopefully the change will go in this time.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148890716
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,51 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +
    +    try {
    +      exitLock.lock();
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            logger.warn("Timed out after %d millis while waiting to exit.", EXIT_TIMEOUT);
    +            exitLock.lock();
    +          }
    +        } catch (InterruptedException e) {
    +          logger.warn("Interrupted while waiting to exit");
    +          exitLock.lock();
    --- End diff --
    
    1. Why calling lock() here ?
    2. Should the code issue an error instead of a warning ?



---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Lead. DRILL-5926 Stabali...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    @Ben-Zvi @paul-rogers 


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r150336286
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
    This is a fix for a test that we don't really even need. We are attempting to verify that we can allocate a single value vector of 2 GB by doubling from a small amount. On the last allocation, we have a 1 GB vector doubling to 2 GB, so we temporarily need 3 GB.
    
    But, vectors should never get this large. The batch size limitation project is doing work to limit vectors to the 16 MB Netty slab size.
    
    For this reason, this is a workaround to a test for a feature that is actually a bug which we will be fixing in a later release.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149166078
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -59,12 +61,14 @@
     public class WorkManager implements AutoCloseable {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkManager.class);
     
    +  public static final long EXIT_TIMEOUT = 30000L;
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149232323
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    final long endTime = startTime + EXIT_TIMEOUT;
    +
    +    exitLock.lock();
    +
    +    try {
    +      long currentTime;
    +      while ((currentTime = System.currentTimeMillis()) < endTime) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          if (!exitCondition.await(endTime - currentTime, TimeUnit.MILLISECONDS)) {
    +            break;
    +          }
    +        } catch (InterruptedException e) {
    +          logger.error("Interrupted while waiting to exit");
    +        }
           }
     
    -      exitLatch = new ExtendedLatch();
    -    }
    +      if (!(queries.isEmpty() && runningFragments.isEmpty())) {
    +        logger.warn("Timed out after {} millis. Shutting down before all fragments and foremen " +
    --- End diff --
    
    It may be good to add what queries and runningFragments are running to the log message.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149234360
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    final long endTime = startTime + EXIT_TIMEOUT;
    +
    +    exitLock.lock();
    +
    +    try {
    +      long currentTime;
    +      while ((currentTime = System.currentTimeMillis()) < endTime) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          if (!exitCondition.await(endTime - currentTime, TimeUnit.MILLISECONDS)) {
    +            break;
    +          }
    +        } catch (InterruptedException e) {
    +          logger.error("Interrupted while waiting to exit");
    +        }
           }
     
    -      exitLatch = new ExtendedLatch();
    -    }
    +      if (!(queries.isEmpty() && runningFragments.isEmpty())) {
    +        logger.warn("Timed out after {} millis. Shutting down before all fragments and foremen " +
    +          "have completed.", EXIT_TIMEOUT);
    +      }
     
    -    // Wait for at most 5 seconds or until the latch is released.
    -    exitLatch.awaitUninterruptibly(5000);
    +    } finally {
    +      exitLock.unlock();
    +    }
       }
     
       /**
    -   * If it is safe to exit, and the exitLatch is in use, signals it so that waitToExit() will
    -   * unblock.
    +   * A thread calling the {@link #waitToExit()} method is notified when a foreman is retired.
        */
       private void indicateIfSafeToExit() {
    -    synchronized(this) {
    -      if (exitLatch != null) {
    -        if (queries.isEmpty() && runningFragments.isEmpty()) {
    -          exitLatch.countDown();
    -        }
    -      }
    -    }
    +    exitLock.lock();
    +    exitCondition.signal();
    --- End diff --
    
    I'd recommend adding try/finally and checking the condition before signaling `exitCondition`. Consider renaming `exitCondition` to `isEmpty` or `isQueriesAndFragmentsEmpty`. 


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r150302692
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
    I could not properly test my PR because of a pre-existing sporadic test failure. This fix was necessary in order to test my changes. Errors with the builds should be fixed immediately, especially when they interfere with testing a change. So I will keep the fix in this PR.


---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabali...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    For commit, let's do this:
    
    * With luck, Arina will commit two PRs this week that may conflict: PR #970, and PR #978.
    * Tim should rebase this PR on top of those changes once they are committed.
    * If the changes are non-trivial, they may require additional review. If so, Paul to do that early the week of the 13th.
    * If there is time to squeeze in this commit before Arina starts the 1.12, release, Paul will do a one-off commit.
    * Otherwise, since Arina wants to do the release during that week, this commit may have to wait until after that release.
    * Once this commit is in, Paul to rebase the PR #914, as it changed some of the same files.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149168081
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -828,8 +828,13 @@ public void close() throws Exception {
             queryManager.writeFinalProfile(uex);
           }
     
    -      // Remove the Foreman from the running query list.
    -      bee.retireForeman(Foreman.this);
    +      try {
    +        queryContext.close();
    +      } catch (Exception e) {
    +        final String message = String.format("Unable to close query context for query {}",
    +          QueryIdHelper.getQueryId(queryId));
    +        logger.error(message, e);
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149167716
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    exitLock.lock();
    +
    +    try {
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149167784
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    exitLock.lock();
    +
    +    try {
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            break;
    +          }
    +        } catch (InterruptedException e) {
    +          logger.error("Interrupted while waiting to exit");
    +        }
           }
     
    -      exitLatch = new ExtendedLatch();
    -    }
    +      if (!(queries.isEmpty() && runningFragments.isEmpty())) {
    +        logger.warn("Timed out after %d millis. Shutting down before all fragments and foremen " +
    +          "have completed.", EXIT_TIMEOUT);
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148889490
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,51 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +
    +    try {
    +      exitLock.lock();
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            logger.warn("Timed out after %d millis while waiting to exit.", EXIT_TIMEOUT);
    +            exitLock.lock();
    +          }
    +        } catch (InterruptedException e) {
    +          logger.warn("Interrupted while waiting to exit");
    +          exitLock.lock();
    +        }
           }
     
    -      exitLatch = new ExtendedLatch();
    -    }
    +      if (!(queries.isEmpty() && runningFragments.isEmpty())) {
    +        logger.warn("Timed out after %d millis. Shutting down before all fragments and foremen " +
    +          "have completed.", EXIT_TIMEOUT);
    --- End diff --
    
    The "time out" time should be `diff`, not EXIT_TIMEOUT 
     


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r150322554
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
    DRILL-5926 affects not only this PR but my PR  #1031 as well even though they are not related at all. At a minimum the change should be in a separate commit that refers to DRILL-5926 as it is a workaround and not an actual fix for the problem. In the future, the workaround may need to be reverted and if it is mixed with other changes, it will not be clear what needs to be reverted. Note that with other changes I would expect tests to require less memory, not more.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148997565
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -828,8 +828,13 @@ public void close() throws Exception {
             queryManager.writeFinalProfile(uex);
           }
     
    -      // Remove the Foreman from the running query list.
    -      bee.retireForeman(Foreman.this);
    +      try {
    +        queryContext.close();
    +      } catch (Exception e) {
    +        final String message = String.format("Unable to close query context for query {}",
    +          QueryIdHelper.getQueryId(queryId));
    +        logger.error(message, e);
    --- End diff --
    
    ```
    logger.error("Unable...query {}", QueryIdHelper...);
    ```
    
    The `{}` is a logger format pattern, not a `String.format()` pattern.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148889092
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,51 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +
    +    try {
    +      exitLock.lock();
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            logger.warn("Timed out after %d millis while waiting to exit.", EXIT_TIMEOUT);
    +            exitLock.lock();
    --- End diff --
    
    Why is this lock necessary?


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
GitHub user ilooner reopened a pull request:

    https://github.com/apache/drill/pull/1023

    DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabalize TestValueVector tests.

    ## DRILL-5922
    
     - QueryContext was never closed when the Foreman finished. So the query child allocator was never closed.
     - The PlanSplitter created a QueryContext temporarily to construct an RPC message but never closed it.
     - The waitForExit method was error prone. Changed it to use the standard condition variable pattern.
    
    ## DRILL-5926
    
    The TestValueVector tests would run out of memory. I simple increased the MaxDirectMemorySize for the forked test processes in the pom to avoid this.
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ilooner/drill DRILL-5922

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/1023.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1023
    
----
commit 005a478238ff8a0ec085e051c099a109a73d3e8a
Author: Timothy Farkas <ti...@...>
Date:   2017-11-10T19:13:18Z

    DRILL-5922:
     - The QueryContext was never closed when the Foreman finished, so it's child allocator was never closed. Now it is.
     - The PlanSplitter created a QueryContext temporarily to construct an RPC message but never closed it. Now the temp QueryContext is closed.
     - The waitForExit method was error prone. Changed it to use the standard condition variable pattern.
     - Fixed timeouts in graceful shutdown tests

commit 551c54eca5ed9a69b990daa2a300c850e11e9376
Author: Timothy Farkas <ti...@...>
Date:   2017-11-10T20:59:42Z

    DRILL-5926: The TestValueVector tests would run out of memory. Increased the MaxDirectMemorySize for the forked test processes in the pom to avoid this.

commit 31175885bcd60abde5ec88ff3fcece589619265f
Author: Timothy Farkas <ti...@...>
Date:   2018-01-09T09:20:09Z

    DRILL-6003: Fixed sporadic test failures caused by the creation of duplicate clusters in TestDynamicUDFSupport.

----


---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabali...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    *Note for batch committer:* Please do not squash the two commits in this PR. Please see the discussion on **pom.xml** for details.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r159131381
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
    Why not just retire the test. As explained above, no vector should ever allocate 2 GB unless we do some fixes to the Netty allocator. (Such a large vector will likely introduce issues in the network layer, but we don't have any user JIRAs to confirm this is, indeed, a problem.)


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148997365
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    exitLock.lock();
    +
    +    try {
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            break;
    +          }
    +        } catch (InterruptedException e) {
    +          logger.error("Interrupted while waiting to exit");
    +        }
           }
     
    -      exitLatch = new ExtendedLatch();
    -    }
    +      if (!(queries.isEmpty() && runningFragments.isEmpty())) {
    +        logger.warn("Timed out after %d millis. Shutting down before all fragments and foremen " +
    +          "have completed.", EXIT_TIMEOUT);
    --- End diff --
    
    The logger has formatting, but does not use `String.format()` formats. Instead, it uses `{}`:
    
    ```
    logger.warn("Timed out after {} millis...", EXIT_TIMEOUT);
    ```


---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabali...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    @arina-ielchiieva Spoke with @paul-rogers . This will likely introduce some conflicts for https://github.com/apache/drill/pull/921 . In an effort to lighten the workload for that contributor we'll delay merging this change. I'll remove the ready-to-commit label on the jira until the other PR is merged. After https://github.com/apache/drill/pull/921 is merged I'll rebase and mark this as read-to-commit once more.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r150339444
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
    @paul-rogers I think there are a few choices available to us:
    
     1. Keep this change in a separate commit (already done) and revert it once the Batch Size Limitation work is done. The benefit is that the build will not be broken up until the Batch Size work is in. I can make a note on the Batch Size jira to revert this change as well.
     1. Remove this change and suffer with a broken build until the Batch Size work is complete. (not fun)
     1. Delete the offending tests instead as part of this PR.
    
    I am in favor of option **1**. I am strongly against option **2** since ignoring build issues makes life much harder for everyone.
    
    Please let me know what path forward you'd like to take.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r150345195
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
    I'm in favor of 1 for now, then we can revert it, if needed, once the batch size work is available and we remove the offending test (option 3).


---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabali...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    Applied review comments


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149173898
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java ---
    @@ -79,6 +80,15 @@ public QueryPlanFragments planFragments(DrillbitContext dContext, QueryId queryI
           responseBuilder.setStatus(QueryState.FAILED);
           responseBuilder.setError(error);
         }
    +
    +    try {
    +      queryContext.close();
    +    } catch (Exception e) {
    +      final String message = String.format("Error closing QueryContext when getting plan fragments for query %s",
    +        QueryIdHelper.getQueryId(queryId));
    +      logger.error(message, e);
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r149168826
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    exitLock.lock();
    +
    +    try {
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            break;
    +          }
    +        } catch (InterruptedException e) {
    +          logger.error("Interrupted while waiting to exit");
    +        }
           }
     
    -      exitLatch = new ExtendedLatch();
    -    }
    +      if (!(queries.isEmpty() && runningFragments.isEmpty())) {
    +        logger.warn("Timed out after %d millis. Shutting down before all fragments and foremen " +
    +          "have completed.", EXIT_TIMEOUT);
    --- End diff --
    
    Done


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r150259975
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
    It may be better to open a separate PR for DRILL-5926


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148998119
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,49 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +    exitLock.lock();
    +
    +    try {
    +      long diff;
    +      while ((diff = (System.currentTimeMillis() - startTime)) < EXIT_TIMEOUT) {
    +        if (queries.isEmpty() && runningFragments.isEmpty()) {
    +          break;
    +        }
    +
    +        try {
    +          final boolean success = exitCondition.await(EXIT_TIMEOUT - diff, TimeUnit.MILLISECONDS);
    +
    +          if (!success) {
    +            break;
    +          }
    --- End diff --
    
    ```
    if (! exitCondition.await(...)) {
      break;
    }
    ```


---

[GitHub] drill issue #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 Stabali...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on the issue:

    https://github.com/apache/drill/pull/1023
  
    @ilooner please resolve conflicts.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148888668
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java ---
    @@ -158,38 +165,51 @@ public DrillbitContext getContext() {
         return dContext;
       }
     
    -  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
    -
       /**
        * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
    -   *
    -   * <p>This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.</p>
    +   * This is intended to be used by {@link org.apache.drill.exec.server.Drillbit#close()}.
        */
       public void waitToExit() {
    -    synchronized(this) {
    -      if (queries.isEmpty() && runningFragments.isEmpty()) {
    -        return;
    +    final long startTime = System.currentTimeMillis();
    +
    +    try {
    +      exitLock.lock();
    --- End diff --
    
    Usually, lock is called outside of try to avoid calling unlock in finally if lock was not successful.


---

[GitHub] drill pull request #1023: DRILL-5922 Fixed Child Allocator Leak. DRILL-5926 ...

Posted by ilooner <gi...@git.apache.org>.
Github user ilooner commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1023#discussion_r148892741
  
    --- Diff: pom.xml ---
    @@ -442,7 +442,7 @@
                   -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
                   -Ddrill.test.query.printing.silent=true
                   -Ddrill.catastrophic_to_standard_out=true
    -              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=3072M
    +              -XX:MaxPermSize=512M -XX:MaxDirectMemorySize=4096M
    --- End diff --
    
    All the unit tests passed locally on my laptop. They also passed on our build server along with all of the functional and TPCH tests.



---