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:19 UTC

[lucene-solr] branch jira/solr-11127-2 updated (16a7ea8 -> 13a1a7b)

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

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


    from 16a7ea8  SOLR-11127: Complete implementation + test.
     add 149469d  LUCENE-8696: Refactor, in preparation for creating a new SegmentEndpoint implementation to fix the problem.
     add 9a6f942  Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr
     add ff799ac  LUCENE-8696: Rework how endpoint circles are represented to allow for consistency on WGS84.
     add 9753e00  SOLR-13074: MoveReplicaHDFSTest leaks threads, falls into an endless loop, logging like crazy (Kevin Risden)
     add 8f78c05  SOLR-9762: Remove the workaround implemented for HADOOP-13346 (Kevin Risden)
     add 58beaeb  SOLR-13227: don't throw exceptions for counting facet.range.other.
     add 1ad45a7  fix typo in javadocs
     add 78a57f7  Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr
     add 00c0229  SOLR-12297: Remove debugging System.out line
     add 15f3c3b  SOLR-7321: Remove reflection in FSHDFSUtils.java (Mike Drob, Kevin Risden)
     add 5b2f064  LUCENE-8709: Handle case of creating a HeapPointWriter with size equal 0
     new de5da1f  Merge branch 'master' into jira/solr-13271
     new b2a8aec  SOLR-13271: Fix issues from review.
     new 7021ef5  SOLR-13271: Use the core.indexEnabled flag to fail-fast ongoing updates.
     new d092fb7  SOLR-13271: Use volatile because this is changed and accessed from different threads.
     new 9182364  Merge branch 'jira/solr-13271' into jira/solr-11127-2
     new 13a1a7b  SOLR-11127: More tests.

The 6 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:
 .../java/org/apache/lucene/search/TotalHits.java   |   2 +-
 .../apache/lucene/util/bkd/HeapPointReader.java    |   7 +-
 .../apache/lucene/util/bkd/HeapPointWriter.java    |   7 +-
 .../lucene/spatial3d/geom/GeoStandardPath.java     | 595 +++++++++++++--------
 .../apache/lucene/spatial3d/geom/GeoPathTest.java  |  19 +-
 solr/CHANGES.txt                                   |   8 +
 .../OverseerCollectionMessageHandler.java          |   5 +-
 .../api/collections/ReindexCollectionCmd.java      |  37 +-
 .../java/org/apache/solr/core/CoreContainer.java   |  33 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |   2 +-
 .../solr/handler/admin/CollectionsHandler.java     |   1 +
 .../handler/component/RangeFacetProcessor.java     |  48 +-
 .../org/apache/solr/security/HadoopAuthPlugin.java |  24 +-
 .../org/apache/solr/security/KerberosPlugin.java   |  32 +-
 .../processor/DistributedUpdateProcessor.java      |  26 +-
 .../src/java/org/apache/solr/util/FSHDFSUtils.java |  37 +-
 .../apache/solr/cloud/CollectionsAPISolrJTest.java |  42 +-
 .../solr/cloud/MoveReplicaHDFSFailoverTest.java    |   3 +-
 .../org/apache/solr/cloud/MoveReplicaHDFSTest.java |  60 +--
 .../org/apache/solr/cloud/MoveReplicaTest.java     |  79 +--
 .../apache/solr/cloud/ReindexCollectionTest.java   | 159 +++++-
 .../solr/cloud/hdfs/HDFSCollectionsAPITest.java    |   5 +-
 .../solr/client/solrj/impl/Http2SolrClient.java    |   1 -
 .../solrj/request/CollectionAdminRequest.java      |  36 +-
 .../apache/solr/common/cloud/DocCollection.java    |   9 +
 .../apache/solr/common/cloud/ZkStateReader.java    |   2 +-
 .../apache/solr/common/params/UpdateParams.java    |   3 -
 27 files changed, 750 insertions(+), 532 deletions(-)


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

Posted by ab...@apache.org.
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";
 }


[lucene-solr] 06/06: SOLR-11127: More tests.

Posted by ab...@apache.org.
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 13a1a7b26e3c187060edb337d5a5160cbc0e355d
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Feb 28 21:52:53 2019 +0100

    SOLR-11127: More tests.
---
 .../api/collections/ReindexCollectionCmd.java      |  37 ++++-
 .../solr/handler/admin/CollectionsHandler.java     |   1 +
 .../apache/solr/cloud/ReindexCollectionTest.java   | 159 ++++++++++++++++++---
 .../solrj/request/CollectionAdminRequest.java      |  32 +++--
 .../apache/solr/common/cloud/DocCollection.java    |   4 +
 5 files changed, 191 insertions(+), 42 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
index 1d90f16..d676791 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
@@ -18,6 +18,7 @@
 package org.apache.solr.cloud.api.collections;
 
 import java.lang.invoke.MethodHandles;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -31,6 +32,7 @@ import java.util.stream.Stream;
 
 import org.apache.http.client.HttpClient;
 import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
@@ -88,7 +90,21 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
   public static final String CHK_COL_PREFIX = ".reindex_ck_";
   public static final String REINDEXING_PROP = CollectionAdminRequest.PROPERTY_PREFIX + "reindexing";
   public static final String REINDEX_PHASE_PROP = CollectionAdminRequest.PROPERTY_PREFIX + "reindex_phase";
-  public static final String READONLY_PROP = CollectionAdminRequest.PROPERTY_PREFIX + ZkStateReader.READ_ONLY_PROP;
+
+  private static final List<String> COLLECTION_PARAMS = Arrays.asList(
+      ZkStateReader.CONFIGNAME_PROP,
+      ZkStateReader.NUM_SHARDS_PROP,
+      ZkStateReader.NRT_REPLICAS,
+      ZkStateReader.PULL_REPLICAS,
+      ZkStateReader.TLOG_REPLICAS,
+      ZkStateReader.REPLICATION_FACTOR,
+      ZkStateReader.MAX_SHARDS_PER_NODE,
+      "shards",
+      Policy.POLICY,
+      CollectionAdminParams.CREATE_NODE_SET_PARAM,
+      CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM,
+      ZkStateReader.AUTO_ADD_REPLICAS
+  );
 
   private final OverseerCollectionMessageHandler ocmh;
 
@@ -182,7 +198,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
         TARGET_COL_PREFIX + collection + "_" + seq : target;
     String chkCollection = CHK_COL_PREFIX + collection + "_" + seq;
     String daemonUrl = null;
-
+    Exception exc = null;
     try {
       // 0. set up target and checkpoint collections
       NamedList<Object> cmdResults = new NamedList<>();
@@ -222,8 +238,11 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
       for (String key : message.keySet()) {
         if (key.startsWith("router.")) {
           propMap.put(key, message.getStr(key));
+        } else if (COLLECTION_PARAMS.contains(key)) {
+          propMap.put(key, message.get(key));
         }
       }
+
       propMap.put(ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode);
       propMap.put(CommonAdminParams.WAIT_FOR_FINAL_STATE, true);
       if (rf != null) {
@@ -276,7 +295,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
       cmd = new ZkNodeProps(
           Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.MODIFYCOLLECTION.toLower(),
           ZkStateReader.COLLECTION_PROP, collection,
-          READONLY_PROP, "true");
+          ZkStateReader.READ_ONLY, "true");
       ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
 
       // 2. copy the documents to target
@@ -349,7 +368,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
             Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.MODIFYCOLLECTION.toLower(),
             ZkStateReader.COLLECTION_PROP, collection,
             REINDEXING_PROP, State.FINISHED.toLower(),
-            READONLY_PROP, "");
+            ZkStateReader.READ_ONLY, "");
         ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
       } else {
         cmd = new ZkNodeProps(
@@ -362,11 +381,17 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
 
       results.add(State.FINISHED.toLower(), collection);
     } catch (Exception e) {
+      log.warn("Error during reindexing of " + collection, e);
+      exc = e;
       aborted = true;
+      throw e;
     } finally {
       if (aborted) {
         cleanup(collection, targetCollection, chkCollection, daemonUrl, targetCollection);
         results.add(State.ABORTED.toLower(), collection);
+        if (exc != null) {
+          results.add("error", exc.toString());
+        }
       }
     }
   }
@@ -501,7 +526,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
     }
     ClusterState clusterState = ocmh.cloudManager.getClusterStateProvider().getClusterState();
     NamedList<Object> cmdResults = new NamedList<>();
-    if (!collection.equals(targetCollection)) {
+    if (!collection.equals(targetCollection) && clusterState.hasCollection(targetCollection)) {
       ZkNodeProps cmd = new ZkNodeProps(
           Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.DELETE.toLower(),
           CommonParams.NAME, targetCollection,
@@ -514,7 +539,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
         Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.MODIFYCOLLECTION.toLower(),
         ZkStateReader.COLLECTION_PROP, collection,
         REINDEXING_PROP, State.ABORTED.toLower(),
-        READONLY_PROP, "");
+        ZkStateReader.READ_ONLY, "");
     ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 6a7ea05..b30c42d 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -543,6 +543,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           CREATE_NODE_SET,
           CREATE_NODE_SET_SHUFFLE,
           AUTO_ADD_REPLICAS,
+          "shards",
           CommonParams.ROWS,
           CommonParams.Q);
       copyPropertiesWithPrefix(req.getParams(), m, "router.");
diff --git a/solr/core/src/test/org/apache/solr/cloud/ReindexCollectionTest.java b/solr/core/src/test/org/apache/solr/cloud/ReindexCollectionTest.java
index ae9d5e0..2a9cfd4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ReindexCollectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ReindexCollectionTest.java
@@ -18,15 +18,24 @@
 package org.apache.solr.cloud;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
 
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.cloud.api.collections.ReindexCollectionCmd;
+import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.ImplicitDocRouter;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -40,8 +49,12 @@ public class ReindexCollectionTest extends SolrCloudTestCase {
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(2)
-        .addConfig("conf1", configset("cloud-dynamic"))
-        .addConfig("conf2", configset("cloud-minimal"))
+        // only *_s
+        .addConfig("conf1", configset("cloud-minimal"))
+        // every combination of field flags
+        .addConfig("conf2", configset("cloud-dynamic"))
+        // catch-all * field, indexed+stored
+        .addConfig("conf3", configset("cloud-minimal-inplace-updates"))
         .configure();
   }
 
@@ -50,8 +63,10 @@ public class ReindexCollectionTest extends SolrCloudTestCase {
 
   @Before
   public void doBefore() throws Exception {
-    solrClient = getCloudSolrClient(cluster);
-    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
+    ZkController zkController = cluster.getJettySolrRunner(0).getCoreContainer().getZkController();
+    cloudManager = zkController.getSolrCloudManager();
+    solrClient = new CloudSolrClientBuilder(Collections.singletonList(zkController.getZkServerAddress()),
+        Optional.empty()).build();
   }
 
   @After
@@ -61,28 +76,16 @@ public class ReindexCollectionTest extends SolrCloudTestCase {
     solrClient.close();
   }
 
-  private static final int NUM_DOCS = 200;
+  private static final int NUM_DOCS = 200; // at least two batches, default batchSize=100
 
   @Test
   public void testBasicReindexing() throws Exception {
     final String sourceCollection = "basicReindexing";
 
-    CollectionAdminRequest.createCollection(sourceCollection, "conf1", 2, 2)
-        .setMaxShardsPerNode(-1)
-        .process(solrClient);
+    createCollection(sourceCollection, "conf1", 2, 2);
 
-    cluster.waitForActiveCollection(sourceCollection, 2, 4);
-
-    // verify that indexing works
-    List<SolrInputDocument> docs = new ArrayList<>();
-    for (int i = 0; i < NUM_DOCS; i++) {
-      docs.add(new SolrInputDocument("id", String.valueOf(i), "string_s", String.valueOf(i)));
-    }
-    solrClient.add(sourceCollection, docs);
-    solrClient.commit(sourceCollection);
-    // verify the docs exist
-    QueryResponse rsp = solrClient.query(sourceCollection, params(CommonParams.Q, "*:*"));
-    assertEquals("initial num docs", NUM_DOCS, rsp.getResults().getNumFound());
+    indexDocs(sourceCollection, NUM_DOCS,
+        i -> new SolrInputDocument("id", String.valueOf(i), "string_s", String.valueOf(i)));
 
     final String targetCollection = "basicReindexingTarget";
 
@@ -95,7 +98,121 @@ public class ReindexCollectionTest extends SolrCloudTestCase {
       return ReindexCollectionCmd.State.FINISHED == state;
     });
     // verify the target docs exist
-    rsp = solrClient.query(targetCollection, params(CommonParams.Q, "*:*"));
+    QueryResponse rsp = solrClient.query(targetCollection, params(CommonParams.Q, "*:*"));
+    assertEquals("copied num docs", NUM_DOCS, rsp.getResults().getNumFound());
+  }
+
+  public void testSameTargetReindexing() throws Exception {
+    final String sourceCollection = "sameTargetReindexing";
+    final String targetCollection = sourceCollection;
+
+    createCollection(sourceCollection, "conf1", 2, 2);
+    indexDocs(sourceCollection, NUM_DOCS,
+        i -> new SolrInputDocument("id", String.valueOf(i), "string_s", String.valueOf(i)));
+
+    CollectionAdminRequest.ReindexCollection req = CollectionAdminRequest.reindexCollection(sourceCollection)
+        .setTarget(targetCollection);
+    req.process(solrClient);
+
+    CloudTestUtils.waitForState(cloudManager, "did not finish copying in time", sourceCollection, (liveNodes, coll) -> {
+      ReindexCollectionCmd.State state = ReindexCollectionCmd.State.get(coll.getStr(ReindexCollectionCmd.REINDEXING_PROP));
+      return ReindexCollectionCmd.State.FINISHED == state;
+    });
+    // verify the target docs exist
+    QueryResponse rsp = solrClient.query(targetCollection, params(CommonParams.Q, "*:*"));
+    assertEquals("copied num docs", NUM_DOCS, rsp.getResults().getNumFound());
+  }
+
+  @Test
+  public void testLossySchema() throws Exception {
+    final String sourceCollection = "sourceLossyReindexing";
+    final String targetCollection = "targetLossyReindexing";
+
+
+    createCollection(sourceCollection, "conf2", 2, 2);
+
+    indexDocs(sourceCollection, NUM_DOCS, i ->
+      new SolrInputDocument(
+          "id", String.valueOf(i),
+          "string_s", String.valueOf(i),
+          "sind", "this is a test " + i)); // "sind": indexed=true, stored=false, will be lost...
+
+    CollectionAdminRequest.ReindexCollection req = CollectionAdminRequest.reindexCollection(sourceCollection)
+        .setTarget(targetCollection)
+        .setConfigName("conf3");
+    req.process(solrClient);
+
+    CloudTestUtils.waitForState(cloudManager, "did not finish copying in time", sourceCollection, (liveNodes, coll) -> {
+      ReindexCollectionCmd.State state = ReindexCollectionCmd.State.get(coll.getStr(ReindexCollectionCmd.REINDEXING_PROP));
+      return ReindexCollectionCmd.State.FINISHED == state;
+    });
+    // verify the target docs exist
+    QueryResponse rsp = solrClient.query(targetCollection, params(CommonParams.Q, "*:*"));
     assertEquals("copied num docs", NUM_DOCS, rsp.getResults().getNumFound());
+    for (SolrDocument doc : rsp.getResults()) {
+      String id = (String)doc.getFieldValue("id");
+      assertEquals(id, doc.getFieldValue("string_s"));
+      assertFalse(doc.containsKey("sind")); // lost in translation ...
+    }
+  }
+
+  @Test
+  public void testReshapeReindexing() throws Exception {
+    final String sourceCollection = "reshapeReindexing";
+    final String targetCollection = sourceCollection;
+    createCollection(sourceCollection, "conf1", 2, 2);
+    indexDocs(sourceCollection, NUM_DOCS,
+        i -> new SolrInputDocument("id", String.valueOf(i), "string_s", String.valueOf(i)));
+
+    CollectionAdminRequest.ReindexCollection req = CollectionAdminRequest.reindexCollection(sourceCollection)
+        .setTarget(targetCollection)
+        .setCollectionParam(ZkStateReader.NUM_SHARDS_PROP, 3)
+        .setCollectionParam(ZkStateReader.REPLICATION_FACTOR, 1)
+        .setCollectionParam("router.name", ImplicitDocRouter.NAME)
+        .setCollectionParam("shards", "foo,bar,baz");
+    req.process(solrClient);
+
+    CloudTestUtils.waitForState(cloudManager, "did not finish copying in time", sourceCollection, (liveNodes, coll) -> {
+      ReindexCollectionCmd.State state = ReindexCollectionCmd.State.get(coll.getStr(ReindexCollectionCmd.REINDEXING_PROP));
+      return ReindexCollectionCmd.State.FINISHED == state;
+    });
+    // verify the target docs exist
+    QueryResponse rsp = solrClient.query(targetCollection, params(CommonParams.Q, "*:*"));
+    assertEquals("copied num docs", NUM_DOCS, rsp.getResults().getNumFound());
+
+    // check the shape of the new collection
+    ClusterState clusterState = solrClient.getClusterStateProvider().getClusterState();
+    List<String> aliases = solrClient.getZkStateReader().getAliases().resolveAliases(targetCollection);
+    assertFalse(aliases.isEmpty());
+    String realTargetCollection = aliases.get(0);
+    DocCollection coll = clusterState.getCollection(realTargetCollection);
+    assertNotNull(coll);
+    assertEquals(3, coll.getSlices().size());
+    assertNotNull("foo", coll.getSlice("foo"));
+    assertNotNull("bar", coll.getSlice("bar"));
+    assertNotNull("baz", coll.getSlice("baz"));
+    assertEquals(new Integer(1), coll.getReplicationFactor());
+    assertEquals(ImplicitDocRouter.NAME, coll.getRouter().getName());
+  }
+
+  private void createCollection(String name, String config, int numShards, int numReplicas) throws Exception {
+    CollectionAdminRequest.createCollection(name, config, numShards, numReplicas)
+        .setMaxShardsPerNode(-1)
+        .process(solrClient);
+
+    cluster.waitForActiveCollection(name, numShards, numShards * numReplicas);
+  }
+
+  private void indexDocs(String collection, int numDocs, Function<Integer, SolrInputDocument> generator) throws Exception {
+    List<SolrInputDocument> docs = new ArrayList<>();
+    for (int i = 0; i < numDocs; i++) {
+      docs.add(generator.apply(i));
+    }
+    solrClient.add(collection, docs);
+    solrClient.commit(collection);
+    // verify the docs exist
+    QueryResponse rsp = solrClient.query(collection, params(CommonParams.Q, "*:*"));
+    assertEquals("num docs", NUM_DOCS, rsp.getResults().getNumFound());
+
   }
 }
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 838a0c1..dee2d1d 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
@@ -796,6 +796,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     String configName;
     Boolean keepSource;
     Integer batchSize;
+    Map<String, Object> collectionParams = new HashMap<>();
 
     private ReindexCollection(String collection) {
       super(CollectionAction.REINDEX_COLLECTION, collection);
@@ -821,24 +822,25 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       return this;
     }
 
+    public ReindexCollection setConfigName(String configName) {
+      this.configName = configName;
+      return this;
+    }
+
+    public ReindexCollection setCollectionParam(String key, Object value) {
+      this.collectionParams.put(key, value);
+      return this;
+    }
+
     @Override
     public SolrParams getParams() {
       ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
-      if (target != null) {
-        params.set("target", target);
-      }
-      if (configName != null) {
-        params.set(ZkStateReader.CONFIGNAME_PROP, configName);
-      }
-      if (query != null) {
-        params.set(CommonParams.Q, query);
-      }
-      if (keepSource != null) {
-        params.set("keepSource", keepSource);
-      }
-      if (batchSize != null) {
-        params.set(CommonParams.ROWS, batchSize);
-      }
+      params.setNonNull("target", target);
+      params.setNonNull(ZkStateReader.CONFIGNAME_PROP, configName);
+      params.setNonNull(CommonParams.Q, query);
+      params.setNonNull("keepSource", keepSource);
+      params.setNonNull(CommonParams.ROWS, batchSize);
+      collectionParams.forEach((k, v) -> params.setNonNull(k, v));
       return params;
     }
   }
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 6a4f701..7dc9f10 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
@@ -306,6 +306,10 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     return slice.getLeader();
   }
 
+  public boolean isReadOnly() {
+    return readOnly;
+  }
+
   /**
    * Check that all replicas in a collection are live
    *


[lucene-solr] 01/06: Merge branch 'master' into jira/solr-13271

Posted by ab...@apache.org.
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 de5da1f8b6b5ed820b96bf0548f38187f2ae75a3
Merge: 35e3683 5b2f064
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Feb 28 10:48:09 2019 +0100

    Merge branch 'master' into jira/solr-13271

 .../java/org/apache/lucene/search/TotalHits.java   |   2 +-
 .../apache/lucene/util/bkd/HeapPointReader.java    |   7 +-
 .../apache/lucene/util/bkd/HeapPointWriter.java    |   7 +-
 .../lucene/spatial3d/geom/GeoStandardPath.java     | 595 +++++++++++++--------
 .../apache/lucene/spatial3d/geom/GeoPathTest.java  |  19 +-
 solr/CHANGES.txt                                   |   8 +
 .../handler/component/RangeFacetProcessor.java     |  48 +-
 .../org/apache/solr/security/HadoopAuthPlugin.java |  24 +-
 .../org/apache/solr/security/KerberosPlugin.java   |  32 +-
 .../src/java/org/apache/solr/util/FSHDFSUtils.java |  37 +-
 .../solr/cloud/MoveReplicaHDFSFailoverTest.java    |   3 +-
 .../org/apache/solr/cloud/MoveReplicaHDFSTest.java |  60 +--
 .../org/apache/solr/cloud/MoveReplicaTest.java     |  79 +--
 .../solr/cloud/hdfs/HDFSCollectionsAPITest.java    |   5 +-
 .../solr/client/solrj/impl/Http2SolrClient.java    |   1 -
 15 files changed, 481 insertions(+), 446 deletions(-)


[lucene-solr] 04/06: SOLR-13271: Use volatile because this is changed and accessed from different threads.

Posted by ab...@apache.org.
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 d092fb79ce1b8172e01530f243eb4c1aea403863
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Feb 28 18:04:10 2019 +0100

    SOLR-13271: Use volatile because this is changed and accessed from different threads.
---
 solr/core/src/java/org/apache/solr/core/SolrCore.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index eb9d40d..a831f75 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -236,7 +236,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
   private String metricTag = Integer.toHexString(hashCode());
 
   public boolean searchEnabled = true;
-  public boolean indexEnabled = true;
+  public volatile boolean indexEnabled = true;
 
   public Set<String> getMetricNames() {
     return metricNames;


[lucene-solr] 05/06: Merge branch 'jira/solr-13271' into jira/solr-11127-2

Posted by ab...@apache.org.
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 918236485f0bab0c4eb4156fd2925a2b4c1a1b57
Merge: 16a7ea8 d092fb7
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Feb 28 18:11:17 2019 +0100

    Merge branch 'jira/solr-13271' into jira/solr-11127-2

 .../java/org/apache/lucene/search/TotalHits.java   |   2 +-
 .../apache/lucene/util/bkd/HeapPointReader.java    |   7 +-
 .../apache/lucene/util/bkd/HeapPointWriter.java    |   7 +-
 .../lucene/spatial3d/geom/GeoStandardPath.java     | 595 +++++++++++++--------
 .../apache/lucene/spatial3d/geom/GeoPathTest.java  |  19 +-
 solr/CHANGES.txt                                   |   8 +
 .../OverseerCollectionMessageHandler.java          |   5 +-
 .../java/org/apache/solr/core/CoreContainer.java   |  33 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |   2 +-
 .../handler/component/RangeFacetProcessor.java     |  48 +-
 .../org/apache/solr/security/HadoopAuthPlugin.java |  24 +-
 .../org/apache/solr/security/KerberosPlugin.java   |  32 +-
 .../processor/DistributedUpdateProcessor.java      |  26 +-
 .../src/java/org/apache/solr/util/FSHDFSUtils.java |  37 +-
 .../apache/solr/cloud/CollectionsAPISolrJTest.java |  42 +-
 .../solr/cloud/MoveReplicaHDFSFailoverTest.java    |   3 +-
 .../org/apache/solr/cloud/MoveReplicaHDFSTest.java |  60 +--
 .../org/apache/solr/cloud/MoveReplicaTest.java     |  79 +--
 .../solr/cloud/hdfs/HDFSCollectionsAPITest.java    |   5 +-
 .../solr/client/solrj/impl/Http2SolrClient.java    |   1 -
 .../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 -
 24 files changed, 559 insertions(+), 490 deletions(-)



[lucene-solr] 03/06: SOLR-13271: Use the core.indexEnabled flag to fail-fast ongoing updates.

Posted by ab...@apache.org.
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 7021ef5cfb2d17220abcb703f770d8cf96c76df3
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Feb 28 17:35:54 2019 +0100

    SOLR-13271: Use the core.indexEnabled flag to fail-fast ongoing updates.
---
 .../update/processor/DistributedUpdateProcessor.java   | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)

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 505097f..5f95f72 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
@@ -184,7 +184,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   private Set<String> skippedCoreNodeNames;
   private boolean isIndexChanged = false;
 
-  private boolean readOnly = false;
+  private boolean readOnlyCollection = false;
 
   /**
    * Number of times requests forwarded to some other shard's leader can be retried
@@ -254,7 +254,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       DocCollection coll = zkController.getClusterState().getCollectionOrNull(collection);
       if (coll != null) {
         // check readOnly property in coll state
-        readOnly = coll.getBool(ZkStateReader.READ_ONLY, false);
+        readOnlyCollection = coll.getBool(ZkStateReader.READ_ONLY, false);
       }
     } else {
       collection = null;
@@ -276,6 +276,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     cloneRequiredOnLeader = shouldClone;
   }
 
+  private boolean isReadOnly() {
+    return readOnlyCollection || !req.getCore().indexEnabled;
+  }
+
   private List<Node> setupRequest(String id, SolrInputDocument doc) {
     return setupRequest(id, doc, null);
   }
@@ -678,7 +682,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
     assert TestInjection.injectFailUpdateRequests();
 
-    if (readOnly) {
+    if (isReadOnly()) {
       throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
     }
 
@@ -1430,7 +1434,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     
     assert TestInjection.injectFailUpdateRequests();
 
-    if (readOnly) {
+    if (isReadOnly()) {
       throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
     }
 
@@ -1943,7 +1947,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     
     assert TestInjection.injectFailUpdateRequests();
 
-    if (readOnly) {
+    if (isReadOnly()) {
       throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
     }
 
@@ -2059,7 +2063,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
   @Override
   public void processMergeIndexes(MergeIndexesCommand cmd) throws IOException {
-    if (readOnly) {
+    if (isReadOnly()) {
       throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
     }
     super.processMergeIndexes(cmd);
@@ -2067,7 +2071,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
   @Override
   public void processRollback(RollbackUpdateCommand cmd) throws IOException {
-    if (readOnly) {
+    if (isReadOnly()) {
       throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
     }
     super.processRollback(cmd);