You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/01/04 06:04:23 UTC

[GitHub] [lucene-solr] muse-dev[bot] commented on a change in pull request #2159: SOLR-14923: Nested docs indexing performance.

muse-dev[bot] commented on a change in pull request #2159:
URL: https://github.com/apache/lucene-solr/pull/2159#discussion_r551131159



##########
File path: solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java
##########
@@ -95,121 +124,129 @@ public SolrInputDocument getSolrInputDocument() {
    }
 
   /**
-   * Creates and returns a lucene Document to index.
-   * Nested documents, if found, will cause an exception to be thrown.  Call {@link #getLuceneDocsIfNested()} for that.
+   * Creates and returns a lucene Document for in-place update.
+   * The SolrInputDocument itself may be modified, which will be reflected in the update log.
    * Any changes made to the returned Document will not be reflected in the SolrInputDocument, or future calls to this
    * method.
-   * Note that the behavior of this is sensitive to {@link #isInPlaceUpdate()}.*/
-   public Document getLuceneDocument() {
-     final boolean ignoreNestedDocs = false; // throw an exception if found
-     SolrInputDocument solrInputDocument = getSolrInputDocument();
-     if (!isInPlaceUpdate() && getReq().getSchema().isUsableForChildDocs()) {
-       addRootField(solrInputDocument, getRootIdUsingRouteParam());
-     }
-     return DocumentBuilder.toDocument(solrInputDocument, req.getSchema(), isInPlaceUpdate(), ignoreNestedDocs);
-   }
-
-  /** Returns the indexed ID for this document.  The returned BytesRef is retained across multiple calls, and should not be modified. */
-   public BytesRef getIndexedId() {
-     if (indexedId == null) {
-       IndexSchema schema = req.getSchema();
-       SchemaField sf = schema.getUniqueKeyField();
-       if (sf != null) {
-         if (solrDoc != null) {
-           SolrInputField field = solrDoc.getField(sf.getName());
-
-           int count = field==null ? 0 : field.getValueCount();
-           if (count == 0) {
-             if (overwrite) {
-               throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"Document is missing mandatory uniqueKey field: " + sf.getName());
-             }
-           } else if (count  > 1) {
-             throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"Document contains multiple values for uniqueKey field: " + field);
-           } else {
-             BytesRefBuilder b = new BytesRefBuilder();
-             sf.getType().readableToIndexed(field.getFirstValue().toString(), b);
-             indexedId = b.get();
-           }
-         }
-       }
+   */
+   Document makeLuceneDocForInPlaceUpdate() {
+     // perhaps this should move to UpdateHandler or DocumentBuilder?
+     assert isInPlaceUpdate();
+     if (req.getSchema().isUsableForChildDocs() && solrDoc.getField(IndexSchema.ROOT_FIELD_NAME) == null) {
+       solrDoc.setField(IndexSchema.ROOT_FIELD_NAME, getIndexedIdStr());
      }
-     return indexedId;
-   }
-
-   public void setIndexedId(BytesRef indexedId) {
-     this.indexedId = indexedId;
+     final boolean forInPlaceUpdate = true;
+     final boolean ignoreNestedDocs = false; // throw an exception if found
+     return DocumentBuilder.toDocument(solrDoc, req.getSchema(), forInPlaceUpdate, ignoreNestedDocs);
    }
 
-   public String getPrintableId() {
-    if (req != null) {
-      IndexSchema schema = req.getSchema();
-      SchemaField sf = schema.getUniqueKeyField();
-      if (solrDoc != null && sf != null) {
-        SolrInputField field = solrDoc.getField(sf.getName());
-        if (field != null) {
-          return field.getFirstValue().toString();
-        }
-      }
-    }
-     return "(null)";
-   }
+  /**
+   * Returns the indexed ID for this document, or the root ID for nested documents.
+   *
+   * @return possibly null if there's no uniqueKey field
+   */
+  public String getIndexedIdStr() {
+    extractIdsIfNeeded();
+    return indexedIdStr;
+  }
 
   /**
+   * Returns the indexed ID for this document, or the root ID for nested documents. The returned
+   * BytesRef should be treated as immutable. It will not be re-used/modified for additional docs.
    *
-   * @return value of _route_ param({@link ShardParams#_ROUTE_}), otherwise doc id.
+   * @return possibly null if there's no uniqueKey field
    */
-  public String getRootIdUsingRouteParam() {
-     return req.getParams().get(ShardParams._ROUTE_, getHashableId());
-   }
+  public BytesRef getIndexedId() {
+    extractIdsIfNeeded();
+    return indexedId;
+  }
 
   /**
-   * @return String id to hash
+   * Returns the ID of the doc itself, possibly different from {@link #getIndexedIdStr()} which
+   * points to the root doc.
+   *
+   * @return possibly null if there's no uniqueKey field
    */
-  public String getHashableId() {
+  public String getChildDocIdStr() {
+    extractIdsIfNeeded();
+    return childDocIdStr;
+  }
+
+  /** The ID for logging purposes. */
+  public String getPrintableId() {
+    extractIdsIfNeeded();
+    if (indexedIdStr == null) {
+      return "(null)";
+    } else if (indexedIdStr.equals(childDocIdStr)) {
+      return indexedIdStr;
+    } else {
+      return childDocIdStr + " (root=" + indexedIdStr + ")";
+    }
+  }
+
+  private void extractIdsIfNeeded() {
+    if (indexedId != null) {
+      return;
+    }
     IndexSchema schema = req.getSchema();
     SchemaField sf = schema.getUniqueKeyField();
     if (sf != null) {
       if (solrDoc != null) {
         SolrInputField field = solrDoc.getField(sf.getName());
-
-        int count = field == null ? 0 : field.getValueCount();
+        // check some uniqueKey constraints
+        int count = field==null ? 0 : field.getValueCount();
         if (count == 0) {
           if (overwrite) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "Document is missing mandatory uniqueKey field: "
-                    + sf.getName());
+            throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"Document is missing mandatory uniqueKey field: " + sf.getName());
           }
-        } else if (count > 1) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Document contains multiple values for uniqueKey field: " + field);
+        } else if (count  > 1) {
+          throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"Document contains multiple values for uniqueKey field: " + field);
         } else {
-          return field.getFirstValue().toString();
+          this.childDocIdStr = field.getFirstValue().toString();

Review comment:
       *NULL_DEREFERENCE:*  object returned by `field.getFirstValue()` could be null and is dereferenced at line 205.

##########
File path: solr/core/src/java/org/apache/solr/update/UpdateLog.java
##########
@@ -561,6 +565,13 @@ public void add(AddUpdateCommand cmd, boolean clearCaches) {
     // TODO: we currently need to log to maintain correct versioning, rtg, etc
     // if ((cmd.getFlags() & UpdateCommand.REPLAY) != 0) return;
 
+    // This hack could be removed after SOLR-15064 when we insist updates to child docs include _root_.
+    // Until then, if we're in a buffering mode, then the solrDoc won't have the _root_ field.
+    // Otherwise, it should already be there, placed by the client.
+    if (usableForChildDocs && cmd.useRouteAsRoot != null && cmd.solrDoc.getField(IndexSchema.ROOT_FIELD_NAME) == null) {

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `UpdateLog.add(...)` indirectly reads without synchronization from container `cmd.solrDoc._fields` via call to `Map.get(...)`. Potentially races with write in method `UpdateLog.add(...)`.
    Reporting because this access may occur on a background thread.

##########
File path: solr/core/src/java/org/apache/solr/update/UpdateLog.java
##########
@@ -561,6 +565,13 @@ public void add(AddUpdateCommand cmd, boolean clearCaches) {
     // TODO: we currently need to log to maintain correct versioning, rtg, etc
     // if ((cmd.getFlags() & UpdateCommand.REPLAY) != 0) return;
 
+    // This hack could be removed after SOLR-15064 when we insist updates to child docs include _root_.
+    // Until then, if we're in a buffering mode, then the solrDoc won't have the _root_ field.
+    // Otherwise, it should already be there, placed by the client.
+    if (usableForChildDocs && cmd.useRouteAsRoot != null && cmd.solrDoc.getField(IndexSchema.ROOT_FIELD_NAME) == null) {
+      cmd.solrDoc.setField(IndexSchema.ROOT_FIELD_NAME, cmd.getIndexedIdStr());

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Unprotected write. Non-private method `UpdateLog.add(...)` indirectly writes to field `cmd.childDocIdStr` outside of synchronization.
    Reporting because this access may occur on a background thread.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org