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 2020/08/30 14:41:43 UTC

[lucene-solr] branch reference_impl_dev updated (036a4d9 -> bc1728a)

This is an automated email from the ASF dual-hosted git repository.

markrmiller pushed a change to branch reference_impl_dev
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


    from 036a4d9  @626 Try to ensure client controls timeouts over server.
     new b44ae65  @627 Handle scheduler threads that can leak.
     new bc1728a  @628 Finish up some XML goodness.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../solr/handler/dataimport/XPathRecordReader.java |  31 ++---
 .../client/solrj/embedded/JettySolrRunner.java     |   4 +-
 .../java/org/apache/solr/core/CoreContainer.java   |   3 +-
 .../handler/DocumentAnalysisRequestHandler.java    |  28 +----
 .../org/apache/solr/handler/loader/XMLLoader.java  |  37 ++----
 .../apache/solr/servlet/SolrDispatchFilter.java    |   3 +-
 solr/solrj/build.gradle                            |   2 +
 solr/solrj/ivy.xml                                 |   1 +
 .../solr/client/solrj/impl/Http2SolrClient.java    |   5 +-
 .../client/solrj/impl/SolrHttpClientScheduler.java |  97 ---------------
 .../solr/client/solrj/impl/XMLResponseParser.java  |  43 +++----
 .../apache/solr/common/EmptyEntityResolver.java    |   4 +-
 .../util/SolrScheduledExecutorScheduler.java       | 138 +++++++++++++++++++++
 13 files changed, 194 insertions(+), 202 deletions(-)
 delete mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientScheduler.java
 create mode 100644 solr/solrj/src/java/org/apache/solr/common/util/SolrScheduledExecutorScheduler.java


[lucene-solr] 01/02: @627 Handle scheduler threads that can leak.

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

markrmiller pushed a commit to branch reference_impl_dev
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit b44ae65b940d0613f8f2265819b59ffde3bd1a03
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Sun Aug 30 08:48:10 2020 -0500

    @627 Handle scheduler threads that can leak.
---
 .../client/solrj/embedded/JettySolrRunner.java     |   4 +-
 .../solr/client/solrj/impl/Http2SolrClient.java    |   5 +-
 .../client/solrj/impl/SolrHttpClientScheduler.java |  97 ---------------
 .../util/SolrScheduledExecutorScheduler.java       | 138 +++++++++++++++++++++
 4 files changed, 144 insertions(+), 100 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java b/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
index c53c17e..52876be 100644
--- a/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
+++ b/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
@@ -21,7 +21,6 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.cloud.SocketProxy;
 import org.apache.solr.client.solrj.impl.Http2SolrClient;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
-import org.apache.solr.client.solrj.impl.SolrHttpClientScheduler;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
@@ -29,6 +28,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.SolrQueuedThreadPool;
+import org.apache.solr.common.util.SolrScheduledExecutorScheduler;
 import org.apache.solr.core.CloudConfig;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.NodeConfig;
@@ -144,7 +144,7 @@ public class JettySolrRunner implements Closeable {
   private volatile boolean isClosed;
 
 
-  private static final Scheduler scheduler = new SolrHttpClientScheduler("JettySolrRunnerScheduler", true, null, new ThreadGroup("JettySolrRunnerScheduler"), 3);
+  private static final Scheduler scheduler = new SolrScheduledExecutorScheduler("JettySolrRunnerScheduler");
   private volatile SolrQueuedThreadPool qtp;
   private volatile boolean closed;
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
index 89002f9..64acbac 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
@@ -44,6 +44,7 @@ import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.SolrQueuedThreadPool;
+import org.apache.solr.common.util.SolrScheduledExecutorScheduler;
 import org.eclipse.jetty.client.HttpClient;
 import org.eclipse.jetty.client.HttpClientTransport;
 import org.eclipse.jetty.client.ProtocolHandlers;
@@ -69,6 +70,7 @@ import org.eclipse.jetty.http2.client.HTTP2Client;
 import org.eclipse.jetty.http2.client.http.HttpClientTransportOverHTTP2;
 import org.eclipse.jetty.util.Fields;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.eclipse.jetty.util.thread.ScheduledExecutorScheduler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -229,6 +231,7 @@ public class Http2SolrClient extends SolrClient {
 
     httpClient.setIdleTimeout(idleTimeout);
     try {
+      httpClient.setScheduler(new SolrScheduledExecutorScheduler("http2client-scheduler"));
       httpClient.setExecutor(httpClientExecutor);
       httpClient.setStrictEventOrdering(true);
       httpClient.setConnectBlocking(false);
@@ -863,7 +866,7 @@ public class Http2SolrClient extends SolrClient {
     return serverBaseUrl;
   }
 
-  private class AsyncTracker {
+  private static class AsyncTracker {
 
     // nocommit - look at outstanding max again
     private static final int MAX_OUTSTANDING_REQUESTS = 30;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientScheduler.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientScheduler.java
deleted file mode 100644
index 54d9f1a..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientScheduler.java
+++ /dev/null
@@ -1,97 +0,0 @@
-package org.apache.solr.client.solrj.impl;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-
-import org.eclipse.jetty.util.component.AbstractLifeCycle;
-import org.eclipse.jetty.util.component.ContainerLifeCycle;
-import org.eclipse.jetty.util.component.Dumpable;
-import org.eclipse.jetty.util.thread.Scheduler;
-
-public class SolrHttpClientScheduler extends AbstractLifeCycle implements Scheduler, Dumpable {
-  private final String name;
-  private final boolean daemon;
-  private final ClassLoader classloader;
-  private final ThreadGroup threadGroup;
-  private volatile ScheduledThreadPoolExecutor scheduler;
-  private volatile Thread thread;
-  private int coreThreads;
-
-  public SolrHttpClientScheduler(String name, boolean daemon, ClassLoader threadFactoryClassLoader) {
-    this(name, daemon, threadFactoryClassLoader, null, 1);
-  }
-
-  public SolrHttpClientScheduler(String name, boolean daemon, ClassLoader threadFactoryClassLoader,
-      ThreadGroup threadGroup, int coreThreads) {
-    this.name = name == null ? "Scheduler-" + hashCode() : name;
-    this.coreThreads = coreThreads;
-    this.daemon = daemon;
-    this.classloader = threadFactoryClassLoader == null ? Thread.currentThread().getContextClassLoader()
-        : threadFactoryClassLoader;
-    this.threadGroup = threadGroup;
-  }
-
-  @Override
-  protected void doStart() throws Exception {
-    scheduler = new ScheduledThreadPoolExecutor(coreThreads, new ThreadFactory() {
-      @Override
-      public Thread newThread(Runnable r) {
-        Thread thread = SolrHttpClientScheduler.this.thread = new Thread(threadGroup, r, name);
-        thread.setDaemon(daemon);
-        thread.setContextClassLoader(classloader);
-        return thread;
-      }
-    });
-    scheduler.setRemoveOnCancelPolicy(true);
-    super.doStart();
-  }
-
-  @Override
-  protected void doStop() throws Exception {
-    scheduler.shutdownNow();
-    super.doStop();
-    scheduler = null;
-  }
-
-  @Override
-  public Task schedule(Runnable task, long delay, TimeUnit unit) {
-    ScheduledThreadPoolExecutor s = scheduler;
-    if (s == null)
-      return () -> false;
-    ScheduledFuture<?> result = s.schedule(task, delay, unit);
-    return new ScheduledFutureTask(result);
-  }
-
-  @Override
-  public String dump() {
-    return ContainerLifeCycle.dump(this);
-  }
-
-  @Override
-  public void dump(Appendable out, String indent) throws IOException {
-    ContainerLifeCycle.dumpObject(out, this);
-    Thread thread = this.thread;
-    if (thread != null) {
-      List<StackTraceElement> frames = Arrays.asList(thread.getStackTrace());
-      ContainerLifeCycle.dump(out, indent, frames);
-    }
-  }
-
-  private static class ScheduledFutureTask implements Task {
-    private final ScheduledFuture<?> scheduledFuture;
-
-    ScheduledFutureTask(ScheduledFuture<?> scheduledFuture) {
-      this.scheduledFuture = scheduledFuture;
-    }
-
-    @Override
-    public boolean cancel() {
-      return scheduledFuture.cancel(false);
-    }
-  }
-}
\ No newline at end of file
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/SolrScheduledExecutorScheduler.java b/solr/solrj/src/java/org/apache/solr/common/util/SolrScheduledExecutorScheduler.java
new file mode 100644
index 0000000..f5e2c3a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/util/SolrScheduledExecutorScheduler.java
@@ -0,0 +1,138 @@
+/*
+ * 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.common.util;
+
+import java.io.IOException;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.eclipse.jetty.util.StringUtil;
+import org.eclipse.jetty.util.annotation.Name;
+import org.eclipse.jetty.util.component.AbstractLifeCycle;
+import org.eclipse.jetty.util.component.Dumpable;
+import org.eclipse.jetty.util.thread.Scheduler;
+
+/**
+ * Implementation of {@link Scheduler} based on Jetty's ScheduledExecutorScheduler which is based on the
+ * on JDK's {@link ScheduledThreadPoolExecutor}.
+ * <p>
+ * While use of {@link ScheduledThreadPoolExecutor} creates futures that will not be used,
+ * it has the advantage of allowing to set a property to remove cancelled tasks from its
+ * queue even if the task did not fire, which provides a huge benefit in the performance
+ * of garbage collection in young generation.
+ */
+public class SolrScheduledExecutorScheduler extends AbstractLifeCycle implements Scheduler, Dumpable {
+  private final String name;
+  private final boolean daemon;
+  private final ClassLoader classloader;
+  private final ThreadGroup threadGroup;
+  private final int threads;
+  private final AtomicInteger count = new AtomicInteger();
+  private volatile ScheduledThreadPoolExecutor scheduler;
+  private volatile Thread thread;
+
+  public SolrScheduledExecutorScheduler() {
+    this(null);
+  }
+
+  public SolrScheduledExecutorScheduler(String name) {
+    this(name, null);
+  }
+
+  public SolrScheduledExecutorScheduler(@Name("name") String name, @Name("threads") int threads) {
+    this(name, null, null, threads);
+  }
+
+  public SolrScheduledExecutorScheduler(String name, ClassLoader classLoader) {
+    this(name, classLoader, null);
+  }
+
+  public SolrScheduledExecutorScheduler(String name, ClassLoader classLoader, ThreadGroup threadGroup) {
+    this(name, classLoader, threadGroup, -1);
+  }
+
+  /**
+   * @param name        The name of the scheduler threads or null for automatic name
+   * @param classLoader The classloader to run the threads with or null to use the current thread context classloader
+   * @param threadGroup The threadgroup to use or null for no thread group
+   * @param threads     The number of threads to pass to the the core {@link ScheduledThreadPoolExecutor} or -1 for a
+   *                    heuristic determined number of threads.
+   */
+  public SolrScheduledExecutorScheduler(@Name("name") String name, @Name("classLoader") ClassLoader classLoader, @Name("threadGroup") ThreadGroup threadGroup, @Name("threads") int threads) {
+    this.name = StringUtil.isBlank(name) ? "Scheduler-" + hashCode() : name;
+    this.daemon = true;
+    this.classloader = classLoader == null ? Thread.currentThread().getContextClassLoader() : classLoader;
+    this.threadGroup = threadGroup;
+    this.threads = threads;
+  }
+
+  @Override
+  protected void doStart() throws Exception {
+    int size = threads > 0 ? threads : 1;
+    scheduler = new ScheduledThreadPoolExecutor(size, r -> {
+      Thread thread = SolrScheduledExecutorScheduler.this.thread = new Thread(threadGroup, r, name + "-" + count.incrementAndGet());
+      thread.setDaemon(daemon);
+      thread.setContextClassLoader(classloader);
+      return thread;
+    });
+    scheduler.setRemoveOnCancelPolicy(true);
+    super.doStart();
+  }
+
+  @Override
+  protected void doStop() throws Exception {
+    scheduler.shutdownNow();
+    super.doStop();
+    ExecutorUtil.awaitTermination(scheduler);
+    scheduler = null;
+  }
+
+  @Override
+  public Task schedule(Runnable task, long delay, TimeUnit unit) {
+    ScheduledThreadPoolExecutor s = scheduler;
+    if (s == null) return () -> false;
+    ScheduledFuture<?> result = s.schedule(task, delay, unit);
+    return new ScheduledFutureTask(result);
+  }
+
+  @Override
+  public String dump() {
+    return Dumpable.dump(this);
+  }
+
+  @Override
+  public void dump(Appendable out, String indent) throws IOException {
+    Thread thread = this.thread;
+    if (thread == null) Dumpable.dumpObject(out, this);
+    else Dumpable.dumpObjects(out, indent, this, (Object[]) thread.getStackTrace());
+  }
+
+  private static class ScheduledFutureTask implements Task {
+    private final ScheduledFuture<?> scheduledFuture;
+
+    ScheduledFutureTask(ScheduledFuture<?> scheduledFuture) {
+      this.scheduledFuture = scheduledFuture;
+    }
+
+    @Override
+    public boolean cancel() {
+      return scheduledFuture.cancel(false);
+    }
+  }
+}


[lucene-solr] 02/02: @628 Finish up some XML goodness.

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

markrmiller pushed a commit to branch reference_impl_dev
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit bc1728a34672df0edb6e70bf4733c74cab72bda0
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Sun Aug 30 09:41:19 2020 -0500

    @628 Finish up some XML goodness.
---
 .../solr/handler/dataimport/XPathRecordReader.java | 31 ++++++----------
 .../java/org/apache/solr/core/CoreContainer.java   |  3 +-
 .../handler/DocumentAnalysisRequestHandler.java    | 28 +-------------
 .../org/apache/solr/handler/loader/XMLLoader.java  | 37 +++++--------------
 .../apache/solr/servlet/SolrDispatchFilter.java    |  3 +-
 solr/solrj/build.gradle                            |  2 +
 solr/solrj/ivy.xml                                 |  1 +
 .../solr/client/solrj/impl/XMLResponseParser.java  | 43 ++++++++++------------
 .../apache/solr/common/EmptyEntityResolver.java    |  4 +-
 9 files changed, 50 insertions(+), 102 deletions(-)

diff --git a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java
index f8f0f7c..10dea2e 100644
--- a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java
+++ b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java
@@ -17,6 +17,7 @@
 package org.apache.solr.handler.dataimport;
 
 import com.ctc.wstx.stax.WstxInputFactory;
+import org.apache.solr.client.solrj.impl.XMLResponseParser;
 import org.apache.solr.common.util.XMLErrorLogger;
 import org.apache.solr.common.EmptyEntityResolver;
 import javax.xml.stream.XMLInputFactory;
@@ -177,12 +178,21 @@ public class XPathRecordReader {
    * @param handler The callback instance
    */
   public void streamRecords(Reader r, Handler handler) {
+    XMLStreamReader parser = null;
     try {
-      XMLStreamReader parser = factory.createXMLStreamReader(r);
+      parser = XMLResponseParser.inputFactory.createXMLStreamReader(r);
       rootNode.parse(parser, handler, new HashMap<>(),
           new Stack<>(), false);
     } catch (Exception e) {
       throw new RuntimeException(e);
+    } finally {
+      if (parser != null) {
+        try {
+          parser.close();
+        } catch (XMLStreamException e) {
+          log.warn("Exception closing parser", e);
+        }
+      }
     }
   }
 
@@ -633,25 +643,6 @@ public class XPathRecordReader {
     return result;
   }
 
-  static XMLInputFactory factory = new WstxInputFactory();
-  static {
-    EmptyEntityResolver.configureXpathFactory(factory);
-    factory.setXMLReporter(XMLLOG);
-    try {
-      // The java 1.6 bundled stax parser (sjsxp) does not currently have a thread-safe
-      // XMLInputFactory, as that implementation tries to cache and reuse the
-      // XMLStreamReader.  Setting the parser-specific "reuse-instance" property to false
-      // prevents this.
-      // All other known open-source stax parsers (and the bea ref impl)
-      // have thread-safe factories.
-      factory.setProperty("reuse-instance", Boolean.FALSE);
-    } catch (IllegalArgumentException ex) {
-      // Other implementations will likely throw this exception since "reuse-instance"
-      // isimplementation specific.
-      log.debug("Unable to set the 'reuse-instance' property for the input chain: {}", factory);
-    }
-  }
-
   /**Implement this interface to stream records as and when one is found.
    *
    */
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index f0736c1..c236161 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -66,6 +66,7 @@ import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.AuthSchemeRegistryProvider;
 import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.CredentialsProviderProvider;
+import org.apache.solr.client.solrj.impl.XMLResponseParser;
 import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
 import org.apache.solr.cloud.CloudDescriptor;
@@ -167,7 +168,7 @@ public class CoreContainer implements Closeable {
 
   static {
     log.warn("expected pre init of xml factories {} {} {} {}", XmlConfigFile.xpathFactory,
-        FieldTypeXmlAdapter.dbf, XMLLoader.inputFactory, XMLLoader.saxFactory);
+        FieldTypeXmlAdapter.dbf, XMLResponseParser.inputFactory, XMLResponseParser.saxFactory);
   }
 
   final SolrCores solrCores = new SolrCores(this);
diff --git a/solr/core/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java
index 09a0039..ea70b86 100644
--- a/solr/core/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/DocumentAnalysisRequestHandler.java
@@ -19,6 +19,7 @@ package org.apache.solr.handler;
 import org.apache.commons.io.IOUtils;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.util.BytesRef;
+import org.apache.solr.client.solrj.impl.XMLResponseParser;
 import org.apache.solr.client.solrj.request.DocumentAnalysisRequest;
 import org.apache.solr.common.EmptyEntityResolver;
 import org.apache.solr.common.ParWork;
@@ -84,30 +85,6 @@ public class DocumentAnalysisRequestHandler extends AnalysisRequestHandlerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final XMLErrorLogger xmllog = new XMLErrorLogger(log);
 
-  private XMLInputFactory inputFactory;
-
-  @Override
-  public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
-    super.init(args);
-
-    inputFactory = XMLInputFactory.newInstance();
-    EmptyEntityResolver.configureXpathFactory(inputFactory);
-    inputFactory.setXMLReporter(xmllog);
-    try {
-      // The java 1.6 bundled stax parser (sjsxp) does not currently have a thread-safe
-      // XMLInputFactory, as that implementation tries to cache and reuse the
-      // XMLStreamReader.  Setting the parser-specific "reuse-instance" property to false
-      // prevents this.
-      // All other known open-source stax parsers (and the bea ref impl)
-      // have thread-safe factories.
-      inputFactory.setProperty("reuse-instance", Boolean.FALSE);
-    } catch (IllegalArgumentException ex) {
-      // Other implementations will likely throw this exception since "reuse-instance"
-      // is implementation specific.
-      log.debug("Unable to set the 'reuse-instance' property for the input factory: {}", inputFactory);
-    }
-  }
-
   @Override
   @SuppressWarnings({"rawtypes"})
   protected NamedList doAnalysis(SolrQueryRequest req) throws Exception {
@@ -152,8 +129,7 @@ public class DocumentAnalysisRequestHandler extends AnalysisRequestHandlerBase {
     try {
       is = stream.getStream();
       final String charset = ContentStreamBase.getCharsetFromContentType(stream.getContentType());
-      parser = (charset == null) ?
-        inputFactory.createXMLStreamReader(is) : inputFactory.createXMLStreamReader(is, charset);
+      parser = (charset == null) ? XMLResponseParser.inputFactory.createXMLStreamReader(is) : XMLResponseParser.inputFactory.createXMLStreamReader(is, charset);
 
       while (true) {
         int event = parser.next();
diff --git a/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java
index 436a090..cf64b8b 100644
--- a/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java
+++ b/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java
@@ -16,10 +16,13 @@
  */
 package org.apache.solr.handler.loader;
 
+import com.ctc.wstx.cfg.InputConfigFlags;
+import com.ctc.wstx.sax.WstxSAXParser;
 import com.ctc.wstx.sax.WstxSAXParserFactory;
 import com.ctc.wstx.stax.WstxInputFactory;
 import com.google.common.collect.Lists;
 import org.apache.commons.io.IOUtils;
+import org.apache.solr.client.solrj.impl.XMLResponseParser;
 import org.apache.solr.common.EmptyEntityResolver;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -48,14 +51,10 @@ import org.codehaus.stax2.XMLStreamReader2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.InputSource;
-import org.xml.sax.SAXNotRecognizedException;
-import org.xml.sax.SAXNotSupportedException;
 import org.xml.sax.XMLReader;
 
 import static org.apache.solr.common.params.CommonParams.ID;
 import static org.apache.solr.common.params.CommonParams.NAME;
-import javax.xml.XMLConstants;
-import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.parsers.SAXParser;
 import javax.xml.parsers.SAXParserFactory;
 import javax.xml.stream.FactoryConfigurationError;
@@ -68,7 +67,6 @@ import javax.xml.transform.TransformerException;
 import javax.xml.transform.dom.DOMResult;
 import javax.xml.transform.dom.DOMSource;
 import javax.xml.transform.sax.SAXSource;
-import javax.xml.transform.stream.StreamSource;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -93,26 +91,9 @@ public class XMLLoader extends ContentStreamLoader {
   public static final int XSLT_CACHE_DEFAULT = 60;
 
   private static int xsltCacheLifetimeSeconds = XSLT_CACHE_DEFAULT;
-  public static WstxInputFactory inputFactory = new WstxInputFactory();
-  public static SAXParserFactory saxFactory = new WstxSAXParserFactory(inputFactory);
-  static {
-
-    inputFactory.configureForSpeed();
-    saxFactory.setNamespaceAware(true);
-    inputFactory.setXMLReporter(xmllog);
-
-    // Init SAX parser (for XSL):
-    saxFactory.setNamespaceAware(true); // XSL needs this!
-
-    EmptyEntityResolver.configureSAXParserFactory(saxFactory);
-    EmptyEntityResolver.configureXpathFactory(inputFactory);
-  }
-
 
   @Override
   public XMLLoader init(SolrParams args) {
-
-
     return this;
   }
 
@@ -144,9 +125,9 @@ public class XMLLoader extends ContentStreamLoader {
         is = stream.getStream();
         final InputSource isrc = new InputSource(is);
         isrc.setEncoding(charset);
-        SAXParser saxParser = saxFactory.newSAXParser();
-
+        WstxSAXParser saxParser = (WstxSAXParser) XMLResponseParser.saxFactory.newSAXParser();
         final XMLReader xmlr = saxParser.getXMLReader();
+        saxParser.getStaxConfig().doSupportExternalEntities(true);
         xmlr.setErrorHandler(xmllog);
         xmlr.setEntityResolver(EmptyEntityResolver.SAX_INSTANCE);
 
@@ -159,13 +140,13 @@ public class XMLLoader extends ContentStreamLoader {
       }
       // second step: feed the intermediate DOM tree into StAX parser:
       try {
-        parser = (XMLStreamReader2) inputFactory.createXMLStreamReader(new DOMSource(result.getNode()));
-
+        parser = (XMLStreamReader2) XMLResponseParser.inputFactory.createXMLStreamReader(new DOMSource(result.getNode()));
+        parser.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, Boolean.TRUE);
         this.processUpdate(req, processor, parser);
       } catch (XMLStreamException e) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e.getMessage(), e);
       } finally {
-        if (parser != null) parser.close();
+        if (parser != null) parser.closeCompletely();
       }
     }
     // Normal XML Loader
@@ -184,7 +165,7 @@ public class XMLLoader extends ContentStreamLoader {
           is = new ByteArrayInputStream(body);
         }
         parser = (XMLStreamReader2) ((charset == null) ?
-                  inputFactory.createXMLStreamReader(is) : inputFactory.createXMLStreamReader(is, charset));
+            XMLResponseParser.inputFactory.createXMLStreamReader(is) : XMLResponseParser.inputFactory.createXMLStreamReader(is, charset));
         parser.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, Boolean.TRUE);
 
         this.processUpdate(req, processor, parser);
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 f254233..74ed713 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -29,6 +29,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.http.HttpHeaders;
 import org.apache.lucene.util.Version;
 import org.apache.solr.api.V2HttpCall;
+import org.apache.solr.client.solrj.impl.XMLResponseParser;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -110,7 +111,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
 
   static {
     log.warn("expected pre init of xml factories {} {} {} {}", XmlConfigFile.xpathFactory,
-        FieldTypeXmlAdapter.dbf, XMLLoader.inputFactory, XMLLoader.saxFactory);
+        FieldTypeXmlAdapter.dbf, XMLResponseParser.inputFactory, XMLResponseParser.saxFactory);
   }
 
   protected volatile CoreContainer cores;
diff --git a/solr/solrj/build.gradle b/solr/solrj/build.gradle
index ec5368e..7eebb23 100644
--- a/solr/solrj/build.gradle
+++ b/solr/solrj/build.gradle
@@ -34,6 +34,8 @@ dependencies {
   api 'org.eclipse.jetty:jetty-alpn-java-client'
   api 'org.eclipse.jetty:jetty-alpn-client'
 
+  api 'com.fasterxml.woodstox:woodstox-core'
+
   api('org.apache.httpcomponents:httpmime', {
     exclude group: "commons-codec", module: "commons-codec"
     exclude group: "commons-logging", module: "commons-logging"
diff --git a/solr/solrj/ivy.xml b/solr/solrj/ivy.xml
index 21f64d0..5ccc05d 100644
--- a/solr/solrj/ivy.xml
+++ b/solr/solrj/ivy.xml
@@ -36,6 +36,7 @@
     <dependency org="commons-io" name="commons-io" rev="${/commons-io/commons-io}" conf="compile"/>
     <dependency org="org.apache.commons" name="commons-math3" rev="${/org.apache.commons/commons-math3}" conf="compile"/>
     <dependency org="org.codehaus.woodstox" name="stax2-api" rev="${/org.codehaus.woodstox/stax2-api}" conf="compile"/>
+    <dependency org="com.fasterxml.woodstox" name="woodstox-core" rev="${/com.fasterxml.woodstox/woodstox-core}" conf="compile"/>
     <dependency org="org.slf4j" name="slf4j-api" rev="${/org.slf4j/slf4j-api}" conf="compile"/>
     <dependency org="org.slf4j" name="jcl-over-slf4j" rev="${/org.slf4j/jcl-over-slf4j}" conf="compile"/>
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/XMLResponseParser.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/XMLResponseParser.java
index b114abe..e1c7137 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/XMLResponseParser.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/XMLResponseParser.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.client.solrj.impl;
 
+import javax.xml.parsers.SAXParserFactory;
 import javax.xml.stream.XMLInputFactory;
 import javax.xml.stream.XMLStreamConstants;
 import javax.xml.stream.XMLStreamException;
@@ -29,6 +30,8 @@ import java.util.Date;
 import java.util.List;
 import java.util.Locale;
 
+import com.ctc.wstx.sax.WstxSAXParserFactory;
+import com.ctc.wstx.stax.WstxInputFactory;
 import org.apache.solr.client.solrj.ResponseParser;
 import org.apache.solr.common.EmptyEntityResolver;
 import org.apache.solr.common.SolrDocument;
@@ -37,6 +40,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.XMLErrorLogger;
+import org.codehaus.stax2.XMLStreamReader2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,28 +55,16 @@ public class XMLResponseParser extends ResponseParser
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final XMLErrorLogger xmllog = new XMLErrorLogger(log);
 
-  // reuse the factory among all parser instances so things like string caches
-  // won't be duplicated
-  static final XMLInputFactory factory;
+  public final static WstxInputFactory inputFactory = new WstxInputFactory();
+  public final static SAXParserFactory saxFactory;
   static {
-    factory = XMLInputFactory.newInstance();
-    EmptyEntityResolver.configureXpathFactory(factory);
+    inputFactory.setXMLReporter(xmllog);
+    inputFactory.configureForSpeed();
+    EmptyEntityResolver.configureInputFactory(inputFactory);
 
-    try {
-      // The java 1.6 bundled stax parser (sjsxp) does not currently have a thread-safe
-      // XMLInputFactory, as that implementation tries to cache and reuse the
-      // XMLStreamReader.  Setting the parser-specific "reuse-instance" property to false
-      // prevents this.
-      // All other known open-source stax parsers (and the bea ref impl)
-      // have thread-safe factories.
-      factory.setProperty("reuse-instance", Boolean.FALSE);
-    }
-    catch( IllegalArgumentException ex ) {
-      // Other implementations will likely throw this exception since "reuse-instance"
-      // isimplementation specific.
-      log.debug( "Unable to set the 'reuse-instance' property for the input factory: {}", factory );
-    }
-    factory.setXMLReporter(xmllog);
+    saxFactory = new WstxSAXParserFactory(inputFactory);
+    saxFactory.setNamespaceAware(true);
+    EmptyEntityResolver.configureSAXParserFactory(saxFactory);
   }
 
   public XMLResponseParser() {}
@@ -90,9 +82,10 @@ public class XMLResponseParser extends ResponseParser
 
   @Override
   public NamedList<Object> processResponse(Reader in) {
-    XMLStreamReader parser = null;
+    XMLStreamReader2 parser = null;
     try {
-      parser = factory.createXMLStreamReader(in);
+      parser = (XMLStreamReader2) inputFactory.createXMLStreamReader(in);
+      parser.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, Boolean.TRUE);
     } catch (XMLStreamException e) {
       throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, "parsing error", e);
     }
@@ -105,7 +98,7 @@ public class XMLResponseParser extends ResponseParser
   {
      XMLStreamReader parser = null;
     try {
-      parser = factory.createXMLStreamReader(in, encoding);
+      parser = inputFactory.createXMLStreamReader(in, encoding);
     } catch (XMLStreamException e) {
       throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, "parsing error", e);
     }
@@ -155,7 +148,9 @@ public class XMLResponseParser extends ResponseParser
       try {
         parser.close();
       }
-      catch( Exception ex ){}
+      catch( Exception ex ){
+        log.warn("Exception closing parser", ex);
+      }
     }
   }
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/EmptyEntityResolver.java b/solr/solrj/src/java/org/apache/solr/common/EmptyEntityResolver.java
index 422b444..7d37d12 100644
--- a/solr/solrj/src/java/org/apache/solr/common/EmptyEntityResolver.java
+++ b/solr/solrj/src/java/org/apache/solr/common/EmptyEntityResolver.java
@@ -98,7 +98,7 @@ public final class EmptyEntityResolver {
   /** Configures the given {@link XMLInputFactory} to not parse external entities.
    * No further configuration on is needed, all required entity resolvers are configured.
    */
-  public static void configureXpathFactory(XMLInputFactory inputFactory) {
+  public static void configureInputFactory(XMLInputFactory inputFactory) {
     // don't enable validation of DTDs:
     trySetStAXProperty(inputFactory, XMLInputFactory.IS_VALIDATING, Boolean.FALSE);
     // enable this to *not* produce parsing failure on external entities:
@@ -107,7 +107,7 @@ public final class EmptyEntityResolver {
   }
 
 
-  public static void configureXpathFactory(XPathFactory xPathFactory) {
+  public static void configureXPathFactory(XPathFactory xPathFactory) {
     // don't enable validation of DTDs:
     trySetStAXProperty(xPathFactory, XMLInputFactory.IS_VALIDATING, Boolean.FALSE);
     // enable this to *not* produce parsing failure on external entities: