You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2013/10/31 20:35:21 UTC

svn commit: r1537597 - in /lucene/dev/branches/branch_4x: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/handler/component/ solr/core/src/java/org/apache/solr/update/processor/ solr/core/src/test-files/solr/collection1/conf/ solr/core/src/test/...

Author: yonik
Date: Thu Oct 31 19:35:20 2013
New Revision: 1537597

URL: http://svn.apache.org/r1537597
Log:
SOLR-5374: user version update processor

Added:
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java
      - copied unchanged from r1537587, lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java
    lucene/dev/branches/branch_4x/solr/core/src/test-files/solr/collection1/conf/solrconfig-externalversionconstraint.xml
      - copied unchanged from r1537587, lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-externalversionconstraint.xml
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/TestDistribDocBasedVersion.java
      - copied unchanged from r1537587, lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestDistribDocBasedVersion.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java
      - copied unchanged from r1537587, lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/update/TestDocBasedVersionConstraints.java
      - copied unchanged from r1537587, lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/TestDocBasedVersionConstraints.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java

Modified: lucene/dev/branches/branch_4x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/CHANGES.txt?rev=1537597&r1=1537596&r2=1537597&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/solr/CHANGES.txt Thu Oct 31 19:35:20 2013
@@ -83,6 +83,11 @@ New Features
  * SOLR-5406: CloudSolrServer failed to propagate request parameters
    along with delete updates. (yonik)
 
+ * SOLR-5374: Support user configured doc-centric versioning rules
+   via the optional DocBasedVersionConstraintsProcessorFactory
+   update processor (Hossman, yonik)
+
+
 
 Bug Fixes
 ----------------------

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java?rev=1537597&r1=1537596&r2=1537597&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java Thu Oct 31 19:35:20 2013
@@ -201,31 +201,46 @@ public class RealTimeGetComponent extend
 
   }
 
+
+  public static SolrInputDocument DELETED = new SolrInputDocument();
+
+  /** returns the SolrInputDocument from the current tlog, or DELETED if it has been deleted, or
+   * null if there is no record of it in the current update log.  If null is returned, it could
+   * still be in the latest index.
+   */
+  public static SolrInputDocument getInputDocumentFromTlog(SolrCore core, BytesRef idBytes) {
+
+    UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
+
+    if (ulog != null) {
+      Object o = ulog.lookup(idBytes);
+      if (o != null) {
+        // should currently be a List<Oper,Ver,Doc/Id>
+        List entry = (List)o;
+        assert entry.size() >= 3;
+        int oper = (Integer)entry.get(0) & UpdateLog.OPERATION_MASK;
+        switch (oper) {
+          case UpdateLog.ADD:
+            return (SolrInputDocument)entry.get(entry.size()-1);
+          case UpdateLog.DELETE:
+            return DELETED;
+          default:
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,  "Unknown Operation! " + oper);
+        }
+      }
+    }
+
+    return null;
+  }
+
   public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes) throws IOException {
     SolrInputDocument sid = null;
     RefCounted<SolrIndexSearcher> searcherHolder = null;
     try {
       SolrIndexSearcher searcher = null;
-      UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
-
-
-      if (ulog != null) {
-        Object o = ulog.lookup(idBytes);
-        if (o != null) {
-          // should currently be a List<Oper,Ver,Doc/Id>
-          List entry = (List)o;
-          assert entry.size() >= 3;
-          int oper = (Integer)entry.get(0) & UpdateLog.OPERATION_MASK;
-          switch (oper) {
-            case UpdateLog.ADD:
-              sid = (SolrInputDocument)entry.get(entry.size()-1);
-              break;
-            case UpdateLog.DELETE:
-              return null;
-            default:
-              throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,  "Unknown Operation! " + oper);
-          }
-        }
+      sid = getInputDocumentFromTlog(core, idBytes);
+      if (sid == DELETED) {
+        return null;
       }
 
       if (sid == null) {

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1537597&r1=1537596&r2=1537597&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Thu Oct 31 19:35:20 2013
@@ -1111,6 +1111,25 @@ public class DistributedUpdateProcessor 
     }
   }
 
+  // internal helper method to tell if we are the leader for an add or deleteById update
+  boolean isLeader(UpdateCommand cmd) {
+    updateCommand = cmd;
+
+    if (zkEnabled) {
+      zkCheck();
+      if (cmd instanceof AddUpdateCommand) {
+        AddUpdateCommand acmd = (AddUpdateCommand)cmd;
+        nodes = setupRequest(acmd.getHashableId(), acmd.getSolrInputDocument());
+      } else if (cmd instanceof DeleteUpdateCommand) {
+        DeleteUpdateCommand dcmd = (DeleteUpdateCommand)cmd;
+        nodes = setupRequest(dcmd.getId(), null);
+      }
+    } else {
+      isLeader = getNonZkLeaderAssumption(req);
+    }
+
+    return isLeader;
+  }
 
   private void zkCheck() {
     if ((updateCommand.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0) {