You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2016/12/13 12:51:22 UTC

lucene-solr:jira/solr-5944: SOLR-5944: Addressing review comments with nocommits

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-5944 a6718e9ee -> ca26b8c5c


SOLR-5944: Addressing review comments with nocommits


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

Branch: refs/heads/jira/solr-5944
Commit: ca26b8c5c46b35d3c96164c0855a685d9f6737d8
Parents: a6718e9
Author: Ishan Chattopadhyaya <ic...@gmail.com>
Authored: Tue Dec 13 19:50:51 2016 +0700
Committer: Ishan Chattopadhyaya <ic...@gmail.com>
Committed: Tue Dec 13 19:50:51 2016 +0700

----------------------------------------------------------------------
 .../solr/update/DirectUpdateHandler2.java       | 14 +----
 .../java/org/apache/solr/update/UpdateLog.java  | 27 ++-------
 .../processor/AtomicUpdateDocumentMerger.java   | 61 ++++----------------
 .../processor/DistributedUpdateProcessor.java   |  2 +-
 ...BasedVersionConstraintsProcessorFactory.java |  7 +--
 .../update/TestInPlaceUpdatesStandalone.java    | 12 ++--
 6 files changed, 27 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca26b8c5/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
index 63e4123..c3186bc 100644
--- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
+++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
@@ -452,18 +452,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       //
       synchronized (solrCoreState.getUpdateLock()) {
 
-        // nocommit: this line is very innocuous and easy to over look
-        // nocommit: if the purpose of this line is to work around LUCENE-7344 there should be a comment
-        // nocommit: ...otherwise someone might not realize why it's here and try to remove it
-        // nocommit: ...likewise if/when LUCENE-7344 is fixed no one will realize it can be removed
-        // 
-        //
-        // nocommit: alternatively: if that's not the reason for this line, then what is?
-        //
-        // nocommit: LUCENE-7344
+        // We are reopening a searcher before applying the deletes to overcome LUCENE-7344.
+        // Once LUCENE-7344 is resolved, we can consider removing this.
         if (ulog != null) ulog.openRealtimeSearcher();
-        // nocommit: END
-        
+
         if (delAll) {
           deleteAll();
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca26b8c5/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index 3264508..6d43ec0 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -787,28 +787,18 @@ public static final int VERSION_IDX = 1;
    * @return Returns 0 if a full document was found in the log, -1 if no full document was found. If full document was supposed
    * to be found in the tlogs, but couldn't be found (because the logs were rotated) then the prevPointer is returned.
    */
-  public long applyPartialUpdates(BytesRef id, long prevPointer, long prevVersion,
+  synchronized public long applyPartialUpdates(BytesRef id, long prevPointer, long prevVersion,
       Set<String> onlyTheseFields, SolrDocumentBase latestPartialDoc) {
     
-    // nocommit: changes that should probably be made (see jira comments for elaboration) ...
-    // 1) "final List<TransactionLog> lookupLogs" should be created once, outside of any looping
-    // 2) "lookupLogs" and any calls to "getEntryFromTLog" that use that List *MUST* happen in the same sync block.
-    // 3) 1+2 => the sync block must wrap the while loop
-    // 4) 3 => we might as well declare the entire method synchronized.
-    //
-    //
-    // nocommit: regardless of any changes, need additional eyeballs (besides ishan & hoss) on the synchronization in this method/class
-
+    // nocommit: need additional eyeballs (besides ishan & hoss) on the synchronization in this method/class
     
     SolrInputDocument partialUpdateDoc = null;
 
+    List<TransactionLog> lookupLogs = Arrays.asList(tlog, prevMapLog, prevMapLog2);
     while (prevPointer >= 0) {
       //go through each partial update and apply it on the incoming doc one after another
       List entry;
-      synchronized (this) {
-        List<TransactionLog> lookupLogs = Arrays.asList(tlog, prevMapLog, prevMapLog2);
-        entry = getEntryFromTLog(prevPointer, prevVersion, lookupLogs);
-      }
+      entry = getEntryFromTLog(prevPointer, prevVersion, lookupLogs);
       if (entry == null) {
         return prevPointer; // a previous update was supposed to be found, but wasn't found (due to log rotation)
       }
@@ -864,14 +854,7 @@ public static final int VERSION_IDX = 1;
    *
    * @return The entry if found, otherwise null
    */
-  private List getEntryFromTLog(long lookupPointer, long lookupVersion, List<TransactionLog> lookupLogs) {
-    // nocommit: faily certain this method should be "synchronized...
-    // nocommit: isn't the only valid usage of that method is when the thread has a lock on "this" ?
-    // nocommit: otherwise some other thread might be decrefing/closing the TransactionLog instances passed to this method
-    //
-    // should have no impact on performance/correctness since the only existing usage of this
-    // method are already in a "synchronized (this)" blocks ... but we should protect against future missuse
-    
+  private synchronized List getEntryFromTLog(long lookupPointer, long lookupVersion, List<TransactionLog> lookupLogs) {
     for (TransactionLog lookupLog : lookupLogs) {
       if (lookupLog != null && lookupLog.getLogSize() > lookupPointer) {
         lookupLog.incref();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca26b8c5/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java b/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
index aa7f899..081dc0b 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
@@ -42,7 +42,6 @@ import org.apache.solr.schema.CopyField;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.NumericValueFieldType;
 import org.apache.solr.schema.SchemaField;
-import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.util.RefCounted;
 import org.slf4j.Logger;
@@ -140,40 +139,18 @@ public class AtomicUpdateDocumentMerger {
    * only then is such an update command executed as an in-place update.
    */
   private static boolean isSupportedFieldForInPlaceUpdate(SchemaField schemaField) {
-    if (schemaField == null) {
-      // nocommit: shouldn't "schemaField == null" trip an assert, or cause IllegalArgumentException
-      // nocommit: if there is a reason for this behavior, it should be noted in javadocs, and explained here in comment
-
-      return false; // nocommit: why?
-    }
     return !(schemaField.indexed() || schemaField.stored() || !schemaField.hasDocValues() || 
         schemaField.multiValued() || !(schemaField.getType() instanceof NumericValueFieldType));
   }
   
   /**
-   * Get a list of the non stored DV Fields in the index from a realtime searcher
-   */
-  private static Set<String> getNonStoredDocValueFieldNamesFromSearcher(SolrCore core) {
-    // nocommit: is this dead code?
-    // nocommit: originally calling code now uses IndexWriter to get field names
-    RefCounted<SolrIndexSearcher> holder = core.getRealtimeSearcher();
-    try {
-      SolrIndexSearcher searcher = holder.get();
-      return Collections.unmodifiableSet(searcher.getNonStoredDVs(false));
-    } finally {
-      holder.decref();
-    }
-  }
-  
-  /**
-   * Given an add update command, is it suitable for an in-place update operation? If so, return the updated fields
+   * Given an add update command, compute a list of fields that can be updated in-place. If there is even a single
+   * field in the update that cannot be updated in-place, the entire update cannot be executed in-place (and empty set
+   * will be returned in that case).
    * 
-   * @return If this is an in-place update, return a set of fields that require in-place update.
-   *         If this is not an in-place update, return an empty set.
+   * @return Return a set of fields that can be in-place updated.
    */
-  public static Set<String> isInPlaceUpdate(AddUpdateCommand cmd) {
-    // nocommit: this method name no longer makes any sense since it doesn't return a boolean
-    
+  public static Set<String> computeInPlaceUpdateableFields(AddUpdateCommand cmd) throws IOException {
     SolrInputDocument sdoc = cmd.getSolrInputDocument();
     BytesRef id = cmd.getIndexedId();
     IndexSchema schema = cmd.getReq().getSchema();
@@ -183,9 +160,6 @@ public class AtomicUpdateDocumentMerger {
     
     Set<String> candidateFields = new HashSet<>();
 
-    // Whether this update command has any update to a supported field. A supported update requires the value be a map.
-    boolean hasAMap = false;
-
     // first pass, check the things that are virtually free,
     // and bail out early if anything is obviously not a valid in-place update
     for (String fieldName : sdoc.getFieldNames()) {
@@ -214,14 +188,10 @@ public class AtomicUpdateDocumentMerger {
 
     // lazy init this so we don't call iw.getFields() (and sync lock on IndexWriter) unless needed
     Set<String> fieldNamesFromIndexWriter = null;
-    // nocommit: see question about why dynamicFields are special below...
-    // nocommit: if dynamicField doesn't actaully matter, then don't bother lazy initing this,
-    // nocommit: just move the init logic here.
-    
     // second pass over the candidates for in-place updates
     // this time more expensive checks
     for (String fieldName: candidateFields) {
-      SchemaField schemaField = schema.getFieldOrNull(fieldName);
+      SchemaField schemaField = schema.getField(fieldName);
 
       if (!isSupportedFieldForInPlaceUpdate(schemaField)) {
         return Collections.emptySet();
@@ -234,20 +204,13 @@ public class AtomicUpdateDocumentMerger {
       }
 
       if (null == fieldNamesFromIndexWriter) { // lazy init fieldNamesFromIndexWriter
+        SolrCore core = cmd.getReq().getCore();
+        RefCounted<IndexWriter> holder = core.getSolrCoreState().getIndexWriter(core);
         try {
-          SolrCore core = cmd.getReq().getCore();
-          RefCounted<IndexWriter> holder = core.getSolrCoreState().getIndexWriter(core);
-          try {
-            IndexWriter iw = holder.get();
-            fieldNamesFromIndexWriter = iw.getFieldNames();
-          } finally {
-            holder.decref();
-          }
-        } catch (IOException e) {
-          throw new RuntimeException(e); // nocommit
-
-          // nocommit: if we're going to throw a runtime excep it should be a SolrException with usefull code/msg
-          // nocommit: but why are we catching/wrapping the IOE?  why aren't we rethrowing?
+          IndexWriter iw = holder.get();
+          fieldNamesFromIndexWriter = iw.getFieldNames();
+        } finally {
+          holder.decref();
         }
       }
       if (! fieldNamesFromIndexWriter.contains(fieldName) ) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca26b8c5/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
----------------------------------------------------------------------
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 e0206af..93c1443 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
@@ -1409,7 +1409,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   boolean getUpdatedDocument(AddUpdateCommand cmd, long versionOnUpdate) throws IOException {
     if (!AtomicUpdateDocumentMerger.isAtomicUpdate(cmd)) return false;
 
-    Set<String> inPlaceUpdatedFields = AtomicUpdateDocumentMerger.isInPlaceUpdate(cmd);
+    Set<String> inPlaceUpdatedFields = AtomicUpdateDocumentMerger.computeInPlaceUpdateableFields(cmd);
     if (inPlaceUpdatedFields.size() > 0) { // non-empty means this is suitable for in-place updates
       if (docMerger.doInPlaceUpdateMerge(cmd, inPlaceUpdatedFields)) {
         return true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca26b8c5/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java
index 4eb3f79..e547541 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java
@@ -261,12 +261,7 @@ public class DocBasedVersionConstraintsProcessorFactory extends UpdateRequestPro
       SolrInputDocument oldDoc = null;
 
       if (useFieldCache) {
-        // nocommit: why is this passing false?
-        // nocommit: prior to this patch, there was no such thing as an inplace update...
-        // nocommit: any call to getInputDocument* would by definition return a "full document"
-        //
-        // (last patch fixed getInputDocument call, but not this getInputDocumentFromTlog; intentional?)
-        oldDoc = RealTimeGetComponent.getInputDocumentFromTlog(core, indexedDocId, null, false);
+        oldDoc = RealTimeGetComponent.getInputDocumentFromTlog(core, indexedDocId, null, true);
         if (oldDoc == RealTimeGetComponent.DELETED) {
           return true;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca26b8c5/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
index 20f4462..a154d11 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
@@ -978,7 +978,7 @@ public class TestInPlaceUpdatesStandalone extends TestRTGBase {
 
   /** 
    * @see #callIsInPlaceUpdate
-   * @see AtomicUpdateDocumentMerger#isInPlaceUpdate 
+   * @see AtomicUpdateDocumentMerger#computeInPlaceUpdateableFields 
    */
   @Test
   public void testIsInPlaceUpdate() throws Exception { // nocommit: rename when isInPlaceUpdate is renamed
@@ -1056,7 +1056,7 @@ public class TestInPlaceUpdatesStandalone extends TestRTGBase {
       AddUpdateCommand cmd = buildAddUpdateCommand(req, sdoc("id", "1", "_version_", 42L,
                                                              "inplace_updatable_float", map("inc", 10)));
       AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req);
-      assertTrue(docMerger.doInPlaceUpdateMerge(cmd, AtomicUpdateDocumentMerger.isInPlaceUpdate(cmd)));
+      assertTrue(docMerger.doInPlaceUpdateMerge(cmd, AtomicUpdateDocumentMerger.computeInPlaceUpdateableFields(cmd)));
       assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
       assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
       assertEquals(210f, cmd.getSolrInputDocument().getFieldValue("inplace_updatable_float"));
@@ -1073,7 +1073,7 @@ public class TestInPlaceUpdatesStandalone extends TestRTGBase {
       AddUpdateCommand cmd = buildAddUpdateCommand(req, sdoc("id", "1", "_version_", 42L,
                                                              "inplace_updatable_float", map("inc", 10)));
       AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req);
-      assertTrue(docMerger.doInPlaceUpdateMerge(cmd, AtomicUpdateDocumentMerger.isInPlaceUpdate(cmd)));
+      assertTrue(docMerger.doInPlaceUpdateMerge(cmd, AtomicUpdateDocumentMerger.computeInPlaceUpdateableFields(cmd)));
       assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
       assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
       assertEquals(210f, cmd.getSolrInputDocument().getFieldValue("inplace_updatable_float"));
@@ -1084,18 +1084,16 @@ public class TestInPlaceUpdatesStandalone extends TestRTGBase {
   }
   
   /** 
-   * Helper method that sets up a req/cmd to run {@link AtomicUpdateDocumentMerger#isInPlaceUpdate} 
+   * Helper method that sets up a req/cmd to run {@link AtomicUpdateDocumentMerger#computeInPlaceUpdateableFields} 
    * on the specified solr input document.
    */
   private static Set<String> callIsInPlaceUpdate(final SolrInputDocument sdoc) throws Exception {
-    // nocommit: rename when isInPlaceUpdate is renamed
-
     try (SolrQueryRequest req = req()) {
       AddUpdateCommand cmd = new AddUpdateCommand(req);
       cmd.solrDoc = sdoc;
       assertTrue(cmd.solrDoc.containsKey(DistributedUpdateProcessor.VERSION_FIELD));
       cmd.setVersion(Long.parseLong(cmd.solrDoc.getFieldValue(DistributedUpdateProcessor.VERSION_FIELD).toString()));
-      return AtomicUpdateDocumentMerger.isInPlaceUpdate(cmd);
+      return AtomicUpdateDocumentMerger.computeInPlaceUpdateableFields(cmd);
     }
   }
 }