You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2019/02/28 20:53:21 UTC

[lucene-solr] 02/06: SOLR-13271: Fix issues from review.

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

ab pushed a commit to branch jira/solr-11127-2
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit b2a8aec5d2b5373bf4dfa585053a64ab6c171d4e
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Feb 28 17:30:19 2019 +0100

    SOLR-13271: Fix issues from review.
---
 .../OverseerCollectionMessageHandler.java          |  5 ++-
 .../java/org/apache/solr/core/CoreContainer.java   | 33 +++++++++++++++--
 .../processor/DistributedUpdateProcessor.java      | 10 ++----
 .../apache/solr/cloud/CollectionsAPISolrJTest.java | 42 +++++++++++-----------
 .../solrj/request/CollectionAdminRequest.java      |  4 ++-
 .../apache/solr/common/cloud/DocCollection.java    |  5 +++
 .../apache/solr/common/cloud/ZkStateReader.java    |  2 +-
 .../apache/solr/common/params/UpdateParams.java    |  3 --
 8 files changed, 67 insertions(+), 37 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
index 03ec81c..c3e53c6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
@@ -94,7 +94,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.LockTree;
@@ -704,8 +703,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
     if (!areChangesVisible)
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not modify collection " + message);
 
-    // if switching TO read-only mode reload the collection
-    if (message.getBool(CollectionAdminRequest.PROPERTY_PREFIX + ZkStateReader.READ_ONLY_PROP, false)) {
+    // if switching to/from read-only mode reload the collection
+    if (message.keySet().contains(ZkStateReader.READ_ONLY)) {
       reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
     }
   }
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 3dc1bb5..545f471 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -57,6 +57,7 @@ import org.apache.http.auth.AuthSchemeProvider;
 import org.apache.http.client.CredentialsProvider;
 import org.apache.http.config.Lookup;
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
@@ -78,6 +79,7 @@ import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Replica.State;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
@@ -121,6 +123,7 @@ import org.apache.solr.update.SolrCoreState;
 import org.apache.solr.update.UpdateShardHandler;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.OrderedExecutor;
+import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.stats.MetricUtils;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.ConnectionLossException;
@@ -1449,10 +1452,36 @@ public class CoreContainer {
         log.info("Reloading SolrCore '{}' using configuration from {}", cd.getName(), coreConfig.getName());
         newCore = core.reload(coreConfig);
 
+        DocCollection docCollection = null;
+        if (getZkController() != null) {
+          docCollection = getZkController().getClusterState().getCollection(cd.getCollectionName());
+          // turn off indexing now, before the new core is registered
+          if (docCollection.getBool(ZkStateReader.READ_ONLY, false)) {
+            newCore.indexEnabled = false;
+          }
+        }
+
         registerCore(cd, newCore, false, false);
 
-        if (getZkController() != null) {
-          DocCollection docCollection = getZkController().getClusterState().getCollection(cd.getCollectionName());
+        // force commit on old core if the new one is readOnly and prevent any new updates
+        if (!newCore.indexEnabled) {
+          RefCounted<IndexWriter> iwRef = core.getSolrCoreState().getIndexWriter(null);
+          if (iwRef != null) {
+            IndexWriter iw = iwRef.get();
+            // switch old core to readOnly
+            core.indexEnabled = false;
+            try {
+              if (iw != null) {
+                iw.commit();
+              }
+            } finally {
+              iwRef.decref();
+            }
+          }
+        }
+
+
+        if (docCollection != null) {
           Replica replica = docCollection.getReplica(cd.getCloudDescriptor().getCoreNodeName());
           assert replica != null;
           if (replica.getType() == Replica.Type.TLOG) { // TODO: needed here?
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 393e48c..505097f 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -40,7 +40,6 @@ import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrRequest.METHOD;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.SimpleSolrResponse;
@@ -237,8 +236,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     // this should always be used - see filterParams
     DistributedUpdateProcessorFactory.addParamToDistributedRequestWhitelist
       (this.req, UpdateParams.UPDATE_CHAIN, TEST_DISTRIB_SKIP_SERVERS, CommonParams.VERSION_FIELD,
-          UpdateParams.EXPUNGE_DELETES, UpdateParams.OPTIMIZE, UpdateParams.MAX_OPTIMIZE_SEGMENTS,
-          UpdateParams.READ_ONLY_IGNORE);
+          UpdateParams.EXPUNGE_DELETES, UpdateParams.OPTIMIZE, UpdateParams.MAX_OPTIMIZE_SEGMENTS);
 
     CoreContainer cc = req.getCore().getCoreContainer();
 
@@ -255,10 +253,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       replicaType = cloudDesc.getReplicaType();
       DocCollection coll = zkController.getClusterState().getCollectionOrNull(collection);
       if (coll != null) {
-        // check readOnly property in coll state, unless overridden by params
-        if (!req.getParams().getBool(UpdateParams.READ_ONLY_IGNORE, false)) {
-          readOnly = coll.getBool(CollectionAdminRequest.PROPERTY_PREFIX + ZkStateReader.READ_ONLY_PROP, false);
-        }
+        // check readOnly property in coll state
+        readOnly = coll.getBool(ZkStateReader.READ_ONLY, false);
       }
     } else {
       collection = null;
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index 52ba89e..2f61c6c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -35,6 +35,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -45,7 +46,6 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.CoreAdminRequest;
 import org.apache.solr.client.solrj.request.CoreStatus;
-import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.CoreAdminResponse;
@@ -60,7 +60,6 @@ import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CoreAdminParams;
-import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.RetryUtil;
 import org.apache.solr.common.util.TimeSource;
@@ -637,16 +636,16 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
     Replica leader
         = solrClient.getZkStateReader().getLeaderRetry(collectionName, "shard1", DEFAULT_TIMEOUT);
 
-    long coreStartTime = getCoreStatus(leader).getCoreStartTime().getTime();
+    final AtomicReference<Long> coreStartTime = new AtomicReference<>(getCoreStatus(leader).getCoreStartTime().getTime());
 
     // Check for value change
     CollectionAdminRequest.modifyCollection(collectionName,
-        Collections.singletonMap(CollectionAdminRequest.PROPERTY_PREFIX + ZkStateReader.READ_ONLY_PROP, "true"))
+        Collections.singletonMap(ZkStateReader.READ_ONLY, "true"))
         .process(solrClient);
 
     DocCollection coll = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
-    assertNotNull(coll.toString(), coll.getProperties().get(CollectionAdminRequest.PROPERTY_PREFIX + ZkStateReader.READ_ONLY_PROP));
-    assertEquals(coll.toString(), coll.getProperties().get(CollectionAdminRequest.PROPERTY_PREFIX + ZkStateReader.READ_ONLY_PROP).toString(), "true");
+    assertNotNull(coll.toString(), coll.getProperties().get(ZkStateReader.READ_ONLY));
+    assertEquals(coll.toString(), coll.getProperties().get(ZkStateReader.READ_ONLY).toString(), "true");
 
     // wait for the expected collection reload
     RetryUtil.retryUntil("Timed out waiting for core to reload", 30, 1000, TimeUnit.MILLISECONDS, () -> {
@@ -657,9 +656,11 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
         log.warn("Exception getting core start time: {}", e.getMessage());
         return false;
       }
-      return restartTime > coreStartTime;
+      return restartTime > coreStartTime.get();
     });
 
+    coreStartTime.set(getCoreStatus(leader).getCoreStartTime().getTime());
+
     // check for docs - reloading should have committed the new docs
     // this also verifies that searching works in read-only mode
     rsp = solrClient.query(params(CommonParams.Q, "*:*"));
@@ -703,24 +704,25 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
       // expected - ignore
     }
 
-    // check that the override works
-    log.info("=== readOnlyIgnore override test");
-    UpdateRequest ureq = new UpdateRequest();
-    ureq.add(new SolrInputDocument("id", "shouldWork"));
-    ureq.setParam(UpdateParams.READ_ONLY_IGNORE, "true");
-    NamedList<Object> res = solrClient.request(ureq, collectionName);
-    ureq = new UpdateRequest();
-    ureq.deleteById("shouldWork");
-    ureq.setParam(UpdateParams.READ_ONLY_IGNORE, "true");
-    res = solrClient.request(ureq, collectionName);
-
     // Check for removing value
     // setting to empty string is equivalent to removing the property, see SOLR-12507
     CollectionAdminRequest.modifyCollection(collectionName,
-        Collections.singletonMap(CollectionAdminRequest.PROPERTY_PREFIX + ZkStateReader.READ_ONLY_PROP, ""))
+        Collections.singletonMap(ZkStateReader.READ_ONLY, ""))
         .process(cluster.getSolrClient());
     coll = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(collectionName);
-    assertNull(coll.toString(), coll.getProperties().get(CollectionAdminRequest.PROPERTY_PREFIX + ZkStateReader.READ_ONLY_PROP));
+    assertNull(coll.toString(), coll.getProperties().get(ZkStateReader.READ_ONLY));
+
+    // wait for the expected collection reload
+    RetryUtil.retryUntil("Timed out waiting for core to reload", 30, 1000, TimeUnit.MILLISECONDS, () -> {
+      long restartTime = 0;
+      try {
+        restartTime = getCoreStatus(leader).getCoreStartTime().getTime();
+      } catch (Exception e) {
+        log.warn("Exception getting core start time: {}", e.getMessage());
+        return false;
+      }
+      return restartTime > coreStartTime.get();
+    });
 
     // check that updates are working now
     docs.clear();
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index ea42590..a5dad2a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -59,6 +59,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
 import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.READ_ONLY;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
@@ -87,7 +88,8 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       POLICY,
       COLL_CONF,
       WITH_COLLECTION,
-      COLOCATED_WITH);
+      COLOCATED_WITH,
+      READ_ONLY);
 
   protected final CollectionAction action;
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
index e896272..6a4f701 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
@@ -39,6 +39,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
 import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.READ_ONLY;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 
@@ -71,6 +72,7 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
   private final Integer maxShardsPerNode;
   private final Boolean autoAddReplicas;
   private final String policy;
+  private final Boolean readOnly;
 
   public DocCollection(String name, Map<String, Slice> slices, Map<String, Object> props, DocRouter router) {
     this(name, slices, props, router, Integer.MAX_VALUE, ZkStateReader.CLUSTER_STATE);
@@ -99,6 +101,8 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     Boolean autoAddReplicas = (Boolean) verifyProp(props, AUTO_ADD_REPLICAS);
     this.policy = (String) props.get(Policy.POLICY);
     this.autoAddReplicas = autoAddReplicas == null ? Boolean.FALSE : autoAddReplicas;
+    Boolean readOnly = (Boolean) verifyProp(props, READ_ONLY);
+    this.readOnly = readOnly == null ? Boolean.FALSE : readOnly;
     
     verifyProp(props, RULE);
     verifyProp(props, SNITCH);
@@ -152,6 +156,7 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
       case TLOG_REPLICAS:
         return Integer.parseInt(o.toString());
       case AUTO_ADD_REPLICAS:
+      case READ_ONLY:
         return Boolean.parseBoolean(o.toString());
       case "snitch":
       case "rule":
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 2069457..37d6515 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -103,7 +103,6 @@ public class ZkStateReader implements Closeable {
   public static final String MAX_AT_ONCE_PROP = "maxAtOnce";
   public static final String MAX_WAIT_SECONDS_PROP = "maxWaitSeconds";
   public static final String STATE_TIMESTAMP_PROP = "stateTimestamp";
-  public static final String READ_ONLY_PROP = "readOnly";
   public static final String COLLECTIONS_ZKNODE = "/collections";
   public static final String LIVE_NODES_ZKNODE = "/live_nodes";
   public static final String ALIASES = "/aliases.json";
@@ -125,6 +124,7 @@ public class ZkStateReader implements Closeable {
   public static final String PULL_REPLICAS = "pullReplicas";
   public static final String NRT_REPLICAS = "nrtReplicas";
   public static final String TLOG_REPLICAS = "tlogReplicas";
+  public static final String READ_ONLY = "readOnly";
 
   public static final String ROLES = "/roles.json";
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/UpdateParams.java b/solr/solrj/src/java/org/apache/solr/common/params/UpdateParams.java
index 867924e..c4633bd 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/UpdateParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/UpdateParams.java
@@ -68,7 +68,4 @@ public interface UpdateParams
 
   /** Return versions of updates? */
   public static final String VERSIONS = "versions";
-
-  /** Ignore the "readOnly" collection status. */
-  public static final String READ_ONLY_IGNORE = "readOnlyIgnore";
 }