You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by GitBox <gi...@apache.org> on 2021/08/25 22:06:46 UTC

[GitHub] [solr] madrob commented on a change in pull request #265: SOLR-15590 - Startup and CoreContainer managed by Core Service

madrob commented on a change in pull request #265:
URL: https://github.com/apache/solr/pull/265#discussion_r695204142



##########
File path: solr/core/src/java/org/apache/solr/servlet/CoreService.java
##########
@@ -0,0 +1,451 @@
+/*
+ * 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.servlet;
+
+import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.http.client.HttpClient;
+import org.apache.lucene.util.Version;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoBean.Group;
+import org.apache.solr.core.SolrXmlConfig;
+import org.apache.solr.metrics.AltBufferPoolMetricSet;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.OperatingSystemMetricSet;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricManager.ResolutionStrategy;
+import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.servlet.RateLimitManager.Builder;
+import org.apache.solr.util.StartupLoggingUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.naming.Context;
+import javax.naming.InitialContext;
+import javax.naming.NamingException;
+import javax.naming.NoInitialContextException;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletContextEvent;
+import javax.servlet.ServletContextListener;
+import javax.servlet.UnavailableException;
+import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.WeakHashMap;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.solr.servlet.SolrCoreUtils.loadNodeConfig;
+import static org.apache.solr.servlet.SolrDispatchFilter.PROPERTIES_ATTRIBUTE;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLRHOME_ATTRIBUTE;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLR_INSTALL_DIR_ATTRIBUTE;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLR_LOG_LEVEL;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLR_LOG_MUTECONSOLE;
+
+/**
+ * A service that can provide access to solr cores. This allows us to have multiple filters and
+ * servlets that depend on SolrCore and CoreContainer, while still only having one CoreContainer per
+ * instance of solr.
+ */
+public class CoreService implements ServletContextListener {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private final String metricTag = SolrMetricProducer.getUniqueMetricTag(this, null);
+  private CoreContainer cores;
+  private Properties extraProperties;
+  private HttpClient httpClient;
+  private SolrMetricManager metricManager;
+  private RateLimitManager rateLimitManager;
+  private final CountDownLatch init = new CountDownLatch(1);
+  private String registryName;
+
+  // AFAIK the only reason we need this is to support JettySolrRunner for tests. In tests we might have
+  // multiple CoreContainers in the same JVM, but I *think* that doesn't happen in a real server.
+  private static final Map<ContextInitializationKey, ServiceHolder> services =
+      Collections.synchronizedMap(new WeakHashMap<>());
+
+  // todo: dependency injection instead, but CDI is not jetty native so for now this method and the associated
+  //  map will have to suffice. Note that this relies on ServletContext.equals() not implementing anything significantly
+  //  different thant Object.equals for its .equals method (I've found no implementation that even immplements it).
+  public static ServiceHolder serviceForContext(ServletContext ctx) throws InterruptedException {
+    ContextInitializationKey key = new ContextInitializationKey(ctx);
+    return services.computeIfAbsent(key, ServiceHolder::new);
+  }
+
+  @Override
+  public void contextInitialized(ServletContextEvent sce) {
+    init(sce.getServletContext());
+  }
+
+  @Override
+  public void contextDestroyed(ServletContextEvent sce) {
+      close();
+  }
+
+  CoreContainer getCoreContainer() throws UnavailableException {
+    waitForCoreContainer(() -> cores,init);
+    return cores;
+  }
+  HttpClient getHttpClient() throws UnavailableException {
+    waitForCoreContainer(() -> cores,init);
+    return httpClient;
+  }
+
+  private static void waitForCoreContainer(CoreContainerProvider provider, CountDownLatch latch) throws UnavailableException {
+    CoreContainer cores = provider.getCores();
+    if (cores == null || cores.isShutDown()) {
+      try {
+        latch.await();
+      } catch (InterruptedException e) { //well, no wait then

Review comment:
       reset interrupted status?

##########
File path: solr/core/src/java/org/apache/solr/servlet/CoreService.java
##########
@@ -0,0 +1,451 @@
+/*
+ * 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.servlet;
+
+import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.http.client.HttpClient;
+import org.apache.lucene.util.Version;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoBean.Group;
+import org.apache.solr.core.SolrXmlConfig;
+import org.apache.solr.metrics.AltBufferPoolMetricSet;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.OperatingSystemMetricSet;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricManager.ResolutionStrategy;
+import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.servlet.RateLimitManager.Builder;
+import org.apache.solr.util.StartupLoggingUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.naming.Context;
+import javax.naming.InitialContext;
+import javax.naming.NamingException;
+import javax.naming.NoInitialContextException;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletContextEvent;
+import javax.servlet.ServletContextListener;
+import javax.servlet.UnavailableException;
+import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.WeakHashMap;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.solr.servlet.SolrCoreUtils.loadNodeConfig;
+import static org.apache.solr.servlet.SolrDispatchFilter.PROPERTIES_ATTRIBUTE;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLRHOME_ATTRIBUTE;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLR_INSTALL_DIR_ATTRIBUTE;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLR_LOG_LEVEL;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLR_LOG_MUTECONSOLE;
+
+/**
+ * A service that can provide access to solr cores. This allows us to have multiple filters and
+ * servlets that depend on SolrCore and CoreContainer, while still only having one CoreContainer per
+ * instance of solr.
+ */
+public class CoreService implements ServletContextListener {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private final String metricTag = SolrMetricProducer.getUniqueMetricTag(this, null);
+  private CoreContainer cores;
+  private Properties extraProperties;
+  private HttpClient httpClient;
+  private SolrMetricManager metricManager;
+  private RateLimitManager rateLimitManager;
+  private final CountDownLatch init = new CountDownLatch(1);
+  private String registryName;
+
+  // AFAIK the only reason we need this is to support JettySolrRunner for tests. In tests we might have
+  // multiple CoreContainers in the same JVM, but I *think* that doesn't happen in a real server.
+  private static final Map<ContextInitializationKey, ServiceHolder> services =
+      Collections.synchronizedMap(new WeakHashMap<>());
+
+  // todo: dependency injection instead, but CDI is not jetty native so for now this method and the associated
+  //  map will have to suffice. Note that this relies on ServletContext.equals() not implementing anything significantly
+  //  different thant Object.equals for its .equals method (I've found no implementation that even immplements it).
+  public static ServiceHolder serviceForContext(ServletContext ctx) throws InterruptedException {
+    ContextInitializationKey key = new ContextInitializationKey(ctx);
+    return services.computeIfAbsent(key, ServiceHolder::new);
+  }
+
+  @Override
+  public void contextInitialized(ServletContextEvent sce) {
+    init(sce.getServletContext());
+  }
+
+  @Override
+  public void contextDestroyed(ServletContextEvent sce) {
+      close();
+  }
+
+  CoreContainer getCoreContainer() throws UnavailableException {
+    waitForCoreContainer(() -> cores,init);
+    return cores;
+  }
+  HttpClient getHttpClient() throws UnavailableException {
+    waitForCoreContainer(() -> cores,init);
+    return httpClient;
+  }
+
+  private static void waitForCoreContainer(CoreContainerProvider provider, CountDownLatch latch) throws UnavailableException {
+    CoreContainer cores = provider.getCores();
+    if (cores == null || cores.isShutDown()) {
+      try {
+        latch.await();

Review comment:
       with timeout?

##########
File path: solr/core/src/java/org/apache/solr/servlet/CoreContainerProvider.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.servlet;
+
+import org.apache.solr.core.CoreContainer;
+
+public interface CoreContainerProvider {
+  CoreContainer getCores();

Review comment:
       Javadoc please!

##########
File path: solr/core/src/java/org/apache/solr/servlet/PathExcluder.java
##########
@@ -0,0 +1,24 @@
+/*
+ * 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.servlet;
+
+import java.util.ArrayList;
+import java.util.regex.Pattern;
+
+public interface PathExcluder {
+  void setExcludePatterns(ArrayList<Pattern> excludePatterns);

Review comment:
       Why ArrayList not List?

##########
File path: solr/core/src/java/org/apache/solr/servlet/CoreService.java
##########
@@ -0,0 +1,451 @@
+/*
+ * 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.servlet;
+
+import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.http.client.HttpClient;
+import org.apache.lucene.util.Version;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoBean.Group;
+import org.apache.solr.core.SolrXmlConfig;
+import org.apache.solr.metrics.AltBufferPoolMetricSet;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.OperatingSystemMetricSet;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricManager.ResolutionStrategy;
+import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.servlet.RateLimitManager.Builder;
+import org.apache.solr.util.StartupLoggingUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.naming.Context;
+import javax.naming.InitialContext;
+import javax.naming.NamingException;
+import javax.naming.NoInitialContextException;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletContextEvent;
+import javax.servlet.ServletContextListener;
+import javax.servlet.UnavailableException;
+import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.WeakHashMap;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.solr.servlet.SolrCoreUtils.loadNodeConfig;
+import static org.apache.solr.servlet.SolrDispatchFilter.PROPERTIES_ATTRIBUTE;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLRHOME_ATTRIBUTE;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLR_INSTALL_DIR_ATTRIBUTE;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLR_LOG_LEVEL;
+import static org.apache.solr.servlet.SolrDispatchFilter.SOLR_LOG_MUTECONSOLE;
+
+/**
+ * A service that can provide access to solr cores. This allows us to have multiple filters and
+ * servlets that depend on SolrCore and CoreContainer, while still only having one CoreContainer per
+ * instance of solr.
+ */
+public class CoreService implements ServletContextListener {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private final String metricTag = SolrMetricProducer.getUniqueMetricTag(this, null);
+  private CoreContainer cores;
+  private Properties extraProperties;
+  private HttpClient httpClient;
+  private SolrMetricManager metricManager;
+  private RateLimitManager rateLimitManager;
+  private final CountDownLatch init = new CountDownLatch(1);
+  private String registryName;
+
+  // AFAIK the only reason we need this is to support JettySolrRunner for tests. In tests we might have
+  // multiple CoreContainers in the same JVM, but I *think* that doesn't happen in a real server.
+  private static final Map<ContextInitializationKey, ServiceHolder> services =
+      Collections.synchronizedMap(new WeakHashMap<>());
+
+  // todo: dependency injection instead, but CDI is not jetty native so for now this method and the associated
+  //  map will have to suffice. Note that this relies on ServletContext.equals() not implementing anything significantly
+  //  different thant Object.equals for its .equals method (I've found no implementation that even immplements it).
+  public static ServiceHolder serviceForContext(ServletContext ctx) throws InterruptedException {
+    ContextInitializationKey key = new ContextInitializationKey(ctx);
+    return services.computeIfAbsent(key, ServiceHolder::new);
+  }
+
+  @Override
+  public void contextInitialized(ServletContextEvent sce) {
+    init(sce.getServletContext());
+  }
+
+  @Override
+  public void contextDestroyed(ServletContextEvent sce) {
+      close();
+  }
+
+  CoreContainer getCoreContainer() throws UnavailableException {
+    waitForCoreContainer(() -> cores,init);
+    return cores;
+  }
+  HttpClient getHttpClient() throws UnavailableException {
+    waitForCoreContainer(() -> cores,init);
+    return httpClient;
+  }
+
+  private static void waitForCoreContainer(CoreContainerProvider provider, CountDownLatch latch) throws UnavailableException {
+    CoreContainer cores = provider.getCores();
+    if (cores == null || cores.isShutDown()) {
+      try {
+        latch.await();
+      } catch (InterruptedException e) { //well, no wait then
+      }
+      cores = provider.getCores();
+      if (cores == null || cores.isShutDown()) {
+        final String msg = "Error processing the request. CoreContainer is either not initialized or shutting down.";
+        log.error(msg);
+        throw new UnavailableException(msg);

Review comment:
       log-and-throw is not great, since we will typically log wherever we catch this bubbling up

##########
File path: solr/test-framework/src/java/org/apache/solr/util/BaseTestHarness.java
##########
@@ -77,14 +77,21 @@ public static String validateXPath(String xml, String... tests)
 
     if (tests==null || tests.length == 0) return null;
 
-    Document document = null;
+    Document document;
     try {
+//      if (xml.startsWith("<html>")) {

Review comment:
       I'm not sure what's going on here, or if we need a replacement?

##########
File path: solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
##########
@@ -527,7 +536,7 @@ public void start(boolean reusePort) throws Exception {
       }
       synchronized (JettySolrRunner.this) {
         int cnt = 0;
-        while (!waitOnSolr || !dispatchFilter.isRunning() || getCoreContainer() == null) {
+        while (!waitOnSolr || !dispatchFilter.isRunning() ) {

Review comment:
       why do we no longer need the null check? this is because it would always throw otherwise, right?




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

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org