You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2016/12/14 18:38:01 UTC

[1/2] lucene-solr:master: SOLR-1953: It may be possible for temporary files to accumulate until the Solr process is shut down.

Repository: lucene-solr
Updated Branches:
  refs/heads/master 22d9af41a -> 7dec783b2


SOLR-1953: It may be possible for temporary files to accumulate until the Solr process is shut down.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/e82399d0
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/e82399d0
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/e82399d0

Branch: refs/heads/master
Commit: e82399d0677651ad4be1d8d2bdc4777b5d90b0fa
Parents: 22d9af4
Author: markrmiller <ma...@apache.org>
Authored: Mon Dec 12 11:10:58 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 14 11:16:50 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../apache/solr/servlet/SolrDispatchFilter.java |  15 ++
 .../apache/solr/servlet/SolrRequestParsers.java |  31 ++--
 .../solr/util/SolrFileCleaningTracker.java      | 147 +++++++++++++++++++
 4 files changed, 182 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 41af0ff..946a04e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -244,6 +244,9 @@ Bug Fixes
 
 * SOLR-9823: CoreContainer incorrectly setting MDCLoggingContext for core (Jessica Cheng Mallet via Erick Erickson)
 
+* SOLR-1953: It may be possible for temporary files to accumulate until the Solr process is shut down.
+  (Karl Wright, Mark Miller)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 5a4cfb6..e8c4657 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -45,6 +45,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.commons.io.FileCleaningTracker;
 import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.commons.io.output.CloseShieldOutputStream;
 import org.apache.commons.lang.StringUtils;
@@ -62,6 +63,7 @@ import org.apache.solr.core.SolrXmlConfig;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
+import org.apache.solr.util.SolrFileCleaningTracker;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -123,6 +125,8 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   {
     log.trace("SolrDispatchFilter.init(): {}", this.getClass().getClassLoader());
 
+    SolrRequestParsers.fileCleaningTracker = new SolrFileCleaningTracker();
+    
     StartupLoggingUtils.checkLogDir();
     logWelcomeBanner();
     String muteConsole = System.getProperty(SOLR_LOG_MUTECONSOLE);
@@ -240,6 +244,17 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   
   @Override
   public void destroy() {
+    try {
+      FileCleaningTracker fileCleaningTracker = SolrRequestParsers.fileCleaningTracker;
+      if (fileCleaningTracker != null) {
+        fileCleaningTracker.exitWhenFinished();
+      }
+    } catch (Exception e) {
+      log.warn("Exception closing FileCleaningTracker", e);
+    } finally {
+      SolrRequestParsers.fileCleaningTracker = null;
+    }
+
     if (cores != null) {
       try {
         cores.shutdown();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
index 9d7e7d9..968320e 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
@@ -42,6 +42,7 @@ import java.util.Map;
 import org.apache.commons.fileupload.FileItem;
 import org.apache.commons.fileupload.disk.DiskFileItemFactory;
 import org.apache.commons.fileupload.servlet.ServletFileUpload;
+import org.apache.commons.io.FileCleaningTracker;
 import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.common.SolrException;
@@ -58,6 +59,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequestBase;
 import org.apache.solr.util.RTimerTree;
+import org.apache.solr.util.SolrFileCleaningTracker;
 
 import static org.apache.solr.common.params.CommonParams.PATH;
 
@@ -88,6 +90,8 @@ public class SolrRequestParsers
   /** Default instance for e.g. admin requests. Limits to 2 MB uploads and does not allow remote streams. */
   public static final SolrRequestParsers DEFAULT = new SolrRequestParsers();
   
+  public static volatile SolrFileCleaningTracker fileCleaningTracker;
+  
   /**
    * Pass in an xml configuration.  A null configuration will enable
    * everything with maximum values.
@@ -532,31 +536,30 @@ public class SolrRequestParsers
   /**
    * Extract Multipart streams
    */
-  static class MultipartRequestParser implements SolrRequestParser
-  {
+  static class MultipartRequestParser implements SolrRequestParser {
     private final int uploadLimitKB;
+    private DiskFileItemFactory factory = new DiskFileItemFactory();
     
-    public MultipartRequestParser( int limit )
-    {
+    public MultipartRequestParser(int limit) {
       uploadLimitKB = limit;
+
+      // Set factory constraints
+      FileCleaningTracker fct = fileCleaningTracker;
+      if (fct != null) {
+        factory.setFileCleaningTracker(fileCleaningTracker);
+      }
+      // TODO - configure factory.setSizeThreshold(yourMaxMemorySize);
+      // TODO - configure factory.setRepository(yourTempDirectory);
     }
     
     @Override
-    public SolrParams parseParamsAndFillStreams( 
-        final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
-    {
+    public SolrParams parseParamsAndFillStreams(
+        final HttpServletRequest req, ArrayList<ContentStream> streams) throws Exception {
       if( !ServletFileUpload.isMultipartContent(req) ) {
         throw new SolrException( ErrorCode.BAD_REQUEST, "Not multipart content! "+req.getContentType() );
       }
       
       MultiMapSolrParams params = parseQueryString( req.getQueryString() );
-      
-      // Create a factory for disk-based file items
-      DiskFileItemFactory factory = new DiskFileItemFactory();
-
-      // Set factory constraints
-      // TODO - configure factory.setSizeThreshold(yourMaxMemorySize);
-      // TODO - configure factory.setRepository(yourTempDirectory);
 
       // Create a new file upload handler
       ServletFileUpload upload = new ServletFileUpload(factory);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java b/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java
new file mode 100644
index 0000000..9c66f0f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java
@@ -0,0 +1,147 @@
+/*
+ * 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.solr.util;
+
+import java.io.File;
+import java.lang.ref.PhantomReference;
+import java.lang.ref.ReferenceQueue;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.commons.io.FileCleaningTracker;
+import org.apache.commons.io.FileDeleteStrategy;
+
+public class SolrFileCleaningTracker extends FileCleaningTracker {
+
+  ReferenceQueue<Object> q = new ReferenceQueue<>();
+
+  final Collection<Tracker> trackers = Collections.synchronizedSet(new HashSet<Tracker>());
+
+  final List<String> deleteFailures = Collections.synchronizedList(new ArrayList<String>());
+
+  volatile boolean exitWhenFinished = false;
+
+  Thread reaper;
+
+  public void track(final File file, final Object marker) {
+    track(file, marker, null);
+  }
+
+  public void track(final File file, final Object marker, final FileDeleteStrategy deleteStrategy) {
+    if (file == null) {
+      throw new NullPointerException("The file must not be null");
+    }
+    addTracker(file.getPath(), marker, deleteStrategy);
+  }
+
+  public void track(final String path, final Object marker) {
+    track(path, marker, null);
+  }
+
+  public void track(final String path, final Object marker, final FileDeleteStrategy deleteStrategy) {
+    if (path == null) {
+      throw new NullPointerException("The path must not be null");
+    }
+    addTracker(path, marker, deleteStrategy);
+  }
+
+  private synchronized void addTracker(final String path, final Object marker,
+      final FileDeleteStrategy deleteStrategy) {
+    if (exitWhenFinished) {
+      throw new IllegalStateException("No new trackers can be added once exitWhenFinished() is called");
+    }
+    if (reaper == null) {
+      reaper = new Reaper();
+      reaper.start();
+    }
+    trackers.add(new Tracker(path, deleteStrategy, marker, q));
+  }
+
+  public int getTrackCount() {
+    return trackers.size();
+  }
+
+  public List<String> getDeleteFailures() {
+    return deleteFailures;
+  }
+
+  public synchronized void exitWhenFinished() {
+    // synchronized block protects reaper
+    exitWhenFinished = true;
+    if (reaper != null) {
+      synchronized (reaper) {
+        reaper.interrupt();
+        try {
+          reaper.join();
+        } catch (InterruptedException e) { 
+          Thread.currentThread().interrupt();
+        }
+      }
+    }
+  }
+
+  private final class Reaper extends Thread {
+    Reaper() {
+      super("MultiPart Upload Tmp File Reaper");
+      setDaemon(true);
+    }
+
+    @Override
+    public void run() {
+      while (exitWhenFinished == false || trackers.size() > 0) {
+        try {
+          // Wait for a tracker to remove.
+          final Tracker tracker = (Tracker) q.remove(); // cannot return null
+          trackers.remove(tracker);
+          if (!tracker.delete()) {
+            deleteFailures.add(tracker.getPath());
+          }
+          tracker.clear();
+        } catch (final InterruptedException e) {
+          Thread.currentThread().interrupt();
+          break;
+        }
+      }
+    }
+  }
+
+  private static final class Tracker extends PhantomReference<Object> {
+
+    private final String path;
+
+    private final FileDeleteStrategy deleteStrategy;
+
+    Tracker(final String path, final FileDeleteStrategy deleteStrategy, final Object marker,
+        final ReferenceQueue<? super Object> queue) {
+      super(marker, queue);
+      this.path = path;
+      this.deleteStrategy = deleteStrategy == null ? FileDeleteStrategy.NORMAL : deleteStrategy;
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+    public boolean delete() {
+      return deleteStrategy.deleteQuietly(new File(path));
+    }
+  }
+
+}
\ No newline at end of file


[2/2] lucene-solr:master: SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests.

Posted by ma...@apache.org.
SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/7dec783b
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/7dec783b
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/7dec783b

Branch: refs/heads/master
Commit: 7dec783b287ab554cc781622b4d6127e553fd2ae
Parents: e82399d
Author: markrmiller <ma...@apache.org>
Authored: Sun Dec 11 22:02:48 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 14 12:57:21 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                            | 2 ++
 .../solr/cloud/OverseerAutoReplicaFailoverThread.java       | 9 +++++++--
 2 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dec783b/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 946a04e..5f0357b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -288,6 +288,8 @@ Other Changes
   response (instead of a SolrException) and includes the remote error message as part of the exception message
   (Tom�s Fern�ndez L�bbe)
 
+* SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests. (Mark Miller)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dec783b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
index 83679a5..10b4bf3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
@@ -89,6 +89,8 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
 
   private final int workLoopDelay;
   private final int waitAfterExpiration;
+
+  private volatile Thread thread;
   
   public OverseerAutoReplicaFailoverThread(CloudConfig config, ZkStateReader zkStateReader,
       UpdateShardHandler updateShardHandler) {
@@ -118,7 +120,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
   
   @Override
   public void run() {
-    
+    this.thread = Thread.currentThread();
     while (!this.isClosed) {
       // work loop
       log.debug("do " + this.getClass().getSimpleName() + " work loop");
@@ -136,7 +138,6 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
         try {
           Thread.sleep(workLoopDelay);
         } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
           return;
         }
       }
@@ -480,6 +481,10 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
   @Override
   public void close() {
     isClosed = true;
+    Thread lThread = thread;
+    if (lThread != null) {
+      lThread.interrupt();
+    }
   }
   
   public boolean isClosed() {