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 2017/03/14 10:13:45 UTC

[1/2] lucene-solr:master: SOLR-10279: The autoAddReplica feature can result in SolrCores being assigned new shards when using legacyCloud=false and will also fail on a state check when taking over a core registration with a new core.

Repository: lucene-solr
Updated Branches:
  refs/heads/master 120274b45 -> 91c3f78f8


SOLR-10279: The autoAddReplica feature can result in SolrCores being assigned new shards when using legacyCloud=false and will also fail on a state check when taking over a core registration with a new core.


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

Branch: refs/heads/master
Commit: bac3424936eacb2381138612ca70276ef5f909d5
Parents: 120274b
Author: markrmiller <ma...@apache.org>
Authored: Tue Mar 14 06:01:06 2017 -0400
Committer: markrmiller <ma...@apache.org>
Committed: Tue Mar 14 06:01:06 2017 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 +++
 .../OverseerAutoReplicaFailoverThread.java      |  6 ++--
 .../org/apache/solr/cloud/ZkController.java     |  8 +-----
 .../cloud/SharedFSAutoReplicaFailoverTest.java  | 29 ++++++++++++++++++++
 4 files changed, 38 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac34249/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6f6516f..280f165 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -243,6 +243,10 @@ Bug Fixes
 
 * SOLR-10269: MetricsHandler JSON output incorrect. (ab)
 
+* SOLR-10279: The autoAddReplica feature can result in SolrCores being assigned new shards when using
+  legacyCloud=false and will also fail on a state check when taking over a core registration with a new
+  core. (Mark Miller, Hrishikesh Gadre, Patrick Dvorack)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac34249/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 10b4bf3..ea09eef 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
@@ -243,13 +243,14 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
     final String dataDir = badReplica.replica.getStr("dataDir");
     final String ulogDir = badReplica.replica.getStr("ulogDir");
     final String coreNodeName = badReplica.replica.getName();
+    final String shardId = badReplica.slice.getName();
     if (dataDir != null) {
       // need an async request - full shard goes down leader election
       final String coreName = badReplica.replica.getStr(ZkStateReader.CORE_NAME_PROP);
       log.debug("submit call to {}", createUrl);
       MDC.put("OverseerAutoReplicaFailoverThread.createUrl", createUrl);
       try {
-        updateExecutor.submit(() -> createSolrCore(collection, createUrl, dataDir, ulogDir, coreNodeName, coreName));
+        updateExecutor.submit(() -> createSolrCore(collection, createUrl, dataDir, ulogDir, coreNodeName, coreName, shardId));
       } finally {
         MDC.remove("OverseerAutoReplicaFailoverThread.createUrl");
       }
@@ -440,7 +441,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
 
   private boolean createSolrCore(final String collection,
       final String createUrl, final String dataDir, final String ulogDir,
-      final String coreNodeName, final String coreName) {
+      final String coreNodeName, final String coreName, final String shardId) {
 
     try (HttpSolrClient client = new HttpSolrClient.Builder(createUrl).build()) {
       log.debug("create url={}", createUrl);
@@ -451,6 +452,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
       createCmd.setCoreNodeName(coreNodeName);
       // TODO: how do we ensure unique coreName
       // for now, the collections API will use unique names
+      createCmd.setShardId(shardId);
       createCmd.setCoreName(coreName);
       createCmd.setDataDir(dataDir);
       createCmd.setUlogDir(ulogDir.substring(0, ulogDir.length() - "/tlog".length()));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac34249/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index a19b351..69a77f9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1449,13 +1449,7 @@ public class ZkController {
             errorMessage.set("coreNodeName " + coreNodeName + " does not exist in shard " + cloudDesc.getShardId());
             return false;
           }
-          String baseUrl = replica.getStr(BASE_URL_PROP);
-          String coreName = replica.getStr(CORE_NAME_PROP);
-          if (baseUrl.equals(this.baseURL) && coreName.equals(cd.getName())) {
-            return true;
-          }
-          errorMessage.set("coreNodeName " + coreNodeName + " exists, but does not match expected node or core name");
-          return false;
+          return true;
         });
       } catch (TimeoutException e) {
         String error = errorMessage.get();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac34249/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
index 18503e7..9c345fd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
@@ -38,6 +38,7 @@ import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
@@ -103,6 +104,11 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
   public void setUp() throws Exception {
     super.setUp();
     collectionUlogDirMap.clear();
+    if (random().nextBoolean()) {
+      CollectionAdminRequest.setClusterProperty("legacyCloud", "false").process(cloudClient);
+    } else {
+      CollectionAdminRequest.setClusterProperty("legacyCloud", "true").process(cloudClient);
+    }
   }
   
   @Override
@@ -313,6 +319,29 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
     assertSliceAndReplicaCount(collection1);
 
     assertUlogDir(collections);
+    
+    // restart all to test core saved state
+    
+    ChaosMonkey.stop(jettys);
+    ChaosMonkey.stop(controlJetty);
+
+    assertTrue("Timeout waiting for all not live", ClusterStateUtil.waitForAllReplicasNotLive(cloudClient.getZkStateReader(), 45000));
+
+    ChaosMonkey.start(jettys);
+    ChaosMonkey.start(controlJetty);
+
+    assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLiveReplicas(cloudClient.getZkStateReader(), collection1, 120000));
+    
+    assertSliceAndReplicaCount(collection1);
+
+    assertUlogDir(collections);
+    
+    assertSliceAndReplicaCount(collection1);
+    assertSingleReplicationAndShardSize(collection3, 5);
+
+    // all docs should be queried
+    assertSingleReplicationAndShardSize(collection4, 5);
+    queryAndAssertResultSize(collection4, numDocs, 10000);
   }
 
   private void queryAndAssertResultSize(String collection, int expectedResultSize, int timeoutMS)


[2/2] lucene-solr:master: SOLR-10076: Hide keystore and truststore passwords from /admin/info/* outputs.

Posted by ma...@apache.org.
SOLR-10076: Hide keystore and truststore passwords from /admin/info/* outputs.


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

Branch: refs/heads/master
Commit: 91c3f78f8fafbd95cd375bb114e80831ba50d525
Parents: bac3424
Author: markrmiller <ma...@apache.org>
Authored: Tue Mar 14 06:13:34 2017 -0400
Committer: markrmiller <ma...@apache.org>
Committed: Tue Mar 14 06:13:34 2017 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 .../handler/admin/PropertiesRequestHandler.java | 27 ++++++--
 .../solr/handler/admin/SystemInfoHandler.java   | 20 +++++-
 .../org/apache/solr/util/RedactionUtils.java    | 51 ++++++++++++++
 .../admin/PropertiesRequestHandlerTest.java     | 73 ++++++++++++++++++++
 5 files changed, 165 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/91c3f78f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 280f165..4c987db 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -181,6 +181,8 @@ New Features
 
 * SOLR-9045: Make RecoveryStrategy settings configurable. (Christine Poerschke)
 
+* SOLR-10076: Hide keystore and truststore passwords from /admin/info/* outputs. (Mano Kovacs via Mark Miller)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/91c3f78f/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java
index c16cded..57a7492 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java
@@ -17,12 +17,14 @@
 package org.apache.solr.handler.admin;
 
 import java.io.IOException;
+import java.util.Enumeration;
 
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.RedactionUtils;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
 
@@ -32,23 +34,36 @@ import static org.apache.solr.common.params.CommonParams.NAME;
  */
 public class PropertiesRequestHandler extends RequestHandlerBase
 {
+
+  public static final String REDACT_STRING = RedactionUtils.getRedactString();
+
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException 
   {
-    Object props = null;
+    NamedList<String> props = new SimpleOrderedMap<>();
     String name = req.getParams().get(NAME);
     if( name != null ) {
-      NamedList<String> p = new SimpleOrderedMap<>();
-      p.add( name, System.getProperty(name) );
-      props = p;
+      String property = getSecuredPropertyValue(name);
+      props.add( name, property);
     }
     else {
-      props = System.getProperties();
+      Enumeration<?> enumeration = System.getProperties().propertyNames();
+      while(enumeration.hasMoreElements()){
+        name = (String) enumeration.nextElement();
+        props.add(name, getSecuredPropertyValue(name));
+      }
     }
     rsp.add( "system.properties", props );
     rsp.setHttpCaching(false);
   }
-  
+
+  private String getSecuredPropertyValue(String name) {
+    if(RedactionUtils.isSystemPropertySensitive(name)){
+      return REDACT_STRING;
+    }
+    return System.getProperty(name);
+  }
+
   //////////////////////// SolrInfoMBeans methods //////////////////////
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/91c3f78f/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
index d031d69..94fb055 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
@@ -36,6 +36,8 @@ import java.text.DecimalFormat;
 import java.text.DecimalFormatSymbols;
 import java.util.Arrays;
 import java.util.Date;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Locale;
 
 import org.apache.commons.io.IOUtils;
@@ -50,7 +52,7 @@ import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.util.RTimer;
-
+import org.apache.solr.util.RedactionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,6 +68,8 @@ public class SystemInfoHandler extends RequestHandlerBase
 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  public static String REDACT_STRING = RedactionUtils.getRedactString();
+
   /**
    * <p>
    * Undocumented expert level system property to prevent doing a reverse lookup of our hostname.
@@ -373,7 +377,7 @@ public class SystemInfoHandler extends RequestHandlerBase
 
       // the input arguments passed to the Java virtual machine
       // which does not include the arguments to the main method.
-      jmx.add( "commandLineArgs", mx.getInputArguments());
+      jmx.add( "commandLineArgs", getInputArgumentsRedacted(mx));
 
       jmx.add( "startTime", new Date(mx.getStartTime()));
       jmx.add( "upTimeMS",  mx.getUptime() );
@@ -436,6 +440,18 @@ public class SystemInfoHandler extends RequestHandlerBase
 
     return newSizeAndUnits;
   }
+
+  private static List<String> getInputArgumentsRedacted(RuntimeMXBean mx) {
+    List<String> list = new LinkedList<>();
+    for (String arg : mx.getInputArguments()) {
+      if (arg.startsWith("-D") && arg.contains("=") && RedactionUtils.isSystemPropertySensitive(arg.substring(2, arg.indexOf("=")))) {
+        list.add(String.format("%s=%s", arg.substring(0, arg.indexOf("=")), REDACT_STRING));
+      } else {
+        list.add(arg);
+      }
+    }
+    return list;
+  }
   
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/91c3f78f/solr/core/src/java/org/apache/solr/util/RedactionUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/RedactionUtils.java b/solr/core/src/java/org/apache/solr/util/RedactionUtils.java
new file mode 100644
index 0000000..afa2abf
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/RedactionUtils.java
@@ -0,0 +1,51 @@
+/*
+ * 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.util.regex.Pattern;
+
+public class RedactionUtils {
+  public static final String SOLR_REDACTION_SYSTEM_PATTERN_PROP = "solr.redaction.system.pattern";
+  private static Pattern pattern = Pattern.compile(System.getProperty(SOLR_REDACTION_SYSTEM_PATTERN_PROP, ".*password.*"), Pattern.CASE_INSENSITIVE);
+  private static final String REDACT_STRING = "--REDACTED--";
+
+  private static boolean redactSystemProperty = Boolean.parseBoolean(System.getProperty("solr.redaction.system.enabled", "true"));
+
+  /**
+   * Returns if the given system property should be redacted.
+   *
+   * @param name The system property that is being checked.
+   * @return true if property should be redacted.
+   */
+  static public boolean isSystemPropertySensitive(String name) {
+    return redactSystemProperty && pattern.matcher(name).matches();
+  }
+
+  /**
+   * @return redaction string to be used instead of the value.
+   */
+  static public String getRedactString() {
+    return REDACT_STRING;
+  }
+
+  public static void setRedactSystemProperty(boolean redactSystemProperty) {
+    RedactionUtils.redactSystemProperty = redactSystemProperty;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/91c3f78f/solr/core/src/test/org/apache/solr/handler/admin/PropertiesRequestHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/PropertiesRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/PropertiesRequestHandlerTest.java
new file mode 100644
index 0000000..1a959a4
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/PropertiesRequestHandlerTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.handler.admin;
+
+import java.io.StringReader;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.impl.XMLResponseParser;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.util.RedactionUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+public class PropertiesRequestHandlerTest extends SolrTestCaseJ4 {
+
+  public static final String PASSWORD = "secret123";
+  public static final String REDACT_STRING = RedactionUtils.getRedactString();
+
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml", "schema.xml");
+  }
+
+  @Test
+  public void testRedaction() throws Exception {
+    RedactionUtils.setRedactSystemProperty(true);
+    for(String propName: new String[]{"some.password", "javax.net.ssl.trustStorePassword"}){
+      System.setProperty(propName, PASSWORD);
+      NamedList<NamedList<NamedList<Object>>> properties = readProperties();
+
+      assertEquals("Failed to redact "+propName, REDACT_STRING, properties.get(propName));
+    }
+  }
+
+  @Test
+  public void testDisabledRedaction() throws Exception {
+    RedactionUtils.setRedactSystemProperty(false);
+    for(String propName: new String[]{"some.password", "javax.net.ssl.trustStorePassword"}){
+      System.setProperty(propName, PASSWORD);
+      NamedList<NamedList<NamedList<Object>>> properties = readProperties();
+
+      assertEquals("Failed to *not* redact "+propName, PASSWORD, properties.get(propName));
+    }
+  }
+
+  private NamedList<NamedList<NamedList<Object>>> readProperties() throws Exception {
+    String xml = h.query(req(
+        CommonParams.QT, "/admin/properties",
+        CommonParams.WT, "xml"
+    ));
+
+    XMLResponseParser parser = new XMLResponseParser();
+    return (NamedList<NamedList<NamedList<Object>>>)
+        parser.processResponse(new StringReader(xml)).get("system.properties");
+  }
+}
\ No newline at end of file